diff --git a/LICENSE-binary b/LICENSE-binary index d84f311ce1909..2df6cf730b7a9 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -364,6 +364,7 @@ org.objenesis:objenesis:2.6 org.xerial.snappy:snappy-java:1.1.10.4 org.yaml:snakeyaml:2.0 org.wildfly.openssl:wildfly-openssl:1.1.3.Final +software.amazon.awssdk:bundle:jar:2.20.160 -------------------------------------------------------------------------------- diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java index c04c1bb47fcea..19ee9d1414ecf 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java @@ -244,6 +244,13 @@ public final class StoreStatisticNames { public static final String OBJECT_MULTIPART_UPLOAD_ABORTED = "object_multipart_aborted"; + /** + * Object multipart list request. + * Value :{@value}. + */ + public static final String OBJECT_MULTIPART_UPLOAD_LIST = + "object_multipart_list"; + /** * Object put/multipart upload count. * Value :{@value}. diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index 1501ee4bc268a..5511c8dd2d3b5 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -1201,17 +1201,24 @@ AWS secret key used by S3A file system. Omit for IAM role-based or provider-based authentication. + + fs.s3a.session.token + Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider + as one of the providers. + + + fs.s3a.aws.credentials.provider org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider, org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider, - com.amazonaws.auth.EnvironmentVariableCredentialsProvider, + software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider, org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider Comma-separated class names of credential provider classes which implement - com.amazonaws.auth.AWSCredentialsProvider. + software.amazon.awssdk.auth.credentials.AwsCredentialsProvider. When S3A delegation tokens are not enabled, this list will be used to directly authenticate with S3 and other AWS services. @@ -1219,43 +1226,6 @@ 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, AWS_SECRET_ACCESS_KEY, - and AWS_SESSION_TOKEN as documented in the AWS SDK. - * org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider: picks up - IAM credentials of any EC2 VM or AWS container in which the process is running. - - - - - fs.s3a.session.token - Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider - as one of the providers. @@ -1353,10 +1323,10 @@ Note: for job submission to actually collect these tokens, Kerberos must be enabled. - Options are: + Bindings available in hadoop-aws 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 + org.apache.hadoop.fs.s3a.auth.delegation.RoleTokenBinding @@ -1428,8 +1398,14 @@ fs.s3a.attempts.maximum - 20 - How many times we should retry commands on transient errors. + 5 + + Number of times the AWS client library should retry errors before + escalating to the S3A code: {@value}. + The S3A connector does its own selective retries; the only time the AWS + SDK operations are not wrapped is during multipart copy via the AWS SDK + transfer manager. + @@ -1562,14 +1538,14 @@ fs.s3a.encryption.algorithm Specify a server-side encryption or client-side encryption algorithm for s3a: file system. Unset by default. It supports the - following values: 'AES256' (for SSE-S3), 'SSE-KMS', 'SSE-C', and 'CSE-KMS' + following values: 'AES256' (for SSE-S3), 'SSE-KMS', 'DSSE-KMS', 'SSE-C', and 'CSE-KMS' fs.s3a.encryption.key Specific encryption key to use if fs.s3a.encryption.algorithm - has been set to 'SSE-KMS', 'SSE-C' or 'CSE-KMS'. In the case of SSE-C + has been set to 'SSE-KMS', 'DSSE-KMS', 'SSE-C' or 'CSE-KMS'. In the case of SSE-C , the value of this property should be the Base64 encoded key. If you are using SSE-KMS and leave this property empty, you'll be using your default's S3 KMS key, otherwise you should set this property to the specific KMS key diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 247f097588ba6..1c913c9a6fc05 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -191,7 +191,9 @@ 1.3.1 1.0-beta-1 900 - 1.12.499 + 1.12.565 + 2.20.160 + 1.0.1 2.7.1 1.11.2 2.1 @@ -1103,15 +1105,31 @@ com.amazonaws - aws-java-sdk-bundle + aws-java-sdk-core ${aws-java-sdk.version} - io.netty + * + * + + + + + software.amazon.awssdk + bundle + ${aws-java-sdk-v2.version} + + + * * + + software.amazon.eventstream + eventstream + ${aws.eventstream.version} + org.apache.mina mina-core diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml index 4bc00a3e5f161..17f7db631c3e9 100644 --- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml +++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml @@ -64,6 +64,11 @@ + + + + + diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index f802695a83818..a73fc30790676 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -483,11 +483,25 @@ test test-jar + + com.amazonaws - aws-java-sdk-bundle + aws-java-sdk-core + provided + + + software.amazon.awssdk + bundle compile + + software.amazon.eventstream + eventstream + test + org.assertj assertj-core diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSBadRequestException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSBadRequestException.java index 482c5a1db7a1f..dfa1725acd11e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSBadRequestException.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSBadRequestException.java @@ -18,7 +18,9 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.AmazonServiceException; +import software.amazon.awssdk.awscore.exception.AwsServiceException; + +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_400_BAD_REQUEST; /** * A 400 "Bad Request" exception was received. @@ -28,7 +30,7 @@ public class AWSBadRequestException extends AWSServiceIOException { /** * HTTP status code which signals this failure mode was triggered: {@value}. */ - public static final int STATUS_CODE = 400; + public static final int STATUS_CODE = SC_400_BAD_REQUEST; /** * Instantiate. @@ -36,7 +38,7 @@ public class AWSBadRequestException extends AWSServiceIOException { * @param cause the underlying cause */ public AWSBadRequestException(String operation, - AmazonServiceException cause) { + AwsServiceException cause) { super(operation, cause); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientIOException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientIOException.java index f31f3c79b355e..43b636c6d5764 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientIOException.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientIOException.java @@ -18,29 +18,28 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.AmazonClientException; -import com.amazonaws.SdkBaseException; +import software.amazon.awssdk.core.exception.SdkException; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import java.io.IOException; /** - * IOException equivalent of an {@link AmazonClientException}. + * IOException equivalent of an {@link SdkException}. */ public class AWSClientIOException extends IOException { private final String operation; public AWSClientIOException(String operation, - SdkBaseException cause) { + SdkException cause) { super(cause); Preconditions.checkArgument(operation != null, "Null 'operation' argument"); Preconditions.checkArgument(cause != null, "Null 'cause' argument"); this.operation = operation; } - public AmazonClientException getCause() { - return (AmazonClientException) super.getCause(); + public SdkException getCause() { + return (SdkException) super.getCause(); } @Override @@ -48,4 +47,11 @@ public String getMessage() { return operation + ": " + getCause().getMessage(); } + /** + * Query inner cause for retryability. + * @return what the cause says. + */ + public boolean retryable() { + return getCause().retryable(); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSCredentialProviderList.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSCredentialProviderList.java index 75b09a4ed7460..1a084cd2357d5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSCredentialProviderList.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSCredentialProviderList.java @@ -19,6 +19,8 @@ package org.apache.hadoop.fs.s3a; import java.io.Closeable; +import java.io.IOException; +import java.nio.file.AccessDeniedException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -27,21 +29,22 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; -import com.amazonaws.AmazonClientException; -import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.AnonymousAWSCredentials; -import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; 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; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; + +import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.exception.SdkException; /** * A list of providers. @@ -51,17 +54,17 @@ *
    *
  1. Allows extra providers to be added dynamically.
  2. *
  3. If any provider in the chain throws an exception other than - * an {@link AmazonClientException}, that is rethrown, rather than + * an {@link SdkException}, that is rethrown, rather than * swallowed.
  4. *
  5. Has some more diagnostics.
  6. - *
  7. On failure, the last "relevant" AmazonClientException raised is + *
  8. On failure, the last "relevant" {@link SdkException} raised is * rethrown; exceptions other than 'no credentials' have priority.
  9. - *
  10. Special handling of {@link AnonymousAWSCredentials}.
  11. + *
  12. Special handling of {@link AnonymousCredentialsProvider}.
  13. *
*/ -@InterfaceAudience.Private +@InterfaceAudience.LimitedPrivate("extensions") @InterfaceStability.Evolving -public final class AWSCredentialProviderList implements AWSCredentialsProvider, +public final class AWSCredentialProviderList implements AwsCredentialsProvider, AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger( @@ -73,9 +76,9 @@ public final class AWSCredentialProviderList implements AWSCredentialsProvider, CREDENTIALS_REQUESTED_WHEN_CLOSED = "Credentials requested after provider list was closed"; - private final List providers = new ArrayList<>(1); + private final List providers = new ArrayList<>(1); private boolean reuseLastProvider = true; - private AWSCredentialsProvider lastProvider; + private AwsCredentialsProvider lastProvider; private final AtomicInteger refCount = new AtomicInteger(1); @@ -98,17 +101,17 @@ public AWSCredentialProviderList() { * @param providers provider list. */ public AWSCredentialProviderList( - Collection providers) { + Collection providers) { this.providers.addAll(providers); } /** - * Create with an initial list of providers. + * Create with an initial list of SDK V2 credential providers. * @param name name for error messages, may be "" * @param providerArgs provider list. */ public AWSCredentialProviderList(final String name, - final AWSCredentialsProvider... providerArgs) { + final AwsCredentialsProvider... providerArgs) { setName(name); Collections.addAll(providers, providerArgs); } @@ -126,11 +129,11 @@ public void setName(final String name) { } /** - * Add a new provider. - * @param p provider + * Add a new SDK V2 provider. + * @param provider provider */ - public void add(AWSCredentialsProvider p) { - providers.add(p); + public void add(AwsCredentialsProvider provider) { + providers.add(provider); } /** @@ -142,16 +145,11 @@ public void addAll(AWSCredentialProviderList other) { } /** - * Refresh all child entries. + * Was an implementation of the v1 refresh; now just + * a no-op. */ - @Override + @Deprecated public void refresh() { - if (isClosed()) { - return; - } - for (AWSCredentialsProvider provider : providers) { - provider.refresh(); - } } /** @@ -160,7 +158,7 @@ public void refresh() { * @return a set of credentials (possibly anonymous), for authenticating. */ @Override - public AWSCredentials getCredentials() { + public AwsCredentials resolveCredentials() { if (isClosed()) { LOG.warn(CREDENTIALS_REQUESTED_WHEN_CLOSED); throw new NoAuthWithAWSException(name + @@ -168,18 +166,18 @@ public AWSCredentials getCredentials() { } checkNotEmpty(); if (reuseLastProvider && lastProvider != null) { - return lastProvider.getCredentials(); + return lastProvider.resolveCredentials(); } - AmazonClientException lastException = null; - for (AWSCredentialsProvider provider : providers) { + SdkException lastException = null; + for (AwsCredentialsProvider provider : providers) { try { - AWSCredentials credentials = provider.getCredentials(); + AwsCredentials credentials = provider.resolveCredentials(); Preconditions.checkNotNull(credentials, "Null credentials returned by %s", provider); - if ((credentials.getAWSAccessKeyId() != null && - credentials.getAWSSecretKey() != null) - || (credentials instanceof AnonymousAWSCredentials)) { + if ((credentials.accessKeyId() != null && credentials.secretAccessKey() != null) || ( + provider instanceof AnonymousCredentialsProvider + || provider instanceof AnonymousAWSCredentialsProvider)) { lastProvider = provider; LOG.debug("Using credentials from {}", provider); return credentials; @@ -196,7 +194,7 @@ public AWSCredentials getCredentials() { } LOG.debug("No credentials from {}: {}", provider, e.toString()); - } catch (AmazonClientException e) { + } catch (SdkException e) { lastException = e; LOG.debug("No credentials provided by {}: {}", provider, e.toString(), e); @@ -222,14 +220,13 @@ public AWSCredentials getCredentials() { * * @return providers */ - @VisibleForTesting - List getProviders() { + public List getProviders() { return providers; } /** * Verify that the provider list is not empty. - * @throws AmazonClientException if there are no providers. + * @throws SdkException if there are no providers. */ public void checkNotEmpty() { if (providers.isEmpty()) { @@ -255,9 +252,11 @@ public String listProviderNames() { */ @Override public String toString() { - return "AWSCredentialProviderList[" + - name + - "refcount= " + refCount.get() + ": [" + + return "AWSCredentialProviderList" + + " name=" + name + + "; refcount= " + refCount.get() + + "; size="+ providers.size() + + ": [" + StringUtils.join(providers, ", ") + ']' + (lastProvider != null ? (" last provider: " + lastProvider) : ""); } @@ -317,7 +316,7 @@ public void close() { } // do this outside the synchronized block. - for (AWSCredentialsProvider p : providers) { + for (AwsCredentialsProvider p : providers) { if (p instanceof Closeable) { IOUtils.closeStream((Closeable) p); } else if (p instanceof AutoCloseable) { @@ -333,4 +332,27 @@ public void close() { public int size() { return providers.size(); } + + + /** + * Translate an exception if it or its inner exception is an + * {@link CredentialInitializationException}. + * If this condition is not met, null is returned. + * @param path path of operation. + * @param throwable exception + * @return a translated exception or null. + */ + public static IOException maybeTranslateCredentialException(String path, + Throwable throwable) { + if (throwable instanceof CredentialInitializationException) { + // the exception raised by AWSCredentialProvider list if the + // credentials were not accepted, + return (AccessDeniedException)new AccessDeniedException(path, null, + throwable.toString()).initCause(throwable); + } else if (throwable.getCause() instanceof CredentialInitializationException) { + return maybeTranslateCredentialException(path, throwable.getCause()); + } else { + return null; + } + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSNoResponseException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSNoResponseException.java index e6a23b2361da9..b8562714b1aae 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSNoResponseException.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSNoResponseException.java @@ -18,14 +18,19 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.AmazonServiceException; +import software.amazon.awssdk.awscore.exception.AwsServiceException; /** * Status code 443, no response from server. This is considered idempotent. */ public class AWSNoResponseException extends AWSServiceIOException { public AWSNoResponseException(String operation, - AmazonServiceException cause) { + AwsServiceException cause) { super(operation, cause); } + + @Override + public boolean retryable() { + return true; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSRedirectException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSRedirectException.java index bb337ee5eebda..cb478482a8ed4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSRedirectException.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSRedirectException.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.AmazonServiceException; +import software.amazon.awssdk.awscore.exception.AwsServiceException; /** * Request is redirected. @@ -32,7 +32,7 @@ public class AWSRedirectException extends AWSServiceIOException { * @param cause the underlying cause */ public AWSRedirectException(String operation, - AmazonServiceException cause) { + AwsServiceException cause) { super(operation, cause); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSS3IOException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSS3IOException.java index 014d217b6a4fb..de1dd8b4a7a18 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSS3IOException.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSS3IOException.java @@ -18,14 +18,13 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.services.s3.model.AmazonS3Exception; +import software.amazon.awssdk.services.s3.model.S3Exception; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import java.util.Map; - /** - * Wrap a {@link AmazonS3Exception} as an IOE, relaying all + * Wrap a {@link S3Exception} as an IOE, relaying all * getters. */ @InterfaceAudience.Public @@ -38,24 +37,12 @@ public class AWSS3IOException extends AWSServiceIOException { * @param cause the underlying cause */ public AWSS3IOException(String operation, - AmazonS3Exception cause) { + S3Exception cause) { super(operation, cause); } - public AmazonS3Exception getCause() { - return (AmazonS3Exception) super.getCause(); - } - - public String getErrorResponseXml() { - return getCause().getErrorResponseXml(); - } - - public Map getAdditionalDetails() { - return getCause().getAdditionalDetails(); - } - - public String getExtendedRequestId() { - return getCause().getExtendedRequestId(); + public S3Exception getCause() { + return (S3Exception) super.getCause(); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSServiceIOException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSServiceIOException.java index a9c2c9840203f..434ec8df292ac 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSServiceIOException.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSServiceIOException.java @@ -18,13 +18,15 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.AmazonServiceException; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.awscore.exception.AwsErrorDetails; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; + /** * A specific exception from AWS operations. - * The exception must always be created with an {@link AmazonServiceException}. + * The exception must always be created with an {@link AwsServiceException}. * The attributes of this exception can all be directly accessed. */ @InterfaceAudience.Public @@ -37,36 +39,27 @@ public class AWSServiceIOException extends AWSClientIOException { * @param cause the underlying cause */ public AWSServiceIOException(String operation, - AmazonServiceException cause) { + AwsServiceException cause) { super(operation, cause); } - public AmazonServiceException getCause() { - return (AmazonServiceException) super.getCause(); - } - - public String getRequestId() { - return getCause().getRequestId(); + public AwsServiceException getCause() { + return (AwsServiceException) super.getCause(); } - public String getServiceName() { - return getCause().getServiceName(); + public String requestId() { + return getCause().requestId(); } - public String getErrorCode() { - return getCause().getErrorCode(); + public AwsErrorDetails awsErrorDetails() { + return getCause().awsErrorDetails(); } - public int getStatusCode() { - return getCause().getStatusCode(); + public int statusCode() { + return getCause().statusCode(); } - public String getRawResponseContent() { - return getCause().getRawResponseContent(); + public String extendedRequestId() { + return getCause().extendedRequestId(); } - - public boolean isRetryable() { - return getCause().isRetryable(); - } - } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSServiceThrottledException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSServiceThrottledException.java index 131cea7562242..5cd2eb9d320f3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSServiceThrottledException.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSServiceThrottledException.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.AmazonServiceException; +import software.amazon.awssdk.awscore.exception.AwsServiceException; /** * Exception raised when a service was throttled. @@ -36,7 +36,12 @@ public class AWSServiceThrottledException extends AWSServiceIOException { * @param cause the underlying cause */ public AWSServiceThrottledException(String operation, - AmazonServiceException cause) { + AwsServiceException cause) { super(operation, cause); } + + @Override + public boolean retryable() { + return true; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java index 83be294fac7cd..f7c72f8530959 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSStatus500Exception.java @@ -18,20 +18,25 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.AmazonServiceException; +import software.amazon.awssdk.awscore.exception.AwsServiceException; /** - * A 500 response came back from a service. - * This is considered probably retriable, That is, we assume - *
    - *
  1. whatever error happened in the service itself to have happened - * before the infrastructure committed the operation.
  2. - *
  3. Nothing else got through either.
  4. - *
+ * A 5xx response came back from a service. + * The 500 error considered retriable by the AWS SDK, which will have already + * tried it {@code fs.s3a.attempts.maximum} times before reaching s3a + * code. + * How it handles other 5xx errors is unknown: S3A FS code will treat them + * as unrecoverable on the basis that they indicate some third-party store + * or gateway problem. */ public class AWSStatus500Exception extends AWSServiceIOException { public AWSStatus500Exception(String operation, - AmazonServiceException cause) { + AwsServiceException cause) { super(operation, cause); } + + @Override + public boolean retryable() { + return false; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSUnsupportedFeatureException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSUnsupportedFeatureException.java new file mode 100644 index 0000000000000..dd1cebf45f9cf --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSUnsupportedFeatureException.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import software.amazon.awssdk.awscore.exception.AwsServiceException; + +/** + * A store returned an error indicating that it does not support a + * specific S3 feature such as the chosen ChangeDetectionPolicy or + * other AWS-S3 feature that the third-party store does not support. + * The workaround is to disable use of the feature. + * Unrecoverable. + */ +public class AWSUnsupportedFeatureException extends AWSServiceIOException { + + /** + * Instantiate. + * @param operation operation which triggered this + * @param cause the underlying cause + */ + public AWSUnsupportedFeatureException(String operation, + AwsServiceException cause) { + super(operation, cause); + } + + @Override + public boolean retryable() { + return false; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java index 564c03bf731d7..dcfc2a03b1232 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java @@ -18,9 +18,10 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.AnonymousAWSCredentials; -import com.amazonaws.auth.AWSCredentials; +import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -35,23 +36,18 @@ * property fs.s3a.aws.credentials.provider. Therefore, changing the class name * would be a backward-incompatible change. * - * @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider - * as part of upgrading S3A to SDK V2. See HADOOP-18073. */ @InterfaceAudience.Private @InterfaceStability.Stable -@Deprecated -public class AnonymousAWSCredentialsProvider implements AWSCredentialsProvider { +public class AnonymousAWSCredentialsProvider implements AwsCredentialsProvider { public static final String NAME = "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider"; - public AWSCredentials getCredentials() { - return new AnonymousAWSCredentials(); + public AwsCredentials resolveCredentials() { + return AnonymousCredentialsProvider.create().resolveCredentials(); } - public void refresh() {} - @Override public String toString() { return getClass().getSimpleName(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ArnResource.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ArnResource.java index a85f26223ffcf..98745b295b5d3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ArnResource.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ArnResource.java @@ -20,7 +20,7 @@ import javax.annotation.Nonnull; -import com.amazonaws.arn.Arn; +import software.amazon.awssdk.arns.Arn; /** * Represents an Arn Resource, this can be an accesspoint or bucket. @@ -126,14 +126,14 @@ public String getEndpoint() { public static ArnResource accessPointFromArn(String arn) throws IllegalArgumentException { Arn parsed = Arn.fromString(arn); - if (parsed.getRegion().isEmpty() || parsed.getAccountId().isEmpty() || - parsed.getResourceAsString().isEmpty()) { + if (!parsed.region().isPresent() || !parsed.accountId().isPresent() || + parsed.resourceAsString().isEmpty()) { throw new IllegalArgumentException( String.format("Access Point Arn %s has an invalid format or missing properties", arn)); } - String resourceName = parsed.getResource().getResource(); - return new ArnResource(resourceName, parsed.getAccountId(), parsed.getRegion(), - parsed.getPartition(), arn); + String resourceName = parsed.resource().resource(); + return new ArnResource(resourceName, parsed.accountId().get(), parsed.region().get(), + parsed.partition(), arn); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index edd63b5f263ca..944ab5dcdcb00 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -140,7 +140,6 @@ private Constants() { public static final String ASSUMED_ROLE_POLICY = "fs.s3a.assumed.role.policy"; - @SuppressWarnings("deprecation") public static final String ASSUMED_ROLE_CREDENTIALS_DEFAULT = SimpleAWSCredentialsProvider.NAME; @@ -229,6 +228,9 @@ private Constants() { /** * Number of times the AWS client library should retry errors before * escalating to the S3A code: {@value}. + * The S3A connector does its own selective retries; the only time the AWS + * SDK operations are not wrapped is during multipart copy via the AWS SDK + * transfer manager. */ public static final String MAX_ERROR_RETRIES = "fs.s3a.attempts.maximum"; @@ -236,7 +238,7 @@ private Constants() { * Default number of times the AWS client library should retry errors before * escalating to the S3A code: {@value}. */ - public static final int DEFAULT_MAX_ERROR_RETRIES = 10; + public static final int DEFAULT_MAX_ERROR_RETRIES = 5; /** * Experimental/Unstable feature: should the AWS client library retry @@ -265,7 +267,7 @@ private Constants() { // milliseconds until we give up trying to establish a connection to s3 public static final String ESTABLISH_TIMEOUT = "fs.s3a.connection.establish.timeout"; - public static final int DEFAULT_ESTABLISH_TIMEOUT = 50000; + public static final int DEFAULT_ESTABLISH_TIMEOUT = 5000; // milliseconds until we give up on a connection to s3 public static final String SOCKET_TIMEOUT = "fs.s3a.connection.timeout"; @@ -597,7 +599,7 @@ private Constants() { public static final String SIGNING_ALGORITHM_STS = "fs.s3a." + Constants.AWS_SERVICE_IDENTIFIER_STS.toLowerCase() - + "signing-algorithm"; + + ".signing-algorithm"; public static final String S3N_FOLDER_SUFFIX = "_$folder$"; public static final String FS_S3A_BLOCK_SIZE = "fs.s3a.block.size"; @@ -739,14 +741,21 @@ private Constants() { public static final String STREAM_READ_GAUGE_INPUT_POLICY = "stream_read_gauge_input_policy"; + /** + * S3 Client Factory implementation class: {@value}. + * Unstable and incompatible between v1 and v2 SDK versions. + */ @InterfaceAudience.Private @InterfaceStability.Unstable public static final String S3_CLIENT_FACTORY_IMPL = "fs.s3a.s3.client.factory.impl"; + /** + * Default factory: + * {@code org.apache.hadoop.fs.s3a.DefaultS3ClientFactory}. + */ @InterfaceAudience.Private @InterfaceStability.Unstable - @SuppressWarnings("deprecation") public static final Class DEFAULT_S3_CLIENT_FACTORY_IMPL = DefaultS3ClientFactory.class; @@ -1170,6 +1179,12 @@ private Constants() { */ public static final String AWS_S3_CENTRAL_REGION = "us-east-1"; + /** + * The default S3 region when using cross region client. + * Value {@value}. + */ + public static final String AWS_S3_DEFAULT_REGION = "us-east-2"; + /** * Require that all S3 access is made through Access Points. */ @@ -1299,4 +1314,23 @@ private Constants() { */ public static final int DEFAULT_PREFETCH_MAX_BLOCKS_COUNT = 4; + /** + * The bucket region header. + */ + public static final String BUCKET_REGION_HEADER = "x-amz-bucket-region"; + + /** + * Should directory operations purge uploads? + * This adds at least one parallelized list operation to the call, + * plus the overhead of deletions. + * Value: {@value}. + */ + public static final String DIRECTORY_OPERATIONS_PURGE_UPLOADS = + "fs.s3a.directory.operations.purge.uploads"; + + /** + * Default value of {@link #DIRECTORY_OPERATIONS_PURGE_UPLOADS}: {@value}. + */ + public static final boolean DIRECTORY_OPERATIONS_PURGE_UPLOADS_DEFAULT = false; + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java index 2f0cfd37ad37c..c6a1b8fcc713b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.AmazonClientException; +import software.amazon.awssdk.core.exception.SdkClientException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -26,22 +26,22 @@ /** * Exception which Hadoop's AWSCredentialsProvider implementations should * throw when there is a problem with the credential setup. This - * is a subclass of {@link AmazonClientException} which sets - * {@link #isRetryable()} to false, so as to fail fast. + * is a subclass of {@link SdkClientException} which sets + * {@link #retryable()} to false, so as to fail fast. * This is used in credential providers and elsewhere. * When passed through {@code S3AUtils.translateException()} it - * is mapped to an AccessDeniedException. As a result, the Invoker - * code will automatically translate + * is mapped to an AccessDeniedException. */ @InterfaceAudience.Public @InterfaceStability.Stable -public class CredentialInitializationException extends AmazonClientException { +public class CredentialInitializationException extends SdkClientException { + public CredentialInitializationException(String message, Throwable t) { - super(message, t); + super(builder().message(message).cause(t)); } public CredentialInitializationException(String message) { - super(message); + super(builder().message(message)); } /** @@ -49,7 +49,7 @@ public CredentialInitializationException(String message) { * @return false, always. */ @Override - public boolean isRetryable() { + public boolean retryable() { return false; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index bab4ec896ff78..6aaa0e2d06605 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -20,32 +20,27 @@ import java.io.IOException; import java.net.URI; +import java.net.URISyntaxException; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.SdkClientException; -import com.amazonaws.client.builder.AwsClientBuilder; -import com.amazonaws.handlers.RequestHandler2; -import com.amazonaws.regions.RegionUtils; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.AmazonS3Builder; -import com.amazonaws.services.s3.AmazonS3Client; -import com.amazonaws.services.s3.AmazonS3ClientBuilder; -import com.amazonaws.services.s3.AmazonS3EncryptionClientV2Builder; -import com.amazonaws.services.s3.AmazonS3EncryptionV2; -import com.amazonaws.services.s3.S3ClientOptions; -import com.amazonaws.services.s3.internal.ServiceUtils; -import com.amazonaws.services.s3.model.CryptoConfigurationV2; -import com.amazonaws.services.s3.model.CryptoMode; -import com.amazonaws.services.s3.model.CryptoRangeGetMode; -import com.amazonaws.services.s3.model.EncryptionMaterialsProvider; -import com.amazonaws.services.s3.model.KMSEncryptionMaterialsProvider; -import com.amazonaws.util.AwsHostNameUtils; -import com.amazonaws.util.RuntimeHttpUtils; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; -import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.fs.s3a.impl.AWSClientConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.awscore.util.AwsHostNameUtils; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3BaseClientBuilder; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.S3Configuration; +import software.amazon.awssdk.services.s3.multipart.MultipartConfiguration; +import software.amazon.awssdk.transfer.s3.S3TransferManager; + import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -54,15 +49,14 @@ import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector; import org.apache.hadoop.fs.store.LogExactlyOnce; -import static com.amazonaws.services.s3.Headers.REQUESTER_PAYS_HEADER; import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION; -import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_CENTRAL_REGION; -import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING; -import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING_DEFAULT; -import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_KEY; -import static org.apache.hadoop.fs.s3a.S3AUtils.getEncryptionAlgorithm; -import static org.apache.hadoop.fs.s3a.S3AUtils.getS3EncryptionKey; -import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; +import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_DEFAULT_REGION; +import static org.apache.hadoop.fs.s3a.Constants.CENTRAL_ENDPOINT; +import static org.apache.hadoop.fs.s3a.impl.AWSHeaders.REQUESTER_PAYS_HEADER; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_SECURE_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.SECURE_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.AWS_SERVICE_IDENTIFIER_S3; + /** * The default {@link S3ClientFactory} implementation. @@ -71,14 +65,13 @@ */ @InterfaceAudience.Private @InterfaceStability.Unstable -@SuppressWarnings("deprecation") public class DefaultS3ClientFactory extends Configured implements S3ClientFactory { - private static final String S3_SERVICE_NAME = "s3"; - private static final String REQUESTER_PAYS_HEADER_VALUE = "requester"; + private static final String S3_SERVICE_NAME = "s3"; + /** * Subclasses refer to this. */ @@ -98,297 +91,237 @@ public class DefaultS3ClientFactory extends Configured "S3A filesystem client is using" + " the SDK region resolution chain."; + /** Exactly once log to inform about ignoring the AWS-SDK Warnings for CSE. */ private static final LogExactlyOnce IGNORE_CSE_WARN = new LogExactlyOnce(LOG); - /** Bucket name. */ - private String bucket; - - /** - * Create the client by preparing the AwsConf configuration - * and then invoking {@code buildAmazonS3Client()}. - */ @Override - public AmazonS3 createS3Client( + public S3Client createS3Client( final URI uri, final S3ClientCreationParameters parameters) throws IOException { - Configuration conf = getConf(); - bucket = uri.getHost(); - final ClientConfiguration awsConf = S3AUtils - .createAwsConf(conf, - bucket, - Constants.AWS_SERVICE_IDENTIFIER_S3); - // add any headers - parameters.getHeaders().forEach((h, v) -> - awsConf.addHeader(h, v)); - if (parameters.isRequesterPays()) { - // All calls must acknowledge requester will pay via header. - awsConf.addHeader(REQUESTER_PAYS_HEADER, REQUESTER_PAYS_HEADER_VALUE); - } + Configuration conf = getConf(); + String bucket = uri.getHost(); + + ApacheHttpClient.Builder httpClientBuilder = AWSClientConfig + .createHttpClientBuilder(conf) + .proxyConfiguration(AWSClientConfig.createProxyConfiguration(conf, bucket)); + return configureClientBuilder(S3Client.builder(), parameters, conf, bucket) + .httpClientBuilder(httpClientBuilder) + .build(); + } - // When EXPERIMENTAL_AWS_INTERNAL_THROTTLING is false - // throttling is explicitly disabled on the S3 client so that - // all failures are collected in S3A instrumentation, and its - // retry policy is the only one used. - // This may cause problems in copy/rename. - awsConf.setUseThrottleRetries( - conf.getBoolean(EXPERIMENTAL_AWS_INTERNAL_THROTTLING, - EXPERIMENTAL_AWS_INTERNAL_THROTTLING_DEFAULT)); + @Override + public S3AsyncClient createS3AsyncClient( + final URI uri, + final S3ClientCreationParameters parameters) throws IOException { - if (!StringUtils.isEmpty(parameters.getUserAgentSuffix())) { - awsConf.setUserAgentSuffix(parameters.getUserAgentSuffix()); - } + Configuration conf = getConf(); + String bucket = uri.getHost(); + + NettyNioAsyncHttpClient.Builder httpClientBuilder = AWSClientConfig + .createAsyncHttpClientBuilder(conf) + .proxyConfiguration(AWSClientConfig.createAsyncProxyConfiguration(conf, bucket)); + + MultipartConfiguration multipartConfiguration = MultipartConfiguration.builder() + .minimumPartSizeInBytes(parameters.getMinimumPartSize()) + .thresholdInBytes(parameters.getMultiPartThreshold()) + .build(); + + return configureClientBuilder(S3AsyncClient.builder(), parameters, conf, bucket) + .httpClientBuilder(httpClientBuilder) + .multipartConfiguration(multipartConfiguration) + .multipartEnabled(parameters.isMultipartCopy()) + .build(); + } - // Get the encryption method for this bucket. - S3AEncryptionMethods encryptionMethods = - getEncryptionAlgorithm(bucket, conf); - try { - // If CSE is enabled then build a S3EncryptionClient. - if (S3AEncryptionMethods.CSE_KMS.getMethod() - .equals(encryptionMethods.getMethod())) { - return buildAmazonS3EncryptionClient( - awsConf, - parameters); - } else { - return buildAmazonS3Client( - awsConf, - parameters); - } - } catch (SdkClientException e) { - // SDK refused to build. - throw translateException("creating AWS S3 client", uri.toString(), e); - } + @Override + public S3TransferManager createS3TransferManager(final S3AsyncClient s3AsyncClient) { + return S3TransferManager.builder() + .s3Client(s3AsyncClient) + .build(); } /** - * Create an {@link AmazonS3} client of type - * {@link AmazonS3EncryptionV2} if CSE is enabled. - * - * @param awsConf AWS configuration. - * @param parameters parameters. - * - * @return new AmazonS3 client. - * @throws IOException if lookupPassword() has any problem. + * Configure a sync or async S3 client builder. + * This method handles all shared configuration. + * @param builder S3 client builder + * @param parameters parameter object + * @param conf configuration object + * @param bucket bucket name + * @return the builder object + * @param S3 client builder type + * @param S3 client type */ - protected AmazonS3 buildAmazonS3EncryptionClient( - final ClientConfiguration awsConf, - final S3ClientCreationParameters parameters) throws IOException { + private , ClientT> BuilderT configureClientBuilder( + BuilderT builder, S3ClientCreationParameters parameters, Configuration conf, String bucket) + throws IOException { - AmazonS3 client; - AmazonS3EncryptionClientV2Builder builder = - new AmazonS3EncryptionClientV2Builder(); - Configuration conf = getConf(); + configureEndpointAndRegion(builder, parameters, conf); - // CSE-KMS Method - String kmsKeyId = getS3EncryptionKey(bucket, conf, true); - // Check if kmsKeyID is not null - Preconditions.checkArgument(!StringUtils.isBlank(kmsKeyId), "CSE-KMS " - + "method requires KMS key ID. Use " + S3_ENCRYPTION_KEY - + " property to set it. "); - - EncryptionMaterialsProvider materialsProvider = - new KMSEncryptionMaterialsProvider(kmsKeyId); - builder.withEncryptionMaterialsProvider(materialsProvider); - //Configure basic params of a S3 builder. - configureBasicParams(builder, awsConf, parameters); - - // Configuring endpoint. - AmazonS3EncryptionClientV2Builder.EndpointConfiguration epr - = createEndpointConfiguration(parameters.getEndpoint(), - awsConf, getConf().getTrimmed(AWS_REGION)); - configureEndpoint(builder, epr); - - // Create cryptoConfig. - CryptoConfigurationV2 cryptoConfigurationV2 = - new CryptoConfigurationV2(CryptoMode.AuthenticatedEncryption) - .withRangeGetMode(CryptoRangeGetMode.ALL); - if (epr != null) { - cryptoConfigurationV2 - .withAwsKmsRegion(RegionUtils.getRegion(epr.getSigningRegion())); - LOG.debug("KMS region used: {}", cryptoConfigurationV2.getAwsKmsRegion()); - } - builder.withCryptoConfiguration(cryptoConfigurationV2); - client = builder.build(); - IGNORE_CSE_WARN.info("S3 client-side encryption enabled: Ignore S3-CSE " - + "Warnings."); + S3Configuration serviceConfiguration = S3Configuration.builder() + .pathStyleAccessEnabled(parameters.isPathStyleAccess()) + .build(); - return client; + return builder + .overrideConfiguration(createClientOverrideConfiguration(parameters, conf)) + .credentialsProvider(parameters.getCredentialSet()) + .serviceConfiguration(serviceConfiguration); } /** - * Use the Builder API to create an AWS S3 client. - *

- * This has a more complex endpoint configuration mechanism - * which initially caused problems; the - * {@code withForceGlobalBucketAccessEnabled(true)} - * command is critical here. - * @param awsConf AWS configuration - * @param parameters parameters - * @return new AmazonS3 client - * @throws SdkClientException if the configuration is invalid. + * Create an override configuration for an S3 client. + * @param parameters parameter object + * @param conf configuration object + * @throws IOException any IOE raised, or translated exception + * @return the override configuration */ - protected AmazonS3 buildAmazonS3Client( - final ClientConfiguration awsConf, - final S3ClientCreationParameters parameters) { - AmazonS3ClientBuilder b = AmazonS3Client.builder(); - configureBasicParams(b, awsConf, parameters); - - // endpoint set up is a PITA - AwsClientBuilder.EndpointConfiguration epr - = createEndpointConfiguration(parameters.getEndpoint(), - awsConf, getConf().getTrimmed(AWS_REGION)); - configureEndpoint(b, epr); - final AmazonS3 client = b.build(); - return client; - } + protected ClientOverrideConfiguration createClientOverrideConfiguration( + S3ClientCreationParameters parameters, Configuration conf) throws IOException { + final ClientOverrideConfiguration.Builder clientOverrideConfigBuilder = + AWSClientConfig.createClientConfigBuilder(conf, AWS_SERVICE_IDENTIFIER_S3); - /** - * A method to configure basic AmazonS3Builder parameters. - * - * @param builder Instance of AmazonS3Builder used. - * @param awsConf ClientConfiguration used. - * @param parameters Parameters used to set in the builder. - */ - private void configureBasicParams(AmazonS3Builder builder, - ClientConfiguration awsConf, S3ClientCreationParameters parameters) { - builder.withCredentials(parameters.getCredentialSet()); - builder.withClientConfiguration(awsConf); - builder.withPathStyleAccessEnabled(parameters.isPathStyleAccess()); + // add any headers + parameters.getHeaders().forEach((h, v) -> clientOverrideConfigBuilder.putHeader(h, v)); - if (parameters.getMetrics() != null) { - builder.withMetricsCollector( - new AwsStatisticsCollector(parameters.getMetrics())); + if (parameters.isRequesterPays()) { + // All calls must acknowledge requester will pay via header. + clientOverrideConfigBuilder.putHeader(REQUESTER_PAYS_HEADER, REQUESTER_PAYS_HEADER_VALUE); } - if (parameters.getRequestHandlers() != null) { - builder.withRequestHandlers( - parameters.getRequestHandlers().toArray(new RequestHandler2[0])); + + if (!StringUtils.isEmpty(parameters.getUserAgentSuffix())) { + clientOverrideConfigBuilder.putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_SUFFIX, + parameters.getUserAgentSuffix()); } - if (parameters.getMonitoringListener() != null) { - builder.withMonitoringListener(parameters.getMonitoringListener()); + + if (parameters.getExecutionInterceptors() != null) { + for (ExecutionInterceptor interceptor : parameters.getExecutionInterceptors()) { + clientOverrideConfigBuilder.addExecutionInterceptor(interceptor); + } } + if (parameters.getMetrics() != null) { + clientOverrideConfigBuilder.addMetricPublisher( + new AwsStatisticsCollector(parameters.getMetrics())); + } + + final RetryPolicy.Builder retryPolicyBuilder = AWSClientConfig.createRetryPolicyBuilder(conf); + clientOverrideConfigBuilder.retryPolicy(retryPolicyBuilder.build()); + + return clientOverrideConfigBuilder.build(); } /** - * A method to configure endpoint and Region for an AmazonS3Builder. + * This method configures the endpoint and region for a S3 client. + * The order of configuration is: * - * @param builder Instance of AmazonS3Builder used. - * @param epr EndpointConfiguration used to set in builder. + *

    + *
  1. If region is configured via fs.s3a.endpoint.region, use it.
  2. + *
  3. If endpoint is configured via via fs.s3a.endpoint, set it. + * If no region is configured, try to parse region from endpoint.
  4. + *
  5. If no region is configured, and it could not be parsed from the endpoint, + * set the default region as US_EAST_2 and enable cross region access.
  6. + *
  7. If configured region is empty, fallback to SDK resolution chain.
  8. + *
+ * + * @param builder S3 client builder. + * @param parameters parameter object + * @param conf conf configuration object + * @param S3 client builder type + * @param S3 client type */ - private void configureEndpoint( - AmazonS3Builder builder, - AmazonS3Builder.EndpointConfiguration epr) { - if (epr != null) { - // an endpoint binding was constructed: use it. - builder.withEndpointConfiguration(epr); - } else { - // no idea what the endpoint is, so tell the SDK - // to work it out at the cost of an extra HEAD request - builder.withForceGlobalBucketAccessEnabled(true); - // HADOOP-17771 force set the region so the build process doesn't halt. - String region = getConf().getTrimmed(AWS_REGION, AWS_S3_CENTRAL_REGION); - LOG.debug("fs.s3a.endpoint.region=\"{}\"", region); - if (!region.isEmpty()) { - // there's either an explicit region or we have fallen back - // to the central one. - LOG.debug("Using default endpoint; setting region to {}", region); - builder.setRegion(region); - } else { - // no region. - // allow this if people really want it; it is OK to rely on this - // when deployed in EC2. - WARN_OF_DEFAULT_REGION_CHAIN.warn(SDK_REGION_CHAIN_IN_USE); - LOG.debug(SDK_REGION_CHAIN_IN_USE); - } + private , ClientT> void configureEndpointAndRegion( + BuilderT builder, S3ClientCreationParameters parameters, Configuration conf) { + URI endpoint = getS3Endpoint(parameters.getEndpoint(), conf); + + String configuredRegion = parameters.getRegion(); + Region region = null; + String origin = ""; + + // If the region was configured, set it. + if (configuredRegion != null && !configuredRegion.isEmpty()) { + origin = AWS_REGION; + region = Region.of(configuredRegion); } - } - /** - * Configure classic S3 client. - *

- * This includes: endpoint, Path Access and possibly other - * options. - * - * @param s3 S3 Client. - * @param endPoint s3 endpoint, may be empty - * @param pathStyleAccess enable path style access? - * @return S3 client - * @throws IllegalArgumentException if misconfigured - */ - protected static AmazonS3 configureAmazonS3Client(AmazonS3 s3, - final String endPoint, - final boolean pathStyleAccess) - throws IllegalArgumentException { - if (!endPoint.isEmpty()) { - try { - s3.setEndpoint(endPoint); - } catch (IllegalArgumentException e) { - String msg = "Incorrect endpoint: " + e.getMessage(); - LOG.error(msg); - throw new IllegalArgumentException(msg, e); + if (endpoint != null) { + builder.endpointOverride(endpoint); + // No region was configured, try to determine it from the endpoint. + if (region == null) { + region = getS3RegionFromEndpoint(parameters.getEndpoint()); + if (region != null) { + origin = "endpoint"; + } } + LOG.debug("Setting endpoint to {}", endpoint); } - if (pathStyleAccess) { - LOG.debug("Enabling path style access!"); - s3.setS3ClientOptions(S3ClientOptions.builder() - .setPathStyleAccess(true) - .build()); + + if (region != null) { + builder.region(region); + } else if (configuredRegion == null) { + // no region is configured, and none could be determined from the endpoint. + // Use US_EAST_2 as default. + region = Region.of(AWS_S3_DEFAULT_REGION); + builder.crossRegionAccessEnabled(true); + builder.region(region); + origin = "cross region access fallback"; + } else if (configuredRegion.isEmpty()) { + // region configuration was set to empty string. + // allow this if people really want it; it is OK to rely on this + // when deployed in EC2. + WARN_OF_DEFAULT_REGION_CHAIN.warn(SDK_REGION_CHAIN_IN_USE); + LOG.debug(SDK_REGION_CHAIN_IN_USE); + origin = "SDK region chain"; } - return s3; + + LOG.debug("Setting region to {} from {}", region, origin); } /** - * Given an endpoint string, return an endpoint config, or null, if none - * is needed. - *

- * This is a pretty painful piece of code. It is trying to replicate - * what AwsClient.setEndpoint() does, because you can't - * call that setter on an AwsClient constructed via - * the builder, and you can't pass a metrics collector - * down except through the builder. - *

- * Note also that AWS signing is a mystery which nobody fully - * understands, especially given all problems surface in a - * "400 bad request" response, which, like all security systems, - * provides minimal diagnostics out of fear of leaking - * secrets. + * Given a endpoint string, create the endpoint URI. * * @param endpoint possibly null endpoint. - * @param awsConf config to build the URI from. - * @param awsRegion AWS S3 Region if the corresponding config is set. - * @return a configuration for the S3 client builder. + * @param conf config to build the URI from. + * @return an endpoint uri */ - @VisibleForTesting - public static AwsClientBuilder.EndpointConfiguration - createEndpointConfiguration( - final String endpoint, final ClientConfiguration awsConf, - String awsRegion) { - LOG.debug("Creating endpoint configuration for \"{}\"", endpoint); + private static URI getS3Endpoint(String endpoint, final Configuration conf) { + + boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS); + + String protocol = secureConnections ? "https" : "http"; + if (endpoint == null || endpoint.isEmpty()) { - // the default endpoint...we should be using null at this point. - LOG.debug("Using default endpoint -no need to generate a configuration"); + // don't set an endpoint if none is configured, instead let the SDK figure it out. return null; } - final URI epr = RuntimeHttpUtils.toUri(endpoint, awsConf); - LOG.debug("Endpoint URI = {}", epr); - String region = awsRegion; - if (StringUtils.isBlank(region)) { - if (!ServiceUtils.isS3USStandardEndpoint(endpoint)) { - LOG.debug("Endpoint {} is not the default; parsing", epr); - region = AwsHostNameUtils.parseRegion( - epr.getHost(), - S3_SERVICE_NAME); - } else { - // US-east, set region == null. - LOG.debug("Endpoint {} is the standard one; declare region as null", - epr); - region = null; - } + if (!endpoint.contains("://")) { + endpoint = String.format("%s://%s", protocol, endpoint); + } + + try { + return new URI(endpoint); + } catch (URISyntaxException e) { + throw new IllegalArgumentException(e); } - LOG.debug("Region for endpoint {}, URI {} is determined as {}", - endpoint, epr, region); - return new AwsClientBuilder.EndpointConfiguration(endpoint, region); } + + /** + * Parses the endpoint to get the region. + * If endpoint is the central one, use US_EAST_1. + * + * @param endpoint the configure endpoint. + * @return the S3 region, null if unable to resolve from endpoint. + */ + private static Region getS3RegionFromEndpoint(String endpoint) { + + if(!endpoint.endsWith(CENTRAL_ENDPOINT)) { + LOG.debug("Endpoint {} is not the default; parsing", endpoint); + return AwsHostNameUtils.parseSigningRegion(endpoint, S3_SERVICE_NAME).orElse(null); + } + + // endpoint is for US_EAST_1; + return Region.US_EAST_1; + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/FailureInjectionPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/FailureInjectionPolicy.java index f3e00cc61d762..f5c9c43a49cb1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/FailureInjectionPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/FailureInjectionPolicy.java @@ -36,7 +36,7 @@ public class FailureInjectionPolicy { public static final String DEFAULT_DELAY_KEY_SUBSTRING = "DELAY_LISTING_ME"; private static final Logger LOG = - LoggerFactory.getLogger(InconsistentAmazonS3Client.class); + LoggerFactory.getLogger(FailureInjectionPolicy.class); /** * Probability of throttling a request. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java deleted file mode 100644 index 317f2c7857630..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java +++ /dev/null @@ -1,345 +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 java.util.List; -import java.util.concurrent.atomic.AtomicLong; - -import com.amazonaws.AmazonClientException; -import com.amazonaws.AmazonServiceException; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.SdkClientException; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.AmazonS3Client; -import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; -import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; -import com.amazonaws.services.s3.model.DeleteObjectRequest; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.DeleteObjectsResult; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; -import com.amazonaws.services.s3.model.InitiateMultipartUploadResult; -import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsV2Request; -import com.amazonaws.services.s3.model.ListObjectsV2Result; -import com.amazonaws.services.s3.model.MultipartUploadListing; -import com.amazonaws.services.s3.model.ObjectListing; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.PutObjectResult; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.S3ObjectSummary; -import com.amazonaws.services.s3.model.UploadPartRequest; -import com.amazonaws.services.s3.model.UploadPartResult; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; - -/** - * A wrapper around {@link com.amazonaws.services.s3.AmazonS3} that injects - * failures. - * It used to also inject inconsistency, but this was removed with S3Guard; - * what is retained is the ability to throttle AWS operations and for the - * input stream to be inconsistent. - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -public class InconsistentAmazonS3Client extends AmazonS3Client { - - private static final Logger LOG = - LoggerFactory.getLogger(InconsistentAmazonS3Client.class); - - private FailureInjectionPolicy policy; - - /** - * Counter of failures since last reset. - */ - private final AtomicLong failureCounter = new AtomicLong(0); - - - /** - * Instantiate. - * This subclasses a deprecated constructor of the parent - * {@code AmazonS3Client} class; we can't use the builder API because, - * that only creates the consistent client. - * @param credentials credentials to auth. - * @param clientConfiguration connection settings - * @param conf hadoop configuration. - */ - @SuppressWarnings("deprecation") - public InconsistentAmazonS3Client(AWSCredentialsProvider credentials, - ClientConfiguration clientConfiguration, Configuration conf) { - super(credentials, clientConfiguration); - policy = new FailureInjectionPolicy(conf); - } - - /** - * A way for tests to patch in a different fault injection policy at runtime. - * @param fs filesystem under test - * @param policy failure injection settings to set - * @throws Exception on failure - */ - public static void setFailureInjectionPolicy(S3AFileSystem fs, - FailureInjectionPolicy policy) throws Exception { - AmazonS3 s3 = fs.getAmazonS3ClientForTesting("s3guard"); - InconsistentAmazonS3Client ic = InconsistentAmazonS3Client.castFrom(s3); - ic.replacePolicy(policy); - } - - private void replacePolicy(FailureInjectionPolicy pol) { - this.policy = pol; - } - - @Override - public String toString() { - return String.format("Inconsistent S3 Client: %s; failure count %d", - policy, failureCounter.get()); - } - - /** - * Convenience function for test code to cast from supertype. - * @param c supertype to cast from - * @return subtype, not null - * @throws Exception on error - */ - public static InconsistentAmazonS3Client castFrom(AmazonS3 c) throws - Exception { - InconsistentAmazonS3Client ic = null; - if (c instanceof InconsistentAmazonS3Client) { - ic = (InconsistentAmazonS3Client) c; - } - Preconditions.checkNotNull(ic, "Not an instance of " + - "InconsistentAmazonS3Client"); - return ic; - } - - @Override - public DeleteObjectsResult deleteObjects(DeleteObjectsRequest - deleteObjectsRequest) - throws AmazonClientException, AmazonServiceException { - maybeFail(); - return super.deleteObjects(deleteObjectsRequest); - } - - @Override - public void deleteObject(DeleteObjectRequest deleteObjectRequest) - throws AmazonClientException, AmazonServiceException { - String key = deleteObjectRequest.getKey(); - LOG.debug("key {}", key); - maybeFail(); - super.deleteObject(deleteObjectRequest); - } - - /* We should only need to override this version of putObject() */ - @Override - public PutObjectResult putObject(PutObjectRequest putObjectRequest) - throws AmazonClientException, AmazonServiceException { - LOG.debug("key {}", putObjectRequest.getKey()); - maybeFail(); - return super.putObject(putObjectRequest); - } - - /* We should only need to override these versions of listObjects() */ - @Override - public ObjectListing listObjects(ListObjectsRequest listObjectsRequest) - throws AmazonClientException, AmazonServiceException { - maybeFail(); - return super.listObjects(listObjectsRequest); - } - - /* consistent listing with possibility of failing. */ - @Override - public ListObjectsV2Result listObjectsV2(ListObjectsV2Request request) - throws AmazonClientException, AmazonServiceException { - maybeFail(); - return super.listObjectsV2(request); - } - - - @Override - public CompleteMultipartUploadResult completeMultipartUpload( - CompleteMultipartUploadRequest completeMultipartUploadRequest) - throws SdkClientException, AmazonServiceException { - maybeFail(); - return super.completeMultipartUpload(completeMultipartUploadRequest); - } - - @Override - public UploadPartResult uploadPart(UploadPartRequest uploadPartRequest) - throws SdkClientException, AmazonServiceException { - maybeFail(); - return super.uploadPart(uploadPartRequest); - } - - @Override - public InitiateMultipartUploadResult initiateMultipartUpload( - InitiateMultipartUploadRequest initiateMultipartUploadRequest) - throws SdkClientException, AmazonServiceException { - maybeFail(); - return super.initiateMultipartUpload(initiateMultipartUploadRequest); - } - - @Override - public MultipartUploadListing listMultipartUploads( - ListMultipartUploadsRequest listMultipartUploadsRequest) - throws SdkClientException, AmazonServiceException { - maybeFail(); - return super.listMultipartUploads(listMultipartUploadsRequest); - } - - /** - * Set the probability of throttling a request. - * @param throttleProbability the probability of a request being throttled. - */ - public void setThrottleProbability(float throttleProbability) { - policy.setThrottleProbability(throttleProbability); - } - - /** - * Conditionally fail the operation. - * @param errorMsg description of failure - * @param statusCode http status code for error - * @throws AmazonClientException if the client chooses to fail - * the request. - */ - private void maybeFail(String errorMsg, int statusCode) - throws AmazonClientException { - // code structure here is to line up for more failures later - AmazonServiceException ex = null; - if (FailureInjectionPolicy.trueWithProbability(policy.getThrottleProbability())) { - // throttle the request - ex = new AmazonServiceException(errorMsg - + " count = " + (failureCounter.get() + 1), null); - ex.setStatusCode(statusCode); - } - - int failureLimit = policy.getFailureLimit(); - if (ex != null) { - long count = failureCounter.incrementAndGet(); - if (failureLimit == 0 - || (failureLimit > 0 && count < failureLimit)) { - throw ex; - } - } - } - - private void maybeFail() { - maybeFail("throttled", 503); - } - - /** - * Set the limit on failures before all operations pass through. - * This resets the failure count. - * @param limit limit; "0" means "no limit" - */ - public void setFailureLimit(int limit) { - policy.setFailureLimit(limit); - failureCounter.set(0); - } - - @Override - public S3Object getObject(GetObjectRequest var1) throws SdkClientException, - AmazonServiceException { - maybeFail(); - return super.getObject(var1); - } - - @Override - public S3Object getObject(String bucketName, String key) - throws SdkClientException, AmazonServiceException { - maybeFail(); - return super.getObject(bucketName, key); - - } - - /** Since ObjectListing is immutable, we just override it with wrapper. */ - @SuppressWarnings("serial") - private static class CustomObjectListing extends ObjectListing { - - private final List customListing; - private final List customPrefixes; - - CustomObjectListing(ObjectListing rawListing, - List customListing, - List customPrefixes) { - super(); - this.customListing = customListing; - this.customPrefixes = customPrefixes; - - this.setBucketName(rawListing.getBucketName()); - this.setCommonPrefixes(rawListing.getCommonPrefixes()); - this.setDelimiter(rawListing.getDelimiter()); - this.setEncodingType(rawListing.getEncodingType()); - this.setMarker(rawListing.getMarker()); - this.setMaxKeys(rawListing.getMaxKeys()); - this.setNextMarker(rawListing.getNextMarker()); - this.setPrefix(rawListing.getPrefix()); - this.setTruncated(rawListing.isTruncated()); - } - - @Override - public List getObjectSummaries() { - return customListing; - } - - @Override - public List getCommonPrefixes() { - return customPrefixes; - } - } - - @SuppressWarnings("serial") - private static class CustomListObjectsV2Result extends ListObjectsV2Result { - - private final List customListing; - private final List customPrefixes; - - CustomListObjectsV2Result(ListObjectsV2Result raw, - List customListing, List customPrefixes) { - super(); - this.customListing = customListing; - this.customPrefixes = customPrefixes; - - this.setBucketName(raw.getBucketName()); - this.setCommonPrefixes(raw.getCommonPrefixes()); - this.setDelimiter(raw.getDelimiter()); - this.setEncodingType(raw.getEncodingType()); - this.setStartAfter(raw.getStartAfter()); - this.setMaxKeys(raw.getMaxKeys()); - this.setContinuationToken(raw.getContinuationToken()); - this.setPrefix(raw.getPrefix()); - this.setTruncated(raw.isTruncated()); - } - - @Override - public List getObjectSummaries() { - return customListing; - } - - @Override - public List getCommonPrefixes() { - return customPrefixes; - } - } -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java deleted file mode 100644 index 4bfcc8aba3af3..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java +++ /dev/null @@ -1,54 +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 com.amazonaws.ClientConfiguration; -import com.amazonaws.services.s3.AmazonS3; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - -/** - * S3 Client factory used for testing with eventual consistency fault injection. - * This client is for testing only; it is in the production - * {@code hadoop-aws} module to enable integration tests to use this - * just by editing the Hadoop configuration used to bring up the client. - * - * The factory uses the older constructor-based instantiation/configuration - * of the client, so does not wire up metrics, handlers etc. - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -public class InconsistentS3ClientFactory extends DefaultS3ClientFactory { - - @Override - protected AmazonS3 buildAmazonS3Client( - final ClientConfiguration awsConf, - final S3ClientCreationParameters parameters) { - LOG.warn("** FAILURE INJECTION ENABLED. Do not run in production! **"); - LOG.warn("List inconsistency is no longer emulated; only throttling and read errors"); - InconsistentAmazonS3Client s3 - = new InconsistentAmazonS3Client( - parameters.getCredentialSet(), awsConf, getConf()); - configureAmazonS3Client(s3, - parameters.getEndpoint(), - parameters.isPathStyleAccess()); - return s3; - } -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java index 67c8e7d809cc8..9b2c95a90c76f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java @@ -24,8 +24,7 @@ import java.util.concurrent.Future; import javax.annotation.Nullable; -import com.amazonaws.AmazonClientException; -import com.amazonaws.SdkBaseException; +import software.amazon.awssdk.core.exception.SdkException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +38,7 @@ import org.apache.hadoop.util.functional.InvocationRaisingIOE; import org.apache.hadoop.util.Preconditions; + import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration; /** @@ -120,7 +120,7 @@ public static T once(String action, String path, throws IOException { try (DurationInfo ignored = new DurationInfo(LOG, false, "%s", action)) { return operation.apply(); - } catch (AmazonClientException e) { + } catch (SdkException e) { throw S3AUtils.translateException(action, path, e); } } @@ -145,7 +145,7 @@ public static T onceTrackingDuration( throws IOException { try { return invokeTrackingDuration(tracker, operation); - } catch (AmazonClientException e) { + } catch (SdkException e) { throw S3AUtils.translateException(action, path, e); } } @@ -170,7 +170,7 @@ public static void once(String action, String path, /** * - * Wait for a future, translating AmazonClientException into an IOException. + * Wait for a future, translating SdkException into an IOException. * @param action action to execute (used in error messages) * @param path path of work (used in error messages) * @param future future to await for @@ -186,7 +186,7 @@ public static T onceInTheFuture(String action, throws IOException { try (DurationInfo ignored = new DurationInfo(LOG, false, "%s", action)) { return FutureIO.awaitFuture(future); - } catch (AmazonClientException e) { + } catch (SdkException e) { throw S3AUtils.translateException(action, path, e); } } @@ -444,7 +444,7 @@ public T retryUntranslated( * @param operation operation to execute * @return the result of the call * @throws IOException any IOE raised - * @throws SdkBaseException any AWS exception raised + * @throws SdkException any AWS exception raised * @throws RuntimeException : these are never caught and retries. */ @Retries.RetryRaw @@ -466,7 +466,7 @@ public T retryUntranslated( } // execute the operation, returning if successful return operation.apply(); - } catch (IOException | SdkBaseException e) { + } catch (IOException | SdkException e) { caught = e; } // you only get here if the operation didn't complete @@ -479,7 +479,7 @@ public T retryUntranslated( translated = (IOException) caught; } else { translated = S3AUtils.translateException(text, "", - (SdkBaseException)caught); + (SdkException) caught); } try { @@ -518,11 +518,10 @@ public T retryUntranslated( if (caught instanceof IOException) { throw (IOException) caught; } else { - throw (SdkBaseException) caught; + throw (SdkException) caught; } } - /** * Execute an operation; any exception raised is simply caught and * logged at debug. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java index 6c39cc4b64240..490deaaab04d9 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java @@ -18,7 +18,8 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.services.s3.model.S3ObjectSummary; +import software.amazon.awssdk.services.s3.model.CommonPrefix; +import software.amazon.awssdk.services.s3.model.S3Object; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.VisibleForTesting; @@ -277,19 +278,19 @@ public S3ListRequest createListObjectsRequest(String key, } /** - * Interface to implement by the logic deciding whether to accept a summary + * Interface to implement the logic deciding whether to accept a s3Object * entry or path as a valid file or directory. */ interface FileStatusAcceptor { /** - * Predicate to decide whether or not to accept a summary entry. + * Predicate to decide whether or not to accept a s3Object entry. * @param keyPath qualified path to the entry - * @param summary summary entry + * @param s3Object s3Object entry * @return true if the entry is accepted (i.e. that a status entry * should be generated. */ - boolean accept(Path keyPath, S3ObjectSummary summary); + boolean accept(Path keyPath, S3Object s3Object); /** * Predicate to decide whether or not to accept a prefix. @@ -451,21 +452,21 @@ private boolean buildNextStatusBatch(S3ListResult objects) { int added = 0, ignored = 0; // list to fill in with results. Initial size will be list maximum. List stats = new ArrayList<>( - objects.getObjectSummaries().size() + + objects.getS3Objects().size() + objects.getCommonPrefixes().size()); // objects - for (S3ObjectSummary summary : objects.getObjectSummaries()) { - String key = summary.getKey(); + for (S3Object s3Object : objects.getS3Objects()) { + String key = s3Object.key(); Path keyPath = getStoreContext().getContextAccessors().keyToPath(key); if (LOG.isDebugEnabled()) { - LOG.debug("{}: {}", keyPath, stringify(summary)); + LOG.debug("{}: {}", keyPath, stringify(s3Object)); } // Skip over keys that are ourselves and old S3N _$folder$ files - if (acceptor.accept(keyPath, summary) && filter.accept(keyPath)) { - S3AFileStatus status = createFileStatus(keyPath, summary, + if (acceptor.accept(keyPath, s3Object) && filter.accept(keyPath)) { + S3AFileStatus status = createFileStatus(keyPath, s3Object, listingOperationCallbacks.getDefaultBlockSize(keyPath), getStoreContext().getUsername(), - summary.getETag(), null, isCSEEnabled); + s3Object.eTag(), null, isCSEEnabled); LOG.debug("Adding: {}", status); stats.add(status); added++; @@ -476,11 +477,11 @@ private boolean buildNextStatusBatch(S3ListResult objects) { } // prefixes: always directories - for (String prefix : objects.getCommonPrefixes()) { + for (CommonPrefix prefix : objects.getCommonPrefixes()) { Path keyPath = getStoreContext() .getContextAccessors() - .keyToPath(prefix); - if (acceptor.accept(keyPath, prefix) && filter.accept(keyPath)) { + .keyToPath(prefix.prefix()); + if (acceptor.accept(keyPath, prefix.prefix()) && filter.accept(keyPath)) { S3AFileStatus status = new S3AFileStatus(Tristate.FALSE, keyPath, getStoreContext().getUsername()); LOG.debug("Adding directory: {}", status); @@ -731,18 +732,18 @@ public AcceptFilesOnly(Path qualifiedPath) { } /** - * Reject a summary entry if the key path is the qualified Path, or + * Reject a s3Object entry if the key path is the qualified Path, or * it ends with {@code "_$folder$"}. * @param keyPath key path of the entry - * @param summary summary entry + * @param s3Object s3Object entry * @return true if the entry is accepted (i.e. that a status entry * should be generated. */ @Override - public boolean accept(Path keyPath, S3ObjectSummary summary) { + public boolean accept(Path keyPath, S3Object s3Object) { return !keyPath.equals(qualifiedPath) - && !summary.getKey().endsWith(S3N_FOLDER_SUFFIX) - && !objectRepresentsDirectory(summary.getKey()); + && !s3Object.key().endsWith(S3N_FOLDER_SUFFIX) + && !objectRepresentsDirectory(s3Object.key()); } /** @@ -767,8 +768,8 @@ public boolean accept(FileStatus status) { */ static class AcceptAllButS3nDirs implements FileStatusAcceptor { - public boolean accept(Path keyPath, S3ObjectSummary summary) { - return !summary.getKey().endsWith(S3N_FOLDER_SUFFIX); + public boolean accept(Path keyPath, S3Object s3Object) { + return !s3Object.key().endsWith(S3N_FOLDER_SUFFIX); } public boolean accept(Path keyPath, String prefix) { @@ -799,17 +800,17 @@ public AcceptAllButSelfAndS3nDirs(Path qualifiedPath) { } /** - * Reject a summary entry if the key path is the qualified Path, or + * Reject a s3Object entry if the key path is the qualified Path, or * it ends with {@code "_$folder$"}. * @param keyPath key path of the entry - * @param summary summary entry + * @param s3Object s3Object entry * @return true if the entry is accepted (i.e. that a status entry * should be generated.) */ @Override - public boolean accept(Path keyPath, S3ObjectSummary summary) { + public boolean accept(Path keyPath, S3Object s3Object) { return !keyPath.equals(qualifiedPath) && - !summary.getKey().endsWith(S3N_FOLDER_SUFFIX); + !s3Object.key().endsWith(S3N_FOLDER_SUFFIX); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java index d8c820cd8a121..b2057c211da7b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java @@ -23,19 +23,20 @@ import java.util.NoSuchElementException; import javax.annotation.Nullable; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; -import com.amazonaws.services.s3.model.MultipartUpload; -import com.amazonaws.services.s3.model.MultipartUploadListing; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest; +import software.amazon.awssdk.services.s3.model.ListMultipartUploadsResponse; +import software.amazon.awssdk.services.s3.model.MultipartUpload; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.store.audit.AuditSpan; -import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_LIST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_MULTIPART_UPLOAD_LIST; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation; @@ -43,7 +44,7 @@ * MultipartUtils upload-specific functions for use by S3AFileSystem and Hadoop * CLI. * The Audit span active when - * {@link #listMultipartUploads(StoreContext, AmazonS3, String, int)} + * {@link #listMultipartUploads(StoreContext, S3Client, String, int)} * was invoked is retained for all subsequent operations. */ public final class MultipartUtils { @@ -65,9 +66,9 @@ private MultipartUtils() { } * @param maxKeys maximum batch size to request at a time from S3. * @return an iterator of matching uploads */ - static MultipartUtils.UploadIterator listMultipartUploads( + static RemoteIterator listMultipartUploads( final StoreContext storeContext, - AmazonS3 s3, + S3Client s3, @Nullable String prefix, int maxKeys) throws IOException { @@ -84,14 +85,14 @@ static MultipartUtils.UploadIterator listMultipartUploads( * at the time the iterator was constructed. */ static class ListingIterator implements - RemoteIterator { + RemoteIterator { private final String prefix; private final RequestFactory requestFactory; private final int maxKeys; - private final AmazonS3 s3; + private final S3Client s3; private final Invoker invoker; private final AuditSpan auditSpan; @@ -101,7 +102,7 @@ static class ListingIterator implements /** * Most recent listing results. */ - private MultipartUploadListing listing; + private ListMultipartUploadsResponse listing; /** * Indicator that this is the first listing. @@ -114,7 +115,7 @@ static class ListingIterator implements private int listCount = 0; ListingIterator(final StoreContext storeContext, - AmazonS3 s3, + S3Client s3, @Nullable String prefix, int maxKeys) throws IOException { this.storeContext = storeContext; @@ -153,7 +154,7 @@ public boolean hasNext() throws IOException { */ @Override @Retries.RetryTranslated - public MultipartUploadListing next() throws IOException { + public ListMultipartUploadsResponse next() throws IOException { if (firstListing) { firstListing = false; } else { @@ -171,32 +172,34 @@ public MultipartUploadListing next() throws IOException { public String toString() { return "Upload iterator: prefix " + prefix + "; list count " + listCount - + "; upload count " + listing.getMultipartUploads().size() + + "; upload count " + listing.uploads().size() + "; isTruncated=" + listing.isTruncated(); } @Retries.RetryTranslated private void requestNextBatch() throws IOException { try (AuditSpan span = auditSpan.activate()) { - ListMultipartUploadsRequest req = requestFactory - .newListMultipartUploadsRequest(prefix); + ListMultipartUploadsRequest.Builder requestBuilder = requestFactory + .newListMultipartUploadsRequestBuilder(prefix); if (!firstListing) { - req.setKeyMarker(listing.getNextKeyMarker()); - req.setUploadIdMarker(listing.getNextUploadIdMarker()); + requestBuilder.keyMarker(listing.nextKeyMarker()); + requestBuilder.uploadIdMarker(listing.nextUploadIdMarker()); } - req.setMaxUploads(maxKeys); + requestBuilder.maxUploads(maxKeys); + + ListMultipartUploadsRequest request = requestBuilder.build(); LOG.debug("[{}], Requesting next {} uploads prefix {}, " + "next key {}, next upload id {}", listCount, maxKeys, prefix, - req.getKeyMarker(), req.getUploadIdMarker()); + request.keyMarker(), request.uploadIdMarker()); listCount++; listing = invoker.retry("listMultipartUploads", prefix, true, trackDurationOfOperation(storeContext.getInstrumentation(), - MULTIPART_UPLOAD_LIST.getSymbol(), - () -> s3.listMultipartUploads(req))); + OBJECT_MULTIPART_UPLOAD_LIST.getSymbol(), + () -> s3.listMultipartUploads(requestBuilder.build()))); LOG.debug("Listing found {} upload(s)", - listing.getMultipartUploads().size()); + listing.uploads().size()); LOG.debug("New listing state: {}", this); } } @@ -216,14 +219,22 @@ public static class UploadIterator */ private ListingIterator lister; /** Current listing: the last upload listing we fetched. */ - private MultipartUploadListing listing; + private ListMultipartUploadsResponse listing; /** Iterator over the current listing. */ private ListIterator batchIterator; + /** + * Construct an iterator to list uploads under a path. + * @param storeContext store context + * @param s3 s3 client + * @param maxKeys max # of keys to list per batch + * @param prefix prefix + * @throws IOException listing failure. + */ @Retries.RetryTranslated public UploadIterator( final StoreContext storeContext, - AmazonS3 s3, + S3Client s3, int maxKeys, @Nullable String prefix) throws IOException { @@ -249,7 +260,7 @@ public MultipartUpload next() throws IOException { private boolean requestNextBatch() throws IOException { if (lister.hasNext()) { listing = lister.next(); - batchIterator = listing.getMultipartUploads().listIterator(); + batchIterator = listing.uploads().listIterator(); return batchIterator.hasNext(); } return false; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java index 0ce022aa88588..7ee6c55c191b7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java @@ -18,59 +18,55 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.event.ProgressEvent; -import com.amazonaws.event.ProgressEventType; -import com.amazonaws.event.ProgressListener; -import com.amazonaws.services.s3.transfer.Upload; +import software.amazon.awssdk.transfer.s3.model.ObjectTransfer; +import software.amazon.awssdk.transfer.s3.progress.TransferListener; import org.apache.hadoop.util.Progressable; import org.slf4j.Logger; -import static com.amazonaws.event.ProgressEventType.TRANSFER_COMPLETED_EVENT; -import static com.amazonaws.event.ProgressEventType.TRANSFER_PART_STARTED_EVENT; /** * Listener to progress from AWS regarding transfers. */ -public class ProgressableProgressListener implements ProgressListener { +public class ProgressableProgressListener implements TransferListener { private static final Logger LOG = S3AFileSystem.LOG; private final S3AFileSystem fs; private final String key; private final Progressable progress; private long lastBytesTransferred; - private final Upload upload; /** * Instantiate. * @param fs filesystem: will be invoked with statistics updates * @param key key for the upload - * @param upload source of events * @param progress optional callback for progress. */ public ProgressableProgressListener(S3AFileSystem fs, String key, - Upload upload, Progressable progress) { this.fs = fs; this.key = key; - this.upload = upload; this.progress = progress; this.lastBytesTransferred = 0; } @Override - public void progressChanged(ProgressEvent progressEvent) { - if (progress != null) { - progress.progress(); - } + public void transferInitiated(TransferListener.Context.TransferInitiated context) { + fs.incrementWriteOperations(); + } + + @Override + public void transferComplete(TransferListener.Context.TransferComplete context) { + fs.incrementWriteOperations(); + } + + @Override + public void bytesTransferred(TransferListener.Context.BytesTransferred context) { - // There are 3 http ops here, but this should be close enough for now - ProgressEventType pet = progressEvent.getEventType(); - if (pet == TRANSFER_PART_STARTED_EVENT || - pet == TRANSFER_COMPLETED_EVENT) { - fs.incrementWriteOperations(); + if(progress != null) { + progress.progress(); } - long transferred = upload.getProgress().getBytesTransferred(); + long transferred = context.progressSnapshot().transferredBytes(); long delta = transferred - lastBytesTransferred; fs.incrementPutProgressStatistics(key, delta); lastBytesTransferred = transferred; @@ -79,11 +75,13 @@ public void progressChanged(ProgressEvent progressEvent) { /** * Method to invoke after upload has completed. * This can handle race conditions in setup/teardown. + * @param upload upload which has just completed. * @return the number of bytes which were transferred after the notification */ - public long uploadCompleted() { - long delta = upload.getProgress().getBytesTransferred() - - lastBytesTransferred; + public long uploadCompleted(ObjectTransfer upload) { + + long delta = + upload.progress().snapshot().transferredBytes() - lastBytesTransferred; if (delta > 0) { LOG.debug("S3A write delta changed after finished: {} bytes", delta); fs.incrementPutProgressStatistics(key, delta); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index 43a2b7e0dbd5b..de0f59154e995 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -31,15 +31,16 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import com.amazonaws.SdkBaseException; -import com.amazonaws.event.ProgressEvent; -import com.amazonaws.event.ProgressEventType; -import com.amazonaws.event.ProgressListener; -import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.PutObjectResult; -import com.amazonaws.services.s3.model.UploadPartRequest; - +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.services.s3.model.CompletedPart; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; +import software.amazon.awssdk.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.services.s3.model.UploadPartResponse; + +import org.apache.hadoop.fs.s3a.impl.ProgressListener; +import org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent; import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.statistics.IOStatisticsAggregator; import org.apache.hadoop.util.Preconditions; @@ -69,6 +70,7 @@ import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; +import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.*; import static org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext.EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation; @@ -192,7 +194,7 @@ class S3ABlockOutputStream extends OutputStream implements this.executorService = MoreExecutors.listeningDecorator( builder.executorService); this.multiPartUpload = null; - final Progressable progress = builder.progress; + Progressable progress = builder.progress; this.progressListener = (progress instanceof ProgressListener) ? (ProgressListener) progress : new ProgressableListener(progress); @@ -439,7 +441,7 @@ public void close() throws IOException { uploadCurrentBlock(true); } // wait for the partial uploads to finish - final List partETags = + final List partETags = multiPartUpload.waitForAllPartUploads(); bytes = bytesSubmitted; @@ -597,27 +599,28 @@ private long putObject() throws IOException { final PutObjectRequest putObjectRequest = uploadData.hasFile() ? writeOperationHelper.createPutObjectRequest( key, - uploadData.getFile(), - builder.putOptions) + uploadData.getFile().length(), + builder.putOptions, + true) : writeOperationHelper.createPutObjectRequest( key, - uploadData.getUploadStream(), size, - builder.putOptions); - BlockUploadProgress callback = - new BlockUploadProgress( - block, progressListener, now()); - putObjectRequest.setGeneralProgressListener(callback); + builder.putOptions, + false); + + BlockUploadProgress progressCallback = + new BlockUploadProgress(block, progressListener, now()); statistics.blockUploadQueued(size); - ListenableFuture putObjectResult = + ListenableFuture putObjectResult = executorService.submit(() -> { try { // the putObject call automatically closes the input // stream afterwards. - return writeOperationHelper.putObject( - putObjectRequest, - builder.putOptions, - statistics); + PutObjectResponse response = + writeOperationHelper.putObject(putObjectRequest, builder.putOptions, uploadData, + uploadData.hasFile(), statistics); + progressCallback.progressChanged(REQUEST_BYTE_TRANSFER_EVENT); + return response; } finally { cleanupWithLogger(LOG, uploadData, block); } @@ -761,7 +764,7 @@ protected IOStatisticsAggregator getThreadIOStatistics() { */ private class MultiPartUpload { private final String uploadId; - private final List> partETagsFutures; + private final List> partETagsFutures; private int partsSubmitted; private int partsUploaded; private long bytesSubmitted; @@ -866,18 +869,19 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block, final int currentPartNumber = partETagsFutures.size() + 1; final UploadPartRequest request; final S3ADataBlocks.BlockUploadData uploadData; + final RequestBody requestBody; try { uploadData = block.startUpload(); - request = writeOperationHelper.newUploadPartRequest( + requestBody = uploadData.hasFile() + ? RequestBody.fromFile(uploadData.getFile()) + : RequestBody.fromInputStream(uploadData.getUploadStream(), size); + + request = writeOperationHelper.newUploadPartRequestBuilder( key, uploadId, currentPartNumber, - size, - uploadData.getUploadStream(), - uploadData.getFile(), - 0L); - request.setLastPart(isLast); - } catch (SdkBaseException aws) { + size).build(); + } catch (SdkException aws) { // catch and translate IOException e = translateException("upload", key, aws); // failure to start the upload. @@ -888,28 +892,38 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block, noteUploadFailure(e); throw e; } - BlockUploadProgress callback = - new BlockUploadProgress( - block, progressListener, now()); - request.setGeneralProgressListener(callback); + + BlockUploadProgress progressCallback = + new BlockUploadProgress(block, progressListener, now()); + statistics.blockUploadQueued(block.dataSize()); - ListenableFuture partETagFuture = + ListenableFuture partETagFuture = executorService.submit(() -> { // this is the queued upload operation // do the upload try { LOG.debug("Uploading part {} for id '{}'", currentPartNumber, uploadId); - PartETag partETag = writeOperationHelper.uploadPart(request, statistics) - .getPartETag(); + + progressCallback.progressChanged(TRANSFER_PART_STARTED_EVENT); + + UploadPartResponse response = writeOperationHelper + .uploadPart(request, requestBody, statistics); LOG.debug("Completed upload of {} to part {}", - block, partETag.getETag()); + block, response.eTag()); LOG.debug("Stream statistics of {}", statistics); partsUploaded++; - return partETag; + + progressCallback.progressChanged(TRANSFER_PART_COMPLETED_EVENT); + + return CompletedPart.builder() + .eTag(response.eTag()) + .partNumber(currentPartNumber) + .build(); } catch (IOException e) { // save immediately. noteUploadFailure(e); + progressCallback.progressChanged(TRANSFER_PART_FAILED_EVENT); throw e; } finally { // close the stream and block @@ -924,7 +938,7 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block, * @return list of results * @throws IOException IO Problems */ - private List waitForAllPartUploads() throws IOException { + private List waitForAllPartUploads() throws IOException { LOG.debug("Waiting for {} uploads to complete", partETagsFutures.size()); try { return Futures.allAsList(partETagsFutures).get(); @@ -948,7 +962,7 @@ private List waitForAllPartUploads() throws IOException { */ private void cancelAllActiveFutures() { LOG.debug("Cancelling futures"); - for (ListenableFuture future : partETagsFutures) { + for (ListenableFuture future : partETagsFutures) { future.cancel(true); } } @@ -960,7 +974,7 @@ private void cancelAllActiveFutures() { * @param partETags list of partial uploads * @throws IOException on any problem */ - private void complete(List partETags) + private void complete(List partETags) throws IOException { maybeRethrowUploadFailure(); AtomicInteger errorCount = new AtomicInteger(0); @@ -1005,22 +1019,24 @@ private IOException abort() { } } + /** * The upload progress listener registered for events returned * during the upload of a single block. * It updates statistics and handles the end of the upload. * Transfer failures are logged at WARN. */ - private final class BlockUploadProgress implements ProgressListener { + private final class BlockUploadProgress { + private final S3ADataBlocks.DataBlock block; private final ProgressListener nextListener; private final Instant transferQueueTime; private Instant transferStartTime; + private long size; /** * Track the progress of a single block upload. * @param block block to monitor - * @param nextListener optional next progress listener * @param transferQueueTime time the block was transferred * into the queue */ @@ -1029,20 +1045,17 @@ private BlockUploadProgress(S3ADataBlocks.DataBlock block, Instant transferQueueTime) { this.block = block; this.transferQueueTime = transferQueueTime; + this.size = block.dataSize(); this.nextListener = nextListener; } - @Override - public void progressChanged(ProgressEvent progressEvent) { - ProgressEventType eventType = progressEvent.getEventType(); - long bytesTransferred = progressEvent.getBytesTransferred(); + public void progressChanged(ProgressListenerEvent eventType) { - long size = block.dataSize(); switch (eventType) { case REQUEST_BYTE_TRANSFER_EVENT: // bytes uploaded - statistics.bytesTransferred(bytesTransferred); + statistics.bytesTransferred(size); break; case TRANSFER_PART_STARTED_EVENT: @@ -1057,6 +1070,7 @@ public void progressChanged(ProgressEvent progressEvent) { statistics.blockUploadCompleted( Duration.between(transferStartTime, now()), size); + statistics.bytesTransferred(size); break; case TRANSFER_PART_FAILED_EVENT: @@ -1071,13 +1085,13 @@ public void progressChanged(ProgressEvent progressEvent) { } if (nextListener != null) { - nextListener.progressChanged(progressEvent); + nextListener.progressChanged(eventType, size); } } } /** - * Bridge from AWS {@code ProgressListener} to Hadoop {@link Progressable}. + * Bridge from {@link ProgressListener} to Hadoop {@link Progressable}. */ private static class ProgressableListener implements ProgressListener { private final Progressable progress; @@ -1086,7 +1100,7 @@ private static class ProgressableListener implements ProgressListener { this.progress = progress; } - public void progressChanged(ProgressEvent progressEvent) { + public void progressChanged(ProgressListenerEvent eventType, int bytesTransferred) { if (progress != null) { progress.progress(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java index ddaf059b055de..5aa43245c2f01 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java @@ -47,7 +47,7 @@ * Set of classes to support output streaming into blocks which are then * uploaded as to S3 as a single PUT, or as part of a multipart request. */ -final class S3ADataBlocks { +public final class S3ADataBlocks { private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class); @@ -101,7 +101,7 @@ static BlockFactory createFactory(S3AFileSystem owner, * It can be one of a file or an input stream. * When closed, any stream is closed. Any source file is untouched. */ - static final class BlockUploadData implements Closeable { + public static final class BlockUploadData implements Closeable { private final File file; private final InputStream uploadStream; @@ -109,7 +109,7 @@ static final class BlockUploadData implements Closeable { * File constructor; input stream will be null. * @param file file to upload */ - BlockUploadData(File file) { + public BlockUploadData(File file) { Preconditions.checkArgument(file.exists(), "No file: " + file); this.file = file; this.uploadStream = null; @@ -119,7 +119,7 @@ static final class BlockUploadData implements Closeable { * Stream constructor, file field will be null. * @param uploadStream stream to upload */ - BlockUploadData(InputStream uploadStream) { + public BlockUploadData(InputStream uploadStream) { Preconditions.checkNotNull(uploadStream, "rawUploadStream"); this.uploadStream = uploadStream; this.file = null; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AEncryptionMethods.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AEncryptionMethods.java index b599790a1cf76..6cacdff6c42d8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AEncryptionMethods.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AEncryptionMethods.java @@ -33,13 +33,14 @@ public enum S3AEncryptionMethods { SSE_KMS("SSE-KMS", true, false), SSE_C("SSE-C", true, true), CSE_KMS("CSE-KMS", false, true), - CSE_CUSTOM("CSE-CUSTOM", false, true); + CSE_CUSTOM("CSE-CUSTOM", false, true), + DSSE_KMS("DSSE-KMS", true, false); /** * Error string when {@link #getMethod(String)} fails. * Used in tests. */ - static final String UNKNOWN_ALGORITHM + public static final String UNKNOWN_ALGORITHM = "Unknown encryption algorithm "; /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 6d9fd76d1346e..db2cedd37fc90 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -21,12 +21,14 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; +import java.io.InputStream; import java.io.InterruptedIOException; import java.io.UncheckedIOException; import java.net.URI; import java.nio.file.AccessDeniedException; import java.text.DateFormat; import java.text.SimpleDateFormat; +import java.time.Instant; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -41,6 +43,7 @@ import java.util.Objects; import java.util.TreeSet; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; @@ -48,42 +51,49 @@ import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.Nullable; -import com.amazonaws.AmazonClientException; -import com.amazonaws.AmazonServiceException; -import com.amazonaws.SdkBaseException; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.Headers; -import com.amazonaws.services.s3.model.CannedAccessControlList; -import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; -import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; -import com.amazonaws.services.s3.model.CopyObjectRequest; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.DeleteObjectsResult; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; -import com.amazonaws.services.s3.model.InitiateMultipartUploadResult; -import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsV2Request; -import com.amazonaws.services.s3.model.MultiObjectDeleteException; -import com.amazonaws.services.s3.model.MultipartUpload; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.PutObjectResult; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.SelectObjectContentRequest; -import com.amazonaws.services.s3.model.SelectObjectContentResult; -import com.amazonaws.services.s3.model.StorageClass; -import com.amazonaws.services.s3.model.UploadPartRequest; -import com.amazonaws.services.s3.model.UploadPartResult; -import com.amazonaws.services.s3.transfer.Copy; -import com.amazonaws.services.s3.transfer.TransferManager; -import com.amazonaws.services.s3.transfer.TransferManagerConfiguration; -import com.amazonaws.services.s3.transfer.Upload; -import com.amazonaws.services.s3.transfer.model.CopyResult; -import com.amazonaws.services.s3.transfer.model.UploadResult; -import com.amazonaws.event.ProgressListener; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse; +import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; +import software.amazon.awssdk.services.s3.model.HeadBucketRequest; +import software.amazon.awssdk.services.s3.model.HeadBucketResponse; +import software.amazon.awssdk.services.s3.model.MultipartUpload; +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadResponse; +import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.CopyObjectResponse; +import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; +import software.amazon.awssdk.services.s3.model.NoSuchBucketException; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; +import software.amazon.awssdk.services.s3.model.S3Error; +import software.amazon.awssdk.services.s3.model.S3Object; +import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; +import software.amazon.awssdk.services.s3.model.SelectObjectContentResponseHandler; +import software.amazon.awssdk.services.s3.model.StorageClass; +import software.amazon.awssdk.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.services.s3.model.UploadPartResponse; +import software.amazon.awssdk.transfer.s3.model.CompletedCopy; +import software.amazon.awssdk.transfer.s3.model.CompletedFileUpload; +import software.amazon.awssdk.transfer.s3.model.Copy; +import software.amazon.awssdk.transfer.s3.S3TransferManager; +import software.amazon.awssdk.transfer.s3.model.CopyRequest; +import software.amazon.awssdk.transfer.s3.model.FileUpload; +import software.amazon.awssdk.transfer.s3.model.UploadFileRequest; import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool; import org.slf4j.Logger; @@ -108,11 +118,12 @@ import org.apache.hadoop.fs.s3a.auth.SignerManager; import org.apache.hadoop.fs.s3a.auth.delegation.DelegationOperations; import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider; +import org.apache.hadoop.fs.s3a.impl.AWSCannedACL; +import org.apache.hadoop.fs.s3a.impl.AWSHeaders; import org.apache.hadoop.fs.s3a.impl.BulkDeleteRetryHandler; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ContextAccessors; import org.apache.hadoop.fs.s3a.impl.CopyFromLocalOperation; -import org.apache.hadoop.fs.s3a.impl.CopyOutcome; import org.apache.hadoop.fs.s3a.impl.CreateFileBuilder; import org.apache.hadoop.fs.s3a.impl.DeleteOperation; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; @@ -122,6 +133,7 @@ import org.apache.hadoop.fs.s3a.impl.InternalConstants; import org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks; import org.apache.hadoop.fs.s3a.impl.MkdirOperation; +import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteException; import org.apache.hadoop.fs.s3a.impl.OpenFileSupport; import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; @@ -131,7 +143,6 @@ import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder; -import org.apache.hadoop.fs.s3a.impl.V2Migration; import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl; @@ -141,6 +152,7 @@ import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.fs.statistics.IOStatisticsContext; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.fs.store.audit.AuditEntryPoint; import org.apache.hadoop.fs.store.audit.ActiveThreadSpanSource; import org.apache.hadoop.fs.store.audit.AuditSpan; @@ -213,7 +225,8 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.INITIALIZE_SPAN; -import static org.apache.hadoop.fs.s3a.auth.RolePolicies.STATEMENT_ALLOW_SSE_KMS_RW; +import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSCredentialProviderList; +import static org.apache.hadoop.fs.s3a.auth.RolePolicies.STATEMENT_ALLOW_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; @@ -224,14 +237,13 @@ import static org.apache.hadoop.fs.s3a.impl.CreateFileBuilder.OPTIONS_CREATE_FILE_OVERWRITE; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AP_INACCESSIBLE; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AP_REQUIRED_EXCEPTION; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.ARN_BUCKET_OPTION; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_403; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_403_FORBIDDEN; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404_NOT_FOUND; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.UPLOAD_PART_COUNT_LIMIT; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.logDnsLookup; @@ -246,6 +258,7 @@ import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; import static org.apache.hadoop.util.Preconditions.checkArgument; +import static org.apache.hadoop.util.functional.RemoteIterators.foreach; import static org.apache.hadoop.util.functional.RemoteIterators.typeCastingRemoteIterator; /** @@ -275,7 +288,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private URI uri; private Path workingDir; private String username; - private AmazonS3 s3; + private S3Client s3Client; + /** Async client is used for transfer manager and s3 select. */ + private S3AsyncClient s3AsyncClient; // initial callback policy is fail-once; it's there just to assist // some mock tests and other codepaths trying to call the low level // APIs on an uninitialized filesystem. @@ -294,7 +309,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private Listing listing; private long partSize; private boolean enableMultiObjectsDelete; - private TransferManager transfers; + private S3TransferManager transferManager; private ExecutorService boundedThreadPool; private ThreadPoolExecutor unboundedThreadPool; @@ -313,10 +328,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private int executorCapacity; private long multiPartThreshold; public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class); + + /** Log to warn of storage class configuration problems. */ + private static final LogExactlyOnce STORAGE_CLASS_WARNING = new LogExactlyOnce(LOG); + private static final Logger PROGRESS = LoggerFactory.getLogger("org.apache.hadoop.fs.s3a.S3AFileSystem.Progress"); private LocalDirAllocator directoryAllocator; - private CannedAccessControlList cannedACL; + private String cannedACL; /** * This must never be null; until initialized it just declares that there @@ -364,6 +383,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private AWSCredentialProviderList credentials; private SignerManager signerManager; + private S3AInternals s3aInternals; + + /** + * Do directory operations purge pending uploads? + */ + private boolean dirOperationsPurgeUploads; /** * Page size for deletions. @@ -418,6 +443,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, */ private boolean isMultipartUploadEnabled = DEFAULT_MULTIPART_UPLOAD_ENABLED; + /** + * Should file copy operations use the S3 transfer manager? + * True unless multipart upload is disabled. + */ + private boolean isMultipartCopyEnabled; + /** * A cache of files that should be deleted when the FileSystem is closed * or the JVM is exited. @@ -510,6 +541,8 @@ public void initialize(URI name, Configuration originalConf) super.initialize(uri, conf); setConf(conf); + s3aInternals = createS3AInternals(); + // look for encryption data // DT Bindings may override this setEncryptionSecrets( @@ -538,6 +571,9 @@ public void initialize(URI name, Configuration originalConf) //check but do not store the block size longBytesOption(conf, FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE, 1); enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true); + // should the delete also purge uploads. + dirOperationsPurgeUploads = conf.getBoolean(DIRECTORY_OPERATIONS_PURGE_UPLOADS, + DIRECTORY_OPERATIONS_PURGE_UPLOADS_DEFAULT); this.isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED, DEFAULT_MULTIPART_UPLOAD_ENABLED); @@ -550,6 +586,10 @@ public void initialize(URI name, Configuration originalConf) this.prefetchBlockSize = (int) prefetchBlockSizeLong; this.prefetchBlockCount = intOption(conf, PREFETCH_BLOCK_COUNT_KEY, PREFETCH_BLOCK_DEFAULT_COUNT, 1); + this.isMultipartUploadEnabled = conf.getBoolean(MULTIPART_UPLOADS_ENABLED, + DEFAULT_MULTIPART_UPLOAD_ENABLED); + // multipart copy and upload are the same; this just makes it explicit + this.isMultipartCopyEnabled = isMultipartUploadEnabled; initThreadPools(conf); @@ -584,9 +624,6 @@ public void initialize(URI name, Configuration originalConf) // the encryption algorithms) bindAWSClient(name, delegationTokensEnabled); - initTransferManager(); - - // This initiates a probe against S3 for the bucket existing. doBucketProbing(); @@ -654,7 +691,7 @@ public void initialize(URI name, Configuration originalConf) AWS_S3_VECTOR_ACTIVE_RANGE_READS, DEFAULT_AWS_S3_VECTOR_ACTIVE_RANGE_READS, 1); vectoredIOContext = populateVectoredIOContext(conf); scheme = (this.uri != null && this.uri.getScheme() != null) ? this.uri.getScheme() : FS_S3A; - } catch (AmazonClientException e) { + } catch (SdkException e) { // amazon client exception: stop all services then throw the translation cleanupWithLogger(LOG, span); stopAllServices(); @@ -706,7 +743,7 @@ private void setCSEGauge() { * bucket existence check is not done to improve performance of * S3AFileSystem initialization. When set to 1 or 2, bucket existence check * will be performed which is potentially slow. - * If 3 or higher: warn and use the v2 check. + * If 3 or higher: warn and skip check. * Also logging DNS address of the s3 endpoint if the bucket probe value is * greater than 0 else skipping it for increased performance. * @throws UnknownStoreException the bucket is absent @@ -723,18 +760,14 @@ private void doBucketProbing() throws IOException { LOG.debug("skipping check for bucket existence"); break; case 1: - logDnsLookup(getConf()); - verifyBucketExists(); - break; case 2: logDnsLookup(getConf()); - verifyBucketExistsV2(); + verifyBucketExists(); break; default: // we have no idea what this is, assume it is from a later release. - LOG.warn("Unknown bucket probe option {}: {}; falling back to check #2", + LOG.warn("Unknown bucket probe option {}: {}; skipping check for bucket existence", S3A_BUCKET_PROBE, bucketProbe); - verifyBucketExistsV2(); break; } } @@ -827,54 +860,34 @@ protected static S3AStorageStatistics createStorageStatistics( } /** - * Verify that the bucket exists. This does not check permissions, - * not even read access. + * Verify that the bucket exists. * Retry policy: retrying, translated. * @throws UnknownStoreException the bucket is absent * @throws IOException any other problem talking to S3 */ @Retries.RetryTranslated - protected void verifyBucketExists() - throws UnknownStoreException, IOException { - if (!invoker.retry("doesBucketExist", bucket, true, - trackDurationOfOperation(getDurationTrackerFactory(), - STORE_EXISTS_PROBE.getSymbol(), - () -> s3.doesBucketExist(bucket)))) { - throw new UnknownStoreException("s3a://" + bucket + "/", " Bucket does " - + "not exist"); - } - } + protected void verifyBucketExists() throws UnknownStoreException, IOException { - /** - * Verify that the bucket exists. This will correctly throw an exception - * when credentials are invalid. - * Retry policy: retrying, translated. - * @throws UnknownStoreException the bucket is absent - * @throws IOException any other problem talking to S3 - */ - @Retries.RetryTranslated - protected void verifyBucketExistsV2() - throws UnknownStoreException, IOException { - if (!invoker.retry("doesBucketExistV2", bucket, true, - trackDurationOfOperation(getDurationTrackerFactory(), - STORE_EXISTS_PROBE.getSymbol(), - () -> { - // Bug in SDK always returns `true` for AccessPoint ARNs with `doesBucketExistV2()` - // expanding implementation to use ARNs and buckets correctly + if(!trackDurationAndSpan( + STORE_EXISTS_PROBE, bucket, null, () -> + invoker.retry("doesBucketExist", bucket, true, () -> { try { - s3.getBucketAcl(bucket); - } catch (AmazonServiceException ex) { - int statusCode = ex.getStatusCode(); - if (statusCode == SC_404 || - (statusCode == SC_403 && ex.getMessage().contains(AP_INACCESSIBLE))) { + s3Client.headBucket(HeadBucketRequest.builder().bucket(bucket).build()); + return true; + } catch (AwsServiceException ex) { + int statusCode = ex.statusCode(); + if (statusCode == SC_404_NOT_FOUND || + (statusCode == SC_403_FORBIDDEN && accessPoint != null)) { return false; } } return true; }))) { - throw new UnknownStoreException("s3a://" + bucket + "/", " Bucket does " - + "not exist"); + + throw new UnknownStoreException("s3a://" + bucket + "/", + " Bucket does " + "not exist. " + "Accessing with " + ENDPOINT + " set to " + + getConf().getTrimmed(ENDPOINT, null)); } } @@ -915,7 +928,6 @@ public Listing getListing() { * @param dtEnabled are delegation tokens enabled? * @throws IOException failure. */ - @SuppressWarnings("deprecation") private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { Configuration conf = getConf(); credentials = null; @@ -928,7 +940,6 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { // with it if so. LOG.debug("Using delegation tokens"); - V2Migration.v1DelegationTokenCredentialProvidersUsed(); S3ADelegationTokens tokens = new S3ADelegationTokens(); this.delegationTokens = Optional.of(tokens); tokens.bindToFileSystem(getCanonicalUri(), @@ -955,7 +966,7 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { uaSuffix = tokens.getUserAgentField(); } else { // DT support is disabled, so create the normal credential chain - credentials = createAWSCredentialProviderSet(name, conf); + credentials = createAWSCredentialProviderList(name, conf); } LOG.debug("Using credential provider {}", credentials); Class s3ClientFactoryClass = conf.getClass( @@ -966,8 +977,12 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { ? conf.getTrimmed(ENDPOINT, DEFAULT_ENDPOINT) : accessPoint.getEndpoint(); - S3ClientFactory.S3ClientCreationParameters parameters = null; - parameters = new S3ClientFactory.S3ClientCreationParameters() + String configuredRegion = accessPoint == null + ? conf.getTrimmed(AWS_REGION) + : accessPoint.getRegion(); + + S3ClientFactory.S3ClientCreationParameters parameters = + new S3ClientFactory.S3ClientCreationParameters() .withCredentialSet(credentials) .withPathUri(name) .withEndpoint(endpoint) @@ -975,11 +990,30 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { .withPathStyleAccess(conf.getBoolean(PATH_STYLE_ACCESS, false)) .withUserAgentSuffix(uaSuffix) .withRequesterPays(conf.getBoolean(ALLOW_REQUESTER_PAYS, DEFAULT_ALLOW_REQUESTER_PAYS)) - .withRequestHandlers(auditManager.createRequestHandlers()); + .withExecutionInterceptors(auditManager.createExecutionInterceptors()) + .withMinimumPartSize(partSize) + .withMultipartCopyEnabled(isMultipartCopyEnabled) + .withMultipartThreshold(multiPartThreshold) + .withTransferManagerExecutor(unboundedThreadPool) + .withRegion(configuredRegion); - s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf) - .createS3Client(getUri(), - parameters); + S3ClientFactory clientFactory = ReflectionUtils.newInstance(s3ClientFactoryClass, conf); + s3Client = clientFactory.createS3Client(getUri(), parameters); + createS3AsyncClient(clientFactory, parameters); + transferManager = clientFactory.createS3TransferManager(getS3AsyncClient()); + } + + /** + * Creates and configures the S3AsyncClient. + * Uses synchronized method to suppress spotbugs error. + * + * @param clientFactory factory used to create S3AsyncClient + * @param parameters parameter object + * @throws IOException on any IO problem + */ + private void createS3AsyncClient(S3ClientFactory clientFactory, + S3ClientFactory.S3ClientCreationParameters parameters) throws IOException { + s3AsyncClient = clientFactory.createS3AsyncClient(getUri(), parameters); } /** @@ -1073,18 +1107,24 @@ protected RequestFactory createRequestFactory() { // Any encoding type String contentEncoding = getConf().getTrimmed(CONTENT_ENCODING, null); + if (contentEncoding != null) { + LOG.debug("Using content encoding set in {} = {}", CONTENT_ENCODING, contentEncoding); + } String storageClassConf = getConf() .getTrimmed(STORAGE_CLASS, "") .toUpperCase(Locale.US); StorageClass storageClass = null; if (!storageClassConf.isEmpty()) { - try { - storageClass = StorageClass.fromValue(storageClassConf); - } catch (IllegalArgumentException e) { - LOG.warn("Unknown storage class property {}: {}; falling back to default storage class", - STORAGE_CLASS, storageClassConf); + storageClass = StorageClass.fromValue(storageClassConf); + LOG.debug("Using storage class {}", storageClass); + if (storageClass.equals(StorageClass.UNKNOWN_TO_SDK_VERSION)) { + STORAGE_CLASS_WARNING.warn("Unknown storage class \"{}\" from option: {};" + + " falling back to default storage class", + storageClassConf, STORAGE_CLASS); + storageClass = null; } + } else { LOG.debug("Unset storage class property {}; falling back to default storage class", STORAGE_CLASS); @@ -1110,6 +1150,14 @@ public RequestFactory getRequestFactory() { return requestFactory; } + /** + * Get the S3 Async client. + * @return the async s3 client. + */ + private S3AsyncClient getS3AsyncClient() { + return s3AsyncClient; + } + /** * Implementation of all operations used by delegation tokens. */ @@ -1150,22 +1198,10 @@ public EncryptionSecrets getEncryptionSecrets() { return encryptionSecrets; } - private void initTransferManager() { - TransferManagerConfiguration transferConfiguration = - new TransferManagerConfiguration(); - transferConfiguration.setMinimumUploadPartSize(partSize); - transferConfiguration.setMultipartUploadThreshold(multiPartThreshold); - transferConfiguration.setMultipartCopyPartSize(partSize); - transferConfiguration.setMultipartCopyThreshold(multiPartThreshold); - - transfers = new TransferManager(s3, unboundedThreadPool); - transfers.setConfiguration(transferConfiguration); - } - private void initCannedAcls(Configuration conf) { String cannedACLName = conf.get(CANNED_ACL, DEFAULT_CANNED_ACL); if (!cannedACLName.isEmpty()) { - cannedACL = CannedAccessControlList.valueOf(cannedACLName); + cannedACL = AWSCannedACL.valueOf(cannedACLName).toString(); } else { cannedACL = null; } @@ -1198,12 +1234,22 @@ private void initMultipartUploads(Configuration conf) throws IOException { public void abortOutstandingMultipartUploads(long seconds) throws IOException { Preconditions.checkArgument(seconds >= 0); - Date purgeBefore = - new Date(new Date().getTime() - seconds * 1000); + Instant purgeBefore = + Instant.now().minusSeconds(seconds); LOG.debug("Purging outstanding multipart uploads older than {}", purgeBefore); invoker.retry("Purging multipart uploads", bucket, true, - () -> transfers.abortMultipartUploads(bucket, purgeBefore)); + () -> { + RemoteIterator uploadIterator = + MultipartUtils.listMultipartUploads(createStoreContext(), s3Client, null, maxKeys); + + while (uploadIterator.hasNext()) { + MultipartUpload upload = uploadIterator.next(); + if (upload.initiated().compareTo(purgeBefore) < 0) { + abortMultipartUpload(upload); + } + } + }); } /** @@ -1252,81 +1298,127 @@ public int getDefaultPort() { } /** - * Returns the S3 client used by this filesystem. - * This is for internal use within the S3A code itself. - * @return AmazonS3Client + * Set the client -used in mocking tests to force in a different client. + * @param client client. */ - private AmazonS3 getAmazonS3Client() { - return s3; + protected void setAmazonS3Client(S3Client client) { + Preconditions.checkNotNull(client, "clientV2"); + LOG.debug("Setting S3V2 client to {}", client); + s3Client = client; } /** - * Returns the S3 client used by this filesystem. - * Warning: this must only be used for testing, as it bypasses core - * S3A operations. - * @param reason a justification for requesting access. - * @return AmazonS3Client + * S3AInternals method. + * {@inheritDoc}. */ - @VisibleForTesting - public AmazonS3 getAmazonS3ClientForTesting(String reason) { - LOG.warn("Access to S3A client requested, reason {}", reason); - V2Migration.v1S3ClientRequested(); - return s3; + @AuditEntryPoint + @Retries.RetryTranslated + public String getBucketLocation() throws IOException { + return s3aInternals.getBucketLocation(bucket); } /** - * Set the client -used in mocking tests to force in a different client. - * @param client client. + * Create the S3AInternals; left as something mocking + * subclasses may want to override. + * @return the internal implementation */ - protected void setAmazonS3Client(AmazonS3 client) { - Preconditions.checkNotNull(client, "client"); - LOG.debug("Setting S3 client to {}", client); - s3 = client; - - // Need to use a new TransferManager that uses the new client. - // Also, using a new TransferManager requires a new threadpool as the old - // TransferManager will shut the thread pool down when it is garbage - // collected. - initThreadPools(getConf()); - initTransferManager(); + protected S3AInternals createS3AInternals() { + return new S3AInternalsImpl(); } /** - * Get the region of a bucket. - * Invoked from StoreContext; consider an entry point. - * @return the region in which a bucket is located - * @throws AccessDeniedException if the caller lacks permission. - * @throws IOException on any failure. + * Get the S3AInternals. + * @return the internal implementation */ - @Retries.RetryTranslated - @InterfaceAudience.LimitedPrivate("diagnostics") - public String getBucketLocation() throws IOException { - return getBucketLocation(bucket); + public S3AInternals getS3AInternals() { + return s3aInternals; } /** - * Get the region of a bucket; fixing up the region so it can be used - * in the builders of other AWS clients. - * Requires the caller to have the AWS role permission - * {@code s3:GetBucketLocation}. - * Retry policy: retrying, translated. - * @param bucketName the name of the bucket - * @return the region in which a bucket is located - * @throws AccessDeniedException if the caller lacks permission. - * @throws IOException on any failure. + * Implementation of the S3A Internals operations; pulled out of S3AFileSystem to + * force code accessing it to call {@link #getS3AInternals()}. */ - @VisibleForTesting - @AuditEntryPoint - @Retries.RetryTranslated - public String getBucketLocation(String bucketName) throws IOException { - final String region = trackDurationAndSpan( - STORE_EXISTS_PROBE, bucketName, null, () -> - invoker.retry("getBucketLocation()", bucketName, true, () -> - // If accessPoint then region is known from Arn - accessPoint != null - ? accessPoint.getRegion() - : s3.getBucketLocation(bucketName))); - return fixBucketRegion(region); + private final class S3AInternalsImpl implements S3AInternals { + + @Override + public S3Client getAmazonS3Client(String reason) { + LOG.debug("Access to S3 client requested, reason {}", reason); + return s3Client; + } + + /** + * S3AInternals method. + * {@inheritDoc}. + */ + @Override + @AuditEntryPoint + @Retries.RetryTranslated + public String getBucketLocation() throws IOException { + return s3aInternals.getBucketLocation(bucket); + } + + /** + * S3AInternals method. + * {@inheritDoc}. + */ + @Override + @AuditEntryPoint + @Retries.RetryTranslated + public String getBucketLocation(String bucketName) throws IOException { + final String region = trackDurationAndSpan( + STORE_EXISTS_PROBE, bucketName, null, () -> + invoker.retry("getBucketLocation()", bucketName, true, () -> + // If accessPoint then region is known from Arn + accessPoint != null + ? accessPoint.getRegion() + : s3Client.getBucketLocation(GetBucketLocationRequest.builder() + .bucket(bucketName) + .build()) + .locationConstraintAsString())); + return fixBucketRegion(region); + } + + /** + * S3AInternals method. + * {@inheritDoc}. + */ + @Override + @AuditEntryPoint + @Retries.RetryTranslated + public HeadObjectResponse getObjectMetadata(Path path) throws IOException { + return trackDurationAndSpan(INVOCATION_GET_FILE_STATUS, path, () -> + S3AFileSystem.this.getObjectMetadata(makeQualified(path), null, invoker, + "getObjectMetadata")); + } + + /** + * S3AInternals method. + * {@inheritDoc}. + */ + @Override + @AuditEntryPoint + @Retries.RetryTranslated + public HeadBucketResponse getBucketMetadata() throws IOException { + return S3AFileSystem.this.getBucketMetadata(); + } + + /** + * Get a shared copy of the AWS credentials, with its reference + * counter updated. + * Caller is required to call {@code close()} on this after + * they have finished using it. + * @param purpose what is this for? This is initially for logging + * @return a reference to shared credentials. + */ + public AWSCredentialProviderList shareCredentials(final String purpose) { + LOG.debug("Sharing credentials for: {}", purpose); + return credentials.share(); + } + + @Override + public boolean isMultipartCopyEnabled() { + return S3AFileSystem.this.isMultipartUploadEnabled; + } } /** @@ -1349,7 +1441,7 @@ public ChangeDetectionPolicy getChangeDetectionPolicy() { } /** - * Get the encryption algorithm of this endpoint. + * Get the encryption algorithm of this connector. * @return the encryption algorithm. */ public S3AEncryptionMethods getS3EncryptionAlgorithm() { @@ -1396,6 +1488,8 @@ private void initLocalDirAllocatorIfNotInitialized(Configuration conf) { * Get the bucket of this filesystem. * @return the bucket */ + @InterfaceAudience.Public + @InterfaceStability.Stable public String getBucket() { return bucket; } @@ -1413,7 +1507,7 @@ protected void setBucket(String bucket) { * Get the canned ACL of this FS. * @return an ACL, if any */ - CannedAccessControlList getCannedACL() { + String getCannedACL() { return cannedACL; } @@ -1643,18 +1737,18 @@ public void close() { } @Override - public GetObjectRequest newGetRequest(final String key) { + public GetObjectRequest.Builder newGetRequestBuilder(final String key) { // active the audit span used for the operation try (AuditSpan span = auditSpan.activate()) { - return getRequestFactory().newGetObjectRequest(key); + return getRequestFactory().newGetObjectRequestBuilder(key); } } @Override - public S3Object getObject(GetObjectRequest request) { + public ResponseInputStream getObject(GetObjectRequest request) { // active the audit span used for the operation try (AuditSpan span = auditSpan.activate()) { - return s3.getObject(request); + return s3Client.getObject(request); } } @@ -1681,18 +1775,19 @@ private final class WriteOperationHelperCallbacksImpl implements WriteOperationHelper.WriteOperationHelperCallbacks { @Override - public SelectObjectContentResult selectObjectContent(SelectObjectContentRequest request) { - return s3.selectObjectContent(request); + public CompletableFuture selectObjectContent( + SelectObjectContentRequest request, + SelectObjectContentResponseHandler responseHandler) { + return getS3AsyncClient().selectObjectContent(request, responseHandler); } @Override - public CompleteMultipartUploadResult completeMultipartUpload( + public CompleteMultipartUploadResponse completeMultipartUpload( CompleteMultipartUploadRequest request) { - return s3.completeMultipartUpload(request); + return s3Client.completeMultipartUpload(request); } } - /** * Create the read context for reading from the referenced file, * using FS state as well as the status. @@ -2054,7 +2149,7 @@ public boolean rename(Path src, Path dst) throws IOException { innerRename(src, dst)); LOG.debug("Copied {} bytes", bytesCopied); return true; - } catch (AmazonClientException e) { + } catch (SdkException e) { throw translateException("rename(" + src +", " + dst + ")", src, e); } catch (RenameFailedException e) { LOG.info("{}", e.getMessage()); @@ -2165,7 +2260,7 @@ private Pair initiateRename( * This operation throws an exception on any failure which needs to be * reported and downgraded to a failure. * Retries: retry translated, assuming all operations it is called do - * so. For safely, consider catch and handle AmazonClientException + * so. For safely, consider catch and handle SdkException * because this is such a complex method there's a risk it could surface. * @param source path to be renamed * @param dest new path after rename @@ -2176,12 +2271,12 @@ private Pair initiateRename( * @return the number of bytes copied. * @throws FileNotFoundException there's no source file. * @throws IOException on IO failure. - * @throws AmazonClientException on failures inside the AWS SDK + * @throws SdkException on failures inside the AWS SDK */ @Retries.RetryMixed private long innerRename(Path source, Path dest) throws RenameFailedException, FileNotFoundException, IOException, - AmazonClientException { + SdkException { Path src = qualify(source); Path dst = qualify(dest); @@ -2194,12 +2289,14 @@ private long innerRename(Path source, Path dest) // Initiate the rename. // this will call back into this class via the rename callbacks + final StoreContext storeContext = createStoreContext(); RenameOperation renameOperation = new RenameOperation( - createStoreContext(), + storeContext, src, srcKey, p.getLeft(), dst, dstKey, p.getRight(), - new OperationCallbacksImpl(), - pageSize); + new OperationCallbacksImpl(storeContext), + pageSize, + dirOperationsPurgeUploads); return renameOperation.execute(); } @@ -2220,8 +2317,19 @@ private final class OperationCallbacksImpl implements OperationCallbacks { /** Audit Span at time of creation. */ private final AuditSpan auditSpan; - private OperationCallbacksImpl() { - auditSpan = getActiveAuditSpan(); + private final StoreContext storeContext; + + private OperationCallbacksImpl(final StoreContext storeContext) { + this.storeContext = requireNonNull(storeContext); + this.auditSpan = storeContext.getActiveAuditSpan(); + } + + /** + * Get the audit span. + * @return the span + */ + private AuditSpan getAuditSpan() { + return auditSpan; } @Override @@ -2276,7 +2384,7 @@ public RemoteIterator listFilesAndDirectoryMarkers( } @Override - public CopyResult copyFile(final String srcKey, + public CopyObjectResponse copyFile(final String srcKey, final String destKey, final S3ObjectAttributes srcAttributes, final S3AReadOpContext readContext) throws IOException { @@ -2287,9 +2395,9 @@ public CopyResult copyFile(final String srcKey, @Override public void removeKeys( - final List keysToDelete, + final List keysToDelete, final boolean deleteFakeDir) - throws MultiObjectDeleteException, AmazonClientException, IOException { + throws MultiObjectDeleteException, SdkException, IOException { auditSpan.activate(); S3AFileSystem.this.removeKeys(keysToDelete, deleteFakeDir); } @@ -2321,7 +2429,29 @@ public RemoteIterator listObjects( Listing.ACCEPT_ALL_BUT_S3N, auditSpan)); } - } + + /** + * Abort multipart uploads under a path. + * @param prefix prefix for uploads to abort + * @return a count of aborts + * @throws IOException trouble; FileNotFoundExceptions are swallowed. + */ + @Override + @Retries.RetryTranslated + public long abortMultipartUploadsUnderPrefix(String prefix) + throws IOException { + getAuditSpan().activate(); + // this deactivates the audit span somehow + final RemoteIterator uploads = + S3AFileSystem.this.listUploadsUnderPrefix(storeContext, prefix); + // so reactivate it. + getAuditSpan().activate(); + return foreach(uploads, upload -> + invoker.retry("Aborting multipart commit", upload.key(), true, () -> + S3AFileSystem.this.abortMultipartUpload(upload))); + } + + } // end OperationCallbacksImpl /** * Callbacks from {@link Listing}. @@ -2388,21 +2518,17 @@ public int getMaxKeys() { * Low-level call to get at the object metadata. * This method is used in some external applications and so * must be viewed as a public entry point. - * Auditing: An audit entry point. + * @deprecated use S3AInternals API. * @param path path to the object. This will be qualified. * @return metadata * @throws IOException IO and object access problems. */ - @VisibleForTesting @AuditEntryPoint @InterfaceAudience.LimitedPrivate("utilities") @Retries.RetryTranslated - @InterfaceStability.Evolving - public ObjectMetadata getObjectMetadata(Path path) throws IOException { - V2Migration.v1GetObjectMetadataCalled(); - return trackDurationAndSpan(INVOCATION_GET_FILE_STATUS, path, () -> - getObjectMetadata(makeQualified(path), null, invoker, - "getObjectMetadata")); + @Deprecated + public HeadObjectResponse getObjectMetadata(Path path) throws IOException { + return getS3AInternals().getObjectMetadata(path); } /** @@ -2415,7 +2541,7 @@ public ObjectMetadata getObjectMetadata(Path path) throws IOException { * @throws IOException IO and object access problems. */ @Retries.RetryTranslated - private ObjectMetadata getObjectMetadata(Path path, + private HeadObjectResponse getObjectMetadata(Path path, ChangeTracker changeTracker, Invoker changeInvoker, String operation) throws IOException { String key = pathToKey(path); @@ -2628,7 +2754,7 @@ protected DurationTrackerFactory nonNullDurationTrackerFactory( @Retries.RetryRaw @VisibleForTesting @InterfaceAudience.LimitedPrivate("external utilities") - ObjectMetadata getObjectMetadata(String key) throws IOException { + HeadObjectResponse getObjectMetadata(String key) throws IOException { return getObjectMetadata(key, null, invoker, "getObjectMetadata"); } @@ -2645,28 +2771,28 @@ ObjectMetadata getObjectMetadata(String key) throws IOException { * @throws RemoteFileChangedException if an unexpected version is detected */ @Retries.RetryRaw - protected ObjectMetadata getObjectMetadata(String key, + protected HeadObjectResponse getObjectMetadata(String key, ChangeTracker changeTracker, Invoker changeInvoker, String operation) throws IOException { - ObjectMetadata meta = changeInvoker.retryUntranslated("GET " + key, true, + HeadObjectResponse response = changeInvoker.retryUntranslated("GET " + key, true, () -> { - GetObjectMetadataRequest request - = getRequestFactory().newGetObjectMetadataRequest(key); + HeadObjectRequest.Builder requestBuilder = + getRequestFactory().newHeadObjectRequestBuilder(key); incrementStatistic(OBJECT_METADATA_REQUESTS); DurationTracker duration = getDurationTrackerFactory() .trackDuration(ACTION_HTTP_HEAD_REQUEST.getSymbol()); try { LOG.debug("HEAD {} with change tracker {}", key, changeTracker); if (changeTracker != null) { - changeTracker.maybeApplyConstraint(request); + changeTracker.maybeApplyConstraint(requestBuilder); } - ObjectMetadata objectMetadata = s3.getObjectMetadata(request); + HeadObjectResponse headObjectResponse = s3Client.headObject(requestBuilder.build()); if (changeTracker != null) { - changeTracker.processMetadata(objectMetadata, operation); + changeTracker.processMetadata(headObjectResponse, operation); } - return objectMetadata; - } catch(AmazonServiceException ase) { + return headObjectResponse; + } catch (AwsServiceException ase) { if (!isObjectNotFound(ase)) { // file not found is not considered a failure of the call, // so only switch the duration tracker to update failure @@ -2680,7 +2806,28 @@ protected ObjectMetadata getObjectMetadata(String key, } }); incrementReadOperations(); - return meta; + return response; + } + + /** + * Request bucket metadata. + * @return the metadata + * @throws UnknownStoreException the bucket is absent + * @throws IOException any other problem talking to S3 + */ + @AuditEntryPoint + @Retries.RetryTranslated + protected HeadBucketResponse getBucketMetadata() throws IOException { + final HeadBucketResponse response = trackDurationAndSpan(STORE_EXISTS_PROBE, bucket, null, + () -> invoker.retry("getBucketMetadata()", bucket, true, () -> { + try { + return s3Client.headBucket( + getRequestFactory().newHeadBucketRequestBuilder(bucket).build()); + } catch (NoSuchBucketException e) { + throw new UnknownStoreException("s3a://" + bucket + "/", " Bucket does " + "not exist"); + } + })); + return response; } /** @@ -2709,9 +2856,9 @@ protected S3ListResult listObjects(S3ListRequest request, OBJECT_LIST_REQUEST, () -> { if (useListV1) { - return S3ListResult.v1(s3.listObjects(request.getV1())); + return S3ListResult.v1(s3Client.listObjects(request.getV1())); } else { - return S3ListResult.v2(s3.listObjectsV2(request.getV2())); + return S3ListResult.v2(s3Client.listObjectsV2(request.getV2())); } })); } @@ -2754,15 +2901,21 @@ protected S3ListResult continueListObjects(S3ListRequest request, OBJECT_CONTINUE_LIST_REQUEST, () -> { if (useListV1) { - return S3ListResult.v1( - s3.listNextBatchOfObjects( - getRequestFactory() - .newListNextBatchOfObjectsRequest( - prevResult.getV1()))); + List prevListResult = prevResult.getV1().contents(); + + // Next markers are only present when a delimiter is specified. + String nextMarker; + if (prevResult.getV1().nextMarker() != null) { + nextMarker = prevResult.getV1().nextMarker(); + } else { + nextMarker = prevListResult.get(prevListResult.size() - 1).key(); + } + + return S3ListResult.v1(s3Client.listObjects( + request.getV1().toBuilder().marker(nextMarker).build())); } else { - request.getV2().setContinuationToken(prevResult.getV2() - .getNextContinuationToken()); - return S3ListResult.v2(s3.listObjectsV2(request.getV2())); + return S3ListResult.v2(s3Client.listObjectsV2(request.getV2().toBuilder() + .continuationToken(prevResult.getV2().nextContinuationToken()).build())); } })); } @@ -2792,14 +2945,14 @@ public void incrementWriteOperations() { * * Retry policy: retry untranslated; delete considered idempotent. * @param key key to blob to delete. - * @throws AmazonClientException problems working with S3 + * @throws SdkException problems working with S3 * @throws InvalidRequestException if the request was rejected due to * a mistaken attempt to delete the root directory. */ @VisibleForTesting @Retries.RetryRaw protected void deleteObject(String key) - throws AmazonClientException, IOException { + throws SdkException, IOException { blockRootDelete(key); incrementWriteOperations(); try (DurationInfo ignored = @@ -2807,14 +2960,21 @@ protected void deleteObject(String key) "deleting %s", key)) { invoker.retryUntranslated(String.format("Delete %s:/%s", bucket, key), DELETE_CONSIDERED_IDEMPOTENT, - ()-> { + () -> { incrementStatistic(OBJECT_DELETE_OBJECTS); trackDurationOfInvocation(getDurationTrackerFactory(), OBJECT_DELETE_REQUEST.getSymbol(), - () -> s3.deleteObject(getRequestFactory() - .newDeleteObjectRequest(key))); + () -> s3Client.deleteObject(getRequestFactory() + .newDeleteObjectRequestBuilder(key) + .build())); return null; }); + } catch (AwsServiceException ase) { + // 404 errors get swallowed; this can be raised by + // third party stores (GCS). + if (!isObjectNotFound(ase)) { + throw ase; + } } } @@ -2825,14 +2985,14 @@ protected void deleteObject(String key) * @param f path path to delete * @param key key of entry * @param isFile is the path a file (used for instrumentation only) - * @throws AmazonClientException problems working with S3 + * @throws SdkException problems working with S3 * @throws IOException from invoker signature only -should not be raised. */ @Retries.RetryRaw void deleteObjectAtPath(Path f, String key, boolean isFile) - throws AmazonClientException, IOException { + throws SdkException, IOException { if (isFile) { instrumentation.fileDeleted(1); } else { @@ -2874,66 +3034,58 @@ private void blockRootDelete(String key) throws InvalidRequestException { * @return the AWS response * @throws MultiObjectDeleteException one or more of the keys could not * be deleted. - * @throws AmazonClientException amazon-layer failure. + * @throws SdkException amazon-layer failure. */ @Retries.RetryRaw - private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest) - throws MultiObjectDeleteException, AmazonClientException, IOException { + private DeleteObjectsResponse deleteObjects(DeleteObjectsRequest deleteRequest) + throws MultiObjectDeleteException, SdkException, IOException { incrementWriteOperations(); BulkDeleteRetryHandler retryHandler = new BulkDeleteRetryHandler(createStoreContext()); - int keyCount = deleteRequest.getKeys().size(); - try(DurationInfo ignored = + int keyCount = deleteRequest.delete().objects().size(); + try (DurationInfo ignored = new DurationInfo(LOG, false, "DELETE %d keys", keyCount)) { - return invoker.retryUntranslated("delete", - DELETE_CONSIDERED_IDEMPOTENT, - (text, e, r, i) -> { - // handle the failure - retryHandler.bulkDeleteRetried(deleteRequest, e); - }, - // duration is tracked in the bulk delete counters - trackDurationOfOperation(getDurationTrackerFactory(), - OBJECT_BULK_DELETE_REQUEST.getSymbol(), () -> { - incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount); - return s3.deleteObjects(deleteRequest); - })); - } catch (MultiObjectDeleteException e) { - // one or more of the keys could not be deleted. - // log and rethrow - List errors = e.getErrors(); - LOG.debug("Partial failure of delete, {} errors", errors.size(), e); - for (MultiObjectDeleteException.DeleteError error : errors) { - LOG.debug("{}: \"{}\" - {}", - error.getKey(), error.getCode(), error.getMessage()); + DeleteObjectsResponse response = + invoker.retryUntranslated("delete", DELETE_CONSIDERED_IDEMPOTENT, + (text, e, r, i) -> { + // handle the failure + retryHandler.bulkDeleteRetried(deleteRequest, e); + }, + // duration is tracked in the bulk delete counters + trackDurationOfOperation(getDurationTrackerFactory(), + OBJECT_BULK_DELETE_REQUEST.getSymbol(), () -> { + incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount); + return s3Client.deleteObjects(deleteRequest); + })); + + if (!response.errors().isEmpty()) { + // one or more of the keys could not be deleted. + // log and then throw + List errors = response.errors(); + LOG.debug("Partial failure of delete, {} errors", errors.size()); + for (S3Error error : errors) { + LOG.debug("{}: \"{}\" - {}", error.key(), error.code(), error.message()); + } + throw new MultiObjectDeleteException(errors); } - throw e; + + return response; } } /** - * Create a putObject request. + * Create a putObject request builder. * Adds the ACL and metadata * @param key key of object - * @param metadata metadata header - * @param srcfile source file + * @param length length of object to be uploaded + * @param isDirectoryMarker true if object to be uploaded is a directory marker * @return the request */ - public PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, File srcfile) { - return requestFactory.newPutObjectRequest(key, metadata, null, srcfile); - } - - /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * - * @param length length of data to set in header. - * @return a new metadata instance - */ - public ObjectMetadata newObjectMetadata(long length) { - return requestFactory.newObjectMetadata(length); + public PutObjectRequest.Builder newPutObjectRequestBuilder(String key, + long length, + boolean isDirectoryMarker) { + return requestFactory.newPutObjectRequestBuilder(key, null, length, isDirectoryMarker); } /** @@ -2950,15 +3102,24 @@ public ObjectMetadata newObjectMetadata(long length) { * Retry policy: N/A: the transfer manager is performing the upload. * Auditing: must be inside an audit span. * @param putObjectRequest the request + * @param file the file to be uploaded + * @param listener the progress listener for the request * @return the upload initiated */ @Retries.OnceRaw - public UploadInfo putObject(PutObjectRequest putObjectRequest) { + public UploadInfo putObject(PutObjectRequest putObjectRequest, File file, + ProgressableProgressListener listener) { long len = getPutRequestLength(putObjectRequest); - LOG.debug("PUT {} bytes to {} via transfer manager ", - len, putObjectRequest.getKey()); + LOG.debug("PUT {} bytes to {} via transfer manager ", len, putObjectRequest.key()); incrementPutStartStatistics(len); - Upload upload = transfers.upload(putObjectRequest); + + FileUpload upload = transferManager.uploadFile( + UploadFileRequest.builder() + .putObjectRequest(putObjectRequest) + .source(file) + .addTransferListener(listener) + .build()); + return new UploadInfo(upload, len); } @@ -2973,30 +3134,37 @@ public UploadInfo putObject(PutObjectRequest putObjectRequest) { * @param putObjectRequest the request * @param putOptions put object options * @param durationTrackerFactory factory for duration tracking + * @param uploadData data to be uploaded + * @param isFile represents if data to be uploaded is a file * @return the upload initiated - * @throws AmazonClientException on problems + * @throws SdkException on problems */ @VisibleForTesting @Retries.OnceRaw("For PUT; post-PUT actions are RetryExceptionsSwallowed") - PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest, + PutObjectResponse putObjectDirect(PutObjectRequest putObjectRequest, PutObjectOptions putOptions, + S3ADataBlocks.BlockUploadData uploadData, boolean isFile, DurationTrackerFactory durationTrackerFactory) - throws AmazonClientException { + throws SdkException { long len = getPutRequestLength(putObjectRequest); - LOG.debug("PUT {} bytes to {}", len, putObjectRequest.getKey()); + LOG.debug("PUT {} bytes to {}", len, putObjectRequest.key()); incrementPutStartStatistics(len); try { - PutObjectResult result = trackDurationOfSupplier( - nonNullDurationTrackerFactory(durationTrackerFactory), - OBJECT_PUT_REQUESTS.getSymbol(), () -> - s3.putObject(putObjectRequest)); + PutObjectResponse response = + trackDurationOfSupplier(nonNullDurationTrackerFactory(durationTrackerFactory), + OBJECT_PUT_REQUESTS.getSymbol(), + () -> isFile ? + s3Client.putObject(putObjectRequest, RequestBody.fromFile(uploadData.getFile())) : + s3Client.putObject(putObjectRequest, + RequestBody.fromInputStream(uploadData.getUploadStream(), + putObjectRequest.contentLength()))); incrementPutCompletedStatistics(true, len); // apply any post-write actions. - finishedWrite(putObjectRequest.getKey(), len, - result.getETag(), result.getVersionId(), + finishedWrite(putObjectRequest.key(), len, + response.eTag(), response.versionId(), putOptions); - return result; - } catch (SdkBaseException e) { + return response; + } catch (SdkException e) { incrementPutCompletedStatistics(false, len); throw e; } @@ -3009,12 +3177,8 @@ PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest, * @throws IllegalArgumentException if the length is negative */ private long getPutRequestLength(PutObjectRequest putObjectRequest) { - long len; - if (putObjectRequest.getFile() != null) { - len = putObjectRequest.getFile().length(); - } else { - len = putObjectRequest.getMetadata().getContentLength(); - } + long len = putObjectRequest.contentLength(); + Preconditions.checkState(len >= 0, "Cannot PUT object of unknown length"); return len; } @@ -3022,28 +3186,29 @@ private long getPutRequestLength(PutObjectRequest putObjectRequest) { /** * Upload part of a multi-partition file. * Increments the write and put counters. - * Important: this call does not close any input stream in the request. + * Important: this call does not close any input stream in the body. * * Retry Policy: none. - * @param request request * @param durationTrackerFactory duration tracker factory for operation + * @param request the upload part request. + * @param body the request body. * @return the result of the operation. - * @throws AmazonClientException on problems + * @throws AwsServiceException on problems */ @Retries.OnceRaw - UploadPartResult uploadPart(UploadPartRequest request, + UploadPartResponse uploadPart(UploadPartRequest request, RequestBody body, final DurationTrackerFactory durationTrackerFactory) - throws AmazonClientException { - long len = request.getPartSize(); + throws AwsServiceException { + long len = request.contentLength(); incrementPutStartStatistics(len); try { - UploadPartResult uploadPartResult = trackDurationOfSupplier( + UploadPartResponse uploadPartResponse = trackDurationOfSupplier( nonNullDurationTrackerFactory(durationTrackerFactory), MULTIPART_UPLOAD_PART_PUT.getSymbol(), () -> - s3.uploadPart(request)); + s3Client.uploadPart(request, body)); incrementPutCompletedStatistics(true, len); - return uploadPartResult; - } catch (AmazonClientException e) { + return uploadPartResponse; + } catch (AwsServiceException e) { incrementPutCompletedStatistics(false, len); throw e; } @@ -3108,56 +3273,57 @@ public void incrementPutProgressStatistics(String key, long bytes) { * be deleted in a multiple object delete operation. * The number of rejected objects will be added to the metric * {@link Statistic#FILES_DELETE_REJECTED}. - * @throws AmazonClientException other amazon-layer failure. + * @throws AwsServiceException other amazon-layer failure. */ @Retries.RetryRaw private void removeKeysS3( - List keysToDelete, + List keysToDelete, boolean deleteFakeDir) - throws MultiObjectDeleteException, AmazonClientException, - IOException { + throws MultiObjectDeleteException, AwsServiceException, IOException { if (LOG.isDebugEnabled()) { LOG.debug("Initiating delete operation for {} objects", keysToDelete.size()); - for (DeleteObjectsRequest.KeyVersion key : keysToDelete) { - LOG.debug(" {} {}", key.getKey(), - key.getVersion() != null ? key.getVersion() : ""); + for (ObjectIdentifier objectIdentifier : keysToDelete) { + LOG.debug(" {} {}", objectIdentifier.key(), + objectIdentifier.versionId() != null ? objectIdentifier.versionId() : ""); } } if (keysToDelete.isEmpty()) { // exit fast if there are no keys to delete return; } - for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) { - blockRootDelete(keyVersion.getKey()); + for (ObjectIdentifier objectIdentifier : keysToDelete) { + blockRootDelete(objectIdentifier.key()); } try { if (enableMultiObjectsDelete) { if (keysToDelete.size() <= pageSize) { deleteObjects(getRequestFactory() - .newBulkDeleteRequest(keysToDelete)); + .newBulkDeleteRequestBuilder(keysToDelete) + .build()); } else { // Multi object deletion of more than 1000 keys is not supported // by s3. So we are paging the keys by page size. LOG.debug("Partitioning the keys to delete as it is more than " + "page size. Number of keys: {}, Page size: {}", keysToDelete.size(), pageSize); - for (List batchOfKeysToDelete : + for (List batchOfKeysToDelete : Lists.partition(keysToDelete, pageSize)) { deleteObjects(getRequestFactory() - .newBulkDeleteRequest(batchOfKeysToDelete)); + .newBulkDeleteRequestBuilder(batchOfKeysToDelete) + .build()); } } } else { - for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) { - deleteObject(keyVersion.getKey()); + for (ObjectIdentifier objectIdentifier : keysToDelete) { + deleteObject(objectIdentifier.key()); } } } catch (MultiObjectDeleteException ex) { // partial delete. // Update the stats with the count of the actual number of successful // deletions. - int rejected = ex.getErrors().size(); + int rejected = ex.errors().size(); noteDeleted(keysToDelete.size() - rejected, deleteFakeDir); incrementStatistic(FILES_DELETE_REJECTED, rejected); throw ex; @@ -3190,15 +3356,15 @@ private void noteDeleted(final int count, final boolean deleteFakeDir) { * a mistaken attempt to delete the root directory. * @throws MultiObjectDeleteException one or more of the keys could not * be deleted in a multiple object delete operation. - * @throws AmazonClientException amazon-layer failure. + * @throws AwsServiceException amazon-layer failure. * @throws IOException other IO Exception. */ @VisibleForTesting @Retries.RetryRaw public void removeKeys( - final List keysToDelete, + final List keysToDelete, final boolean deleteFakeDir) - throws MultiObjectDeleteException, AmazonClientException, + throws MultiObjectDeleteException, AwsServiceException, IOException { try (DurationInfo ignored = new DurationInfo(LOG, false, "Deleting %d keys", keysToDelete.size())) { @@ -3246,14 +3412,17 @@ protected boolean deleteWithoutCloseCheck(Path f, boolean recursive) throws IOEx // span covers delete, getFileStatus, fake directory operations. try (AuditSpan span = createSpan(INVOCATION_DELETE.getSymbol(), path.toString(), null)) { + // SC will include active span + final StoreContext storeContext = createStoreContext(); boolean outcome = trackDuration(getDurationTrackerFactory(), INVOCATION_DELETE.getSymbol(), new DeleteOperation( - createStoreContext(), + storeContext, innerGetFileStatus(path, true, StatusProbeEnum.ALL), recursive, - new OperationCallbacksImpl(), - pageSize)); + new OperationCallbacksImpl(storeContext), + pageSize, + dirOperationsPurgeUploads)); if (outcome) { try { maybeCreateFakeParentDirectory(path); @@ -3268,7 +3437,7 @@ protected boolean deleteWithoutCloseCheck(Path f, boolean recursive) throws IOEx LOG.debug("Couldn't delete {} - does not exist: {}", path, e.toString()); instrumentation.errorIgnored(); return false; - } catch (AmazonClientException e) { + } catch (SdkException e) { throw translateException("delete", path, e); } } @@ -3282,7 +3451,7 @@ protected boolean deleteWithoutCloseCheck(Path f, boolean recursive) throws IOEx */ @Retries.RetryTranslated private void createFakeDirectoryIfNecessary(Path f) - throws IOException, AmazonClientException { + throws IOException, SdkException { String key = pathToKey(f); // we only make the LIST call; the codepaths to get here should not // be reached if there is an empty dir marker -and if they do, it @@ -3302,7 +3471,7 @@ private void createFakeDirectoryIfNecessary(Path f) @Retries.RetryTranslated @VisibleForTesting protected void maybeCreateFakeParentDirectory(Path path) - throws IOException, AmazonClientException { + throws IOException, SdkException { Path parent = path.getParent(); if (parent != null && !parent.isRoot() && !isUnderMagicCommitPath(parent)) { createFakeDirectoryIfNecessary(parent); @@ -3356,11 +3525,11 @@ public FileStatus[] listStatus(Path f) throws FileNotFoundException, * @return the statuses of the files/directories in the given patch * @throws FileNotFoundException when the path does not exist; * @throws IOException due to an IO problem. - * @throws AmazonClientException on failures inside the AWS SDK + * @throws SdkException on failures inside the AWS SDK */ private RemoteIterator innerListStatus(Path f) throws FileNotFoundException, - IOException, AmazonClientException { + IOException, SdkException { Path path = qualify(f); LOG.debug("List status for path: {}", path); @@ -3424,15 +3593,15 @@ public S3ListRequest createListObjectsRequest(String key, private S3ListRequest createListObjectsRequest(String key, String delimiter, int limit) { if (!useListV1) { - ListObjectsV2Request request = - getRequestFactory().newListObjectsV2Request( + ListObjectsV2Request.Builder requestBuilder = + getRequestFactory().newListObjectsV2RequestBuilder( key, delimiter, limit); - return S3ListRequest.v2(request); + return S3ListRequest.v2(requestBuilder.build()); } else { - ListObjectsRequest request = - getRequestFactory().newListObjectsV1Request( + ListObjectsRequest.Builder requestBuilder = + getRequestFactory().newListObjectsV1RequestBuilder( key, delimiter, limit); - return S3ListRequest.v1(request); + return S3ListRequest.v1(requestBuilder.build()); } } @@ -3720,31 +3889,31 @@ S3AFileStatus s3GetFileStatus(final Path path, && probes.contains(StatusProbeEnum.Head)) { try { // look for the simple file - ObjectMetadata meta = getObjectMetadata(key); + HeadObjectResponse meta = getObjectMetadata(key); LOG.debug("Found exact file: normal file {}", key); - long contentLength = meta.getContentLength(); + long contentLength = meta.contentLength(); // check if CSE is enabled, then strip padded length. - if (isCSEEnabled - && meta.getUserMetaDataOf(Headers.CRYPTO_CEK_ALGORITHM) != null + if (isCSEEnabled && + meta.metadata().get(AWSHeaders.CRYPTO_CEK_ALGORITHM) != null && contentLength >= CSE_PADDING_LENGTH) { contentLength -= CSE_PADDING_LENGTH; } return new S3AFileStatus(contentLength, - dateToLong(meta.getLastModified()), + meta.lastModified().toEpochMilli(), path, getDefaultBlockSize(path), username, - meta.getETag(), - meta.getVersionId()); - } catch (AmazonServiceException e) { + meta.eTag(), + meta.versionId()); + } catch (AwsServiceException e) { // if the response is a 404 error, it just means that there is // no file at that path...the remaining checks will be needed. // But: an empty bucket is also a 404, so check for that // and fail. - if (e.getStatusCode() != SC_404 || isUnknownBucket(e)) { + if (e.statusCode() != SC_404_NOT_FOUND || isUnknownBucket(e)) { throw translateException("getFileStatus", path, e); } - } catch (AmazonClientException e) { + } catch (SdkException e) { throw translateException("getFileStatus", path, e); } } @@ -3787,11 +3956,11 @@ S3AFileStatus s3GetFileStatus(final Path path, LOG.debug("Found root directory"); return new S3AFileStatus(Tristate.TRUE, path, username); } - } catch (AmazonServiceException e) { - if (e.getStatusCode() != SC_404 || isUnknownBucket(e)) { + } catch (AwsServiceException e) { + if (e.statusCode() != SC_404_NOT_FOUND || isUnknownBucket(e)) { throw translateException("getFileStatus", path, e); } - } catch (AmazonClientException e) { + } catch (SdkException e) { throw translateException("getFileStatus", path, e); } } @@ -3835,7 +4004,7 @@ private boolean s3Exists(final Path path, final Set probes) * @throws IOException IO problem * @throws FileAlreadyExistsException the destination file exists and * overwrite==false - * @throws AmazonClientException failure in the AWS SDK + * @throws SdkException failure in the AWS SDK */ @Override @AuditEntryPoint @@ -3890,13 +4059,12 @@ public void copyLocalFileFromTo(File file, Path from, Path to) throws IOExceptio to, () -> { final String key = pathToKey(to); - final ObjectMetadata om = newObjectMetadata(file.length()); Progressable progress = null; - PutObjectRequest putObjectRequest = newPutObjectRequest(key, om, file); - S3AFileSystem.this.invoker.retry( - "putObject(" + "" + ")", to.toString(), - true, - () -> executePut(putObjectRequest, progress, putOptionsForPath(to))); + PutObjectRequest.Builder putObjectRequestBuilder = + newPutObjectRequestBuilder(key, file.length(), false); + S3AFileSystem.this.invoker.retry("putObject(" + "" + ")", to.toString(), true, + () -> executePut(putObjectRequestBuilder.build(), progress, putOptionsForPath(to), + file)); return null; }); @@ -3921,40 +4089,35 @@ public boolean createEmptyDir(Path path, StoreContext storeContext) /** * Execute a PUT via the transfer manager, blocking for completion. - * If the waiting for completion is interrupted, the upload will be - * aborted before an {@code InterruptedIOException} is thrown. * @param putObjectRequest request * @param progress optional progress callback * @param putOptions put object options * @return the upload result - * @throws InterruptedIOException if the blocking was interrupted. + * @throws IOException IO failure */ @Retries.OnceRaw("For PUT; post-PUT actions are RetrySwallowed") - UploadResult executePut( + PutObjectResponse executePut( final PutObjectRequest putObjectRequest, final Progressable progress, - final PutObjectOptions putOptions) - throws InterruptedIOException { - String key = putObjectRequest.getKey(); + final PutObjectOptions putOptions, + final File file) + throws IOException { + String key = putObjectRequest.key(); long len = getPutRequestLength(putObjectRequest); - UploadInfo info = putObject(putObjectRequest); - Upload upload = info.getUpload(); - ProgressableProgressListener listener = new ProgressableProgressListener( - this, key, upload, progress); - upload.addProgressListener(listener); - UploadResult result = waitForUploadCompletion(key, info); - listener.uploadCompleted(); + ProgressableProgressListener listener = + new ProgressableProgressListener(this, putObjectRequest.key(), progress); + UploadInfo info = putObject(putObjectRequest, file, listener); + PutObjectResponse result = waitForUploadCompletion(key, info).response(); + listener.uploadCompleted(info.getFileUpload()); // post-write actions finishedWrite(key, len, - result.getETag(), result.getVersionId(), putOptions); + result.eTag(), result.versionId(), putOptions); return result; } /** * Wait for an upload to complete. - * If the waiting for completion is interrupted, the upload will be - * aborted before an {@code InterruptedIOException} is thrown. * If the upload (or its result collection) failed, this is where * the failure is raised as an AWS exception. * Calls {@link #incrementPutCompletedStatistics(boolean, long)} @@ -3962,24 +4125,20 @@ UploadResult executePut( * @param key destination key * @param uploadInfo upload to wait for * @return the upload result - * @throws InterruptedIOException if the blocking was interrupted. + * @throws IOException IO failure */ @Retries.OnceRaw - UploadResult waitForUploadCompletion(String key, UploadInfo uploadInfo) - throws InterruptedIOException { - Upload upload = uploadInfo.getUpload(); + CompletedFileUpload waitForUploadCompletion(String key, UploadInfo uploadInfo) + throws IOException { + FileUpload upload = uploadInfo.getFileUpload(); try { - UploadResult result = upload.waitForUploadResult(); + CompletedFileUpload result = upload.completionFuture().join(); incrementPutCompletedStatistics(true, uploadInfo.getLength()); return result; - } catch (InterruptedException e) { + } catch (CompletionException e) { LOG.info("Interrupted: aborting upload"); incrementPutCompletedStatistics(false, uploadInfo.getLength()); - upload.abort(); - throw (InterruptedIOException) - new InterruptedIOException("Interrupted in PUT to " - + keyToQualifiedPath(key)) - .initCause(e); + throw extractException("upload", key, e); } } @@ -4072,17 +4231,13 @@ public void close() throws IOException { * both the expected state of this FS and of failures while being stopped. */ protected synchronized void stopAllServices() { - // shutting down the transfer manager also shuts - // down the S3 client it is bonded to. - if (transfers != null) { - try { - transfers.shutdownNow(true); - } catch (RuntimeException e) { - // catch and swallow for resilience. - LOG.debug("When shutting down", e); - } - transfers = null; - } + closeAutocloseables(LOG, transferManager, + s3Client, + getS3AsyncClient()); + transferManager = null; + s3Client = null; + s3AsyncClient = null; + // At this point the S3A client is shut down, // now the executor pools are closed HadoopExecutors.shutdown(boundedThreadPool, LOG, @@ -4108,13 +4263,13 @@ protected synchronized void stopAllServices() { } /** - * Verify that the input stream is open. Non blocking; this gives + * Verify that the filesystem has not been closed. Non blocking; this gives * the last state of the volatile {@link #closed} field. - * @throws IOException if the connection is closed. + * @throws PathIOException if the FS is closed. */ - private void checkNotClosed() throws IOException { + private void checkNotClosed() throws PathIOException { if (isClosed) { - throw new IOException(uri + ": " + E_FS_CLOSED); + throw new PathIOException(uri.toString(), E_FS_CLOSED); } } @@ -4218,7 +4373,7 @@ public List listAWSPolicyRules( // 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); + statements.add(STATEMENT_ALLOW_KMS_RW); return statements; } @@ -4237,21 +4392,11 @@ public List listAWSPolicyRules( * @throws IOException Other IO problems */ @Retries.RetryTranslated - private CopyResult copyFile(String srcKey, String dstKey, long size, + private CopyObjectResponse copyFile(String srcKey, String dstKey, long size, S3ObjectAttributes srcAttributes, S3AReadOpContext readContext) - throws IOException, InterruptedIOException { + throws IOException { LOG.debug("copyFile {} -> {} ", srcKey, dstKey); - ProgressListener progressListener = progressEvent -> { - switch (progressEvent.getEventType()) { - case TRANSFER_PART_COMPLETED_EVENT: - incrementWriteOperations(); - break; - default: - break; - } - }; - ChangeTracker changeTracker = new ChangeTracker( keyToQualifiedPath(srcKey).toString(), changeDetectionPolicy, @@ -4263,7 +4408,7 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, String action = "copyFile(" + srcKey + ", " + dstKey + ")"; Invoker readInvoker = readContext.getReadInvoker(); - ObjectMetadata srcom; + HeadObjectResponse srcom; try { srcom = once(action, srcKey, () -> @@ -4274,7 +4419,6 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, // This means the File was deleted since LIST enumerated it. LOG.debug("getObjectMetadata({}) failed to find an expected file", srcKey, e); - // We create an exception, but the text depends on the S3Guard state throw new RemoteFileChangedException( keyToQualifiedPath(srcKey).toString(), action, @@ -4282,38 +4426,68 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, e); } - return readInvoker.retry( - action, srcKey, - true, - () -> { - CopyObjectRequest copyObjectRequest = - getRequestFactory().newCopyObjectRequest(srcKey, dstKey, srcom); - changeTracker.maybeApplyConstraint(copyObjectRequest); - incrementStatistic(OBJECT_COPY_REQUESTS); - Copy copy = transfers.copy(copyObjectRequest, - getAuditManager().createStateChangeListener()); - copy.addProgressListener(progressListener); - CopyOutcome copyOutcome = CopyOutcome.waitForCopy(copy); - InterruptedException interruptedException = - copyOutcome.getInterruptedException(); - if (interruptedException != null) { - // copy interrupted: convert to an IOException. - throw (IOException)new InterruptedIOException( - "Interrupted copying " + srcKey - + " to " + dstKey + ", cancelling") - .initCause(interruptedException); - } - SdkBaseException awsException = copyOutcome.getAwsException(); - if (awsException != null) { - changeTracker.processException(awsException, "copy"); - throw awsException; - } - CopyResult result = copyOutcome.getCopyResult(); - changeTracker.processResponse(result); - incrementWriteOperations(); - instrumentation.filesCopied(1, size); - return result; - }); + CopyObjectRequest.Builder copyObjectRequestBuilder = + getRequestFactory().newCopyObjectRequestBuilder(srcKey, dstKey, srcom); + changeTracker.maybeApplyConstraint(copyObjectRequestBuilder); + final CopyObjectRequest copyRequest = copyObjectRequestBuilder.build(); + LOG.debug("Copy Request: {}", copyRequest); + CopyObjectResponse response; + + // transfer manager is skipped if disabled or the file is too small to worry about + final boolean useTransferManager = isMultipartCopyEnabled && size >= multiPartThreshold; + if (useTransferManager) { + // use transfer manager + response = readInvoker.retry( + action, srcKey, + true, + () -> { + incrementStatistic(OBJECT_COPY_REQUESTS); + + Copy copy = transferManager.copy( + CopyRequest.builder() + .copyObjectRequest(copyRequest) + .build()); + + try { + CompletedCopy completedCopy = copy.completionFuture().join(); + return completedCopy.response(); + } catch (CompletionException e) { + Throwable cause = e.getCause(); + if (cause instanceof SdkException) { + // if this is a 412 precondition failure, it may + // be converted to a RemoteFileChangedException + SdkException awsException = (SdkException)cause; + changeTracker.processException(awsException, "copy"); + throw awsException; + } + throw extractException(action, srcKey, e); + } + }); + } else { + // single part copy bypasses transfer manager + // note, this helps with some mock testing, e.g. HBoss. as there is less to mock. + response = readInvoker.retry( + action, srcKey, + true, + () -> { + LOG.debug("copyFile: single part copy {} -> {} of size {}", srcKey, dstKey, size); + incrementStatistic(OBJECT_COPY_REQUESTS); + try { + return s3Client.copyObject(copyRequest); + } catch (SdkException awsException) { + // if this is a 412 precondition failure, it may + // be converted to a RemoteFileChangedException + changeTracker.processException(awsException, "copy"); + // otherwise, rethrow + throw awsException; + } + }); + } + + changeTracker.processResponse(response); + incrementWriteOperations(); + instrumentation.filesCopied(1, size); + return response; } /** @@ -4321,16 +4495,16 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, * Retry policy: none + untranslated. * @param request request to initiate * @return the result of the call - * @throws AmazonClientException on failures inside the AWS SDK + * @throws SdkException on failures inside the AWS SDK * @throws IOException Other IO problems */ @Retries.OnceRaw - InitiateMultipartUploadResult initiateMultipartUpload( - InitiateMultipartUploadRequest request) throws IOException { - LOG.debug("Initiate multipart upload to {}", request.getKey()); + CreateMultipartUploadResponse initiateMultipartUpload( + CreateMultipartUploadRequest request) throws IOException { + LOG.debug("Initiate multipart upload to {}", request.key()); return trackDurationOfSupplier(getDurationTrackerFactory(), OBJECT_MULTIPART_UPLOAD_INITIATED.getSymbol(), - () -> getAmazonS3Client().initiateMultipartUpload(request)); + () -> s3Client.createMultipartUpload(request)); } /** @@ -4403,22 +4577,22 @@ private PutObjectOptions putOptionsForPath(Path path) { */ @Retries.RetryExceptionsSwallowed private void deleteUnnecessaryFakeDirectories(Path path) { - List keysToRemove = new ArrayList<>(); + List keysToRemove = new ArrayList<>(); while (!path.isRoot()) { String key = pathToKey(path); key = (key.endsWith("/")) ? key : (key + "/"); LOG.trace("To delete unnecessary fake directory {} for {}", key, path); - keysToRemove.add(new DeleteObjectsRequest.KeyVersion(key)); + keysToRemove.add(ObjectIdentifier.builder().key(key).build()); path = path.getParent(); } try { removeKeys(keysToRemove, true); - } catch(AmazonClientException | IOException e) { + } catch (AwsServiceException | IOException e) { instrumentation.errorIgnored(); if (LOG.isDebugEnabled()) { StringBuilder sb = new StringBuilder(); - for(DeleteObjectsRequest.KeyVersion kv : keysToRemove) { - sb.append(kv.getKey()).append(","); + for (ObjectIdentifier objectIdentifier : keysToRemove) { + sb.append(objectIdentifier.key()).append(","); } LOG.debug("While deleting keys {} ", sb.toString(), e); } @@ -4451,11 +4625,18 @@ private void createFakeDirectory(final String objectName, @Retries.RetryTranslated private void createEmptyObject(final String objectName, PutObjectOptions putOptions) throws IOException { - invoker.retry("PUT 0-byte object ", objectName, - true, () -> - putObjectDirect(getRequestFactory().newDirectoryMarkerRequest(objectName), - putOptions, - getDurationTrackerFactory())); + final InputStream im = new InputStream() { + @Override + public int read() throws IOException { + return -1; + } + }; + + S3ADataBlocks.BlockUploadData uploadData = new S3ADataBlocks.BlockUploadData(im); + + invoker.retry("PUT 0-byte object ", objectName, true, + () -> putObjectDirect(getRequestFactory().newDirectoryMarkerRequest(objectName).build(), + putOptions, uploadData, false, getDurationTrackerFactory())); incrementPutProgressStatistics(objectName, 0); instrumentation.directoryCreated(); } @@ -4712,10 +4893,10 @@ public EtagChecksum getFileChecksum(Path f, final long length) ETAG_CHECKSUM_ENABLED_DEFAULT)) { return trackDurationAndSpan(INVOCATION_GET_FILE_CHECKSUM, path, () -> { LOG.debug("getFileChecksum({})", path); - ObjectMetadata headers = getObjectMetadata(path, null, + HeadObjectResponse headers = getObjectMetadata(path, null, invoker, "getFileChecksum are"); - String eTag = headers.getETag(); + String eTag = headers.eTag(); return eTag != null ? new EtagChecksum(eTag) : null; }); } else { @@ -4797,11 +4978,18 @@ protected final class HeaderProcessingCallbacksImpl implements HeaderProcessing.HeaderProcessingCallbacks { @Override - public ObjectMetadata getObjectMetadata(final String key) + public HeadObjectResponse getObjectMetadata(final String key) throws IOException { return once("getObjectMetadata", key, () -> S3AFileSystem.this.getObjectMetadata(key)); } + + @Override + public HeadBucketResponse getBucketMetadata() + throws IOException { + return once("getBucketMetadata", bucket, () -> + S3AFileSystem.this.getBucketMetadata()); + } } /** * {@inheritDoc}. @@ -4914,7 +5102,7 @@ private RemoteIterator innerListFiles( // If we have reached here, it means either there are files // in this directory or it is empty. return listFilesAssumingDir; - } catch (AmazonClientException e) { + } catch (SdkException e) { throw translateException("listFiles", path, e); } } @@ -5007,14 +5195,39 @@ S3ALocatedFileStatus toLocatedFileStatus(S3AFileStatus status) @InterfaceAudience.Private @Retries.RetryTranslated @AuditEntryPoint - public MultipartUtils.UploadIterator listUploads(@Nullable String prefix) + public RemoteIterator listUploads(@Nullable String prefix) throws IOException { // span is picked up retained in the listing. - return trackDurationAndSpan(MULTIPART_UPLOAD_LIST, prefix, null, () -> - MultipartUtils.listMultipartUploads( - createStoreContext(), - s3, prefix, maxKeys - )); + checkNotClosed(); + try (AuditSpan span = createSpan(MULTIPART_UPLOAD_LIST.getSymbol(), + prefix, null)) { + return listUploadsUnderPrefix(createStoreContext(), prefix); + } + } + + /** + * List any pending multipart uploads whose keys begin with prefix, using + * an iterator that can handle an unlimited number of entries. + * See {@link #listMultipartUploads(String)} for a non-iterator version of + * this. + * @param storeContext store conext. + * @param prefix optional key prefix to search + * @return Iterator over multipart uploads. + * @throws IOException on failure + */ + @InterfaceAudience.Private + @Retries.RetryTranslated + public RemoteIterator listUploadsUnderPrefix( + final StoreContext storeContext, + final @Nullable String prefix) + throws IOException { + // span is picked up retained in the listing. + String p = prefix; + if (prefix != null && !prefix.isEmpty() && !prefix.endsWith("/")) { + p = prefix + "/"; + } + // duration tracking is done in iterator. + return MultipartUtils.listMultipartUploads(storeContext, s3Client, p, maxKeys); } /** @@ -5024,7 +5237,7 @@ public MultipartUtils.UploadIterator listUploads(@Nullable String prefix) * Retry policy: retry, translated. * @return a listing of multipart uploads. * @param prefix prefix to scan for, "" for none - * @throws IOException IO failure, including any uprated AmazonClientException + * @throws IOException IO failure, including any uprated SdkException */ @InterfaceAudience.Private @Retries.RetryTranslated @@ -5036,9 +5249,10 @@ public List listMultipartUploads(String prefix) } String p = prefix; return invoker.retry("listMultipartUploads", p, true, () -> { - ListMultipartUploadsRequest request = getRequestFactory() - .newListMultipartUploadsRequest(p); - return s3.listMultipartUploads(request).getMultipartUploads(); + final ListMultipartUploadsRequest request = getRequestFactory() + .newListMultipartUploadsRequestBuilder(p).build(); + return trackDuration(getInstrumentation(), MULTIPART_UPLOAD_LIST.getSymbol(), () -> + s3Client.listMultipartUploads(request).uploads()); }); } @@ -5047,37 +5261,35 @@ public List listMultipartUploads(String prefix) * Retry policy: none. * @param destKey destination key * @param uploadId Upload ID + * @throws IOException IO failure, including any uprated SdkException */ - @Retries.OnceRaw - void abortMultipartUpload(String destKey, String uploadId) { - LOG.info("Aborting multipart upload {} to {}", uploadId, destKey); - getAmazonS3Client().abortMultipartUpload( - getRequestFactory().newAbortMultipartUploadRequest( - destKey, - uploadId)); + @Retries.OnceTranslated + public void abortMultipartUpload(String destKey, String uploadId) throws IOException { + LOG.debug("Aborting multipart upload {} to {}", uploadId, destKey); + trackDuration(getInstrumentation(), OBJECT_MULTIPART_UPLOAD_ABORTED.getSymbol(), () -> + s3Client.abortMultipartUpload( + getRequestFactory().newAbortMultipartUploadRequestBuilder( + destKey, + uploadId).build())); } /** * Abort a multipart upload. * Retry policy: none. * @param upload the listed upload to abort. + * @throws IOException IO failure, including any uprated SdkException */ - @Retries.OnceRaw - void abortMultipartUpload(MultipartUpload upload) { - String destKey; - String uploadId; - destKey = upload.getKey(); - uploadId = upload.getUploadId(); - if (LOG.isInfoEnabled()) { + @Retries.OnceTranslated + public void abortMultipartUpload(MultipartUpload upload) throws IOException { + String destKey = upload.key(); + String uploadId = upload.uploadId(); + if (LOG.isDebugEnabled()) { DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); LOG.debug("Aborting multipart upload {} to {} initiated by {} on {}", - uploadId, destKey, upload.getInitiator(), - df.format(upload.getInitiated())); + uploadId, destKey, upload.initiator(), + df.format(Date.from(upload.initiated()))); } - getAmazonS3Client().abortMultipartUpload( - getRequestFactory().newAbortMultipartUploadRequest( - destKey, - uploadId)); + abortMultipartUpload(destKey, uploadId); } /** @@ -5123,13 +5335,17 @@ public boolean hasPathCapability(final Path path, final String capability) case STORE_CAPABILITY_DIRECTORY_MARKER_AWARE: return true; + // Do directory operations purge uploads. + case DIRECTORY_OPERATIONS_PURGE_UPLOADS: + return dirOperationsPurgeUploads; + // etags are avaialable in listings, but they // are not consistent across renames. // therefore, only availability is declared case CommonPathCapabilities.ETAGS_AVAILABLE: return true; - /* + /* * Marker policy capabilities are handed off. */ case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP: @@ -5399,7 +5615,7 @@ public MarkerToolOperations createMarkerToolOperations(final String target) throws IOException { createSpan("marker-tool-scan", target, null); - return new MarkerToolOperationsImpl(new OperationCallbacksImpl()); + return new MarkerToolOperationsImpl(new OperationCallbacksImpl(createStoreContext())); } /** @@ -5464,4 +5680,5 @@ public boolean isCSEEnabled() { public boolean isMultipartUploadEnabled() { return isMultipartUploadEnabled; } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java index 4b50ab2c04bd9..2ed9083efcddd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java @@ -22,6 +22,7 @@ import java.io.Closeable; import java.io.EOFException; import java.io.IOException; +import java.io.InputStream; import java.io.InterruptedIOException; import java.net.SocketTimeoutException; import java.nio.ByteBuffer; @@ -31,9 +32,9 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.IntFunction; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.S3ObjectInputStream; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,7 +47,6 @@ import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.FileRange; -import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.impl.CombinedFileRange; import org.apache.hadoop.fs.VectoredReadUtils; @@ -61,6 +61,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.functional.CallableRaisingIOE; + import static java.util.Objects.requireNonNull; import static org.apache.commons.lang3.StringUtils.isNotEmpty; import static org.apache.hadoop.fs.VectoredReadUtils.isOrderedDisjoint; @@ -125,14 +126,9 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, */ private volatile boolean closed; /** - * wrappedStream is associated with an object (instance of S3Object). When - * the object is garbage collected, the associated wrappedStream will be - * closed. Keep a reference to this object to prevent the wrapperStream - * still in use from being closed unexpectedly due to garbage collection. - * See HADOOP-17338 for details. + * Input stream returned by a getObject call. */ - private S3Object object; - private S3ObjectInputStream wrappedStream; + private ResponseInputStream wrappedStream; private final S3AReadOpContext context; private final InputStreamCallbacks client; @@ -271,28 +267,22 @@ private synchronized void reopen(String reason, long targetPos, long length, uri, reason, targetPos, contentRangeFinish, length, pos, nextReadPos, inputPolicy); + GetObjectRequest request = client.newGetRequestBuilder(key) + .range(S3AUtils.formatRange(targetPos, contentRangeFinish - 1)) + .applyMutation(changeTracker::maybeApplyConstraint) + .build(); long opencount = streamStatistics.streamOpened(); - GetObjectRequest request = client.newGetRequest(key) - .withRange(targetPos, contentRangeFinish - 1); String operation = opencount == 0 ? OPERATION_OPEN : OPERATION_REOPEN; String text = String.format("%s %s at %d", operation, uri, targetPos); - changeTracker.maybeApplyConstraint(request); - - object = onceTrackingDuration(text, uri, + wrappedStream = onceTrackingDuration(text, uri, streamStatistics.initiateGetRequest(), () -> client.getObject(request)); - - changeTracker.processResponse(object, operation, + changeTracker.processResponse(wrappedStream.response(), operation, targetPos); - wrappedStream = object.getObjectContent(); - contentRangeStart = targetPos; - if (wrappedStream == null) { - throw new PathIOException(uri, - "Null IO stream from " + operation + " of (" + reason + ") "); - } + contentRangeStart = targetPos; this.pos = targetPos; } @@ -505,14 +495,15 @@ public synchronized int read() throws IOException { */ @Retries.OnceTranslated private void onReadFailure(IOException ioe, boolean forceAbort) { + GetObjectResponse objectResponse = wrappedStream == null ? null : wrappedStream.response(); if (LOG.isDebugEnabled()) { LOG.debug("Got exception while trying to read from stream {}, " + "client: {} object: {}, trying to recover: ", - uri, client, object, ioe); + uri, client, objectResponse, ioe); } else { LOG.info("Got exception while trying to read from stream {}, " + "client: {} object: {}, trying to recover: " + ioe, - uri, client, object); + uri, client, objectResponse); } streamStatistics.readException(); closeStream("failure recovery", forceAbort, false); @@ -672,7 +663,6 @@ private CompletableFuture closeStream( CompletableFuture operation; SDKStreamDrainer drainer = new SDKStreamDrainer( uri, - object, wrappedStream, shouldAbort, (int) remaining, @@ -694,7 +684,6 @@ private CompletableFuture closeStream( // either the stream is closed in the blocking call or the async call is // submitted with its own copy of the references wrappedStream = null; - object = null; return operation; } @@ -910,23 +899,19 @@ public void readVectored(List ranges, private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRange, IntFunction allocate) { LOG.debug("Start reading combined range {} from path {} ", combinedFileRange, pathStr); - // This reference must be kept till all buffers are populated as this is a - // finalizable object which closes the internal stream when gc triggers. - S3Object objectRange = null; - S3ObjectInputStream objectContent = null; + ResponseInputStream rangeContent = null; try { - objectRange = getS3ObjectAndValidateNotNull("readCombinedFileRange", + rangeContent = getS3ObjectInputStream("readCombinedFileRange", combinedFileRange.getOffset(), combinedFileRange.getLength()); - objectContent = objectRange.getObjectContent(); - populateChildBuffers(combinedFileRange, objectContent, allocate); + populateChildBuffers(combinedFileRange, rangeContent, allocate); } catch (Exception ex) { LOG.debug("Exception while reading a range {} from path {} ", combinedFileRange, pathStr, ex); for(FileRange child : combinedFileRange.getUnderlying()) { child.getData().completeExceptionally(ex); } } finally { - IOUtils.cleanupWithLogger(LOG, objectRange, objectContent); + IOUtils.cleanupWithLogger(LOG, rangeContent); } LOG.debug("Finished reading range {} from path {} ", combinedFileRange, pathStr); } @@ -939,7 +924,7 @@ private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRa * @throws IOException any IOE. */ private void populateChildBuffers(CombinedFileRange combinedFileRange, - S3ObjectInputStream objectContent, + InputStream objectContent, IntFunction allocate) throws IOException { // If the combined file range just contains a single child // range, we only have to fill that one child buffer else @@ -971,7 +956,7 @@ private void populateChildBuffers(CombinedFileRange combinedFileRange, * @param drainQuantity how many bytes to drain. * @throws IOException any IOE. */ - private void drainUnnecessaryData(S3ObjectInputStream objectContent, long drainQuantity) + private void drainUnnecessaryData(InputStream objectContent, long drainQuantity) throws IOException { int drainBytes = 0; int readCount; @@ -1013,28 +998,24 @@ private void validateRangeRequest(FileRange range) throws EOFException { */ private void readSingleRange(FileRange range, ByteBuffer buffer) { LOG.debug("Start reading range {} from path {} ", range, pathStr); - // This reference must be kept till all buffers are populated as this is a - // finalizable object which closes the internal stream when gc triggers. - S3Object objectRange = null; - S3ObjectInputStream objectContent = null; + ResponseInputStream objectRange = null; try { long position = range.getOffset(); int length = range.getLength(); - objectRange = getS3ObjectAndValidateNotNull("readSingleRange", position, length); - objectContent = objectRange.getObjectContent(); - populateBuffer(length, buffer, objectContent); + objectRange = getS3ObjectInputStream("readSingleRange", position, length); + populateBuffer(length, buffer, objectRange); range.getData().complete(buffer); } catch (Exception ex) { LOG.warn("Exception while reading a range {} from path {} ", range, pathStr, ex); range.getData().completeExceptionally(ex); } finally { - IOUtils.cleanupWithLogger(LOG, objectRange, objectContent); + IOUtils.cleanupWithLogger(LOG, objectRange); } LOG.debug("Finished reading range {} from path {} ", range, pathStr); } /** - * Get the s3 object for S3 server for a specified range. + * Get the s3 object input stream for S3 server for a specified range. * Also checks if the vectored io operation has been stopped before and after * the http get request such that we don't waste time populating the buffers. * @param operationName name of the operation for which get object on S3 is called. @@ -1043,15 +1024,11 @@ private void readSingleRange(FileRange range, ByteBuffer buffer) { * @return result s3 object. * @throws IOException exception if any. */ - private S3Object getS3ObjectAndValidateNotNull(final String operationName, - final long position, - final int length) throws IOException { + private ResponseInputStream getS3ObjectInputStream( + final String operationName, final long position, final int length) throws IOException { checkIfVectoredIOStopped(); - S3Object objectRange = getS3Object(operationName, position, length); - if (objectRange.getObjectContent() == null) { - throw new PathIOException(uri, - "Null IO stream received during " + operationName); - } + ResponseInputStream objectRange = + getS3Object(operationName, position, length); checkIfVectoredIOStopped(); return objectRange; } @@ -1066,7 +1043,7 @@ private S3Object getS3ObjectAndValidateNotNull(final String operationName, */ private void populateBuffer(int length, ByteBuffer buffer, - S3ObjectInputStream objectContent) throws IOException { + InputStream objectContent) throws IOException { if (buffer.isDirect()) { VectoredReadUtils.readInDirectBuffer(length, buffer, @@ -1091,7 +1068,7 @@ private void populateBuffer(int length, * @param length number of bytes to fill in dest. * @throws IOException any IOE. */ - private void readByteArray(S3ObjectInputStream objectContent, + private void readByteArray(InputStream objectContent, byte[] dest, int offset, int length) throws IOException { @@ -1118,13 +1095,16 @@ private void readByteArray(S3ObjectInputStream objectContent, * @return S3Object result s3 object. * @throws IOException exception if any. */ - private S3Object getS3Object(String operationName, long position, - int length) throws IOException { - final GetObjectRequest request = client.newGetRequest(key) - .withRange(position, position + length - 1); - changeTracker.maybeApplyConstraint(request); + private ResponseInputStream getS3Object(String operationName, + long position, + int length) + throws IOException { + final GetObjectRequest request = client.newGetRequestBuilder(key) + .range(S3AUtils.formatRange(position, position + length - 1)) + .applyMutation(changeTracker::maybeApplyConstraint) + .build(); DurationTracker tracker = streamStatistics.initiateGetRequest(); - S3Object objectRange; + ResponseInputStream objectRange; Invoker invoker = context.getReadInvoker(); try { objectRange = invoker.retry(operationName, pathStr, true, @@ -1139,7 +1119,7 @@ private S3Object getS3Object(String operationName, long position, } finally { tracker.close(); } - changeTracker.processResponse(objectRange, operationName, + changeTracker.processResponse(objectRange.response(), operationName, position); return objectRange; } @@ -1293,11 +1273,11 @@ public IOStatistics getIOStatistics() { public interface InputStreamCallbacks extends Closeable { /** - * Create a GET request. + * Create a GET request builder. * @param key object key - * @return the request + * @return the request builder */ - GetObjectRequest newGetRequest(String key); + GetObjectRequest.Builder newGetRequestBuilder(String key); /** * Execute the request. @@ -1305,7 +1285,7 @@ public interface InputStreamCallbacks extends Closeable { * @return the response */ @Retries.OnceRaw - S3Object getObject(GetObjectRequest request); + ResponseInputStream getObject(GetObjectRequest request); /** * Submit some asynchronous work, for example, draining a stream. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInternals.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInternals.java new file mode 100644 index 0000000000000..18d6c1af586ff --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInternals.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.IOException; +import java.nio.file.AccessDeniedException; + +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.HeadBucketResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.store.audit.AuditEntryPoint; + +/** + * This is an unstable interface for access to S3A Internal state, S3 operations + * and the S3 client connector itself. + */ +@InterfaceStability.Unstable +@InterfaceAudience.LimitedPrivate("testing/diagnostics") +public interface S3AInternals { + + /** + * Returns the S3 client used by this filesystem. + * Will log at debug. + *

+ * Warning + * This bypasses core S3A operations, including auditing. + * It is intended for use in testing, diagnostics and for accessing + * operations not available through the S3A connector itself. + *

+ * Unless audit spans are created through the S3AFileSystem, make + * sure that {@code fs.s3a.audit.reject.out.of.span.operations} is + * set to false. + *

+ * Mocking note: this is the same S3Client as is used by the owning + * filesystem; changes to this client will be reflected by changes + * in the behavior of that filesystem. + * @param reason a justification for requesting access. + * @return S3Client + */ + S3Client getAmazonS3Client(String reason); + + /** + * Get the region of a bucket. + * Invoked from StoreContext; consider an entry point. + * @return the region in which a bucket is located + * @throws AccessDeniedException if the caller lacks permission. + * @throws IOException on any failure. + */ + @Retries.RetryTranslated + @AuditEntryPoint + String getBucketLocation() throws IOException; + + /** + * Get the region of a bucket; fixing up the region so it can be used + * in the builders of other AWS clients. + * Requires the caller to have the AWS role permission + * {@code s3:GetBucketLocation}. + * Retry policy: retrying, translated. + * @param bucketName the name of the bucket + * @return the region in which a bucket is located + * @throws AccessDeniedException if the caller lacks permission. + * @throws IOException on any failure. + */ + @AuditEntryPoint + @Retries.RetryTranslated + String getBucketLocation(String bucketName) throws IOException; + + /** + * Low-level call to get at the object metadata. + * Auditing: An audit entry point. + * @param path path to the object. This will be qualified. + * @return metadata + * @throws IOException IO and object access problems. + */ + @AuditEntryPoint + @Retries.RetryTranslated + HeadObjectResponse getObjectMetadata(Path path) throws IOException; + + /** + * Get a shared copy of the AWS credentials, with its reference + * counter updated. + * Caller is required to call {@code close()} on this after + * they have finished using it. + * @param purpose what is this for? This is for logging + * @return a reference to shared credentials. + */ + AWSCredentialProviderList shareCredentials(String purpose); + + /** + * Request bucket metadata. + * @return the metadata + * @throws UnknownStoreException the bucket is absent + * @throws IOException any other problem talking to S3 + */ + @AuditEntryPoint + @Retries.RetryTranslated + HeadBucketResponse getBucketMetadata() throws IOException; + + /** + * Is multipart copy enabled? + * @return true if the transfer manager is used to copy files. + */ + boolean isMultipartCopyEnabled(); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java index 528a99f5e0966..cc30da4fbe04e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java @@ -22,7 +22,10 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InterruptedIOException; +import java.net.BindException; +import java.net.ConnectException; import java.net.NoRouteToHostException; +import java.net.SocketException; import java.net.SocketTimeoutException; import java.net.UnknownHostException; import java.nio.file.AccessDeniedException; @@ -30,7 +33,7 @@ import java.util.Map; import java.util.concurrent.TimeUnit; -import com.amazonaws.AmazonClientException; +import software.amazon.awssdk.core.exception.SdkException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,15 +45,16 @@ import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.net.ConnectTimeoutException; -import org.apache.hadoop.util.Preconditions; -import static org.apache.hadoop.io.retry.RetryPolicies.*; +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.io.retry.RetryPolicies.*; import static org.apache.hadoop.fs.s3a.Constants.*; + /** * The S3A request retry policy. - * + *

* This uses the retry options in the configuration file to determine retry * count and delays for "normal" retries and separately, for throttling; * the latter is best handled for longer with an exponential back-off. @@ -65,20 +69,25 @@ * For non-idempotent operations, only failures due to throttling or * from failures which are known to only arise prior to talking to S3 * are retried. - * + *

* The retry policy is all built around that of the normal IO exceptions, * particularly those extracted from - * {@link S3AUtils#translateException(String, Path, AmazonClientException)}. + * {@link S3AUtils#translateException(String, Path, SdkException)}. * Because the {@link #shouldRetry(Exception, int, int, boolean)} method - * does this translation if an {@code AmazonClientException} is processed, + * does this translation if an {@code SdkException} is processed, * the policy defined for the IOEs also applies to the original exceptions. - * + *

* Put differently: this retry policy aims to work for handlers of the * untranslated exceptions, as well as the translated ones. + *

+ * Note that because delete is considered idempotent, all s3a operations currently + * declare themselves idempotent. + * This means the retry policy here is more complex than it needs to be -but it + * does force us to consider when retrying operations would not be safe. * @see S3 Error responses - * @see Amazon S3 Error Best Practices + * @see Amazon S3 Error Best Practices */ -@SuppressWarnings("visibilitymodifier") // I want a struct of finals, for real. +@SuppressWarnings("visibilitymodifier") // we want a struct of finals, for real. public class S3ARetryPolicy implements RetryPolicy { private static final Logger LOG = LoggerFactory.getLogger( @@ -111,13 +120,17 @@ public class S3ARetryPolicy implements RetryPolicy { */ protected final RetryPolicy connectivityFailure; + /** + * Handling of AWSClientIOException and subclasses. + */ + protected final RetryPolicy retryAwsClientExceptions; + /** * Instantiate. * @param conf configuration to read. */ public S3ARetryPolicy(Configuration conf) { - Preconditions.checkArgument(conf != null, "Null configuration"); - this.configuration = conf; + this.configuration = requireNonNull(conf, "Null configuration"); // base policy from configuration int limit = conf.getInt(RETRY_LIMIT, RETRY_LIMIT_DEFAULT); @@ -138,6 +151,11 @@ public S3ARetryPolicy(Configuration conf) { retryIdempotentCalls = new FailNonIOEs( new IdempotencyRetryFilter(baseExponentialRetry)); + // retry on AWSClientIOException and possibly subclasses; + // See: HADOOP-18871. S3ARetryPolicy to use sdk exception retryable() if it is valid + // currently the normal retryIdempotentCalls policy is used. + retryAwsClientExceptions = retryIdempotentCalls; + // and a separate policy for throttle requests, which are considered // repeatable, even for non-idempotent calls, as the service // rejected the call entirely @@ -177,35 +195,57 @@ protected Map, RetryPolicy> createExceptionMap() { // inherit policies. Map, RetryPolicy> policyMap = new HashMap<>(); - // failfast exceptions which we consider unrecoverable - policyMap.put(UnknownHostException.class, fail); - policyMap.put(NoRouteToHostException.class, fail); - policyMap.put(InterruptedException.class, fail); - // note this does not pick up subclasses (like socket timeout) - policyMap.put(InterruptedIOException.class, fail); // Access denial and auth exceptions are not retried policyMap.put(AccessDeniedException.class, fail); - policyMap.put(NoAuthWithAWSException.class, fail); - policyMap.put(FileNotFoundException.class, fail); - policyMap.put(UnknownStoreException.class, fail); - policyMap.put(InvalidRequestException.class, fail); - // once the file has changed, trying again is not going to help - policyMap.put(RemoteFileChangedException.class, fail); + // Treated as an immediate failure + policyMap.put(AWSBadRequestException.class, fail); - // likely only recovered by changing the policy configuration or s3 - // implementation - policyMap.put(NoVersionAttributeException.class, fail); + // use specific retry policy for aws client exceptions + // nested IOExceptions will already have been extracted and used + // in this map. + policyMap.put(AWSClientIOException.class, retryAwsClientExceptions); + + // server didn't respond. + policyMap.put(AWSNoResponseException.class, retryIdempotentCalls); // should really be handled by resubmitting to new location; // that's beyond the scope of this retry policy policyMap.put(AWSRedirectException.class, fail); + // generic exception from the service + policyMap.put(AWSServiceIOException.class, retryAwsClientExceptions); + // throttled requests are can be retried, always policyMap.put(AWSServiceThrottledException.class, throttlePolicy); + // Status 5xx error code is an immediate failure + // this is sign of a server-side problem, and while + // rare in AWS S3, it does happen on third party stores. + // (out of disk space, etc). + // by the time we get here, the aws sdk will have + // already retried. + // there is specific handling for some 5XX codes (501, 503); + // this is for everything else + policyMap.put(AWSStatus500Exception.class, fail); + + // subclass of AWSServiceIOException whose cause is always S3Exception + policyMap.put(AWSS3IOException.class, retryIdempotentCalls); + + // server doesn't support a feature. + // raised from a number of HTTP error codes -mostly from + // third-party stores which only support a subset of AWS S3 + // operations. + policyMap.put(AWSUnsupportedFeatureException.class, fail); + + // socket exception subclass we consider unrecoverable + // though this is normally only found when opening a port for listening. + // which is never done in S3A. + policyMap.put(BindException.class, fail); + // connectivity problems are retried without worrying about idempotency policyMap.put(ConnectTimeoutException.class, connectivityFailure); + policyMap.put(ConnectException.class, connectivityFailure); // this can be a sign of an HTTP connection breaking early. // which can be reacted to by another attempt if the request was idempotent. @@ -213,25 +253,38 @@ protected Map, RetryPolicy> createExceptionMap() { // which isn't going to be recovered from policyMap.put(EOFException.class, retryIdempotentCalls); - // policy on a 400/bad request still ambiguous. - // Treated as an immediate failure - policyMap.put(AWSBadRequestException.class, fail); + // object not found. 404 when not unknown bucket; 410 "gone" + policyMap.put(FileNotFoundException.class, fail); - // Status 500 error code is also treated as a connectivity problem - policyMap.put(AWSStatus500Exception.class, connectivityFailure); + // Interrupted, usually by other threads + policyMap.put(InterruptedException.class, fail); + // note this does not pick up subclasses (like socket timeout) + policyMap.put(InterruptedIOException.class, fail); + policyMap.put(InvalidRequestException.class, fail); - // server didn't respond. - policyMap.put(AWSNoResponseException.class, retryIdempotentCalls); + // auth failure. Possibly recoverable by reattempting with + // the credential provider, but not covered here. + policyMap.put(NoAuthWithAWSException.class, fail); - // other operations - policyMap.put(AWSClientIOException.class, retryIdempotentCalls); - policyMap.put(AWSServiceIOException.class, retryIdempotentCalls); - policyMap.put(AWSS3IOException.class, retryIdempotentCalls); - policyMap.put(SocketTimeoutException.class, retryIdempotentCalls); + // network routing. + policyMap.put(NoRouteToHostException.class, fail); + // likely only recovered by changing the policy configuration or s3 + // implementation + policyMap.put(NoVersionAttributeException.class, fail); + // once the file has changed, trying again is not going to help + policyMap.put(RemoteFileChangedException.class, fail); + // general socket exceptions + policyMap.put(SocketException.class, connectivityFailure); + policyMap.put(SocketTimeoutException.class, connectivityFailure); + + // assume that DNS wil not recover; SDK is likely to have retried. + policyMap.put(UnknownHostException.class, fail); + policyMap.put(UnknownStoreException.class, fail); // Unsupported requests do not work, however many times you try policyMap.put(UnsupportedRequestException.class, fail); + return policyMap; } @@ -240,15 +293,19 @@ public RetryAction shouldRetry(Exception exception, int retries, int failovers, boolean idempotent) throws Exception { - Preconditions.checkArgument(exception != null, "Null exception"); - Exception ex = exception; - if (exception instanceof AmazonClientException) { - // uprate the amazon client exception for the purpose of exception + Exception ex = requireNonNull(exception, "Null exception"); + if (exception instanceof SdkException) { + // update the sdk exception for the purpose of exception // processing. - ex = S3AUtils.translateException("", "", - (AmazonClientException) exception); + ex = S3AUtils.translateException("", "", (SdkException) exception); } - return retryPolicy.shouldRetry(ex, retries, failovers, idempotent); + LOG.debug("Retry probe for {} with {} retries and {} failovers," + + " idempotent={}, due to {}", + ex.getClass().getSimpleName(), retries, failovers, idempotent, ex, ex); + // look in the retry policy map + final RetryAction action = retryPolicy.shouldRetry(ex, retries, failovers, idempotent); + LOG.debug("Retry action is {}", action); + return action; } /** @@ -315,4 +372,28 @@ public RetryAction shouldRetry(Exception e, } } + /** + * Policy where AWS SDK exceptions are retried if they state that they are retryable. + * See HADOOP-18871. S3ARetryPolicy to use sdk exception retryable() if it is valid. + */ + private static final class RetryFromAWSClientExceptionPolicy implements RetryPolicy { + + private final RetryPolicy next; + + private RetryFromAWSClientExceptionPolicy(RetryPolicy next) { + this.next = next; + } + + @Override + public RetryAction shouldRetry(Exception e, + int retries, + int failovers, + boolean isIdempotentOrAtMostOnce) throws Exception { + return + e instanceof AWSClientIOException ? + next.shouldRetry(e, retries, failovers, ((AWSClientIOException)e).retryable()) + : RetryAction.FAIL; + } + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java index dc0c211fcab50..f3e23f1cbf9fb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java @@ -18,20 +18,12 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.AbortedException; -import com.amazonaws.AmazonClientException; -import com.amazonaws.AmazonServiceException; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.Protocol; -import com.amazonaws.SdkBaseException; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.EnvironmentVariableCredentialsProvider; -import com.amazonaws.retry.RetryUtils; -import com.amazonaws.services.s3.model.AmazonS3Exception; -import com.amazonaws.services.s3.model.MultiObjectDeleteException; -import com.amazonaws.services.s3.model.S3ObjectSummary; -import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.AbortedException; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.core.retry.RetryUtils; +import software.amazon.awssdk.services.s3.model.S3Exception; +import software.amazon.awssdk.services.s3.model.S3Object; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; @@ -44,16 +36,12 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.util.functional.RemoteIterators; -import org.apache.hadoop.fs.s3a.audit.AuditFailureException; -import org.apache.hadoop.fs.s3a.audit.AuditIntegration; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; -import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider; -import org.apache.hadoop.fs.s3a.impl.NetworkBinding; -import org.apache.hadoop.fs.s3a.impl.V2Migration; +import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteException; import org.apache.hadoop.fs.s3native.S3xLoginHelper; import org.apache.hadoop.net.ConnectTimeoutException; import org.apache.hadoop.security.ProviderUtils; -import org.apache.hadoop.util.VersionInfo; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; import org.slf4j.Logger; @@ -74,23 +62,25 @@ 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.CompletionException; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; import static org.apache.commons.lang3.StringUtils.isEmpty; +import static org.apache.hadoop.fs.s3a.AWSCredentialProviderList.maybeTranslateCredentialException; import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.maybeTranslateAuditException; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH; -import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.translateDeleteException; +import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.instantiationException; +import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.isAbstract; +import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.isNotInstanceOf; +import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.unsupportedConstructor; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.*; +import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.maybeExtractIOException; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator; @@ -102,13 +92,7 @@ public final class S3AUtils { private static final Logger LOG = LoggerFactory.getLogger(S3AUtils.class); - static final String CONSTRUCTOR_EXCEPTION = "constructor exception"; - static final String INSTANTIATION_EXCEPTION - = "instantiation exception"; - static final String NOT_AWS_PROVIDER = - "does not implement AWSCredentialsProvider"; - static final String ABSTRACT_PROVIDER = - "is abstract and therefore cannot be created"; + static final String ENDPOINT_KEY = "Endpoint"; /** Filesystem is closed; kept here to keep the errors close. */ @@ -143,21 +127,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() { } /** * Translate an exception raised in an operation into an IOException. * The specific type of IOException depends on the class of - * {@link AmazonClientException} passed in, and any status codes included + * {@link SdkException} passed in, and any status codes included * in the operation. That is: HTTP error codes are examined and can be * used to build a more specific response. * @@ -170,14 +146,14 @@ private S3AUtils() { */ public static IOException translateException(String operation, Path path, - AmazonClientException exception) { + SdkException exception) { return translateException(operation, path.toString(), exception); } /** * Translate an exception raised in an operation into an IOException. * The specific type of IOException depends on the class of - * {@link AmazonClientException} passed in, and any status codes included + * {@link SdkException} passed in, and any status codes included * in the operation. That is: HTTP error codes are examined and can be * used to build a more specific response. * @param operation operation @@ -188,12 +164,14 @@ public static IOException translateException(String operation, @SuppressWarnings("ThrowableInstanceNeverThrown") public static IOException translateException(@Nullable String operation, String path, - SdkBaseException exception) { + SdkException exception) { String message = String.format("%s%s: %s", operation, StringUtils.isNotEmpty(path)? (" on " + path) : "", exception); - if (!(exception instanceof AmazonServiceException)) { + if (!(exception instanceof AwsServiceException)) { + // exceptions raised client-side: connectivity, auth, network problems... + Exception innerCause = containsInterruptedException(exception); if (innerCause != null) { // interrupted IO, or a socket exception underneath that class @@ -205,62 +183,72 @@ public static IOException translateException(@Nullable String operation, } // if the exception came from the auditor, hand off translation // to it. - if (exception instanceof AuditFailureException) { - return AuditIntegration.translateAuditException(path, (AuditFailureException) exception); + IOException ioe = maybeTranslateAuditException(path, exception); + if (ioe != null) { + return ioe; } - if (exception instanceof CredentialInitializationException) { - // the exception raised by AWSCredentialProvider list if the - // credentials were not accepted, - return (AccessDeniedException)new AccessDeniedException(path, null, - exception.toString()).initCause(exception); + ioe = maybeTranslateCredentialException(path, exception); + if (ioe != null) { + return ioe; } + // network problems covered by an IOE inside the exception chain. + ioe = maybeExtractIOException(path, exception); + if (ioe != null) { + return ioe; + } + // no custom handling. return new AWSClientIOException(message, exception); } else { + // "error response returned by an S3 or other service." + // These contain more details and should be translated based + // on the HTTP status code and other details. IOException ioe; - AmazonServiceException ase = (AmazonServiceException) exception; + AwsServiceException ase = (AwsServiceException) exception; // this exception is non-null if the service exception is an s3 one - AmazonS3Exception s3Exception = ase instanceof AmazonS3Exception - ? (AmazonS3Exception) ase + S3Exception s3Exception = ase instanceof S3Exception + ? (S3Exception) ase : null; - int status = ase.getStatusCode(); - message = message + ":" + ase.getErrorCode(); + int status = ase.statusCode(); + if (ase.awsErrorDetails() != null) { + message = message + ":" + ase.awsErrorDetails().errorCode(); + } + + // big switch on the HTTP status code. switch (status) { - case 301: - case 307: + case SC_301_MOVED_PERMANENTLY: + case SC_307_TEMPORARY_REDIRECT: if (s3Exception != null) { - if (s3Exception.getAdditionalDetails() != null && - s3Exception.getAdditionalDetails().containsKey(ENDPOINT_KEY)) { - message = String.format("Received permanent redirect response to " - + "endpoint %s. This likely indicates that the S3 endpoint " - + "configured in %s does not match the AWS region containing " - + "the bucket.", - s3Exception.getAdditionalDetails().get(ENDPOINT_KEY), ENDPOINT); - } + message = String.format("Received permanent redirect response to " + + "region %s. This likely indicates that the S3 region " + + "configured in %s does not match the AWS region containing " + "the bucket.", + s3Exception.awsErrorDetails().sdkHttpResponse().headers().get(BUCKET_REGION_HEADER), + AWS_REGION); ioe = new AWSRedirectException(message, s3Exception); } else { ioe = new AWSRedirectException(message, ase); } break; - case 400: + case SC_400_BAD_REQUEST: ioe = new AWSBadRequestException(message, ase); break; // permissions - case 401: - case 403: + case SC_401_UNAUTHORIZED: + case SC_403_FORBIDDEN: ioe = new AccessDeniedException(path, null, message); ioe.initCause(ase); break; // the object isn't there - case 404: + case SC_404_NOT_FOUND: if (isUnknownBucket(ase)) { // this is a missing bucket ioe = new UnknownStoreException(path, message, ase); } else { - // a normal unknown object + // a normal unknown object. + // Can also be raised by third-party stores when aborting an unknown multipart upload ioe = new FileNotFoundException(message); ioe.initCause(ase); } @@ -268,20 +256,23 @@ public static IOException translateException(@Nullable String operation, // this also surfaces sometimes and is considered to // be ~ a not found exception. - case 410: + case SC_410_GONE: ioe = new FileNotFoundException(message); ioe.initCause(ase); break; - // method not allowed; seen on S3 Select. - // treated as a bad request - case 405: - ioe = new AWSBadRequestException(message, s3Exception); + // errors which stores can return from requests which + // the store does not support. + case SC_405_METHOD_NOT_ALLOWED: + case SC_412_PRECONDITION_FAILED: + case SC_415_UNSUPPORTED_MEDIA_TYPE: + case SC_501_NOT_IMPLEMENTED: + ioe = new AWSUnsupportedFeatureException(message, s3Exception); break; // out of range. This may happen if an object is overwritten with // a shorter one while it is being read. - case 416: + case SC_416_RANGE_NOT_SATISFIABLE: ioe = new EOFException(message); ioe.initCause(ase); break; @@ -289,32 +280,40 @@ public static IOException translateException(@Nullable String operation, // this has surfaced as a "no response from server" message. // so rare we haven't replicated it. // Treating as an idempotent proxy error. - case 443: - case 444: + case SC_443_NO_RESPONSE: + case SC_444_NO_RESPONSE: ioe = new AWSNoResponseException(message, ase); break; // throttling - case 503: + case SC_429_TOO_MANY_REQUESTS_GCS: // google cloud through this connector + case SC_503_SERVICE_UNAVAILABLE: // AWS ioe = new AWSServiceThrottledException(message, ase); break; // internal error - case 500: + case SC_500_INTERNAL_SERVER_ERROR: ioe = new AWSStatus500Exception(message, ase); break; - case 200: + case SC_200_OK: if (exception instanceof MultiObjectDeleteException) { // failure during a bulk delete - return translateDeleteException(message, - (MultiObjectDeleteException) exception); + return ((MultiObjectDeleteException) exception) + .translateException(message); } // other 200: FALL THROUGH default: - // no specific exit code. Choose an IOE subclass based on the class - // of the caught exception + // no specifically handled exit code. + + // convert all unknown 500+ errors to a 500 exception + if (status > SC_500_INTERNAL_SERVER_ERROR) { + ioe = new AWSStatus500Exception(message, ase); + break; + } + + // Choose an IOE subclass based on the class of the caught exception ioe = s3Exception != null ? new AWSS3IOException(message, s3Exception) : new AWSServiceIOException(message, ase); @@ -334,10 +333,35 @@ public static IOException translateException(@Nullable String operation, public static IOException extractException(String operation, String path, ExecutionException ee) { + return convertExceptionCause(operation, path, ee.getCause()); + } + + /** + * Extract an exception from a failed future, and convert to an IOE. + * @param operation operation which failed + * @param path path operated on (may be null) + * @param ce completion exception + * @return an IOE which can be thrown + */ + public static IOException extractException(String operation, + String path, + CompletionException ce) { + return convertExceptionCause(operation, path, ce.getCause()); + } + + /** + * Convert the cause of a concurrent exception to an IOE. + * @param operation operation which failed + * @param path path operated on (may be null) + * @param cause cause of a concurrent exception + * @return an IOE which can be thrown + */ + private static IOException convertExceptionCause(String operation, + String path, + Throwable cause) { IOException ioe; - Throwable cause = ee.getCause(); - if (cause instanceof AmazonClientException) { - ioe = translateException(operation, path, (AmazonClientException) cause); + if (cause instanceof SdkException) { + ioe = translateException(operation, path, (SdkException) cause); } else if (cause instanceof IOException) { ioe = (IOException) cause; } else { @@ -375,7 +399,7 @@ static Exception containsInterruptedException(Throwable thrown) { * @return an IOE which can be rethrown */ private static InterruptedIOException translateInterruptedException( - SdkBaseException exception, + SdkException exception, final Exception innerCause, String message) { InterruptedIOException ioe; @@ -386,6 +410,7 @@ private static InterruptedIOException translateInterruptedException( if (name.endsWith(".ConnectTimeoutException") || name.endsWith(".ConnectionPoolTimeoutException") || name.endsWith("$ConnectTimeoutException")) { + // TODO: review in v2 // TCP connection http timeout from the shaded or unshaded filenames // com.amazonaws.thirdparty.apache.http.conn.ConnectTimeoutException ioe = new ConnectTimeoutException(message); @@ -409,10 +434,10 @@ private static InterruptedIOException translateInterruptedException( */ public static boolean isThrottleException(Exception ex) { return ex instanceof AWSServiceThrottledException - || (ex instanceof AmazonServiceException - && 503 == ((AmazonServiceException)ex).getStatusCode()) - || (ex instanceof SdkBaseException - && RetryUtils.isThrottlingException((SdkBaseException) ex)); + || (ex instanceof AwsServiceException + && 503 == ((AwsServiceException)ex).statusCode()) + || (ex instanceof SdkException + && RetryUtils.isThrottlingException((SdkException) ex)); } /** @@ -422,7 +447,8 @@ public static boolean isThrottleException(Exception ex) { * @param ex exception * @return true if this is believed to be a sign the connection was broken. */ - public static boolean isMessageTranslatableToEOF(SdkBaseException ex) { + public static boolean isMessageTranslatableToEOF(SdkException ex) { + // TODO: review in v2 return ex.toString().contains(EOF_MESSAGE_IN_XML_PARSER) || ex.toString().contains(EOF_READ_DIFFERENT_LENGTH); } @@ -432,47 +458,26 @@ public static boolean isMessageTranslatableToEOF(SdkBaseException ex) { * @param e exception * @return string details */ - public static String stringify(AmazonServiceException e) { + public static String stringify(AwsServiceException e) { StringBuilder builder = new StringBuilder( - String.format("%s: %s error %d: %s; %s%s%n", - e.getErrorType(), - e.getServiceName(), - e.getStatusCode(), - e.getErrorCode(), - e.getErrorMessage(), - (e.isRetryable() ? " (retryable)": "") + String.format("%s error %d: %s; %s%s%n", + e.awsErrorDetails().serviceName(), + e.statusCode(), + e.awsErrorDetails().errorCode(), + e.awsErrorDetails().errorMessage(), + (e.retryable() ? " (retryable)": "") )); - String rawResponseContent = e.getRawResponseContent(); + String rawResponseContent = e.awsErrorDetails().rawResponse().asUtf8String(); if (rawResponseContent != null) { builder.append(rawResponseContent); } return builder.toString(); } - /** - * Get low level details of an amazon exception for logging; multi-line. - * @param e exception - * @return string details - */ - public static String stringify(AmazonS3Exception e) { - // get the low level details of an exception, - StringBuilder builder = new StringBuilder( - stringify((AmazonServiceException) e)); - Map details = e.getAdditionalDetails(); - if (details != null) { - builder.append('\n'); - for (Map.Entry d : details.entrySet()) { - builder.append(d.getKey()).append('=') - .append(d.getValue()).append('\n'); - } - } - return builder.toString(); - } - /** * Create a files status instance from a listing. * @param keyPath path to entry - * @param summary summary from AWS + * @param s3Object s3Object entry * @param blockSize block size to declare. * @param owner owner of the file * @param eTag S3 object eTag or null if unavailable @@ -481,20 +486,20 @@ public static String stringify(AmazonS3Exception e) { * @return a status entry */ public static S3AFileStatus createFileStatus(Path keyPath, - S3ObjectSummary summary, + S3Object s3Object, long blockSize, String owner, String eTag, String versionId, boolean isCSEEnabled) { - long size = summary.getSize(); + long size = s3Object.size(); // check if cse is enabled; strip out constant padding length. if (isCSEEnabled && size >= CSE_PADDING_LENGTH) { size -= CSE_PADDING_LENGTH; } return createFileStatus(keyPath, - objectRepresentsDirectory(summary.getKey()), - size, summary.getLastModified(), blockSize, owner, eTag, versionId); + objectRepresentsDirectory(s3Object.key()), + size, Date.from(s3Object.lastModified()), blockSize, owner, eTag, versionId); } /** @@ -556,114 +561,7 @@ public static long dateToLong(final Date date) { } /** - * The standard AWS provider list for AWS connections. - */ - @SuppressWarnings("deprecation") - public static final List> - 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 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( - @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 {}", - binding, credentials); - return credentials; - } - - /** - * Load list of AWS credential provider/credential provider factory classes. - * @param conf configuration - * @param key key - * @param defaultValue list of default values - * @return the list of classes, possibly empty - * @throws IOException on a failure to load the list. - */ - public static List> loadAWSProviderClasses(Configuration conf, - String key, - Class... defaultValue) throws IOException { - try { - 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> defaultValues, - final Set> forbidden) throws IOException { - - // build up the base provider - List> 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) { - - // List of V1 credential providers that will be migrated with V2 upgrade - if (!Arrays.asList("EnvironmentVariableCredentialsProvider", - "EC2ContainerCredentialsProviderWrapper", "InstanceProfileCredentialsProvider") - .contains(aClass.getSimpleName()) && aClass.getName().contains(AWS_AUTH_CLASS_PREFIX)) { - V2Migration.v1ProviderReferenced(aClass.getName()); - } - - 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 + * Creates an instance of a class using reflection. The * class must implement one of the following means of construction, which are * attempted in order: * @@ -672,92 +570,87 @@ public static AWSCredentialProviderList buildAWSProviderList( * org.apache.hadoop.conf.Configuration *

  • a public constructor accepting * org.apache.hadoop.conf.Configuration
  • - *
  • a public static method named getInstance that accepts no + *
  • a public static method named as per methodName, that accepts no * arguments and returns an instance of - * com.amazonaws.auth.AWSCredentialsProvider, or
  • + * specified type, or *
  • a public default constructor.
  • * * + * @param className name of class for which instance is to be created * @param conf configuration - * @param credClass credential class * @param uri URI of the FS - * @return the instantiated class - * @throws IOException on any instantiation failure. + * @param interfaceImplemented interface that this class implements + * @param methodName name of factory method to be invoked + * @param configKey config key under which this class is specified + * @param Instance of class + * @return instance of the specified class + * @throws IOException on any problem */ - private static AWSCredentialsProvider createAWSCredentialProvider( + @SuppressWarnings("unchecked") + public static InstanceT getInstanceFromReflection(String className, Configuration conf, - Class credClass, - @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); - } - if (Modifier.isAbstract(credClass.getModifiers())) { - throw new IOException("Class " + credClass + " " + ABSTRACT_PROVIDER); - } - LOG.debug("Credential provider class is {}", className); - + @Nullable URI uri, + Class interfaceImplemented, + String methodName, + String configKey) throws IOException { try { - // new X(uri, conf) - Constructor cons = getConstructor(credClass, URI.class, - Configuration.class); - if (cons != null) { - credentials = (AWSCredentialsProvider)cons.newInstance(uri, conf); - return credentials; + Class instanceClass = S3AUtils.class.getClassLoader().loadClass(className); + if (Modifier.isAbstract(instanceClass.getModifiers())) { + throw isAbstract(uri, className, configKey); } - // new X(conf) - cons = getConstructor(credClass, Configuration.class); - if (cons != null) { - credentials = (AWSCredentialsProvider)cons.newInstance(conf); - return credentials; + if (!interfaceImplemented.isAssignableFrom(instanceClass)) { + throw isNotInstanceOf(uri, className, interfaceImplemented.getName(), configKey); + + } + Constructor cons; + if (conf != null) { + // new X(uri, conf) + cons = getConstructor(instanceClass, URI.class, Configuration.class); + + if (cons != null) { + return (InstanceT) cons.newInstance(uri, conf); + } + // new X(conf) + cons = getConstructor(instanceClass, Configuration.class); + if (cons != null) { + return (InstanceT) cons.newInstance(conf); + } } - // X.getInstance() - Method factory = getFactoryMethod(credClass, AWSCredentialsProvider.class, - "getInstance"); + // X.methodName() + Method factory = getFactoryMethod(instanceClass, interfaceImplemented, methodName); if (factory != null) { - credentials = (AWSCredentialsProvider)factory.invoke(null); - return credentials; + return (InstanceT) factory.invoke(null); } // new X() - cons = getConstructor(credClass); + cons = getConstructor(instanceClass); if (cons != null) { - credentials = (AWSCredentialsProvider)cons.newInstance(); - return credentials; + return (InstanceT) cons.newInstance(); } // 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 " - + "of a supported signature, or a public factory method named " - + "getInstance that accepts no arguments.", - className, AWS_CREDENTIALS_PROVIDER)); + throw unsupportedConstructor(uri, className, configKey); } catch (InvocationTargetException e) { Throwable targetException = e.getTargetException(); if (targetException == null) { - targetException = e; + targetException = e; } if (targetException instanceof IOException) { throw (IOException) targetException; - } else if (targetException instanceof SdkBaseException) { - throw translateException("Instantiate " + className, "", - (SdkBaseException) targetException); + } else if (targetException instanceof SdkException) { + throw translateException("Instantiate " + className, "", (SdkException) targetException); } else { // supported constructor or factory method found, but the call failed - throw new IOException(className + " " + INSTANTIATION_EXCEPTION - + ": " + targetException, - targetException); + throw instantiationException(uri, className, configKey, targetException); } } catch (ReflectiveOperationException | IllegalArgumentException e) { // supported constructor or factory method found, but the call failed - throw new IOException(className + " " + INSTANTIATION_EXCEPTION - + ": " + e, - e); + throw instantiationException(uri, className, configKey, e); } } + /** * Set a key if the value is non-empty. * @param config config to patch @@ -941,13 +834,13 @@ static String lookupPassword(Configuration conf, String key, String defVal) /** * String information about a summary entry for debug messages. - * @param summary summary object + * @param s3Object s3Object entry * @return string value */ - public static String stringify(S3ObjectSummary summary) { - StringBuilder builder = new StringBuilder(summary.getKey().length() + 100); - builder.append(summary.getKey()).append(' '); - builder.append("size=").append(summary.getSize()); + public static String stringify(S3Object s3Object) { + StringBuilder builder = new StringBuilder(s3Object.key().length() + 100); + builder.append(s3Object.key()).append(' '); + builder.append("size=").append(s3Object.size()); return builder.toString(); } @@ -1215,215 +1108,6 @@ public static void deleteWithWarning(FileSystem fs, } } - /** - * Create a new AWS {@code ClientConfiguration}. - * All clients to AWS services MUST use this for consistent setup - * of connectivity, UA, proxy settings. - * @param conf The Hadoop configuration - * @param bucket Optional bucket to use to look up per-bucket proxy secrets - * @return new AWS client configuration - * @throws IOException problem creating AWS client configuration - * - * @deprecated use {@link #createAwsConf(Configuration, String, String)} - */ - @Deprecated - public static ClientConfiguration createAwsConf(Configuration conf, - String bucket) - throws IOException { - return createAwsConf(conf, bucket, null); - } - - /** - * Create a new AWS {@code ClientConfiguration}. All clients to AWS services - * MUST use this or the equivalents for the specific service for - * consistent setup of connectivity, UA, proxy settings. - * - * @param conf The Hadoop configuration - * @param bucket Optional bucket to use to look up per-bucket proxy secrets - * @param awsServiceIdentifier a string representing the AWS service (S3, - * etc) for which the ClientConfiguration is being created. - * @return new AWS client configuration - * @throws IOException problem creating AWS client configuration - */ - public static ClientConfiguration createAwsConf(Configuration conf, - String bucket, String awsServiceIdentifier) - throws IOException { - final ClientConfiguration awsConf = new ClientConfiguration(); - initConnectionSettings(conf, awsConf); - initProxySupport(conf, bucket, awsConf); - initUserAgent(conf, awsConf); - if (StringUtils.isNotEmpty(awsServiceIdentifier)) { - String configKey = null; - switch (awsServiceIdentifier) { - case AWS_SERVICE_IDENTIFIER_S3: - configKey = SIGNING_ALGORITHM_S3; - break; - case AWS_SERVICE_IDENTIFIER_STS: - configKey = SIGNING_ALGORITHM_STS; - break; - default: - // Nothing to do. The original signer override is already setup - } - if (configKey != null) { - String signerOverride = conf.getTrimmed(configKey, ""); - if (!signerOverride.isEmpty()) { - LOG.debug("Signer override for {}} = {}", awsServiceIdentifier, - signerOverride); - awsConf.setSignerOverride(signerOverride); - } - } - } - return awsConf; - } - - /** - * Initializes all AWS SDK settings related to connection management. - * - * @param conf Hadoop configuration - * @param awsConf AWS SDK configuration - * - * @throws IOException if there was an error initializing the protocol - * settings - */ - public static void initConnectionSettings(Configuration conf, - ClientConfiguration awsConf) throws IOException { - awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS, - DEFAULT_MAXIMUM_CONNECTIONS, 1)); - awsConf.setConnectionTTL(longOption(conf, CONNECTION_TTL, - DEFAULT_CONNECTION_TTL, -1)); - initProtocolSettings(conf, awsConf); - awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES, - DEFAULT_MAX_ERROR_RETRIES, 0)); - awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT, - DEFAULT_ESTABLISH_TIMEOUT, 0)); - awsConf.setSocketTimeout(intOption(conf, SOCKET_TIMEOUT, - DEFAULT_SOCKET_TIMEOUT, 0)); - int sockSendBuffer = intOption(conf, SOCKET_SEND_BUFFER, - DEFAULT_SOCKET_SEND_BUFFER, 2048); - int sockRecvBuffer = intOption(conf, SOCKET_RECV_BUFFER, - DEFAULT_SOCKET_RECV_BUFFER, 2048); - long requestTimeoutMillis = conf.getTimeDuration(REQUEST_TIMEOUT, - DEFAULT_REQUEST_TIMEOUT, TimeUnit.SECONDS, TimeUnit.MILLISECONDS); - - if (requestTimeoutMillis > Integer.MAX_VALUE) { - LOG.debug("Request timeout is too high({} ms). Setting to {} ms instead", - requestTimeoutMillis, Integer.MAX_VALUE); - requestTimeoutMillis = Integer.MAX_VALUE; - } - awsConf.setRequestTimeout((int) requestTimeoutMillis); - awsConf.setSocketBufferSizeHints(sockSendBuffer, sockRecvBuffer); - String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, ""); - if (!signerOverride.isEmpty()) { - LOG.debug("Signer override = {}", signerOverride); - awsConf.setSignerOverride(signerOverride); - } - } - - /** - * Initializes the connection protocol settings when connecting to S3 (e.g. - * either HTTP or HTTPS). If secure connections are enabled, this method - * will load the configured SSL providers. - * - * @param conf Hadoop configuration - * @param awsConf AWS SDK configuration - * - * @throws IOException if there is an error initializing the configured - * {@link javax.net.ssl.SSLSocketFactory} - */ - private static void initProtocolSettings(Configuration conf, - ClientConfiguration awsConf) throws IOException { - boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS, - DEFAULT_SECURE_CONNECTIONS); - awsConf.setProtocol(secureConnections ? Protocol.HTTPS : Protocol.HTTP); - if (secureConnections) { - NetworkBinding.bindSSLChannelMode(conf, awsConf); - } - } - - /** - * Initializes AWS SDK proxy support in the AWS client configuration - * if the S3A settings enable it. - * - * @param conf Hadoop configuration - * @param bucket Optional bucket to use to look up per-bucket proxy secrets - * @param awsConf AWS SDK configuration to update - * @throws IllegalArgumentException if misconfigured - * @throws IOException problem getting username/secret from password source. - */ - public static void initProxySupport(Configuration conf, - String bucket, - ClientConfiguration awsConf) throws IllegalArgumentException, - IOException { - String proxyHost = conf.getTrimmed(PROXY_HOST, ""); - int proxyPort = conf.getInt(PROXY_PORT, -1); - if (!proxyHost.isEmpty()) { - awsConf.setProxyHost(proxyHost); - if (proxyPort >= 0) { - awsConf.setProxyPort(proxyPort); - } else { - if (conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS)) { - LOG.warn("Proxy host set without port. Using HTTPS default 443"); - awsConf.setProxyPort(443); - } else { - LOG.warn("Proxy host set without port. Using HTTP default 80"); - awsConf.setProxyPort(80); - } - } - final String proxyUsername = lookupPassword(bucket, conf, PROXY_USERNAME, - null, null); - final String proxyPassword = lookupPassword(bucket, conf, PROXY_PASSWORD, - null, null); - if ((proxyUsername == null) != (proxyPassword == null)) { - String msg = "Proxy error: " + PROXY_USERNAME + " or " + - PROXY_PASSWORD + " set without the other."; - LOG.error(msg); - throw new IllegalArgumentException(msg); - } - boolean isProxySecured = conf.getBoolean(PROXY_SECURED, false); - awsConf.setProxyUsername(proxyUsername); - awsConf.setProxyPassword(proxyPassword); - awsConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN)); - awsConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION)); - awsConf.setProxyProtocol(isProxySecured ? Protocol.HTTPS : Protocol.HTTP); - if (LOG.isDebugEnabled()) { - LOG.debug("Using proxy server {}://{}:{} as user {} with password {} " - + "on domain {} as workstation {}", - awsConf.getProxyProtocol(), - awsConf.getProxyHost(), - awsConf.getProxyPort(), - String.valueOf(awsConf.getProxyUsername()), - awsConf.getProxyPassword(), awsConf.getProxyDomain(), - awsConf.getProxyWorkstation()); - } - } else if (proxyPort >= 0) { - String msg = - "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST; - LOG.error(msg); - throw new IllegalArgumentException(msg); - } - } - - /** - * Initializes the User-Agent header to send in HTTP requests to AWS - * services. We always include the Hadoop version number. The user also - * may set an optional custom prefix to put in front of the Hadoop version - * number. The AWS SDK internally appends its own information, which seems - * to include the AWS SDK version, OS and JVM version. - * - * @param conf Hadoop configuration - * @param awsConf AWS SDK configuration to update - */ - private static void initUserAgent(Configuration conf, - ClientConfiguration awsConf) { - String userAgent = "Hadoop " + VersionInfo.getVersion(); - String userAgentPrefix = conf.getTrimmed(USER_AGENT_PREFIX, ""); - if (!userAgentPrefix.isEmpty()) { - userAgent = userAgentPrefix + ", " + userAgent; - } - LOG.debug("Using User-Agent: {}", userAgent); - awsConf.setUserAgentPrefix(userAgent); - } - /** * Convert the data of an iterator of {@link S3AFileStatus} to * an array. @@ -1746,6 +1430,11 @@ public static EncryptionSecrets buildEncryptionSecrets(String bucket, diagnostics); break; + case DSSE_KMS: + LOG.debug("Using DSSE-KMS with {}", + diagnostics); + break; + case NONE: default: LOG.debug("Data is unencrypted"); @@ -1917,4 +1606,15 @@ public String toString() { } }; + /** + * Format a byte range for a request header. + * See https://www.rfc-editor.org/rfc/rfc9110.html#section-14.1.2 + * + * @param rangeStart the start byte offset + * @param rangeEnd the end byte offset (inclusive) + * @return a formatted byte range + */ + public static String formatRange(long rangeStart, long rangeEnd) { + return String.format("bytes=%d-%d", rangeStart, rangeEnd); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java index 9010f34dc259c..451eab8e9ad58 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java @@ -24,11 +24,13 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.Executor; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.handlers.RequestHandler2; -import com.amazonaws.monitoring.MonitoringListener; -import com.amazonaws.services.s3.AmazonS3; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.transfer.s3.S3TransferManager; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -37,7 +39,7 @@ import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ENDPOINT; /** - * Factory for creation of {@link AmazonS3} client instances. + * Factory for creation of {@link S3Client} client instances. * Important: HBase's HBoss module implements this interface in its * tests. * Take care when updating this interface to ensure that a client @@ -49,20 +51,44 @@ */ @InterfaceAudience.LimitedPrivate("HBoss") @InterfaceStability.Evolving -@Deprecated public interface S3ClientFactory { /** - * Creates a new {@link AmazonS3} client. + * Creates a new {@link S3Client}. + * The client returned supports synchronous operations. For + * asynchronous operations, use + * {@link #createS3AsyncClient(URI, S3ClientCreationParameters)}. * * @param uri S3A file system URI * @param parameters parameter object * @return S3 client - * @throws IOException IO problem + * @throws IOException on any IO problem */ - AmazonS3 createS3Client(URI uri, + S3Client createS3Client(URI uri, S3ClientCreationParameters parameters) throws IOException; + /** + * Creates a new {@link S3AsyncClient}. + * The client returned supports asynchronous operations. For + * synchronous operations, use + * {@link #createS3Client(URI, S3ClientCreationParameters)}. + * + * @param uri S3A file system URI + * @param parameters parameter object + * @return Async S3 client + * @throws IOException on any IO problem + */ + S3AsyncClient createS3AsyncClient(URI uri, + S3ClientCreationParameters parameters) throws IOException; + + /** + * Creates a new {@link S3TransferManager}. + * + * @param s3AsyncClient the async client to be used by the TM. + * @return S3 transfer manager + */ + S3TransferManager createS3TransferManager(S3AsyncClient s3AsyncClient); + /** * Settings for the S3 Client. * Implemented as a class to pass in so that adding @@ -74,7 +100,7 @@ final class S3ClientCreationParameters { /** * Credentials. */ - private AWSCredentialsProvider credentialSet; + private AwsCredentialsProvider credentialSet; /** * Endpoint. @@ -86,11 +112,6 @@ final class S3ClientCreationParameters { */ private final Map headers = new HashMap<>(); - /** - * Monitoring listener. - */ - private MonitoringListener monitoringListener; - /** * RequestMetricCollector metrics...if not-null will be wrapped * with an {@code AwsStatisticsCollector} and passed to @@ -109,9 +130,9 @@ final class S3ClientCreationParameters { private boolean requesterPays; /** - * Request handlers; used for auditing, X-Ray etc. - */ - private List requestHandlers; + * Execution interceptors; used for auditing, X-Ray etc. + * */ + private List executionInterceptors; /** * Suffix to UA. @@ -125,37 +146,48 @@ final class S3ClientCreationParameters { private URI pathUri; /** - * List of request handlers to include in the chain - * of request execution in the SDK. - * @return the handler list + * Minimum part size for transfer parts. */ - public List getRequestHandlers() { - return requestHandlers; - } + private long minimumPartSize; /** - * List of request handlers. - * @param handlers handler list. - * @return this object + * Threshold for multipart operations. */ - public S3ClientCreationParameters withRequestHandlers( - @Nullable final List handlers) { - requestHandlers = handlers; - return this; - } + private long multiPartThreshold; + + /** + * Multipart upload enabled. + */ + private boolean multipartCopy = true; - public MonitoringListener getMonitoringListener() { - return monitoringListener; + /** + * Executor that the transfer manager will use to execute background tasks. + */ + private Executor transferManagerExecutor; + + /** + * Region of the S3 bucket. + */ + private String region; + + + /** + * List of execution interceptors to include in the chain + * of interceptors in the SDK. + * @return the interceptors list + */ + public List getExecutionInterceptors() { + return executionInterceptors; } /** - * listener for AWS monitoring events. - * @param listener listener + * List of execution interceptors. + * @param interceptors interceptors list. * @return this object */ - public S3ClientCreationParameters withMonitoringListener( - @Nullable final MonitoringListener listener) { - monitoringListener = listener; + public S3ClientCreationParameters withExecutionInterceptors( + @Nullable final List interceptors) { + executionInterceptors = interceptors; return this; } @@ -191,7 +223,7 @@ public boolean isRequesterPays() { return requesterPays; } - public AWSCredentialsProvider getCredentialSet() { + public AwsCredentialsProvider getCredentialSet() { return credentialSet; } @@ -202,7 +234,7 @@ public AWSCredentialsProvider getCredentialSet() { */ public S3ClientCreationParameters withCredentialSet( - final AWSCredentialsProvider value) { + final AwsCredentialsProvider value) { credentialSet = value; return this; } @@ -294,5 +326,101 @@ public S3ClientCreationParameters withPathUri( pathUri = value; return this; } + + /** + * Get the minimum part size for transfer parts. + * @return part size + */ + public long getMinimumPartSize() { + return minimumPartSize; + } + + /** + * Set the minimum part size for transfer parts. + * @param value new value + * @return the builder + */ + public S3ClientCreationParameters withMinimumPartSize( + final long value) { + minimumPartSize = value; + return this; + } + + /** + * Get the threshold for multipart operations. + * @return multipart threshold + */ + public long getMultiPartThreshold() { + return multiPartThreshold; + } + + /** + * Set the threshold for multipart operations. + * @param value new value + * @return the builder + */ + public S3ClientCreationParameters withMultipartThreshold( + final long value) { + multiPartThreshold = value; + return this; + } + + /** + * Get the executor that the transfer manager will use to execute background tasks. + * @return part size + */ + public Executor getTransferManagerExecutor() { + return transferManagerExecutor; + } + + /** + * Set the executor that the transfer manager will use to execute background tasks. + * @param value new value + * @return the builder + */ + public S3ClientCreationParameters withTransferManagerExecutor( + final Executor value) { + transferManagerExecutor = value; + return this; + } + + /** + * Set the multipart flag.. + * + * @param value new value + * @return the builder + */ + public S3ClientCreationParameters withMultipartCopyEnabled(final boolean value) { + this.multipartCopy = value; + return this; + } + + /** + * Get the multipart flag. + * @return multipart flag + */ + public boolean isMultipartCopy() { + return multipartCopy; + } + + /** + * Set region. + * + * @param value new value + * @return the builder + */ + public S3ClientCreationParameters withRegion( + final String value) { + region = value; + return this; + } + + /** + * Get the region. + * @return invoker + */ + public String getRegion() { + return region; + } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java index d51211516f251..c729f3de15f08 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java @@ -18,8 +18,8 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; /** * API version-independent container for S3 List requests. @@ -78,14 +78,14 @@ public ListObjectsV2Request getV2() { public String toString() { if (isV1()) { return String.format(DESCRIPTION, - v1Request.getBucketName(), v1Request.getPrefix(), - v1Request.getDelimiter(), v1Request.getMaxKeys(), - v1Request.isRequesterPays()); + v1Request.bucket(), v1Request.prefix(), + v1Request.delimiter(), v1Request.maxKeys(), + v1Request.requestPayerAsString()); } else { return String.format(DESCRIPTION, - v2Request.getBucketName(), v2Request.getPrefix(), - v2Request.getDelimiter(), v2Request.getMaxKeys(), - v2Request.isRequesterPays()); + v2Request.bucket(), v2Request.prefix(), + v2Request.delimiter(), v2Request.maxKeys(), + v2Request.requestPayerAsString()); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java index 69c42bfe1471a..0300b78bec250 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java @@ -22,19 +22,23 @@ import java.util.List; import java.util.stream.Collectors; -import com.amazonaws.services.s3.model.ListObjectsV2Result; -import com.amazonaws.services.s3.model.ObjectListing; -import com.amazonaws.services.s3.model.S3ObjectSummary; +import software.amazon.awssdk.services.s3.model.CommonPrefix; +import software.amazon.awssdk.services.s3.model.ListObjectsResponse; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; +import software.amazon.awssdk.services.s3.model.S3Object; + import org.slf4j.Logger; +import static java.util.Objects.requireNonNull; + /** * API version-independent container for S3 List responses. */ public class S3ListResult { - private ObjectListing v1Result; - private ListObjectsV2Result v2Result; + private ListObjectsResponse v1Result; + private ListObjectsV2Response v2Result; - protected S3ListResult(ObjectListing v1, ListObjectsV2Result v2) { + protected S3ListResult(ListObjectsResponse v1, ListObjectsV2Response v2) { v1Result = v1; v2Result = v2; } @@ -44,8 +48,8 @@ protected S3ListResult(ObjectListing v1, ListObjectsV2Result v2) { * @param result v1 result * @return new list result container */ - public static S3ListResult v1(ObjectListing result) { - return new S3ListResult(result, null); + public static S3ListResult v1(ListObjectsResponse result) { + return new S3ListResult(requireNonNull(result), null); } /** @@ -53,8 +57,8 @@ public static S3ListResult v1(ObjectListing result) { * @param result v2 result * @return new list result container */ - public static S3ListResult v2(ListObjectsV2Result result) { - return new S3ListResult(null, result); + public static S3ListResult v2(ListObjectsV2Response result) { + return new S3ListResult(null, requireNonNull(result)); } /** @@ -65,19 +69,19 @@ public boolean isV1() { return v1Result != null; } - public ObjectListing getV1() { + public ListObjectsResponse getV1() { return v1Result; } - public ListObjectsV2Result getV2() { + public ListObjectsV2Response getV2() { return v2Result; } - public List getObjectSummaries() { + public List getS3Objects() { if (isV1()) { - return v1Result.getObjectSummaries(); + return v1Result.contents(); } else { - return v2Result.getObjectSummaries(); + return v2Result.contents(); } } @@ -89,21 +93,21 @@ public boolean isTruncated() { } } - public List getCommonPrefixes() { + public List getCommonPrefixes() { if (isV1()) { - return v1Result.getCommonPrefixes(); + return v1Result.commonPrefixes(); } else { - return v2Result.getCommonPrefixes(); + return v2Result.commonPrefixes(); } } /** - * Get the list of keys in the object summary. + * Get the list of keys in the list result. * @return a possibly empty list */ - private List objectSummaryKeys() { - return getObjectSummaries().stream() - .map(S3ObjectSummary::getKey) + private List objectKeys() { + return getS3Objects().stream() + .map(S3Object::key) .collect(Collectors.toList()); } @@ -112,9 +116,8 @@ private List objectSummaryKeys() { * @return true if the result is non-empty */ public boolean hasPrefixesOrObjects() { - return !(getCommonPrefixes()).isEmpty() - || !getObjectSummaries().isEmpty(); + || !getS3Objects().isEmpty(); } /** @@ -128,7 +131,7 @@ public boolean representsEmptyDirectory( // no children. // So the listing must contain the marker entry only as an object, // and prefixes is null - List keys = objectSummaryKeys(); + List keys = objectKeys(); return keys.size() == 1 && keys.contains(dirKey) && getCommonPrefixes().isEmpty(); } @@ -138,15 +141,15 @@ public boolean representsEmptyDirectory( * @param log log to use */ public void logAtDebug(Logger log) { - Collection prefixes = getCommonPrefixes(); - Collection summaries = getObjectSummaries(); + Collection prefixes = getCommonPrefixes(); + Collection s3Objects = getS3Objects(); log.debug("Prefix count = {}; object count={}", - prefixes.size(), summaries.size()); - for (S3ObjectSummary summary : summaries) { - log.debug("Summary: {} {}", summary.getKey(), summary.getSize()); + prefixes.size(), s3Objects.size()); + for (S3Object s3Object : s3Objects) { + log.debug("Summary: {} {}", s3Object.key(), s3Object.size()); } - for (String prefix : prefixes) { - log.debug("Prefix: {}", prefix); + for (CommonPrefix prefix : prefixes) { + log.debug("Prefix: {}", prefix.prefix()); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java index 0a0454854b2ac..4fc5b8658b605 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java @@ -18,8 +18,6 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.services.s3.transfer.model.CopyResult; - import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.Path; @@ -74,31 +72,6 @@ public S3ObjectAttributes( this.len = len; } - /** - * Construct from the result of a copy and those parameters - * which aren't included in an AWS SDK response. - * @param path path - * @param copyResult copy result. - * @param serverSideEncryptionAlgorithm current encryption algorithm - * @param serverSideEncryptionKey any server side encryption key? - * @param len object length - */ - public S3ObjectAttributes( - final Path path, - final CopyResult copyResult, - final S3AEncryptionMethods serverSideEncryptionAlgorithm, - final String serverSideEncryptionKey, - final long len) { - this.bucket = copyResult.getDestinationBucketName(); - this.key = copyResult.getDestinationKey(); - this.path = path; - this.serverSideEncryptionAlgorithm = serverSideEncryptionAlgorithm; - this.serverSideEncryptionKey = serverSideEncryptionKey; - this.eTag = copyResult.getETag(); - this.versionId = copyResult.getVersionId(); - this.len = len; - } - public String getBucket() { return bucket; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SharedInstanceCredentialProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SharedInstanceCredentialProvider.java index 6579a2bc3e7d2..b7de937f5137a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SharedInstanceCredentialProvider.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SharedInstanceCredentialProvider.java @@ -39,6 +39,5 @@ */ @InterfaceAudience.Public @InterfaceStability.Evolving -@SuppressWarnings("deprecation") public final class SharedInstanceCredentialProvider extends IAMInstanceCredentialsProvider { } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SimpleAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SimpleAWSCredentialsProvider.java index 657bd437b5be9..087d034975adb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SimpleAWSCredentialsProvider.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SimpleAWSCredentialsProvider.java @@ -18,9 +18,11 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.BasicAWSCredentials; + +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; + import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.commons.lang3.StringUtils; @@ -42,13 +44,10 @@ * property fs.s3a.aws.credentials.provider. Therefore, changing the class name * would be a backward-incompatible change. * - * @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider - * as part of upgrading S3A to SDK V2. See HADOOP-18073. */ @InterfaceAudience.Public @InterfaceStability.Stable -@Deprecated -public class SimpleAWSCredentialsProvider implements AWSCredentialsProvider { +public class SimpleAWSCredentialsProvider implements AwsCredentialsProvider { public static final String NAME = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider"; @@ -63,7 +62,7 @@ public class SimpleAWSCredentialsProvider implements AWSCredentialsProvider { */ public SimpleAWSCredentialsProvider(final URI uri, final Configuration conf) throws IOException { - this(getAWSAccessKeys(uri, conf)); + this(getAWSAccessKeys(uri, conf)); } /** @@ -75,25 +74,25 @@ public SimpleAWSCredentialsProvider(final URI uri, final Configuration conf) @VisibleForTesting SimpleAWSCredentialsProvider(final S3xLoginHelper.Login login) throws IOException { - this.accessKey = login.getUser(); - this.secretKey = login.getPassword(); + this.accessKey = login.getUser(); + this.secretKey = login.getPassword(); } @Override - public AWSCredentials getCredentials() { + public AwsCredentials resolveCredentials() { if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) { - return new BasicAWSCredentials(accessKey, secretKey); + return AwsBasicCredentials.create(accessKey, secretKey); } throw new NoAwsCredentialsException("SimpleAWSCredentialsProvider", "No AWS credentials in the Hadoop configuration"); } - @Override - public void refresh() {} - @Override public String toString() { - return getClass().getSimpleName(); + return "SimpleAWSCredentialsProvider{" + + "accessKey.empty=" + accessKey.isEmpty() + + ", secretKey.empty=" + secretKey.isEmpty() + + '}'; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index f9694871d7304..72fc75b642415 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -242,7 +242,10 @@ public enum Statistic { StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_ABORTED, "Object multipart upload aborted", TYPE_DURATION), - OBJECT_PUT_REQUESTS( + OBJECT_MULTIPART_UPLOAD_LIST( + StoreStatisticNames.OBJECT_MULTIPART_UPLOAD_LIST, + "Object multipart list request issued", + TYPE_DURATION), OBJECT_PUT_REQUESTS( StoreStatisticNames.OBJECT_PUT_REQUEST, "Object put/multipart upload count", TYPE_DURATION), @@ -577,7 +580,7 @@ public enum Statistic { TYPE_COUNTER), MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED( StoreStatisticNames.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED, - "Multipart Upload Abort Unner Path Invoked", + "Multipart Upload Abort Under Path Invoked", TYPE_COUNTER), MULTIPART_UPLOAD_COMPLETED( StoreStatisticNames.MULTIPART_UPLOAD_COMPLETED, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java index db3d0bb13297c..5587b11a36f88 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java @@ -21,7 +21,7 @@ import javax.annotation.Nullable; import java.io.IOException; -import com.amazonaws.auth.AWSCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; import java.net.URI; @@ -44,12 +44,9 @@ * This credential provider must not fail in creation because that will * break a chain of credential providers. * - * @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider - * as part of upgrading S3A to SDK V2. See HADOOP-18073. */ @InterfaceAudience.Public @InterfaceStability.Stable -@Deprecated public class TemporaryAWSCredentialsProvider extends AbstractSessionCredentialsProvider { public static final String NAME @@ -92,7 +89,7 @@ public TemporaryAWSCredentialsProvider( * @throws NoAwsCredentialsException the credentials are actually empty. */ @Override - protected AWSCredentials createCredentials(Configuration config) + protected AwsCredentials createCredentials(Configuration config) throws IOException { MarshalledCredentials creds = MarshalledCredentialBinding.fromFileSystem( getUri(), config); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Tristate.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Tristate.java index 0462ccfd7cbbd..d7123787735a1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Tristate.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Tristate.java @@ -18,15 +18,58 @@ package org.apache.hadoop.fs.s3a; +import java.util.Optional; + +import static java.util.Optional.empty; +import static java.util.Optional.of; + /** * Simple enum to express {true, false, don't know}. */ public enum Tristate { + // Do not add additional values here. Logic will assume there are exactly // three possibilities. - TRUE, FALSE, UNKNOWN; + TRUE(of(Boolean.TRUE)), + FALSE(of(Boolean.FALSE)), + UNKNOWN(empty()); + + /** + * Mapping to an optional boolean. + */ + @SuppressWarnings("NonSerializableFieldInSerializableClass") + private final Optional mapping; + + Tristate(final Optional t) { + mapping = t; + } + + /** + * Get the boolean mapping, if present. + * @return the boolean value, if present. + */ + public Optional getMapping() { + return mapping; + } + + /** + * Does this value map to a boolean. + * @return true if the state is one of true or false. + */ + public boolean isBoolean() { + return mapping.isPresent(); + } public static Tristate fromBool(boolean v) { return v ? TRUE : FALSE; } + + /** + * Build a tristate from a boolean. + * @param b source optional + * @return a tristate derived from the argument. + */ + public static Tristate from(Optional b) { + return b.map(Tristate::fromBool).orElse(UNKNOWN); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java index 238cd97ed8f48..85ed69e14bc79 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java @@ -18,22 +18,22 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.services.s3.transfer.Upload; +import software.amazon.awssdk.transfer.s3.model.FileUpload; /** * Simple struct that contains information about a S3 upload. */ public class UploadInfo { - private final Upload upload; + private final FileUpload fileUpload; private final long length; - public UploadInfo(Upload upload, long length) { - this.upload = upload; + public UploadInfo(FileUpload upload, long length) { + this.fileUpload = upload; this.length = length; } - public Upload getUpload() { - return upload; + public FileUpload getFileUpload() { + return fileUpload; } public long getLength() { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java index 0bbb8a35f51cb..1eacc4cbe07b7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java @@ -19,29 +19,28 @@ package org.apache.hadoop.fs.s3a; import javax.annotation.Nullable; -import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; -import java.io.InputStream; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; -import com.amazonaws.services.s3.model.AmazonS3Exception; -import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; -import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; -import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; -import com.amazonaws.services.s3.model.MultipartUpload; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.PutObjectResult; -import com.amazonaws.services.s3.model.SelectObjectContentRequest; -import com.amazonaws.services.s3.model.SelectObjectContentResult; -import com.amazonaws.services.s3.model.UploadPartRequest; -import com.amazonaws.services.s3.model.UploadPartResult; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse; +import software.amazon.awssdk.services.s3.model.CompletedPart; +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.MultipartUpload; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; +import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; +import software.amazon.awssdk.services.s3.model.SelectObjectContentResponseHandler; +import software.amazon.awssdk.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.services.s3.model.UploadPartResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -50,6 +49,8 @@ import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.select.SelectEventStreamPublisher; +import org.apache.hadoop.fs.s3a.select.SelectObjectContentHelper; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.select.SelectBinding; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; @@ -239,49 +240,25 @@ private void deactivateAuditSpan() { /** * Create a {@link PutObjectRequest} request against the specific key. * @param destKey destination key - * @param inputStream source data. * @param length size, if known. Use -1 for not known * @param options options for the request + * @param isFile is data to be uploaded a file * @return the request */ @Retries.OnceRaw public PutObjectRequest createPutObjectRequest(String destKey, - InputStream inputStream, long length, - final PutObjectOptions options) { - activateAuditSpan(); - ObjectMetadata objectMetadata = newObjectMetadata(length); - return getRequestFactory().newPutObjectRequest( - destKey, - objectMetadata, - options, - inputStream); - } + final PutObjectOptions options, + boolean isFile) { - /** - * Create a {@link PutObjectRequest} request to upload a file. - * @param dest key to PUT to. - * @param sourceFile source file - * @param options options for the request - * @return the request - */ - @Retries.OnceRaw - public PutObjectRequest createPutObjectRequest( - String dest, - File sourceFile, - final PutObjectOptions options) { activateAuditSpan(); - final ObjectMetadata objectMetadata = - newObjectMetadata((int) sourceFile.length()); - - PutObjectRequest putObjectRequest = getRequestFactory(). - newPutObjectRequest(dest, - objectMetadata, - options, - sourceFile); - return putObjectRequest; + + return getRequestFactory() + .newPutObjectRequestBuilder(destKey, options, length, false) + .build(); } + /** * Callback on a successful write. * @param length length of the write @@ -297,17 +274,6 @@ public void writeFailed(Exception ex) { LOG.debug("Write to {} failed", this, ex); } - /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * @param length size, if known. Use -1 for not known - * @return a new metadata instance - */ - public ObjectMetadata newObjectMetadata(long length) { - return getRequestFactory().newObjectMetadata(length); - } - /** * {@inheritDoc} */ @@ -320,11 +286,11 @@ public String initiateMultiPartUpload( try (AuditSpan span = activateAuditSpan()) { return retry("initiate MultiPartUpload", destKey, true, () -> { - final InitiateMultipartUploadRequest initiateMPURequest = - getRequestFactory().newMultipartUploadRequest( + final CreateMultipartUploadRequest.Builder initiateMPURequestBuilder = + getRequestFactory().newMultipartUploadRequestBuilder( destKey, options); - return owner.initiateMultipartUpload(initiateMPURequest) - .getUploadId(); + return owner.initiateMultipartUpload(initiateMPURequestBuilder.build()) + .uploadId(); }); } } @@ -345,10 +311,10 @@ public String initiateMultiPartUpload( * @throws IOException on problems. */ @Retries.RetryTranslated - private CompleteMultipartUploadResult finalizeMultipartUpload( + private CompleteMultipartUploadResponse finalizeMultipartUpload( String destKey, String uploadId, - List partETags, + List partETags, long length, PutObjectOptions putOptions, Retried retrying) throws IOException { @@ -357,18 +323,18 @@ private CompleteMultipartUploadResult finalizeMultipartUpload( "No upload parts in multipart upload"); } try (AuditSpan span = activateAuditSpan()) { - CompleteMultipartUploadResult uploadResult; + CompleteMultipartUploadResponse uploadResult; uploadResult = invoker.retry("Completing multipart upload", destKey, true, retrying, () -> { - final CompleteMultipartUploadRequest request = - getRequestFactory().newCompleteMultipartUploadRequest( + final CompleteMultipartUploadRequest.Builder requestBuilder = + getRequestFactory().newCompleteMultipartUploadRequestBuilder( destKey, uploadId, partETags); - return writeOperationHelperCallbacks.completeMultipartUpload(request); + return writeOperationHelperCallbacks.completeMultipartUpload(requestBuilder.build()); }); - owner.finishedWrite(destKey, length, uploadResult.getETag(), - uploadResult.getVersionId(), + owner.finishedWrite(destKey, length, uploadResult.eTag(), + uploadResult.versionId(), putOptions); return uploadResult; } @@ -391,10 +357,10 @@ private CompleteMultipartUploadResult finalizeMultipartUpload( * the retry count was exceeded */ @Retries.RetryTranslated - public CompleteMultipartUploadResult completeMPUwithRetries( + public CompleteMultipartUploadResponse completeMPUwithRetries( String destKey, String uploadId, - List partETags, + List partETags, long length, AtomicInteger errorCount, PutObjectOptions putOptions) @@ -452,7 +418,7 @@ public void abortMultipartUpload(String destKey, String uploadId, @Retries.RetryTranslated public void abortMultipartUpload(MultipartUpload upload) throws IOException { - invoker.retry("Aborting multipart commit", upload.getKey(), true, + invoker.retry("Aborting multipart commit", upload.key(), true, withinAuditSpan(getAuditSpan(), () -> owner.abortMultipartUpload(upload))); } @@ -477,7 +443,7 @@ public int abortMultipartUploadsUnderPath(String prefix) abortMultipartUpload(upload); count++; } catch (FileNotFoundException e) { - LOG.debug("Already aborted: {}", upload.getKey(), e); + LOG.debug("Already aborted: {}", upload.key(), e); } } return count; @@ -506,45 +472,31 @@ public void abortMultipartCommit(String destKey, String uploadId) } /** - * Create and initialize a part request of a multipart upload. - * Exactly one of: {@code uploadStream} or {@code sourceFile} - * must be specified. - * A subset of the file may be posted, by providing the starting point - * in {@code offset} and a length of block in {@code size} equal to - * or less than the remaining bytes. + * Create and initialize a part request builder of a multipart upload. * The part number must be less than 10000. * Retry policy is once-translated; to much effort * @param destKey destination key of ongoing operation * @param uploadId ID of ongoing upload * @param partNumber current part number of the upload * @param size amount of data - * @param uploadStream source of data to upload - * @param sourceFile optional source file. - * @param offset offset in file to start reading. - * @return the request. + * @return the request builder. * @throws IllegalArgumentException if the parameters are invalid. * @throws PathIOException if the part number is out of range. */ @Override @Retries.OnceTranslated - public UploadPartRequest newUploadPartRequest( + public UploadPartRequest.Builder newUploadPartRequestBuilder( String destKey, String uploadId, int partNumber, - long size, - InputStream uploadStream, - File sourceFile, - Long offset) throws IOException { + long size) throws IOException { return once("upload part request", destKey, withinAuditSpan(getAuditSpan(), () -> - getRequestFactory().newUploadPartRequest( + getRequestFactory().newUploadPartRequestBuilder( destKey, uploadId, partNumber, - size, - uploadStream, - sourceFile, - offset))); + size))); } /** @@ -566,18 +518,20 @@ public String toString() { * @param putObjectRequest the request * @param putOptions put object options * @param durationTrackerFactory factory for duration tracking + * @param uploadData data to be uploaded + * @param isFile is data to be uploaded a file + * * @return the upload initiated * @throws IOException on problems */ @Retries.RetryTranslated - public PutObjectResult putObject(PutObjectRequest putObjectRequest, - PutObjectOptions putOptions, + public PutObjectResponse putObject(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions, S3ADataBlocks.BlockUploadData uploadData, boolean isFile, DurationTrackerFactory durationTrackerFactory) throws IOException { - return retry("Writing Object", - putObjectRequest.getKey(), true, - withinAuditSpan(getAuditSpan(), () -> - owner.putObjectDirect(putObjectRequest, putOptions, durationTrackerFactory))); + return retry("Writing Object", putObjectRequest.key(), true, withinAuditSpan(getAuditSpan(), + () -> owner.putObjectDirect(putObjectRequest, putOptions, uploadData, isFile, + durationTrackerFactory))); } /** @@ -613,10 +567,10 @@ public void revertCommit(String destKey) throws IOException { * the retry count was exceeded */ @Retries.RetryTranslated - public CompleteMultipartUploadResult commitUpload( + public CompleteMultipartUploadResponse commitUpload( String destKey, String uploadId, - List partETags, + List partETags, long length) throws IOException { checkNotNull(uploadId); @@ -635,19 +589,21 @@ public CompleteMultipartUploadResult commitUpload( * Upload part of a multi-partition file. * @param request request * @param durationTrackerFactory duration tracker factory for operation + * @param request the upload part request. + * @param body the request body. * @return the result of the operation. * @throws IOException on problems */ @Retries.RetryTranslated - public UploadPartResult uploadPart(UploadPartRequest request, + public UploadPartResponse uploadPart(UploadPartRequest request, RequestBody body, final DurationTrackerFactory durationTrackerFactory) throws IOException { - return retry("upload part #" + request.getPartNumber() - + " upload ID " + request.getUploadId(), - request.getKey(), + return retry("upload part #" + request.partNumber() + + " upload ID " + request.uploadId(), + request.key(), true, withinAuditSpan(getAuditSpan(), - () -> owner.uploadPart(request, durationTrackerFactory))); + () -> owner.uploadPart(request, body, durationTrackerFactory))); } /** @@ -659,15 +615,9 @@ public Configuration getConf() { return conf; } - /** - * Create a S3 Select request for the destination path. - * This does not build the query. - * @param path pre-qualified path for query - * @return the request - */ - public SelectObjectContentRequest newSelectRequest(Path path) { + public SelectObjectContentRequest.Builder newSelectRequestBuilder(Path path) { try (AuditSpan span = getAuditSpan()) { - return getRequestFactory().newSelectRequest( + return getRequestFactory().newSelectRequestBuilder( storeContext.pathToKey(path)); } } @@ -676,26 +626,27 @@ public SelectObjectContentRequest newSelectRequest(Path path) { * Execute an S3 Select operation. * On a failure, the request is only logged at debug to avoid the * select exception being printed. - * @param source source for selection + * + * @param source source for selection * @param request Select request to issue. - * @param action the action for use in exception creation + * @param action the action for use in exception creation * @return response * @throws IOException failure */ @Retries.RetryTranslated - public SelectObjectContentResult select( + public SelectEventStreamPublisher select( final Path source, final SelectObjectContentRequest request, final String action) throws IOException { // no setting of span here as the select binding is (statically) created // without any span. - String bucketName = request.getBucketName(); + String bucketName = request.bucket(); Preconditions.checkArgument(bucket.equals(bucketName), "wrong bucket: %s", bucketName); if (LOG.isDebugEnabled()) { LOG.debug("Initiating select call {} {}", - source, request.getExpression()); + source, request.expression()); LOG.debug(SelectBinding.toString(request)); } return invoker.retry( @@ -706,8 +657,9 @@ public SelectObjectContentResult select( try (DurationInfo ignored = new DurationInfo(LOG, "S3 Select operation")) { try { - return writeOperationHelperCallbacks.selectObjectContent(request); - } catch (AmazonS3Exception e) { + return SelectObjectContentHelper.select( + writeOperationHelperCallbacks, source, request, action); + } catch (Throwable e) { LOG.error("Failure of S3 Select request against {}", source); LOG.debug("S3 Select request against {}:\n{}", @@ -756,16 +708,18 @@ public interface WriteOperationHelperCallbacks { /** * Initiates a select request. * @param request selectObjectContent request + * @param t selectObjectContent request handler * @return selectObjectContentResult */ - SelectObjectContentResult selectObjectContent(SelectObjectContentRequest request); + CompletableFuture selectObjectContent(SelectObjectContentRequest request, + SelectObjectContentResponseHandler t); /** * Initiates a complete multi-part upload request. * @param request Complete multi-part upload request * @return completeMultipartUploadResult */ - CompleteMultipartUploadResult completeMultipartUpload(CompleteMultipartUploadRequest request); + CompleteMultipartUploadResponse completeMultipartUpload(CompleteMultipartUploadRequest request); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java index 1c3d368857583..0fda4921a30da 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java @@ -20,29 +20,27 @@ import javax.annotation.Nullable; import java.io.Closeable; -import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; -import java.io.InputStream; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; -import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; -import com.amazonaws.services.s3.model.MultipartUpload; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.PutObjectResult; -import com.amazonaws.services.s3.model.SelectObjectContentRequest; -import com.amazonaws.services.s3.model.SelectObjectContentResult; -import com.amazonaws.services.s3.model.UploadPartRequest; -import com.amazonaws.services.s3.model.UploadPartResult; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse; +import software.amazon.awssdk.services.s3.model.CompletedPart; +import software.amazon.awssdk.services.s3.model.MultipartUpload; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; +import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; +import software.amazon.awssdk.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.services.s3.model.UploadPartResponse; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.s3a.select.SelectEventStreamPublisher; import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.util.functional.CallableRaisingIOE; @@ -77,27 +75,15 @@ T retry(String action, /** * Create a {@link PutObjectRequest} request against the specific key. * @param destKey destination key - * @param inputStream source data. * @param length size, if known. Use -1 for not known * @param options options for the request + * @param isFile is data to be uploaded a file * @return the request */ PutObjectRequest createPutObjectRequest(String destKey, - InputStream inputStream, long length, - @Nullable PutObjectOptions options); - - /** - * Create a {@link PutObjectRequest} request to upload a file. - * @param dest key to PUT to. - * @param sourceFile source file - * @param options options for the request - * @return the request - */ - PutObjectRequest createPutObjectRequest( - String dest, - File sourceFile, - @Nullable PutObjectOptions options); + @Nullable PutObjectOptions options, + boolean isFile); /** * Callback on a successful write. @@ -111,15 +97,6 @@ PutObjectRequest createPutObjectRequest( */ void writeFailed(Exception ex); - /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * @param length size, if known. Use -1 for not known - * @return a new metadata instance - */ - ObjectMetadata newObjectMetadata(long length); - /** * Start the multipart upload process. * Retry policy: retrying, translated. @@ -148,10 +125,10 @@ PutObjectRequest createPutObjectRequest( * the retry count was exceeded */ @Retries.RetryTranslated - CompleteMultipartUploadResult completeMPUwithRetries( + CompleteMultipartUploadResponse completeMPUwithRetries( String destKey, String uploadId, - List partETags, + List partETags, long length, AtomicInteger errorCount, PutObjectOptions putOptions) @@ -213,31 +190,20 @@ void abortMultipartCommit(String destKey, String uploadId) throws IOException; /** - * Create and initialize a part request of a multipart upload. - * Exactly one of: {@code uploadStream} or {@code sourceFile} - * must be specified. - * A subset of the file may be posted, by providing the starting point - * in {@code offset} and a length of block in {@code size} equal to - * or less than the remaining bytes. + * Create and initialize a part request builder of a multipart upload. * @param destKey destination key of ongoing operation * @param uploadId ID of ongoing upload * @param partNumber current part number of the upload * @param size amount of data - * @param uploadStream source of data to upload - * @param sourceFile optional source file. - * @param offset offset in file to start reading. - * @return the request. + * @return the request builder. * @throws IllegalArgumentException if the parameters are invalid * @throws PathIOException if the part number is out of range. */ - UploadPartRequest newUploadPartRequest( + UploadPartRequest.Builder newUploadPartRequestBuilder( String destKey, String uploadId, int partNumber, - long size, - InputStream uploadStream, - File sourceFile, - Long offset) throws IOException; + long size) throws IOException; /** * PUT an object directly (i.e. not via the transfer manager). @@ -246,12 +212,14 @@ UploadPartRequest newUploadPartRequest( * @param putObjectRequest the request * @param putOptions put object options * @param durationTrackerFactory factory for duration tracking + * @param uploadData data to be uploaded + * @param isFile is data to be uploaded a file * @return the upload initiated * @throws IOException on problems */ @Retries.RetryTranslated - PutObjectResult putObject(PutObjectRequest putObjectRequest, - PutObjectOptions putOptions, + PutObjectResponse putObject(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions, S3ADataBlocks.BlockUploadData uploadData, boolean isFile, DurationTrackerFactory durationTrackerFactory) throws IOException; @@ -279,22 +247,23 @@ PutObjectResult putObject(PutObjectRequest putObjectRequest, * the retry count was exceeded */ @Retries.RetryTranslated - CompleteMultipartUploadResult commitUpload( + CompleteMultipartUploadResponse commitUpload( String destKey, String uploadId, - List partETags, + List partETags, long length) throws IOException; /** * Upload part of a multi-partition file. - * @param request request + * @param request the upload part request. + * @param body the request body. * @param durationTrackerFactory factory for duration tracking * @return the result of the operation. * @throws IOException on problems */ @Retries.RetryTranslated - UploadPartResult uploadPart(UploadPartRequest request, + UploadPartResponse uploadPart(UploadPartRequest request, RequestBody body, DurationTrackerFactory durationTrackerFactory) throws IOException; @@ -306,25 +275,26 @@ UploadPartResult uploadPart(UploadPartRequest request, Configuration getConf(); /** - * Create a S3 Select request for the destination path. + * Create a S3 Select request builder for the destination path. * This does not build the query. * @param path pre-qualified path for query - * @return the request + * @return the request builder */ - SelectObjectContentRequest newSelectRequest(Path path); + SelectObjectContentRequest.Builder newSelectRequestBuilder(Path path); /** * Execute an S3 Select operation. * On a failure, the request is only logged at debug to avoid the * select exception being printed. - * @param source source for selection + * + * @param source source for selection * @param request Select request to issue. - * @param action the action for use in exception creation + * @param action the action for use in exception creation * @return response * @throws IOException failure */ @Retries.RetryTranslated - SelectObjectContentResult select( + SelectEventStreamPublisher select( Path source, SelectObjectContentRequest request, String action) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/AwsV1BindingSupport.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/AwsV1BindingSupport.java new file mode 100644 index 0000000000000..177952cb9d22b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/AwsV1BindingSupport.java @@ -0,0 +1,118 @@ +/* + * 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.adapter; + +import java.io.IOException; +import java.net.URI; +import javax.annotation.Nullable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.impl.InstantiationIOException; + +import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.unavailable; + +/** + * Binding support; the sole way which the rest of the code should instantiate v1 SDK libraries. + * Uses this class's Classloader for its analysis/loading. + */ +@SuppressWarnings("StaticNonFinalField") +public final class AwsV1BindingSupport { + + private static final Logger LOG = LoggerFactory.getLogger( + AwsV1BindingSupport.class); + + /** + * V1 credential provider classname: {@code}. + */ + public static final String CREDENTIAL_PROVIDER_CLASSNAME = + "com.amazonaws.auth.AWSCredentialsProvider"; + + /** + * SDK availability. + */ + private static final boolean SDK_V1_FOUND = checkForAwsV1Sdk(); + + private AwsV1BindingSupport() { + } + + /** + * Probe for the AWS v1 SDK being available by looking for + * the class {@link #CREDENTIAL_PROVIDER_CLASSNAME}. + * @return true if it was found in the classloader + */ + private static boolean checkForAwsV1Sdk() { + + try { + ClassLoader cl = AwsV1BindingSupport.class.getClassLoader(); + cl.loadClass(CREDENTIAL_PROVIDER_CLASSNAME); + LOG.debug("v1 SDK class {} found", CREDENTIAL_PROVIDER_CLASSNAME); + return true; + } catch (Exception e) { + LOG.debug("v1 SDK class {} not found", CREDENTIAL_PROVIDER_CLASSNAME, e); + return false; + } + } + + /** + * Is the AWS v1 SDK available? + * @return true if it was found in the classloader + */ + public static synchronized boolean isAwsV1SdkAvailable() { + return SDK_V1_FOUND; + } + + + /** + * Create an AWS credential provider from its class by using reflection. The + * class must implement one of the following means of construction, which are + * attempted in order: + * + *
      + *
    1. a public constructor accepting java.net.URI and + * org.apache.hadoop.conf.Configuration
    2. + *
    3. a public constructor accepting + * org.apache.hadoop.conf.Configuration
    4. + *
    5. a public static method named getInstance that accepts no + * arguments and returns an instance of + * com.amazonaws.auth.AWSCredentialsProvider, or
    6. + *
    7. a public default constructor.
    8. + *
    + * @param conf configuration + * @param className credential classname + * @param uri URI of the FS + * @param key configuration key to use + * @return the instantiated class + * @throws InstantiationIOException on any instantiation failure, including v1 SDK not found + * @throws IOException anything else. + */ + public static AwsCredentialsProvider createAWSV1CredentialProvider( + Configuration conf, + String className, + @Nullable URI uri, + final String key) throws IOException { + if (!isAwsV1SdkAvailable()) { + throw unavailable(uri, className, key, "No AWS v1 SDK available"); + } + return V1ToV2AwsCredentialProviderAdapter.create(conf, className, uri); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1ToV2AwsCredentialProviderAdapter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1ToV2AwsCredentialProviderAdapter.java new file mode 100644 index 0000000000000..2cbd9836d3a43 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1ToV2AwsCredentialProviderAdapter.java @@ -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.adapter; + +import java.io.Closeable; +import java.io.IOException; +import java.net.URI; +import javax.annotation.Nullable; + +import com.amazonaws.SdkClientException; +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSSessionCredentials; +import com.amazonaws.auth.AnonymousAWSCredentials; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.CredentialInitializationException; +import org.apache.hadoop.fs.s3a.S3AUtils; +import org.apache.hadoop.fs.s3a.impl.InstantiationIOException; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER; + +/** + * Adapts a V1 {@link AWSCredentialsProvider} to the V2 {@link AwsCredentialsProvider} interface. + */ +public final class V1ToV2AwsCredentialProviderAdapter + implements AwsCredentialsProvider, Closeable { + + private static final Logger LOG = LoggerFactory.getLogger( + V1ToV2AwsCredentialProviderAdapter.class); + + /** + * The V1 credential provider constructed. + */ + private final AWSCredentialsProvider v1CredentialsProvider; + + + private V1ToV2AwsCredentialProviderAdapter(AWSCredentialsProvider v1CredentialsProvider) { + this.v1CredentialsProvider = requireNonNull(v1CredentialsProvider); + } + + + /** + * Collect v1 credentials and convert to v2. + * @return v2 credentials + * @throws CredentialInitializationException if the inner retrieval raised an exception + */ + @Override + public AwsCredentials resolveCredentials() { + try { + // get the wrapped credentials + AWSCredentials toAdapt = v1CredentialsProvider.getCredentials(); + return convertToV2Credentials(toAdapt); + } catch (SdkClientException e) { + // wrap with a v2 exception so that code which adds a try/catch for v2 sdk exceptions + // gets a compatible exception. + throw new CredentialInitializationException(e.toString(), e); + } + } + + /** + * Close the wrapped provider if it implements Closeable/AutoCloseable. + * @throws IOException failure + */ + @Override + public void close() throws IOException { + if (v1CredentialsProvider instanceof Closeable) { + ((Closeable) v1CredentialsProvider).close(); + } else if (v1CredentialsProvider instanceof AutoCloseable) { + S3AUtils.closeAutocloseables(LOG, (AutoCloseable) v1CredentialsProvider); + } + } + + /** + * Convert v1 credentials to v2, including support for session and anonymous + * credentials. + * @param toAdapt credentials to adapt. + * @return v2 credentials. + */ + static AwsCredentials convertToV2Credentials(final AWSCredentials toAdapt) { + if (toAdapt instanceof AWSSessionCredentials) { + return AwsSessionCredentials.create(toAdapt.getAWSAccessKeyId(), + toAdapt.getAWSSecretKey(), + ((AWSSessionCredentials) toAdapt).getSessionToken()); + } else if (toAdapt instanceof AnonymousAWSCredentials) { + return AnonymousCredentialsProvider.create().resolveCredentials(); + } else { + return AwsBasicCredentials.create(toAdapt.getAWSAccessKeyId(), toAdapt.getAWSSecretKey()); + } + } + + @Override + public String toString() { + return "V1ToV2AwsCredentialProviderAdapter{" + + "v1CredentialsProvider=" + v1CredentialsProvider + + '}'; + } + + /** + * @param v1CredentialsProvider V1 credential provider to adapt. + * @return A new instance of the credentials provider adapter. + */ + static AwsCredentialsProvider create(AWSCredentialsProvider v1CredentialsProvider) { + return new V1ToV2AwsCredentialProviderAdapter(v1CredentialsProvider); + } + + /** + * Create an AWS credential provider from its class by using reflection. The + * class must implement one of the following means of construction, which are + * attempted in order: + * + *
      + *
    1. a public constructor accepting java.net.URI and + * org.apache.hadoop.conf.Configuration
    2. + *
    3. a public constructor accepting + * org.apache.hadoop.conf.Configuration
    4. + *
    5. a public static method named getInstance that accepts no + * arguments and returns an instance of + * com.amazonaws.auth.AWSCredentialsProvider, or
    6. + *
    7. a public default constructor.
    8. + *
    + * @param conf configuration + * @param className classname + * @param uri URI of the FS + * @return the instantiated class + * @throws InstantiationIOException on construction and instantiation failures, + * including v1 SDK exceptions. + * @throws IOException if raised by a constructor/factory method. + */ + static AwsCredentialsProvider create( + Configuration conf, + String className, + @Nullable URI uri) throws InstantiationIOException, IOException { + + final AWSCredentialsProvider instance = + S3AUtils.getInstanceFromReflection(className, conf, uri, AWSCredentialsProvider.class, + "getInstance", AWS_CREDENTIALS_PROVIDER); + return create(instance); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/package-info.java new file mode 100644 index 0000000000000..124534188bccc --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/package-info.java @@ -0,0 +1,31 @@ +/* + * 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. + */ + +/** + * Adapter classes for allowing V1 credential providers to be used with SDKV2. + * This is the only package where use of aws v1 classes are permitted; + * all instantiations of objects here must use reflection to probe for + * availability or be prepared to catch exceptions which may be raised + * if the v1 SDK isn't found on the classpath + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.adapter; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java index 2a4771925f086..99a898f728166 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java @@ -19,41 +19,37 @@ package org.apache.hadoop.fs.s3a.api; import javax.annotation.Nullable; -import java.io.File; -import java.io.InputStream; import java.util.List; -import java.util.Optional; - -import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; -import com.amazonaws.services.s3.model.CannedAccessControlList; -import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; -import com.amazonaws.services.s3.model.CopyObjectRequest; -import com.amazonaws.services.s3.model.DeleteObjectRequest; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; -import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; -import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsV2Request; -import com.amazonaws.services.s3.model.ObjectListing; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; -import com.amazonaws.services.s3.model.SSECustomerKey; -import com.amazonaws.services.s3.model.SelectObjectContentRequest; -import com.amazonaws.services.s3.model.StorageClass; -import com.amazonaws.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CompletedPart; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadBucketRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; +import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; +import software.amazon.awssdk.services.s3.model.StorageClass; +import software.amazon.awssdk.services.s3.model.UploadPartRequest; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; /** - * Factory for S3 objects. + * Factory for S3 request objects. * * This is where the owner FS's {@code prepareRequest()} * callback is invoked to mark up a request for this span. @@ -67,6 +63,8 @@ * as there are no guarantees how they are processed. * That is: no guarantees of retry or translation. */ +@InterfaceStability.Unstable +@InterfaceAudience.LimitedPrivate("testing/diagnostics") public interface RequestFactory { /** @@ -79,22 +77,7 @@ public interface RequestFactory { * Get the canned ACL of this FS. * @return an ACL, if any */ - CannedAccessControlList getCannedACL(); - - /** - * 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 - */ - Optional generateSSEAwsKeyParams(); - - /** - * 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 an optional customer key. - */ - Optional generateSSECustomerKey(); + String getCannedACL(); /** * Get the encryption algorithm of this endpoint. @@ -115,79 +98,58 @@ public interface RequestFactory { StorageClass getStorageClass(); /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * - * @param length length of data to set in header; Ignored if negative - * @return a new metadata instance - */ - ObjectMetadata newObjectMetadata(long length); - - /** - * Create a copy request. + * Create a copy request builder. * This includes the work of copying the relevant parts * of the metadata from the source * @param srcKey source * @param dstKey destination * @param srcom source object metadata. - * @return the request + * @return the request builder */ - CopyObjectRequest newCopyObjectRequest(String srcKey, + CopyObjectRequest.Builder newCopyObjectRequestBuilder(String srcKey, String dstKey, - ObjectMetadata srcom); + HeadObjectResponse srcom); - /** - * Create a putObject request. - * Adds the ACL and metadata - * @param key key of object - * @param metadata metadata header - * @param options options for the request - * @param srcfile source file - * @return the request - */ - PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, PutObjectOptions options, File srcfile); /** - * Create a {@link PutObjectRequest} request. + * Create a {@link PutObjectRequest} request builder. * The metadata is assumed to have been configured with the size of the * operation. * @param key key of object - * @param metadata metadata header * @param options options for the request - * @param inputStream source data. - * @return the request + * @param length length of object to be uploaded + * @param isDirectoryMarker true if object to be uploaded is a directory marker + * @return the request builder */ - PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, + PutObjectRequest.Builder newPutObjectRequestBuilder(String key, PutObjectOptions options, - InputStream inputStream); + long length, + boolean isDirectoryMarker); /** * Create a {@link PutObjectRequest} request for creating * an empty directory. * * @param directory destination directory. - * @return request for a zero byte upload. + * @return request builder for a zero byte upload. */ - PutObjectRequest newDirectoryMarkerRequest(String directory); + PutObjectRequest.Builder newDirectoryMarkerRequest(String directory); /** * List all multipart uploads under a prefix. * @param prefix prefix to list under - * @return the request. + * @return the request builder. */ - ListMultipartUploadsRequest newListMultipartUploadsRequest( + ListMultipartUploadsRequest.Builder newListMultipartUploadsRequestBuilder( @Nullable String prefix); /** * Abort a multipart upload. * @param destKey destination object key * @param uploadId ID of initiated upload - * @return the request. + * @return the request builder. */ - AbortMultipartUploadRequest newAbortMultipartUploadRequest( + AbortMultipartUploadRequest.Builder newAbortMultipartUploadRequestBuilder( String destKey, String uploadId); @@ -195,10 +157,10 @@ AbortMultipartUploadRequest newAbortMultipartUploadRequest( * Start a multipart upload. * @param destKey destination object key * @param options options for the request - * @return the request. + * @return the request builder. * @throws PathIOException if multipart uploads are disabled */ - InitiateMultipartUploadRequest newMultipartUploadRequest( + CreateMultipartUploadRequest.Builder newMultipartUploadRequestBuilder( String destKey, @Nullable PutObjectOptions options) throws PathIOException; @@ -207,107 +169,95 @@ InitiateMultipartUploadRequest newMultipartUploadRequest( * @param destKey destination object key * @param uploadId ID of initiated upload * @param partETags ordered list of etags - * @return the request. + * @return the request builder. */ - CompleteMultipartUploadRequest newCompleteMultipartUploadRequest( + CompleteMultipartUploadRequest.Builder newCompleteMultipartUploadRequestBuilder( String destKey, String uploadId, - List partETags); + List partETags); /** - * Create a HEAD request. + * Create a HEAD object request builder. * @param key key, may have trailing / - * @return the request. + * @return the request builder. + */ + HeadObjectRequest.Builder newHeadObjectRequestBuilder(String key); + + /** + * Create a HEAD bucket request builder. + * @param bucket bucket to get metadata for + * @return the request builder. */ - GetObjectMetadataRequest newGetObjectMetadataRequest(String key); + HeadBucketRequest.Builder newHeadBucketRequestBuilder(String bucket); + /** - * Create a GET request. + * Create a GET request builder. * @param key object key - * @return the request. + * @return the request builder. */ - GetObjectRequest newGetObjectRequest(String key); + GetObjectRequest.Builder newGetObjectRequestBuilder(String key); /** - * Create and initialize a part request of a multipart upload. - * Exactly one of: {@code uploadStream} or {@code sourceFile} - * must be specified. - * A subset of the file may be posted, by providing the starting point - * in {@code offset} and a length of block in {@code size} equal to - * or less than the remaining bytes. - * @param destKey destination key of ongoing operation - * @param uploadId ID of ongoing upload - * @param partNumber current part number of the upload - * @param size amount of data - * @param uploadStream source of data to upload - * @param sourceFile optional source file. - * @param offset offset in file to start reading. - * @return the request. + * Create and initialize a part request builder of a multipart upload. + * + * @param destKey destination key of ongoing operation + * @param uploadId ID of ongoing upload + * @param partNumber current part number of the upload + * @param size amount of data + * @return the request builder. * @throws PathIOException if the part number is out of range. */ - UploadPartRequest newUploadPartRequest( + UploadPartRequest.Builder newUploadPartRequestBuilder( String destKey, String uploadId, int partNumber, - long size, - InputStream uploadStream, - File sourceFile, - long offset) throws PathIOException; + long size) throws PathIOException; /** - * Create a S3 Select request for the destination object. + * Create a S3 Select request builder for the destination object. * This does not build the query. * @param key object key - * @return the request + * @return the request builder */ - SelectObjectContentRequest newSelectRequest(String key); + SelectObjectContentRequest.Builder newSelectRequestBuilder(String key); /** - * Create the (legacy) V1 list request. + * Create the (legacy) V1 list request builder. * @param key key to list under * @param delimiter delimiter for keys * @param maxKeys maximum number in a list page. - * @return the request + * @return the request builder. */ - ListObjectsRequest newListObjectsV1Request(String key, + ListObjectsRequest.Builder newListObjectsV1RequestBuilder(String key, String delimiter, int maxKeys); /** - * Create the next V1 page list request, following - * on from the previous response. - * @param prev previous response - * @return the request - */ - - ListNextBatchOfObjectsRequest newListNextBatchOfObjectsRequest( - ObjectListing prev); - - /** - * Create a V2 list request. + * Create a V2 list request builder. * This will be recycled for any subsequent requests. * @param key key to list under * @param delimiter delimiter for keys * @param maxKeys maximum number in a list page. - * @return the request + * @return the request builder. */ - ListObjectsV2Request newListObjectsV2Request(String key, + ListObjectsV2Request.Builder newListObjectsV2RequestBuilder(String key, String delimiter, int maxKeys); /** - * Create a request to delete a single object. + * Create a request builder to delete a single object. * @param key object to delete - * @return the request + * @return the request builder. */ - DeleteObjectRequest newDeleteObjectRequest(String key); + DeleteObjectRequest.Builder newDeleteObjectRequestBuilder(String key); /** - * Bulk delete request. + * Create a request builder to delete objects in bulk. * @param keysToDelete list of keys to delete. - * @return the request + * @return the request builder. */ - DeleteObjectsRequest newBulkDeleteRequest( - List keysToDelete); + DeleteObjectsRequest.Builder newBulkDeleteRequestBuilder( + List keysToDelete); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSAuditEventCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSAuditEventCallbacks.java index 8134d5cea942e..712246c21961c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSAuditEventCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSAuditEventCallbacks.java @@ -18,15 +18,9 @@ package org.apache.hadoop.fs.s3a.audit; -import com.amazonaws.AmazonWebServiceRequest; -import com.amazonaws.Request; -import com.amazonaws.Response; -import com.amazonaws.SdkBaseException; -import com.amazonaws.handlers.HandlerAfterAttemptContext; -import com.amazonaws.handlers.HandlerBeforeAttemptContext; -import com.amazonaws.http.HttpResponse; +import software.amazon.awssdk.core.SdkRequest; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; -import org.apache.hadoop.fs.s3a.Retries; /** * Callbacks for audit spans. This is implemented @@ -37,10 +31,10 @@ * detect this and raise an exception. * * Look at the documentation for - * {@code com.amazonaws.handlers.IRequestHandler2} for details + * {@code ExecutionInterceptor} for details * on the callbacks. */ -public interface AWSAuditEventCallbacks { +public interface AWSAuditEventCallbacks extends ExecutionInterceptor { /** * Return a span ID which must be unique for all spans within @@ -66,95 +60,8 @@ public interface AWSAuditEventCallbacks { * It is not invoked on any AWS requests created in the SDK. * Avoid raising exceptions or talking to any remote service; * this callback is for annotation rather than validation. - * @param request request request. - * @param type of request - * @return the request, possibly modified. + * @param builder the request builder. */ - default T requestCreated(T request) { - return request; - } + default void requestCreated(SdkRequest.Builder builder) {} - /** - * Preflight preparation of AWS request. - * @param request request - * @param type of request - * @return an updated request. - * @throws AuditFailureException for generic audit failures - * @throws SdkBaseException for other reasons. - */ - @Retries.OnceRaw - default T beforeExecution(T request) - throws AuditFailureException, SdkBaseException { - return request; - } - - /** - * Callback after S3 responded to a request. - * @param request request - * @param response response. - * @throws AuditFailureException for generic audit failures - * @throws SdkBaseException for other reasons. - */ - default void afterResponse(Request request, - Response response) - throws AuditFailureException, SdkBaseException { - } - - /** - * Callback after a request resulted in an error. - * @param request request - * @param response response. - * @param exception exception raised. - * @throws AuditFailureException for generic audit failures - * @throws SdkBaseException for other reasons. - */ - default void afterError(Request request, - Response response, - Exception exception) - throws AuditFailureException, SdkBaseException { - } - - /** - * Request before marshalling. - * @param request request - * @return possibly modified request. - */ - default AmazonWebServiceRequest beforeMarshalling( - AmazonWebServiceRequest request) { - return request; - } - - /** - * Request before marshalling. - * @param request request - */ - default void beforeRequest(Request request) { - } - - /** - * Before any attempt is made. - * @param context full context, including the request. - */ - default void beforeAttempt(HandlerBeforeAttemptContext context) { - } - - /** - * After any attempt is made. - * @param context full context, including the request. - */ - default void afterAttempt( - HandlerAfterAttemptContext context) { - } - - /** - * Before unmarshalling the response. - * @param request request made. - * @param httpResponse response received - * @return updated response. - */ - default HttpResponse beforeUnmarshalling( - final Request request, - final HttpResponse httpResponse) { - return httpResponse; - } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java index b4be341c912e0..8a24a4e14db1c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java @@ -20,24 +20,23 @@ import java.util.List; -import com.amazonaws.AmazonWebServiceRequest; -import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; -import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; -import com.amazonaws.services.s3.model.CopyPartRequest; -import com.amazonaws.services.s3.model.DeleteObjectRequest; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.GetBucketLocationRequest; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; -import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; -import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsV2Request; -import com.amazonaws.services.s3.model.ObjectListing; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.SelectObjectContentRequest; -import com.amazonaws.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.core.SdkRequest; +import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; +import software.amazon.awssdk.services.s3.model.UploadPartCopyRequest; +import software.amazon.awssdk.services.s3.model.UploadPartRequest; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_HEAD_REQUEST; @@ -64,102 +63,85 @@ public class AWSRequestAnalyzer { * read/write and path. * @param request request. * @return information about the request. - * @param type of request. */ - public RequestInfo analyze(T request) { + public RequestInfo analyze(SdkRequest request) { // this is where Scala's case statement would massively // simplify life. // Please Keep in Alphabetical Order. if (request instanceof AbortMultipartUploadRequest) { return writing(MULTIPART_UPLOAD_ABORTED, - ((AbortMultipartUploadRequest) request).getKey(), + ((AbortMultipartUploadRequest) request).key(), 0); } else if (request instanceof CompleteMultipartUploadRequest) { CompleteMultipartUploadRequest r = (CompleteMultipartUploadRequest) request; return writing(MULTIPART_UPLOAD_COMPLETED, - r.getKey(), - r.getPartETags().size()); + r.key(), + r.multipartUpload().parts().size()); + } else if (request instanceof CreateMultipartUploadRequest) { + return writing(MULTIPART_UPLOAD_STARTED, + ((CreateMultipartUploadRequest) request).key(), + 0); } else if (request instanceof DeleteObjectRequest) { // DeleteObject: single object return writing(OBJECT_DELETE_REQUEST, - ((DeleteObjectRequest) request).getKey(), + ((DeleteObjectRequest) request).key(), 1); } else if (request instanceof DeleteObjectsRequest) { // DeleteObjects: bulk delete // use first key as the path DeleteObjectsRequest r = (DeleteObjectsRequest) request; - List keys - = r.getKeys(); + List objectIdentifiers + = r.delete().objects(); return writing(OBJECT_BULK_DELETE_REQUEST, - keys.isEmpty() ? null : keys.get(0).getKey(), - keys.size()); + objectIdentifiers.isEmpty() ? null : objectIdentifiers.get(0).key(), + objectIdentifiers.size()); } else if (request instanceof GetBucketLocationRequest) { GetBucketLocationRequest r = (GetBucketLocationRequest) request; return reading(STORE_EXISTS_PROBE, - r.getBucketName(), + r.bucket(), 0); - } else if (request instanceof GetObjectMetadataRequest) { - return reading(ACTION_HTTP_HEAD_REQUEST, - ((GetObjectMetadataRequest) request).getKey(), 0); } else if (request instanceof GetObjectRequest) { GetObjectRequest r = (GetObjectRequest) request; - long[] range = r.getRange(); - long size = range == null - ? -1 - : range[1] - range[0]; return reading(ACTION_HTTP_GET_REQUEST, - r.getKey(), - size); - } else if (request instanceof InitiateMultipartUploadRequest) { - return writing(MULTIPART_UPLOAD_STARTED, - ((InitiateMultipartUploadRequest) request).getKey(), - 0); + r.key(), + sizeFromRangeHeader(r.range())); + } else if (request instanceof HeadObjectRequest) { + return reading(ACTION_HTTP_HEAD_REQUEST, + ((HeadObjectRequest) request).key(), 0); } else if (request instanceof ListMultipartUploadsRequest) { ListMultipartUploadsRequest r = (ListMultipartUploadsRequest) request; return reading(MULTIPART_UPLOAD_LIST, - r.getPrefix(), - r.getMaxUploads()); + r.prefix(), + r.maxUploads()); } else if (request instanceof ListObjectsRequest) { ListObjectsRequest r = (ListObjectsRequest) request; return reading(OBJECT_LIST_REQUEST, - r.getPrefix(), - r.getMaxKeys()); - } else if (request instanceof ListNextBatchOfObjectsRequest) { - ListNextBatchOfObjectsRequest r = (ListNextBatchOfObjectsRequest) request; - ObjectListing l = r.getPreviousObjectListing(); - String prefix = ""; - int size = 0; - if (l != null) { - prefix = l.getPrefix(); - size = l.getMaxKeys(); - } - return reading(OBJECT_LIST_REQUEST, - prefix, - size); + r.prefix(), + r.maxKeys()); } else if (request instanceof ListObjectsV2Request) { ListObjectsV2Request r = (ListObjectsV2Request) request; return reading(OBJECT_LIST_REQUEST, - r.getPrefix(), - r.getMaxKeys()); + r.prefix(), + r.maxKeys()); } else if (request instanceof PutObjectRequest) { PutObjectRequest r = (PutObjectRequest) request; return writing(OBJECT_PUT_REQUEST, - r.getKey(), + r.key(), 0); } else if (request instanceof SelectObjectContentRequest) { SelectObjectContentRequest r = (SelectObjectContentRequest) request; return reading(OBJECT_SELECT_REQUESTS, - r.getKey(), + r.key(), 1); } else if (request instanceof UploadPartRequest) { UploadPartRequest r = (UploadPartRequest) request; return writing(MULTIPART_UPLOAD_PART_PUT, - r.getKey(), - r.getPartSize()); + r.key(), + r.contentLength()); } // no explicit support, return classname return writing(request.getClass().getName(), null, 0); @@ -212,7 +194,7 @@ private RequestInfo writing(final String verb, */ public static boolean isRequestNotAlwaysInSpan(final Object request) { - return request instanceof CopyPartRequest + return request instanceof UploadPartCopyRequest || request instanceof CompleteMultipartUploadRequest || request instanceof GetBucketLocationRequest; } @@ -225,9 +207,9 @@ private RequestInfo writing(final String verb, * @return true if the transfer manager creates them. */ public static boolean isRequestMultipartIO(final Object request) { - return request instanceof CopyPartRequest + return request instanceof UploadPartCopyRequest || request instanceof CompleteMultipartUploadRequest - || request instanceof InitiateMultipartUploadRequest + || request instanceof CreateMultipartUploadRequest || request instanceof UploadPartRequest; } @@ -307,4 +289,23 @@ public String toString() { private static long toSafeLong(final Number size) { return size != null ? size.longValue() : 0; } + + private static final String BYTES_PREFIX = "bytes="; + + private static Number sizeFromRangeHeader(String rangeHeader) { + if (rangeHeader != null && rangeHeader.startsWith(BYTES_PREFIX)) { + String[] values = rangeHeader + .substring(BYTES_PREFIX.length()) + .split("-"); + if (values.length == 2) { + try { + long start = Long.parseUnsignedLong(values[0]); + long end = Long.parseUnsignedLong(values[0]); + return end - start; + } catch(NumberFormatException e) { + } + } + } + return -1; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java index c66f45eb309ff..dfbe8e1a13502 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditIntegration.java @@ -23,7 +23,7 @@ import java.lang.reflect.InvocationTargetException; import java.nio.file.AccessDeniedException; -import com.amazonaws.HandlerContextAware; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,10 +34,11 @@ import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditManagerS3A; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; + import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_ENABLED; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_ENABLED_DEFAULT; -import static org.apache.hadoop.fs.s3a.audit.impl.S3AInternalAuditConstants.AUDIT_SPAN_HANDLER_CONTEXT; +import static org.apache.hadoop.fs.s3a.audit.impl.S3AInternalAuditConstants.AUDIT_SPAN_EXECUTION_ATTRIBUTE; /** * Support for integrating auditing within the S3A code. @@ -123,25 +124,24 @@ public static OperationAuditor createAndInitAuditor( } /** - * Get the span from a handler context. - * @param request request - * @param type of request. + * Get the span from the execution attributes. + * @param executionAttributes the execution attributes * @return the span callbacks or null */ - public static AWSAuditEventCallbacks - retrieveAttachedSpan(final T request) { - return request.getHandlerContext(AUDIT_SPAN_HANDLER_CONTEXT); + public static AuditSpanS3A + retrieveAttachedSpan(final ExecutionAttributes executionAttributes) { + return executionAttributes.getAttribute(AUDIT_SPAN_EXECUTION_ATTRIBUTE); } /** - * Attach a span to a handler context. - * @param request request + * Attach a span to the execution attributes. + * @param executionAttributes the execution attributes * @param span span to attach - * @param type of request. */ - public static void attachSpanToRequest( - final T request, final AWSAuditEventCallbacks span) { - request.addHandlerContext(AUDIT_SPAN_HANDLER_CONTEXT, span); + public static void attachSpanToRequest( + final ExecutionAttributes executionAttributes, + final AuditSpanS3A span) { + executionAttributes.putAttribute(AUDIT_SPAN_EXECUTION_ATTRIBUTE, span); } /** @@ -160,4 +160,30 @@ public static IOException translateAuditException(String path, return (AccessDeniedException)new AccessDeniedException(path, null, exception.toString()).initCause(exception); } + + /** + * Translate an exception if it or its inner exception is an + * {@link AuditFailureException}. + * If this condition is not met, null is returned. + * @param path path of operation. + * @param exception exception + * @return a translated exception or null. + */ + public static IOException maybeTranslateAuditException(String path, + Exception exception) { + if (exception instanceof AuditFailureException) { + return translateAuditException(path, (AuditFailureException) exception); + } else if (exception.getCause() instanceof AuditFailureException) { + return translateAuditException(path, + (AuditFailureException) exception.getCause()); + } else { + return null; + } + } + + public static boolean containsAuditException(Exception exception) { + return exception instanceof AuditFailureException + || exception.getCause() instanceof AuditFailureException; + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java index c1302d57454b1..bb7f94cfc206d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditManagerS3A.java @@ -21,8 +21,8 @@ import java.io.IOException; import java.util.List; -import com.amazonaws.handlers.RequestHandler2; -import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.transfer.s3.progress.TransferListener; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.Path; @@ -32,6 +32,7 @@ import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.service.Service; + /** * Interface for Audit Managers auditing operations through the * AWS libraries. @@ -56,24 +57,24 @@ public interface AuditManagerS3A extends Service, OperationAuditor getAuditor(); /** - * Create the request handler(s) for this audit service. - * The list returned is mutable; new handlers may be added. - * @return list of handlers for the SDK. + * Create the execution interceptor(s) for this audit service. + * The list returned is mutable; new interceptors may be added. + * @return list of interceptors for the SDK. * @throws IOException failure. */ - List createRequestHandlers() throws IOException; + List createExecutionInterceptors() throws IOException; /** - * Return a transfer state change callback which + * Return a transfer callback which * fixes the active span context to be that in which - * the state change listener was created. + * the transfer listener was created. * This can be used to audit the creation of the multipart * upload initiation request which the transfer manager * makes when a file to be copied is split up. * This must be invoked/used within the active span. - * @return a state change listener. + * @return a transfer listener. */ - TransferStateChangeListener createStateChangeListener(); + TransferListener createTransferListener(); /** * Check for permission to access a path. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3AAuditConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3AAuditConstants.java index 1d76833f8ceab..55deb0a1989de 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3AAuditConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3AAuditConstants.java @@ -66,13 +66,20 @@ private S3AAuditConstants() { "org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor"; /** - * List of extra AWS SDK request handlers: {@value}. - * These are added to the SDK request chain after - * any audit service. + * Deprecated list of extra AWS SDK request handlers: {@value}. + * Use {@link #AUDIT_EXECUTION_INTERCEPTORS} instead. */ public static final String AUDIT_REQUEST_HANDLERS = "fs.s3a.audit.request.handlers"; + /** + * List of extra AWS SDK execution interceptors: {@value}. + * These are added to the SDK request chain after + * any audit service. + */ + public static final String AUDIT_EXECUTION_INTERCEPTORS = + "fs.s3a.audit.execution.interceptors"; + /** * Should operations outside spans be rejected? * This is for testing coverage of the span code; if used diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java index ffd08d37e4aa2..1da203dc80249 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/ActiveAuditManagerS3A.java @@ -25,16 +25,16 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; -import com.amazonaws.AmazonWebServiceRequest; -import com.amazonaws.HandlerContextAware; -import com.amazonaws.Request; -import com.amazonaws.Response; -import com.amazonaws.SdkBaseException; -import com.amazonaws.handlers.HandlerAfterAttemptContext; -import com.amazonaws.handlers.HandlerBeforeAttemptContext; -import com.amazonaws.handlers.RequestHandler2; -import com.amazonaws.http.HttpResponse; -import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; +import software.amazon.awssdk.core.SdkRequest; +import software.amazon.awssdk.core.SdkResponse; +import software.amazon.awssdk.core.interceptor.Context; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.http.SdkHttpRequest; +import software.amazon.awssdk.http.SdkHttpResponse; +import software.amazon.awssdk.transfer.s3.progress.TransferListener; + +import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,6 +56,7 @@ import org.apache.hadoop.fs.s3a.audit.OperationAuditor; import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions; import org.apache.hadoop.fs.s3a.audit.S3AAuditConstants; +import org.apache.hadoop.fs.s3a.impl.V2Migration; import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.service.CompositeService; @@ -66,6 +67,7 @@ import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION; import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.attachSpanToRequest; import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.retrieveAttachedSpan; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_EXECUTION_INTERCEPTORS; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS; /** @@ -82,10 +84,11 @@ * will deactivate the wrapped span and then * switch the active span to the unbounded span. * - * The inner class {@link AWSAuditEventCallbacks} is returned - * as a request handler in {@link #createRequestHandlers()}; - * this forwards all requests to the outer {@code ActiveAuditManagerS3A}, - * which then locates the active span and forwards the request. + * This class also implements {@link ExecutionInterceptor} and + * returns itself in {@link #createExecutionInterceptors()}; + * once registered with the S3 client, the implemented methods + * will be called during different parts of an SDK request lifecycle, + * which then locate the active span and forward the request. * If any such invocation raises an {@link AuditFailureException} * then the IOStatistics counter for {@code AUDIT_FAILURE} * is incremented. @@ -390,25 +393,39 @@ public AuditSpanS3A createSpan(final String operation, } /** - * Return a request handler for the AWS SDK which + * Return a list of execution interceptors for the AWS SDK which * relays to this class. - * @return a request handler. + * @return a list of execution interceptors. */ @Override - public List createRequestHandlers() + public List createExecutionInterceptors() throws IOException { // wire up the AWS SDK To call back into this class when // preparing to make S3 calls. - List requestHandlers = new ArrayList<>(); - requestHandlers.add(new SdkRequestHandler()); - // now look for any more handlers - final Class[] handlers = getConfig().getClasses(AUDIT_REQUEST_HANDLERS); - if (handlers != null) { - for (Class handler : handlers) { + List executionInterceptors = new ArrayList<>(); + executionInterceptors.add(this); + + final String handlers = getConfig().getTrimmed(AUDIT_REQUEST_HANDLERS, ""); + if (!handlers.isEmpty()) { + // warn and ignore v1 handlers. + V2Migration.v1RequestHandlersUsed(handlers); + } + + // V2 SDK supports global/service interceptors, but they need to be configured on the + // classpath and don't get the filesystem/job configuration passed down. + final Class[] interceptors = getConfig().getClasses(AUDIT_EXECUTION_INTERCEPTORS); + if (interceptors != null) { + for (Class handler : interceptors) { try { + LOG.debug("Adding intercept of class {}", handler); Constructor ctor = handler.getConstructor(); - requestHandlers.add((RequestHandler2)ctor.newInstance()); + final ExecutionInterceptor interceptor = (ExecutionInterceptor) ctor.newInstance(); + if (interceptor instanceof Configurable) { + // pass in the configuration. + ((Configurable) interceptor).setConf(getConfig()); + } + executionInterceptors.add(interceptor); } catch (ExceptionInInitializerError e) { throw FutureIO.unwrapInnerException(e); } catch (Exception e) { @@ -416,13 +433,18 @@ public List createRequestHandlers() } } } - return requestHandlers; + return executionInterceptors; } @Override - public TransferStateChangeListener createStateChangeListener() { + public TransferListener createTransferListener() { final WrappingAuditSpan span = activeSpan(); - return (transfer, state) -> switchToActiveSpan(span); + return new TransferListener() { + @Override + public void transferInitiated(Context.TransferInitiated context) { + switchToActiveSpan(span); + } + }; } @Override @@ -434,20 +456,18 @@ public boolean checkAccess(final Path path, } /** - * Attach a reference to the active thread span, then - * invoke the same callback on that active thread. + * Audit the creation of a request and retrieve + * a reference to the active thread span. */ @Override - public T requestCreated( - final T request) { + public void requestCreated(final SdkRequest.Builder builder) { AuditSpanS3A span = getActiveAuditSpan(); if (LOG.isTraceEnabled()) { LOG.trace("Created Request {} in span {}", - analyzer.analyze(request), span); + analyzer.analyze(builder.build()), span); } - attachSpanToRequest(request, span); try { - return span.requestCreated(request); + span.requestCreated(builder); } catch (AuditFailureException e) { ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); throw e; @@ -463,14 +483,13 @@ public T requestCreated( * {@inheritDoc} */ @Override - public T beforeExecution( - final T request) { + public void beforeExecution(Context.BeforeExecution context, + ExecutionAttributes executionAttributes) { ioStatisticsStore.incrementCounter(AUDIT_REQUEST_EXECUTION.getSymbol()); - - // identify the span and invoke the callback + AuditSpanS3A span = getActiveAuditSpan(); + attachSpanToRequest(executionAttributes, span); try { - return extractAndActivateSpanFromRequest(request) - .beforeExecution(request); + span.beforeExecution(context, executionAttributes); } catch (AuditFailureException e) { ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); throw e; @@ -479,16 +498,14 @@ public T beforeExecution( /** * Forward to active span. - * @param request request - * @param response response. + * {@inheritDoc} */ @Override - public void afterResponse(final Request request, - final Response response) - throws AuditFailureException, SdkBaseException { + public void afterExecution(Context.AfterExecution context, + ExecutionAttributes executionAttributes) { try { - extractAndActivateSpanFromRequest(request) - .afterResponse(request, response); + extractAndActivateSpanFromRequest(context.request(), executionAttributes) + .afterExecution(context, executionAttributes); } catch (AuditFailureException e) { ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); throw e; @@ -496,18 +513,19 @@ public void afterResponse(final Request request, } /** - * Get the active span from the handler context, + * Get the active span from the execution attributes, * falling back to the active thread span if there - * is nothing in the context. - * Provided the span is a wrapped span, the + * is nothing in the attributes. + * Provided the span is a wrapped span, the span is + * activated. * @param request request - * @param type of request. - * @return the callbacks + * @param executionAttributes the execution attributes + * @return the active span */ - private AWSAuditEventCallbacks - extractAndActivateSpanFromRequest(final T request) { - AWSAuditEventCallbacks span; - span = retrieveAttachedSpan(request); + private AuditSpanS3A extractAndActivateSpanFromRequest( + final SdkRequest request, + final ExecutionAttributes executionAttributes) { + AuditSpanS3A span = retrieveAttachedSpan(executionAttributes); if (span == null) { // no span is attached. Not unusual for the copy operations, // or for calls to GetBucketLocation made by the AWS client @@ -530,18 +548,16 @@ public void afterResponse(final Request request, /** * Forward to active span. - * @param request request - * @param response response. - * @param exception exception raised. + * @param context execution context + * @param executionAttributes the execution attributes + * {@inheritDoc} */ @Override - public void afterError(final Request request, - final Response response, - final Exception exception) - throws AuditFailureException, SdkBaseException { + public void onExecutionFailure(Context.FailedExecution context, + ExecutionAttributes executionAttributes) { try { - extractAndActivateSpanFromRequest(request) - .afterError(request, response, exception); + extractAndActivateSpanFromRequest(context.request(), executionAttributes).onExecutionFailure( + context, executionAttributes); } catch (AuditFailureException e) { ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); throw e; @@ -549,11 +565,12 @@ public void afterError(final Request request, } @Override - public AmazonWebServiceRequest beforeMarshalling( - final AmazonWebServiceRequest request) { + public SdkRequest modifyRequest(Context.ModifyRequest context, + ExecutionAttributes executionAttributes) { try { - return extractAndActivateSpanFromRequest(request) - .beforeMarshalling(request); + return extractAndActivateSpanFromRequest(context.request(), + executionAttributes) + .modifyRequest(context, executionAttributes); } catch (AuditFailureException e) { ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); throw e; @@ -561,10 +578,12 @@ public AmazonWebServiceRequest beforeMarshalling( } @Override - public void beforeRequest(final Request request) { + public void beforeMarshalling(Context.BeforeMarshalling context, + ExecutionAttributes executionAttributes) { try { - extractAndActivateSpanFromRequest(request) - .beforeRequest(request); + extractAndActivateSpanFromRequest(context.request(), + executionAttributes) + .beforeMarshalling(context, executionAttributes); } catch (AuditFailureException e) { ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); throw e; @@ -572,10 +591,12 @@ public void beforeRequest(final Request request) { } @Override - public void beforeAttempt(final HandlerBeforeAttemptContext context) { + public void afterMarshalling(Context.AfterMarshalling context, + ExecutionAttributes executionAttributes) { try { - extractAndActivateSpanFromRequest(context.getRequest()) - .beforeAttempt(context); + extractAndActivateSpanFromRequest(context.request(), + executionAttributes) + .afterMarshalling(context, executionAttributes); } catch (AuditFailureException e) { ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); throw e; @@ -583,10 +604,12 @@ public void beforeAttempt(final HandlerBeforeAttemptContext context) { } @Override - public void afterAttempt(final HandlerAfterAttemptContext context) { + public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context, + ExecutionAttributes executionAttributes) { try { - extractAndActivateSpanFromRequest(context.getRequest()) - .afterAttempt(context); + return extractAndActivateSpanFromRequest(context.request(), + executionAttributes) + .modifyHttpRequest(context, executionAttributes); } catch (AuditFailureException e) { ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); throw e; @@ -594,73 +617,80 @@ public void afterAttempt(final HandlerAfterAttemptContext context) { } @Override - public HttpResponse beforeUnmarshalling(final Request request, - final HttpResponse httpResponse) { + public void beforeTransmission(Context.BeforeTransmission context, + ExecutionAttributes executionAttributes) { try { - extractAndActivateSpanFromRequest(request.getOriginalRequest()) - .beforeUnmarshalling(request, httpResponse); + extractAndActivateSpanFromRequest(context.request(), + executionAttributes) + .beforeTransmission(context, executionAttributes); } catch (AuditFailureException e) { ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); throw e; } - return httpResponse; } - /** - * Callbacks from the AWS SDK; all forward to the ActiveAuditManagerS3A. - * We need a separate class because the SDK requires the handler list - * to be list of {@code RequestHandler2} instances. - */ - private class SdkRequestHandler extends RequestHandler2 { - - @Override - public AmazonWebServiceRequest beforeExecution( - final AmazonWebServiceRequest request) { - return ActiveAuditManagerS3A.this.beforeExecution(request); - } - - @Override - public void afterResponse(final Request request, - final Response response) { - ActiveAuditManagerS3A.this.afterResponse(request, response); - } - - @Override - public void afterError(final Request request, - final Response response, - final Exception e) { - ActiveAuditManagerS3A.this.afterError(request, response, e); - } - - @Override - public AmazonWebServiceRequest beforeMarshalling( - final AmazonWebServiceRequest request) { - return ActiveAuditManagerS3A.this.beforeMarshalling(request); + @Override + public void afterTransmission(Context.AfterTransmission context, + ExecutionAttributes executionAttributes) { + try { + extractAndActivateSpanFromRequest(context.request(), + executionAttributes) + .afterTransmission(context, executionAttributes); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; } + } - @Override - public void beforeRequest(final Request request) { - ActiveAuditManagerS3A.this.beforeRequest(request); + @Override + public SdkHttpResponse modifyHttpResponse(Context.ModifyHttpResponse context, + ExecutionAttributes executionAttributes) { + try { + return extractAndActivateSpanFromRequest(context.request(), + executionAttributes) + .modifyHttpResponse(context, executionAttributes); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; } + } - @Override - public void beforeAttempt( - final HandlerBeforeAttemptContext context) { - ActiveAuditManagerS3A.this.beforeAttempt(context); + @Override + public void beforeUnmarshalling(Context.BeforeUnmarshalling context, + ExecutionAttributes executionAttributes) { + try { + extractAndActivateSpanFromRequest(context.request(), + executionAttributes) + .beforeUnmarshalling(context, executionAttributes); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; } + } - @Override - public HttpResponse beforeUnmarshalling( - final Request request, - final HttpResponse httpResponse) { - return ActiveAuditManagerS3A.this.beforeUnmarshalling(request, - httpResponse); + @Override + public void afterUnmarshalling(Context.AfterUnmarshalling context, + ExecutionAttributes executionAttributes) { + try { + extractAndActivateSpanFromRequest(context.request(), + executionAttributes) + .afterUnmarshalling(context, executionAttributes); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; } + } - @Override - public void afterAttempt( - final HandlerAfterAttemptContext context) { - ActiveAuditManagerS3A.this.afterAttempt(context); + @Override + public SdkResponse modifyResponse(Context.ModifyResponse context, + ExecutionAttributes executionAttributes) { + try { + return extractAndActivateSpanFromRequest(context.request(), + executionAttributes) + .modifyResponse(context, executionAttributes); + } catch (AuditFailureException e) { + ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol()); + throw e; } } @@ -748,9 +778,8 @@ public void deactivate() { * {@inheritDoc} */ @Override - public T requestCreated( - final T request) { - return span.requestCreated(request); + public void requestCreated(final SdkRequest.Builder builder) { + span.requestCreated(builder); } /** @@ -774,79 +803,132 @@ public void set(final String key, final String value) { /** * Forward to the inner span. - * @param request request - * @param type of request - * @return an updated request. + * {@inheritDoc} */ @Override - public T beforeExecution( - final T request) { - return span.beforeExecution(request); + public void beforeExecution(Context.BeforeExecution context, + ExecutionAttributes executionAttributes) { + span.beforeExecution(context, executionAttributes); } /** * Forward to the inner span. - * @param request request - * @param response response. + * {@inheritDoc} */ @Override - public void afterResponse(final Request request, - final Response response) { - span.afterResponse(request, response); + public void afterExecution(Context.AfterExecution context, + ExecutionAttributes executionAttributes) { + span.afterExecution(context, executionAttributes); } /** * Forward to the inner span. - * @param request request - * @param response response. - * @param exception exception raised. + * {@inheritDoc} */ @Override - public void afterError(final Request request, - final Response response, - final Exception exception) { - span.afterError(request, response, exception); + public void onExecutionFailure(Context.FailedExecution context, + ExecutionAttributes executionAttributes) { + span.onExecutionFailure(context, executionAttributes); } /** * Forward to the inner span. - * @param request request - * @return request to marshall + * {@inheritDoc} */ @Override - public AmazonWebServiceRequest beforeMarshalling( - final AmazonWebServiceRequest request) { - return span.beforeMarshalling(request); + public void beforeMarshalling(Context.BeforeMarshalling context, + ExecutionAttributes executionAttributes) { + span.beforeMarshalling(context, executionAttributes); } /** * Forward to the inner span. - * @param request request + * {@inheritDoc} */ @Override - public void beforeRequest(final Request request) { - span.beforeRequest(request); + public SdkRequest modifyRequest(Context.ModifyRequest context, + ExecutionAttributes executionAttributes) { + return span.modifyRequest(context, executionAttributes); } /** * Forward to the inner span. - * @param context full context, including the request. + * {@inheritDoc} */ @Override - public void beforeAttempt( - final HandlerBeforeAttemptContext context) { - span.beforeAttempt(context); + public void afterMarshalling(Context.AfterMarshalling context, + ExecutionAttributes executionAttributes) { + span.afterMarshalling(context, executionAttributes); } /** * Forward to the inner span. - * - * @param context full context, including the request. + * {@inheritDoc} */ @Override - public void afterAttempt( - final HandlerAfterAttemptContext context) { - span.afterAttempt(context); + public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context, + ExecutionAttributes executionAttributes) { + return span.modifyHttpRequest(context, executionAttributes); + } + + /** + * Forward to the inner span. + * {@inheritDoc} + */ + @Override + public void beforeTransmission(Context.BeforeTransmission context, + ExecutionAttributes executionAttributes) { + span.beforeTransmission(context, executionAttributes); + } + + /** + * Forward to the inner span. + * {@inheritDoc} + */ + @Override + public void afterTransmission(Context.AfterTransmission context, + ExecutionAttributes executionAttributes) { + span.afterTransmission(context, executionAttributes); + } + + /** + * Forward to the inner span. + * {@inheritDoc} + */ + @Override + public SdkHttpResponse modifyHttpResponse(Context.ModifyHttpResponse context, + ExecutionAttributes executionAttributes) { + return span.modifyHttpResponse(context, executionAttributes); + } + + /** + * Forward to the inner span. + * {@inheritDoc} + */ + @Override + public void beforeUnmarshalling(Context.BeforeUnmarshalling context, + ExecutionAttributes executionAttributes) { + span.beforeUnmarshalling(context, executionAttributes); + } + + /** + * Forward to the inner span. + * {@inheritDoc} + */ + @Override + public void afterUnmarshalling(Context.AfterUnmarshalling context, + ExecutionAttributes executionAttributes) { + span.afterUnmarshalling(context, executionAttributes); + } + + /** + * Forward to the inner span. + * {@inheritDoc} + */ + @Override + public SdkResponse modifyResponse(Context.ModifyResponse context, + ExecutionAttributes executionAttributes) { + return span.modifyResponse(context, executionAttributes); } @Override @@ -859,5 +941,4 @@ public String toString() { return sb.toString(); } } - } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java index fcf2015487c48..3a2d9d7f823ee 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java @@ -24,10 +24,14 @@ import java.util.HashMap; import java.util.Map; -import com.amazonaws.AmazonWebServiceRequest; -import com.amazonaws.services.s3.model.DeleteObjectRequest; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.awscore.AwsExecutionAttribute; +import software.amazon.awssdk.core.SdkRequest; +import software.amazon.awssdk.core.interceptor.Context; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.http.SdkHttpRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -258,21 +262,22 @@ private class LoggingAuditSpan extends AbstractAuditSpanImpl { /** * Attach Range of data for GetObject Request. - * @param request given get object request + * @param request the sdk request to be modified + * @param executionAttributes execution attributes for this request */ - private void attachRangeFromRequest(AmazonWebServiceRequest request) { - if (request instanceof GetObjectRequest) { - long[] rangeValue = ((GetObjectRequest) request).getRange(); - if (rangeValue == null || rangeValue.length == 0) { - return; - } - if (rangeValue.length != 2) { - WARN_INCORRECT_RANGE.warn("Expected range to contain 0 or 2 elements." - + " Got {} elements. Ignoring.", rangeValue.length); - return; + private void attachRangeFromRequest(SdkHttpRequest request, + ExecutionAttributes executionAttributes) { + + String operationName = executionAttributes.getAttribute(AwsExecutionAttribute.OPERATION_NAME); + + if (operationName != null && operationName.equals("GetObject")) { + if (request.headers() != null && request.headers().get("Range") != null) { + String[] rangeHeader = request.headers().get("Range").get(0).split("="); + // only set header if range unit is bytes + if (rangeHeader[0].equals("bytes")) { + referrer.set(AuditConstants.PARAM_RANGE, rangeHeader[1]); + } } - String combinedRangeValue = String.format("%d-%d", rangeValue[0], rangeValue[1]); - referrer.set(AuditConstants.PARAM_RANGE, combinedRangeValue); } } @@ -346,64 +351,78 @@ public void set(final String key, final String value) { referrer.set(key, value); } + + /** - * Before execution, the logging auditor always builds - * the referrer header, saves to the outer class - * (where {@link #getLastHeader()} can retrieve it, + * Before transmitting a request, the logging auditor + * always builds the referrer header, saves to the outer + * class (where {@link #getLastHeader()} can retrieve it, * and logs at debug. * If configured to add the header to the S3 logs, it will * be set as the HTTP referrer. - * @param request request - * @param type of request. - * @return the request with any extra headers. + * @param context The current state of the execution, + * including the SDK and current HTTP request. + * @param executionAttributes A mutable set of attributes scoped + * to one specific request/response + * cycle that can be used to give data + * to future lifecycle methods. + * @return The potentially-modified HTTP request that should be + * sent to the service. Must not be null. */ @Override - public T beforeExecution( - final T request) { + public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context, + ExecutionAttributes executionAttributes) { + SdkHttpRequest httpRequest = context.httpRequest(); + SdkRequest sdkRequest = context.request(); + // attach range for GetObject requests - attachRangeFromRequest(request); + attachRangeFromRequest(httpRequest, executionAttributes); + // for delete op, attach the number of files to delete - attachDeleteKeySizeAttribute(request); + attachDeleteKeySizeAttribute(sdkRequest); + // build the referrer header final String header = referrer.buildHttpReferrer(); // update the outer class's field. setLastHeader(header); if (headerEnabled) { // add the referrer header - request.putCustomRequestHeader(HEADER_REFERRER, - header); + httpRequest = httpRequest.toBuilder() + .appendHeader(HEADER_REFERRER, header) + .build(); } if (LOG.isDebugEnabled()) { LOG.debug("[{}] {} Executing {} with {}; {}", currentThreadID(), getSpanId(), getOperationName(), - analyzer.analyze(request), + analyzer.analyze(context.request()), header); } + // now see if the request is actually a blocked multipart request - if (!isMultipartUploadEnabled && isRequestMultipartIO(request)) { + if (!isMultipartUploadEnabled && isRequestMultipartIO(sdkRequest)) { throw new AuditOperationRejectedException("Multipart IO request " - + request + " rejected " + header); + + sdkRequest + " rejected " + header); } - return request; + return httpRequest; } /** * For delete requests, attach delete key size as a referrer attribute. * * @param request the request object. - * @param type of the request. */ - private void attachDeleteKeySizeAttribute(T request) { + private void attachDeleteKeySizeAttribute(SdkRequest request) { + if (request instanceof DeleteObjectsRequest) { - int keySize = ((DeleteObjectsRequest) request).getKeys().size(); - this.set(DELETE_KEYS_SIZE, String.valueOf(keySize)); + int keySize = ((DeleteObjectsRequest) request).delete().objects().size(); + referrer.set(DELETE_KEYS_SIZE, String.valueOf(keySize)); } else if (request instanceof DeleteObjectRequest) { - String key = ((DeleteObjectRequest) request).getKey(); + String key = ((DeleteObjectRequest) request).key(); if (key != null && key.length() > 0) { - this.set(DELETE_KEYS_SIZE, "1"); + referrer.set(DELETE_KEYS_SIZE, "1"); } } } @@ -460,15 +479,13 @@ public boolean isValidSpan() { } @Override - public T requestCreated( - final T request) { + public void requestCreated(final SdkRequest.Builder builder) { String error = "Creating a request outside an audit span " - + analyzer.analyze(request); + + analyzer.analyze(builder.build()); LOG.info(error); if (LOG.isDebugEnabled()) { LOG.debug(error, new AuditFailureException("unaudited")); } - return request; } /** @@ -476,20 +493,22 @@ public T requestCreated( * increment the failure count. * Some requests (e.g. copy part) are not expected in spans due * to how they are executed; these do not trigger failures. - * @param request request - * @param type of request - * @return an updated request. - * @throws AuditFailureException if failure is enabled. + * @param context The current state of the execution, including + * the unmodified SDK request from the service + * client call. + * @param executionAttributes A mutable set of attributes scoped + * to one specific request/response + * cycle that can be used to give data + * to future lifecycle methods. */ @Override - public T beforeExecution( - final T request) { - + public void beforeExecution(Context.BeforeExecution context, + ExecutionAttributes executionAttributes) { String error = "executing a request outside an audit span " - + analyzer.analyze(request); + + analyzer.analyze(context.request()); final String unaudited = getSpanId() + " " + UNAUDITED_OPERATION + " " + error; - if (isRequestNotAlwaysInSpan(request)) { + if (isRequestNotAlwaysInSpan(context.request())) { // can get by auditing during a copy, so don't overreact LOG.debug(unaudited); } else { @@ -500,7 +519,7 @@ public T beforeExecution( } } // now hand off to the superclass for its normal preparation - return super.beforeExecution(request); + super.beforeExecution(context, executionAttributes); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java index d1ebd922e073d..e58c906460daa 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/NoopAuditManagerS3A.java @@ -24,9 +24,8 @@ import java.util.List; import java.util.UUID; -import com.amazonaws.handlers.RequestHandler2; -import com.amazonaws.services.s3.transfer.Transfer; -import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.transfer.s3.progress.TransferListener; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -39,6 +38,7 @@ import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions; import org.apache.hadoop.service.CompositeService; + import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; /** @@ -121,17 +121,13 @@ public AuditSpanS3A createSpan(final String operation, } @Override - public List createRequestHandlers() throws IOException { + public List createExecutionInterceptors() throws IOException { return new ArrayList<>(); } @Override - public TransferStateChangeListener createStateChangeListener() { - return new TransferStateChangeListener() { - public void transferStateChanged(final Transfer transfer, - final Transfer.TransferState state) { - } - }; + public TransferListener createTransferListener() { + return new TransferListener() {}; } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/S3AInternalAuditConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/S3AInternalAuditConstants.java index f82e3d7f1e5e8..c170a2be6611d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/S3AInternalAuditConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/S3AInternalAuditConstants.java @@ -18,10 +18,10 @@ package org.apache.hadoop.fs.s3a.audit.impl; -import com.amazonaws.handlers.HandlerContextKey; +import software.amazon.awssdk.core.interceptor.ExecutionAttribute; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.fs.s3a.audit.AWSAuditEventCallbacks; +import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; /** * Internal constants; not intended for public use, or @@ -34,11 +34,11 @@ private S3AInternalAuditConstants() { } /** - * Handler key for audit span callbacks. - * This is used to bind the handler in the AWS code. + * Exceution attribute for audit span callbacks. + * This is used to retrieve the span in the AWS code. */ - public static final HandlerContextKey - AUDIT_SPAN_HANDLER_CONTEXT = - new HandlerContextKey<>( - "org.apache.hadoop.fs.s3a.audit.AWSAuditEventCallbacks"); + public static final ExecutionAttribute + AUDIT_SPAN_EXECUTION_ATTRIBUTE = + new ExecutionAttribute<>( + "org.apache.hadoop.fs.s3a.audit.AuditSpanS3A"); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractAWSCredentialProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractAWSCredentialProvider.java index 1815285738b0e..4754427a4b118 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractAWSCredentialProvider.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractAWSCredentialProvider.java @@ -21,7 +21,7 @@ import javax.annotation.Nullable; import java.net.URI; -import com.amazonaws.auth.AWSCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import org.apache.hadoop.conf.Configuration; @@ -29,12 +29,9 @@ * Base class for AWS credential providers which * take a URI and config in their constructor. * - * @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider - * as part of upgrading S3A to SDK V2. See HADOOP-18073. */ -@Deprecated public abstract class AbstractAWSCredentialProvider - implements AWSCredentialsProvider { + implements AwsCredentialsProvider { private final URI binding; @@ -65,10 +62,4 @@ public URI getUri() { return binding; } - /** - * Refresh is a no-op by default. - */ - @Override - public void refresh() { - } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractSessionCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractSessionCredentialsProvider.java index 2cdf0880affbb..4ef9d49df1aeb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractSessionCredentialsProvider.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractSessionCredentialsProvider.java @@ -23,8 +23,8 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; -import com.amazonaws.SdkBaseException; -import com.amazonaws.auth.AWSCredentials; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.auth.credentials.AwsCredentials; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.classification.InterfaceAudience; @@ -33,19 +33,17 @@ import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.Retries; + /** * Base class for session credential support. * - * @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider - * as part of upgrading S3A to SDK V2. See HADOOP-18073. */ @InterfaceAudience.Private -@Deprecated public abstract class AbstractSessionCredentialsProvider extends AbstractAWSCredentialProvider { /** Credentials, created in {@link #init()}. */ - private volatile AWSCredentials awsCredentials; + private volatile AwsCredentials awsCredentials; /** Atomic flag for on-demand initialization. */ private final AtomicBoolean initialized = new AtomicBoolean(false); @@ -105,7 +103,7 @@ public boolean isInitialized() { * @return the credentials * @throws IOException on any failure. */ - protected abstract AWSCredentials createCredentials(Configuration config) + protected abstract AwsCredentials createCredentials(Configuration config) throws IOException; /** @@ -115,10 +113,10 @@ protected abstract AWSCredentials createCredentials(Configuration config) * 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 SdkException if one was raised during init * @throws CredentialInitializationException on other failures. */ - public AWSCredentials getCredentials() throws SdkBaseException { + public AwsCredentials resolveCredentials() throws SdkException { // do an on-demand init then raise an AWS SDK exception if // there was a failure. try { @@ -126,8 +124,8 @@ public AWSCredentials getCredentials() throws SdkBaseException { init(); } } catch (IOException e) { - if (e.getCause() instanceof SdkBaseException) { - throw (SdkBaseException) e.getCause(); + if (e.getCause() instanceof SdkException) { + throw (SdkException) e.getCause(); } else { throw new CredentialInitializationException(e.getMessage(), e); } @@ -165,15 +163,16 @@ public IOException getInitializationException() { * 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 { + protected static final class NoCredentials implements AwsCredentials { @Override - public String getAWSAccessKeyId() { + public String accessKeyId() { return null; } @Override - public String getAWSSecretKey() { + public String secretAccessKey() { return null; } } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java index fe816ecfcd90e..626fe4d44683f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java @@ -26,21 +26,23 @@ import java.util.Locale; import java.util.concurrent.TimeUnit; -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 org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.thirdparty.com.google.common.collect.Sets; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; +import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; +import software.amazon.awssdk.services.sts.model.StsException; 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.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.AWSCredentialProviderList; import org.apache.hadoop.fs.s3a.CredentialInitializationException; import org.apache.hadoop.fs.s3a.Retries; @@ -49,9 +51,10 @@ import org.apache.hadoop.fs.s3a.S3ARetryPolicy; import org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.thirdparty.com.google.common.collect.Sets; import static org.apache.hadoop.fs.s3a.Constants.*; -import static org.apache.hadoop.fs.s3a.S3AUtils.buildAWSProviderList; +import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.buildAWSProviderList; /** * Support IAM Assumed roles by instantiating an instance of @@ -61,13 +64,10 @@ * * Classname is used in configuration files; do not move. * - * @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider - * as part of upgrading S3A to SDK V2. See HADOOP-18073. */ @InterfaceAudience.Public @InterfaceStability.Evolving -@Deprecated -public class AssumedRoleCredentialProvider implements AWSCredentialsProvider, +public final class AssumedRoleCredentialProvider implements AwsCredentialsProvider, Closeable { private static final Logger LOG = @@ -78,7 +78,7 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider, public static final String E_NO_ROLE = "Unset property " + ASSUMED_ROLE_ARN; - private final STSAssumeRoleSessionCredentialsProvider stsProvider; + private final StsAssumeRoleCredentialsProvider stsProvider; private final String sessionName; @@ -90,22 +90,24 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider, private final Invoker invoker; + private final StsClient stsClient; + /** * Instantiate. - * This calls {@link #getCredentials()} to fail fast on the inner + * This calls {@link #resolveCredentials()} to fail fast on the inner * role credential retrieval. * @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 + * @throws StsException problems getting credentials */ public AssumedRoleCredentialProvider(@Nullable URI fsUri, Configuration conf) throws IOException { arn = conf.getTrimmed(ASSUMED_ROLE_ARN, ""); if (StringUtils.isEmpty(arn)) { - throw new IOException(E_NO_ROLE); + throw new PathIOException(String.valueOf(fsUri), E_NO_ROLE); } // build up the base provider @@ -114,8 +116,8 @@ public AssumedRoleCredentialProvider(@Nullable URI fsUri, Configuration conf) Arrays.asList( SimpleAWSCredentialsProvider.class, EnvironmentVariableCredentialsProvider.class), - Sets.newHashSet(this.getClass())); - LOG.debug("Credentials to obtain role credentials: {}", credentialsToSTS); + Sets.newHashSet(getClass())); + LOG.debug("Credentials used to obtain role credentials: {}", credentialsToSTS); // then the STS binding sessionName = conf.getTrimmed(ASSUMED_ROLE_SESSION_NAME, @@ -125,29 +127,31 @@ public AssumedRoleCredentialProvider(@Nullable URI fsUri, Configuration conf) String policy = conf.getTrimmed(ASSUMED_ROLE_POLICY, ""); LOG.debug("{}", this); - STSAssumeRoleSessionCredentialsProvider.Builder builder - = new STSAssumeRoleSessionCredentialsProvider.Builder(arn, sessionName); - builder.withRoleSessionDurationSeconds((int) duration); + + AssumeRoleRequest.Builder requestBuilder = + AssumeRoleRequest.builder().roleArn(arn).roleSessionName(sessionName) + .durationSeconds((int) duration); + if (StringUtils.isNotEmpty(policy)) { LOG.debug("Scope down policy {}", policy); - builder.withScopeDownPolicy(policy); + requestBuilder.policy(policy); } + 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 = + stsClient = STSClientFactory.builder( conf, fsUri != null ? fsUri.getHost() : "", credentialsToSTS, endpoint, - region); - // the STS client is not tracked for a shutdown in close(), because it - // (currently) throws an UnsupportedOperationException in shutdown(). - builder.withStsClient(stsbuilder.build()); + region).build(); //now build the provider - stsProvider = builder.build(); + stsProvider = StsAssumeRoleCredentialsProvider.builder() + .refreshRequest(requestBuilder.build()) + .stsClient(stsClient).build(); // to handle STS throttling by the AWS account, we // need to retry @@ -155,21 +159,21 @@ public AssumedRoleCredentialProvider(@Nullable URI fsUri, Configuration conf) // and force in a fail-fast check just to keep the stack traces less // convoluted - getCredentials(); + resolveCredentials(); } /** * Get credentials. * @return the credentials - * @throws AWSSecurityTokenServiceException if none could be obtained. + * @throws StsException if none could be obtained. */ @Override @Retries.RetryRaw - public AWSCredentials getCredentials() { + public AwsCredentials resolveCredentials() { try { - return invoker.retryUntranslated("getCredentials", + return invoker.retryUntranslated("resolveCredentials", true, - stsProvider::getCredentials); + stsProvider::resolveCredentials); } catch (IOException e) { // this is in the signature of retryUntranslated; // its hard to see how this could be raised, but for @@ -178,35 +182,28 @@ public AWSCredentials getCredentials() { throw new CredentialInitializationException( "getCredentials failed: " + e, e); - } catch (AWSSecurityTokenServiceException e) { - LOG.error("Failed to get credentials for role {}", + } catch (SdkClientException e) { + LOG.error("Failed to resolve credentials for role {}", arn, e); throw e; } } - @Override - public void refresh() { - stsProvider.refresh(); - } - /** * Propagate the close() call to the inner stsProvider. */ @Override public void close() { - S3AUtils.closeAutocloseables(LOG, stsProvider, credentialsToSTS); + S3AUtils.closeAutocloseables(LOG, stsProvider, credentialsToSTS, stsClient); } @Override public String toString() { - final StringBuilder sb = new StringBuilder( - "AssumedRoleCredentialProvider{"); - sb.append("role='").append(arn).append('\''); - sb.append(", session'").append(sessionName).append('\''); - sb.append(", duration=").append(duration); - sb.append('}'); - return sb.toString(); + String sb = "AssumedRoleCredentialProvider{" + "role='" + arn + '\'' + + ", session'" + sessionName + '\'' + + ", duration=" + duration + + '}'; + return sb; } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/CredentialProviderListFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/CredentialProviderListFactory.java new file mode 100644 index 0000000000000..b106777dd29cc --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/CredentialProviderListFactory.java @@ -0,0 +1,303 @@ +/* + * 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.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import javax.annotation.Nullable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider; +import software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider; + +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.AWSCredentialProviderList; +import org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider; +import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.fs.s3a.S3AUtils; +import org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider; +import org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider; +import org.apache.hadoop.fs.s3a.adapter.AwsV1BindingSupport; +import org.apache.hadoop.fs.s3a.impl.InstantiationIOException; +import org.apache.hadoop.fs.s3native.S3xLoginHelper; +import org.apache.hadoop.fs.store.LogExactlyOnce; + +import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER; +import static org.apache.hadoop.fs.s3a.adapter.AwsV1BindingSupport.isAwsV1SdkAvailable; + +/** + * This class provides methods to create a {@link AWSCredentialProviderList} + * list of AWS credential providers. + */ +public final class CredentialProviderListFactory { + + private static final Logger LOG = LoggerFactory.getLogger(CredentialProviderListFactory.class); + + /** + * A v1 entry has been remapped. warn once about this and then shut up. + */ + private static final LogExactlyOnce LOG_REMAPPED_ENTRY = new LogExactlyOnce(LOG); + + /** + * 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"; + + /** + * The standard AWS provider list for AWS connections. + */ + public static final List> + STANDARD_AWS_PROVIDERS = Collections.unmodifiableList( + Arrays.asList( + EnvironmentVariableCredentialsProvider.class, + IAMInstanceCredentialsProvider.class, + SimpleAWSCredentialsProvider.class, + TemporaryAWSCredentialsProvider.class)); + + /** V1 credential provider: {@value}. */ + public static final String ANONYMOUS_CREDENTIALS_V1 = + "com.amazonaws.auth.AnonymousAWSCredentials"; + + /** V1 credential provider: {@value}. */ + public static final String EC2_CONTAINER_CREDENTIALS_V1 = + "com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper"; + + /** V1 credential provider: {@value}. */ + public static final String EC2_IAM_CREDENTIALS_V1 = + "com.amazonaws.auth.InstanceProfileCredentialsProvider"; + + /** V2 EC2 instance/container credential provider. */ + public static final String EC2_IAM_CREDENTIALS_V2 = + IAMInstanceCredentialsProvider.class.getName(); + + /** V1 env var credential provider: {@value}. */ + public static final String ENVIRONMENT_CREDENTIALS_V1 = + "com.amazonaws.auth.EnvironmentVariableCredentialsProvider"; + + /** V2 environment variables credential provider. */ + public static final String ENVIRONMENT_CREDENTIALS_V2 = + EnvironmentVariableCredentialsProvider.class.getName(); + + /** V1 profile credential provider: {@value}. */ + public static final String PROFILE_CREDENTIALS_V1 = + "com.amazonaws.auth.profile.ProfileCredentialsProvider"; + + /** V2 environment variables credential provider. */ + public static final String PROFILE_CREDENTIALS_V2 = + ProfileCredentialsProvider.class.getName(); + + /** + * Private map of v1 to v2 credential provider name mapping. + */ + private static final Map V1_V2_CREDENTIAL_PROVIDER_MAP = + initCredentialProvidersMap(); + + private CredentialProviderListFactory() { + } + + /** + * 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 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 createAWSCredentialProviderList( + @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 {}", + binding, credentials); + return credentials; + } + + /** + * Load list of AWS credential provider/credential provider factory classes. + * @param conf configuration + * @param key key + * @param defaultValue list of default values + * @return the list of classes, empty if the default list is empty and + * there was no match for the key in the configuration. + * @throws IOException on a failure to load the list. + */ + private static Collection loadAWSProviderClasses(Configuration conf, + String key, + Class... defaultValue) throws IOException { + final Collection classnames = conf.getTrimmedStringCollection(key); + if (classnames.isEmpty()) { + // empty list; return the defaults + return Arrays.stream(defaultValue).map(c -> c.getName()).collect(Collectors.toList()); + } else { + return classnames; + } + } + + /** + * Maps V1 credential providers to either their equivalent SDK V2 class or hadoop provider. + */ + private static Map initCredentialProvidersMap() { + Map v1v2CredentialProviderMap = new HashMap<>(); + + v1v2CredentialProviderMap.put(ANONYMOUS_CREDENTIALS_V1, + AnonymousAWSCredentialsProvider.NAME); + v1v2CredentialProviderMap.put(EC2_CONTAINER_CREDENTIALS_V1, + EC2_IAM_CREDENTIALS_V2); + v1v2CredentialProviderMap.put(EC2_IAM_CREDENTIALS_V1, + EC2_IAM_CREDENTIALS_V2); + v1v2CredentialProviderMap.put(ENVIRONMENT_CREDENTIALS_V1, + ENVIRONMENT_CREDENTIALS_V2); + v1v2CredentialProviderMap.put(PROFILE_CREDENTIALS_V1, + PROFILE_CREDENTIALS_V2); + + return v1v2CredentialProviderMap; + } + + /** + * 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 configuration key to use + * @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> defaultValues, + final Set> forbidden) throws IOException { + + // build up the base provider + Collection awsClasses = loadAWSProviderClasses(conf, + key, + defaultValues.toArray(new Class[defaultValues.size()])); + + Map v1v2CredentialProviderMap = V1_V2_CREDENTIAL_PROVIDER_MAP; + final Set forbiddenClassnames = + forbidden.stream().map(c -> c.getName()).collect(Collectors.toSet()); + + + // iterate through, checking for forbidden values and then instantiating + // each provider + AWSCredentialProviderList providers = new AWSCredentialProviderList(); + for (String className : awsClasses) { + if (v1v2CredentialProviderMap.containsKey(className)) { + // mapping + + final String mapped = v1v2CredentialProviderMap.get(className); + LOG_REMAPPED_ENTRY.warn("Credentials option {} contains AWS v1 SDK entry {}; mapping to {}", + key, className, mapped); + className = mapped; + } + // now scan the forbidden list. doing this after any mappings ensures the v1 names + // are also blocked + if (forbiddenClassnames.contains(className)) { + throw new InstantiationIOException(InstantiationIOException.Kind.Forbidden, + binding, className, key, E_FORBIDDEN_AWS_PROVIDER, null); + } + + AwsCredentialsProvider provider; + try { + provider = createAWSV2CredentialProvider(conf, className, binding, key); + } catch (InstantiationIOException e) { + // failed to create a v2; try to see if it is a v1 + if (e.getKind() == InstantiationIOException.Kind.IsNotImplementation) { + if (isAwsV1SdkAvailable()) { + // try to create v1 + LOG.debug("Failed to create {} as v2 credentials, trying to instantiate as v1", + className); + try { + provider = + AwsV1BindingSupport.createAWSV1CredentialProvider(conf, className, binding, key); + LOG_REMAPPED_ENTRY.warn("Credentials option {} contains AWS v1 SDK entry {}", + key, className); + } catch (InstantiationIOException ex) { + // if it is something other than non-implementation, throw. + // that way, non-impl messages are about v2 not v1 in the error + if (ex.getKind() != InstantiationIOException.Kind.IsNotImplementation) { + throw ex; + } else { + throw e; + } + } + } else { + LOG.warn("Failed to instantiate {} as AWS v2 SDK credential provider;" + + " AWS V1 SDK is not on the classpth so unable to attempt to" + + " instantiate as a v1 provider", className, e); + throw e; + } + } else { + // any other problem + throw e; + + } + LOG.debug("From provider class {} created Aws provider {}", className, provider); + } + providers.add(provider); + } + return providers; + } + + /** + * Create an AWS v2 credential provider from its class by using reflection. + * @param conf configuration + * @param className credential class name + * @param uri URI of the FS + * @param key configuration key to use + * @return the instantiated class + * @throws IOException on any instantiation failure. + * @see S3AUtils#getInstanceFromReflection + */ + private static AwsCredentialsProvider createAWSV2CredentialProvider(Configuration conf, + String className, + @Nullable URI uri, final String key) throws IOException { + LOG.debug("Credential provider class is {}", className); + return S3AUtils.getInstanceFromReflection(className, conf, uri, AwsCredentialsProvider.class, + "create", key); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java index ca9c518d30048..080b79e7f20d5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java @@ -21,39 +21,69 @@ 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.EC2ContainerCredentialsProviderWrapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider; +import software.amazon.awssdk.auth.credentials.HttpCredentialsProvider; +import software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider; +import software.amazon.awssdk.core.exception.SdkClientException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.maybeExtractIOException; + /** * This is an IAM credential provider which wraps - * an {@code EC2ContainerCredentialsProviderWrapper} + * an {@code ContainerCredentialsProvider} * to provide credentials when the S3A connector is instantiated on AWS EC2 * or the AWS container services. *

    - * When it fails to authenticate, it raises a - * {@link NoAwsCredentialsException} which can be recognized by retry handlers + * The provider is initialized with async credential refresh enabled to be less + * brittle against transient network issues. + *

    + * If the ContainerCredentialsProvider fails to authenticate, then an instance of + * {@link InstanceProfileCredentialsProvider} is created and attemped to + * be used instead, again with async credential refresh enabled. + *

    + * If both credential providers fail, a {@link NoAwsCredentialsException} + * is thrown, which can be recognized by retry handlers * as a non-recoverable failure. *

    * It is implicitly public; marked evolving as we can change its semantics. - * - * @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider - * as part of upgrading S3A to SDK V2. See HADOOP-18073. */ @InterfaceAudience.Public @InterfaceStability.Evolving -@Deprecated public class IAMInstanceCredentialsProvider - implements AWSCredentialsProvider, Closeable { + implements AwsCredentialsProvider, Closeable { - private final AWSCredentialsProvider provider = - new EC2ContainerCredentialsProviderWrapper(); + private static final Logger LOG = + LoggerFactory.getLogger(IAMInstanceCredentialsProvider.class); + + /** + * The credentials provider. + * Initially a container credentials provider, but if that fails + * fall back to the instance profile provider. + */ + private HttpCredentialsProvider iamCredentialsProvider; + /** + * Is the container credentials provider in use? + */ + private boolean isContainerCredentialsProvider; + + /** + * Constructor. + * Build credentials provider with async refresh, + * mark {@link #isContainerCredentialsProvider} as true. + */ public IAMInstanceCredentialsProvider() { + isContainerCredentialsProvider = true; + iamCredentialsProvider = ContainerCredentialsProvider.builder() + .asyncCredentialUpdateEnabled(true) + .build(); } /** @@ -63,23 +93,76 @@ public IAMInstanceCredentialsProvider() { * @throws NoAwsCredentialsException on auth failure to indicate non-recoverable. */ @Override - public AWSCredentials getCredentials() { + public AwsCredentials resolveCredentials() { try { - return provider.getCredentials(); - } catch (AmazonClientException e) { + return getCredentials(); + } catch (SdkClientException e) { + + // if the exception contains an IOE, extract it + // so its type is the immediate cause of this new exception. + Throwable t = e; + final IOException ioe = maybeExtractIOException("IAM endpoint", e); + if (ioe != null) { + t = ioe; + } throw new NoAwsCredentialsException("IAMInstanceCredentialsProvider", - e.getMessage(), - e); + e.getMessage(), t); } } + /** + * First try {@link ContainerCredentialsProvider}, which will throw an exception if credentials + * cannot be retrieved from the container. Then resolve credentials + * using {@link InstanceProfileCredentialsProvider}. + * + * @return credentials + */ + private synchronized AwsCredentials getCredentials() { + try { + return iamCredentialsProvider.resolveCredentials(); + } catch (SdkClientException e) { + LOG.debug("Failed to get credentials from container provider,", e); + if (isContainerCredentialsProvider) { + // create instance profile provider + LOG.debug("Switching to instance provider", e); + + // close it to shut down any thread + iamCredentialsProvider.close(); + isContainerCredentialsProvider = false; + iamCredentialsProvider = InstanceProfileCredentialsProvider.builder() + .asyncCredentialUpdateEnabled(true) + .build(); + return iamCredentialsProvider.resolveCredentials(); + } else { + // already using instance profile provider, so fail + throw e; + } + + } + } + + /** + * Is this a container credentials provider? + * @return true if the container credentials provider is in use; + * false for InstanceProfileCredentialsProvider + */ + public boolean isContainerCredentialsProvider() { + return isContainerCredentialsProvider; + } + @Override - public void refresh() { - provider.refresh(); + public synchronized void close() throws IOException { + // this be true but just for safety... + if (iamCredentialsProvider != null) { + iamCredentialsProvider.close(); + } } @Override - public void close() throws IOException { - // no-op. + public String toString() { + return "IAMInstanceCredentialsProvider{" + + "credentialsProvider=" + iamCredentialsProvider + + ", isContainerCredentialsProvider=" + isContainerCredentialsProvider + + '}'; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialBinding.java index 72d29df3fe0e7..46d35e156d069 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialBinding.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialBinding.java @@ -24,19 +24,18 @@ import java.util.Map; import java.util.concurrent.TimeUnit; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.SdkClientException; -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 software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.model.Credentials; import org.apache.hadoop.thirdparty.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.Invoker; import org.apache.hadoop.fs.s3a.Retries; @@ -77,10 +76,10 @@ private MarshalledCredentialBinding() { public static MarshalledCredentials fromSTSCredentials( final Credentials credentials) { MarshalledCredentials marshalled = new MarshalledCredentials( - credentials.getAccessKeyId(), - credentials.getSecretAccessKey(), - credentials.getSessionToken()); - Date date = credentials.getExpiration(); + credentials.accessKeyId(), + credentials.secretAccessKey(), + credentials.sessionToken()); + Date date = Date.from(credentials.expiration()); marshalled.setExpiration(date != null ? date.getTime() : 0); return marshalled; } @@ -91,11 +90,11 @@ public static MarshalledCredentials fromSTSCredentials( * @return a set of marshalled credentials. */ public static MarshalledCredentials fromAWSCredentials( - final AWSSessionCredentials credentials) { + final AwsSessionCredentials credentials) { return new MarshalledCredentials( - credentials.getAWSAccessKeyId(), - credentials.getAWSSecretKey(), - credentials.getSessionToken()); + credentials.accessKeyId(), + credentials.secretAccessKey(), + credentials.sessionToken()); } /** @@ -156,7 +155,7 @@ public static MarshalledCredentials fromFileSystem( * @throws NoAuthWithAWSException validation failure * @throws NoAwsCredentialsException the credentials are actually empty. */ - public static AWSCredentials toAWSCredentials( + public static AwsCredentials toAWSCredentials( final MarshalledCredentials marshalled, final MarshalledCredentials.CredentialTypeRequired typeRequired, final String component) @@ -173,46 +172,49 @@ public static AWSCredentials toAWSCredentials( final String secretKey = marshalled.getSecretKey(); if (marshalled.hasSessionToken()) { // a session token was supplied, so return session credentials - return new BasicSessionCredentials(accessKey, secretKey, + return AwsSessionCredentials.create(accessKey, secretKey, marshalled.getSessionToken()); } else { // these are full credentials - return new BasicAWSCredentials(accessKey, secretKey); + return AwsBasicCredentials.create(accessKey, secretKey); } } /** * Request a set of credentials from an STS endpoint. * @param parentCredentials the parent credentials needed to talk to STS - * @param awsConf AWS client configuration + * @param configuration AWS client configuration * @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. + * @param bucket bucket name. * @return the credentials * @throws IOException on a failure of the request */ @Retries.RetryTranslated public static MarshalledCredentials requestSessionCredentials( - final AWSCredentialsProvider parentCredentials, - final ClientConfiguration awsConf, + final AwsCredentialsProvider parentCredentials, + final Configuration configuration, final String stsEndpoint, final String stsRegion, final int duration, - final Invoker invoker) throws IOException { + final Invoker invoker, + final String bucket) throws IOException { try { - final AWSSecurityTokenService tokenService = + final StsClient tokenService = STSClientFactory.builder(parentCredentials, - awsConf, + configuration, stsEndpoint.isEmpty() ? null : stsEndpoint, - stsRegion) + stsRegion, + bucket) .build(); try (STSClientFactory.STSClient stsClient = STSClientFactory.createClientConnection( tokenService, invoker)) { return fromSTSCredentials(stsClient.requestSessionCredentials(duration, TimeUnit.SECONDS)); } - } catch (SdkClientException e) { + } catch (SdkException e) { if (stsRegion.isEmpty()) { LOG.error("Region must be provided when requesting session credentials.", e); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialProvider.java index cde4cc94b7852..7dffa157c046c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialProvider.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialProvider.java @@ -21,7 +21,7 @@ import java.io.IOException; import java.net.URI; -import com.amazonaws.auth.AWSCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -40,7 +40,6 @@ */ @InterfaceAudience.Private @InterfaceStability.Unstable -@SuppressWarnings("deprecation") public class MarshalledCredentialProvider extends AbstractSessionCredentialsProvider { @@ -85,7 +84,7 @@ public MarshalledCredentialProvider( * @throws IOException on a failure */ @Override - protected AWSCredentials createCredentials(final Configuration config) + protected AwsCredentials createCredentials(final Configuration config) throws IOException { return toAWSCredentials(credentials, typeRequired, component); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAuthWithAWSException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAuthWithAWSException.java index 7ec13b092c9bc..8f92153b2e1d6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAuthWithAWSException.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAuthWithAWSException.java @@ -21,7 +21,7 @@ import org.apache.hadoop.fs.s3a.CredentialInitializationException; /** - * A specific subclass of {@code AmazonClientException} which is + * A specific subclass of {@code SdkException} which is * used in the S3A retry policy to fail fast when there is any * authentication problem. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RolePolicies.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RolePolicies.java index aa88a320ff2a8..d46c9f6f92152 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RolePolicies.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/RolePolicies.java @@ -80,7 +80,7 @@ private RolePolicies() { * Statement to allow KMS R/W access access, so full use of * SSE-KMS. */ - public static final Statement STATEMENT_ALLOW_SSE_KMS_RW = + public static final Statement STATEMENT_ALLOW_KMS_RW = statement(true, KMS_ALL_KEYS, KMS_ALL_OPERATIONS); /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java index 4779f3c1cb0cd..6f1c7ef801dc3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java @@ -20,31 +20,38 @@ import java.io.Closeable; import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; 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 software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.http.apache.ProxyConfiguration; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.StsClientBuilder; +import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; +import software.amazon.awssdk.services.sts.model.Credentials; +import software.amazon.awssdk.services.sts.model.GetSessionTokenRequest; +import software.amazon.awssdk.thirdparty.org.apache.http.client.utils.URIBuilder; +import org.apache.hadoop.fs.s3a.impl.AWSClientConfig; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.s3a.Constants; 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.Constants.AWS_SERVICE_IDENTIFIER_STS; import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.*; /** @@ -71,17 +78,15 @@ public class STSClientFactory { * @return the builder to call {@code build()} * @throws IOException problem reading proxy secrets */ - public static AWSSecurityTokenServiceClientBuilder builder( + public static StsClientBuilder builder( final Configuration conf, final String bucket, - final AWSCredentialsProvider credentials) throws IOException { - final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket, - Constants.AWS_SERVICE_IDENTIFIER_STS); + final AwsCredentialsProvider credentials) throws IOException { 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); + return builder(credentials, conf, endpoint, region, bucket); } /** @@ -96,64 +101,89 @@ public static AWSSecurityTokenServiceClientBuilder builder( * @return the builder to call {@code build()} * @throws IOException problem reading proxy secrets */ - public static AWSSecurityTokenServiceClientBuilder builder( + public static StsClientBuilder builder( final Configuration conf, final String bucket, - final AWSCredentialsProvider credentials, + final AwsCredentialsProvider credentials, final String stsEndpoint, final String stsRegion) throws IOException { - final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket, - Constants.AWS_SERVICE_IDENTIFIER_STS); - return builder(credentials, awsConf, stsEndpoint, stsRegion); + return builder(credentials, conf, stsEndpoint, stsRegion, bucket); } /** * 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 conf 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. + * @param bucket bucket name * @return the builder to call {@code build()} + * @throws IOException problem reading proxy secrets */ - public static AWSSecurityTokenServiceClientBuilder builder( - final AWSCredentialsProvider credentials, - final ClientConfiguration awsConf, - final String stsEndpoint, - final String stsRegion) { - final AWSSecurityTokenServiceClientBuilder builder - = AWSSecurityTokenServiceClientBuilder.standard(); + public static StsClientBuilder builder(final AwsCredentialsProvider credentials, + final Configuration conf, final String stsEndpoint, final String stsRegion, + final String bucket) throws IOException { + final StsClientBuilder stsClientBuilder = StsClient.builder(); + Preconditions.checkArgument(credentials != null, "No credentials"); - builder.withClientConfiguration(awsConf); - builder.withCredentials(credentials); + + final ClientOverrideConfiguration.Builder clientOverrideConfigBuilder = + AWSClientConfig.createClientConfigBuilder(conf, AWS_SERVICE_IDENTIFIER_STS); + + final ApacheHttpClient.Builder httpClientBuilder = + AWSClientConfig.createHttpClientBuilder(conf); + + final RetryPolicy.Builder retryPolicyBuilder = AWSClientConfig.createRetryPolicyBuilder(conf); + + final ProxyConfiguration proxyConfig = AWSClientConfig.createProxyConfiguration(conf, bucket); + + clientOverrideConfigBuilder.retryPolicy(retryPolicyBuilder.build()); + httpClientBuilder.proxyConfiguration(proxyConfig); + + stsClientBuilder.httpClientBuilder(httpClientBuilder) + .overrideConfiguration(clientOverrideConfigBuilder.build()) + .credentialsProvider(credentials); + 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); + 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)); + stsClientBuilder.endpointOverride(getSTSEndpoint(stsEndpoint)).region(Region.of(stsRegion)); } else { Preconditions.checkArgument(isEmpty(stsRegion), - "STS signing region set set to %s but no STS endpoint specified", - stsRegion); + "STS signing region set set to %s but no STS endpoint specified", stsRegion); + } + return stsClientBuilder; + } + + /** + * Given a endpoint string, create the endpoint URI. + * + * @param endpoint possibly null endpoint. + * @return an endpoint uri + */ + private static URI getSTSEndpoint(String endpoint) { + try { + return new URIBuilder().setScheme("https").setHost(endpoint).build(); + } catch (URISyntaxException e) { + throw new IllegalArgumentException(e); } - return builder; } + /** * Create an STS Client instance. - * @param tokenService STS instance + * @param stsClient STS instance * @param invoker invoker to use * @return an STS client bonded to that interface. */ public static STSClient createClientConnection( - final AWSSecurityTokenService tokenService, + final StsClient stsClient, final Invoker invoker) { - return new STSClient(tokenService, invoker); + return new STSClient(stsClient, invoker); } /** @@ -161,21 +191,19 @@ public static STSClient createClientConnection( */ public static final class STSClient implements Closeable { - private final AWSSecurityTokenService tokenService; + private final StsClient stsClient; private final Invoker invoker; - private STSClient(final AWSSecurityTokenService tokenService, + private STSClient(final StsClient stsClient, final Invoker invoker) { - this.tokenService = tokenService; + this.stsClient = stsClient; this.invoker = invoker; } @Override public void close() throws IOException { - // Since we are not using AbstractAWSSecurityTokenService, we - // don't need to worry about catching UnsupportedOperationException. - tokenService.shutdown(); + stsClient.close(); } /** @@ -192,13 +220,13 @@ public Credentials requestSessionCredentials( 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); + final GetSessionTokenRequest request = + GetSessionTokenRequest.builder().durationSeconds(durationSeconds).build(); return invoker.retry("request session credentials", "", true, () ->{ LOG.info("Requesting Amazon STS Session credentials"); - return tokenService.getSessionToken(request).getCredentials(); + return stsClient.getSessionToken(request).credentials(); }); } @@ -222,15 +250,14 @@ public Credentials requestRole( 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); + AssumeRoleRequest.Builder requestBuilder = + AssumeRoleRequest.builder().durationSeconds((int) timeUnit.toSeconds(duration)) + .roleArn(roleARN).roleSessionName(sessionName); if (isNotEmpty(policy)) { - request.setPolicy(policy); + requestBuilder.policy(policy); } return invoker.retry("request role credentials", "", true, - () -> tokenService.assumeRole(request).getCredentials()); + () -> stsClient.assumeRole(requestBuilder.build()).credentials()); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerFactory.java new file mode 100644 index 0000000000000..21c390c07940b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerFactory.java @@ -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; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.signer.Aws4Signer; +import software.amazon.awssdk.auth.signer.Aws4UnsignedPayloadSigner; +import software.amazon.awssdk.auth.signer.AwsS3V4Signer; +import software.amazon.awssdk.core.signer.NoOpSigner; +import software.amazon.awssdk.core.signer.Signer; + +import org.apache.hadoop.fs.s3a.S3AUtils; +import org.apache.hadoop.fs.s3a.impl.InstantiationIOException; + +import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.unavailable; +import static org.apache.hadoop.util.Preconditions.checkArgument; + +/** + * Signer factory used to register and create signers. + */ +public final class SignerFactory { + + private static final Logger LOG = LoggerFactory.getLogger(SignerFactory.class); + public static final String VERSION_FOUR_SIGNER = "AWS4SignerType"; + public static final String VERSION_FOUR_UNSIGNED_PAYLOAD_SIGNER = "AWS4UnsignedPayloadSignerType"; + public static final String NO_OP_SIGNER = "NoOpSignerType"; + private static final String S3_V4_SIGNER = "AWSS3V4SignerType"; + + /** The v2 signer is no longer available: {@value}. */ + public static final String S3_V2_SIGNER = "S3SignerType"; + + private static final Map> SIGNERS + = new ConcurrentHashMap<>(); + + static { + // Register the standard signer types. + SIGNERS.put(VERSION_FOUR_SIGNER, Aws4Signer.class); + SIGNERS.put(VERSION_FOUR_UNSIGNED_PAYLOAD_SIGNER, Aws4UnsignedPayloadSigner.class); + SIGNERS.put(NO_OP_SIGNER, NoOpSigner.class); + SIGNERS.put(S3_V4_SIGNER, AwsS3V4Signer.class); + } + + + private SignerFactory() { + } + + /** + * Register an implementation class for the given signer type. + * + * @param signerType The name of the signer type to register. + * @param signerClass The class implementing the given signature protocol. + */ + public static void registerSigner( + final String signerType, + final Class signerClass) { + + checkArgument(signerType != null, "signerType cannot be null"); + checkArgument(signerClass != null, "signerClass cannot be null"); + + SIGNERS.put(signerType, signerClass); + } + + /** + * Check if the signer has already been registered. + * @param signerType signer to get + * @return true if the signer is registered. + */ + public static boolean isSignerRegistered(String signerType) { + return SIGNERS.containsKey(signerType); + } + + /** + * Create an instance of the given signer. + * + * @param signerType The signer type. + * @param configKey Config key used to configure the signer. + * @return The new signer instance. + * @throws InstantiationIOException instantiation problems. + * @throws IOException on any other problem. + * + */ + public static Signer createSigner(String signerType, String configKey) throws IOException { + if (S3_V2_SIGNER.equals(signerType)) { + throw unavailable(null, null, configKey, S3_V2_SIGNER + " is no longer supported"); + } + if (!isSignerRegistered(signerType)) { + throw unavailable(null, null, configKey, "unknown signer type: " + signerType); + } + Class signerClass = SIGNERS.get(signerType); + + String className = signerClass.getName(); + + LOG.debug("Signer class from {} and key {} is {}", signerType, configKey, className); + + Signer signer = + S3AUtils.getInstanceFromReflection(className, null, null, Signer.class, "create", + configKey); + + return signer; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerManager.java index e162428787cc4..f0a71007b4cf8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerManager.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerManager.java @@ -22,14 +22,12 @@ import java.util.LinkedList; import java.util.List; -import com.amazonaws.auth.Signer; -import com.amazonaws.auth.SignerFactory; +import software.amazon.awssdk.core.signer.Signer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider; -import org.apache.hadoop.fs.s3a.impl.V2Migration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.ReflectionUtils; @@ -71,8 +69,6 @@ public void initCustomSigners() { return; } - V2Migration.v1CustomSignerUsed(); - for (String customSigner : customSigners) { String[] parts = customSigner.split(":"); if (!(parts.length == 1 || parts.length == 2 || parts.length == 3)) { @@ -113,17 +109,20 @@ public void initCustomSigners() { } } - /* - * Make sure the signer class is registered once with the AWS SDK + /** + * Make sure the signer class is registered once with the AWS SDK. + * @param signerName signer name + * @param signerClassName classname + * @param conf source configuration + * @throws RuntimeException if the class is not found */ private static void maybeRegisterSigner(String signerName, String signerClassName, Configuration conf) { - try { - SignerFactory.getSignerByTypeAndService(signerName, null); - } catch (IllegalArgumentException e) { + + if (!SignerFactory.isSignerRegistered(signerName)) { // Signer is not registered with the AWS SDK. // Load the class and register the signer. - Class clazz = null; + Class clazz; try { clazz = (Class) conf.getClassByName(signerClassName); } catch (ClassNotFoundException cnfe) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java index 6526f9a947815..8a55a970134f3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java @@ -20,9 +20,6 @@ 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; /** @@ -35,37 +32,31 @@ 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. + /*** + * Gets the SSE-C client side key if present. + * * @param secrets source of the encryption secrets. + * @return an optional key to attach to a request. */ - public static Optional createSSECustomerKey( - final EncryptionSecrets secrets) { - if (secrets.hasEncryptionKey() && - secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_C) { - return Optional.of(new SSECustomerKey(secrets.getEncryptionKey())); + public static Optional getSSECustomerKey(final EncryptionSecrets secrets) { + if (secrets.hasEncryptionKey() && secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_C) { + return Optional.of(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. + * Gets the SSE-KMS key if present, else let S3 use AWS managed key. + * * @param secrets source of the encryption secrets. + * @return an optional key to attach to a request. */ - public static Optional 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()); - } + public static Optional getSSEAwsKMSKey(final EncryptionSecrets secrets) { + if ((secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_KMS + || secrets.getEncryptionMethod() == S3AEncryptionMethods.DSSE_KMS) + && secrets.hasEncryptionKey()) { + return Optional.of(secrets.getEncryptionKey()); } else { return Optional.empty(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/RoleTokenBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/RoleTokenBinding.java index 8a9ee30f64e5c..75d907aac81b0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/RoleTokenBinding.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/RoleTokenBinding.java @@ -23,7 +23,7 @@ import java.util.UUID; import java.util.concurrent.TimeUnit; -import com.amazonaws.services.securitytoken.model.Credentials; +import software.amazon.awssdk.services.sts.model.Credentials; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java index 2fe6be61e2583..f46c1f9405eb2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java @@ -120,11 +120,6 @@ public class S3ADelegationTokens extends AbstractDTService { */ private AbstractDelegationTokenBinding tokenBinding; - /** - * List of cred providers; unset until {@link #bindToDelegationToken(Token)}. - */ - //private Optional credentialProviders = Optional.empty(); - /** * delegation binding information; unset until {@link #bindToDelegationToken(Token)}. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java index 87fbdf53ccb11..2b21ea173ef01 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java @@ -26,21 +26,18 @@ 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 software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; +import software.amazon.awssdk.services.sts.StsClient; import org.apache.hadoop.thirdparty.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.Constants; 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; @@ -50,8 +47,8 @@ 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.CredentialProviderListFactory.STANDARD_AWS_PROVIDERS; +import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.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.*; @@ -105,7 +102,8 @@ public class SessionTokenBinding extends AbstractDelegationTokenBinding { private boolean hasSessionCreds; /** - * The auth chain for the parent options. + * The parent authentication chain: that used to request + * session/role credentials when deployed unbonded. */ private AWSCredentialProviderList parentAuthChain; @@ -164,12 +162,14 @@ protected void serviceStart() throws Exception { DEFAULT_DELEGATION_TOKEN_REGION); // create the provider set for session credentials. - parentAuthChain = buildAWSProviderList( + final AWSCredentialProviderList chain = buildAWSProviderList( getCanonicalUri(), conf, AWS_CREDENTIALS_PROVIDER, STANDARD_AWS_PROVIDERS, new HashSet<>()); + LOG.debug("Setting parent authentication chain to {}", chain); + setParentAuthChain(chain); } @Override @@ -192,7 +192,7 @@ protected void serviceStop() throws Exception { public AWSCredentialProviderList deployUnbonded() throws IOException { requireServiceStarted(); - return parentAuthChain; + return getParentAuthChain(); } /** @@ -292,23 +292,22 @@ private synchronized Optional maybeInitSTS() // chain. // As no codepath (session propagation, STS creation) will work, // throw this. - final AWSCredentials parentCredentials = once("get credentials", + final AwsCredentials parentCredentials = once("get credentials", "", - () -> parentAuthChain.getCredentials()); - hasSessionCreds = parentCredentials instanceof AWSSessionCredentials; + () -> getParentAuthChain().resolveCredentials()); + hasSessionCreds = parentCredentials instanceof AwsSessionCredentials; if (!hasSessionCreds) { LOG.debug("Creating STS client for {}", getDescription()); invoker = new Invoker(new S3ARetryPolicy(conf), LOG_EVENT); - ClientConfiguration awsConf = - S3AUtils.createAwsConf(conf, uri.getHost(), - Constants.AWS_SERVICE_IDENTIFIER_STS); - AWSSecurityTokenService tokenService = - STSClientFactory.builder(parentAuthChain, - awsConf, + + StsClient tokenService = + STSClientFactory.builder(getParentAuthChain(), + conf, endpoint, - region) + region, + uri.getHost()) .build(); stsClient = Optional.of( STSClientFactory.createClientConnection(tokenService, invoker)); @@ -374,11 +373,11 @@ public SessionTokenIdentifier createTokenIdentifier( + " -duration unknown", getCanonicalUri()); } origin += " " + CREDENTIALS_CONVERTED_TO_DELEGATION_TOKEN; - final AWSCredentials awsCredentials - = parentAuthChain.getCredentials(); - if (awsCredentials instanceof AWSSessionCredentials) { + final AwsCredentials awsCredentials + = getParentAuthChain().resolveCredentials(); + if (awsCredentials instanceof AwsSessionCredentials) { marshalledCredentials = fromAWSCredentials( - (AWSSessionCredentials) awsCredentials); + (AwsSessionCredentials) awsCredentials); } else { throw new DelegationTokenIOException( "AWS Authentication chain is no longer supplying session secrets"); @@ -425,4 +424,16 @@ protected void setTokenIdentifier(Optional tokenIdentifier) { this.tokenIdentifier = tokenIdentifier; } + + /** + * The auth chain for the parent options. + * @return the parent authentication chain. + */ + protected AWSCredentialProviderList getParentAuthChain() { + return parentAuthChain; + } + + protected void setParentAuthChain(AWSCredentialProviderList parentAuthChain) { + this.parentAuthChain = parentAuthChain; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java index e53c690431ee0..09664a6dbdf63 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java @@ -27,10 +27,11 @@ import java.util.List; import java.util.UUID; -import com.amazonaws.services.s3.model.MultipartUpload; +import software.amazon.awssdk.services.s3.model.MultipartUpload; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.VisibleForTesting; @@ -975,7 +976,7 @@ protected void abortPendingUploadsInCleanup( .executeWith(commitContext.getOuterSubmitter()) .suppressExceptions(suppressExceptions) .run(u -> commitContext.abortMultipartCommit( - u.getKey(), u.getUploadId())); + u.key(), u.uploadId())); } else { LOG.info("No pending uploads were found"); } @@ -1300,8 +1301,8 @@ protected void warnOnActiveUploads(final Path path) { DateFormat df = DateFormat.getDateTimeInstance(); pending.forEach(u -> LOG.info("[{}] {}", - df.format(u.getInitiated()), - u.getKey())); + df.format(Date.from(u.initiated())), + u.key())); if (shouldAbortUploadsInCleanup()) { LOG.warn("This committer will abort these uploads in job cleanup"); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java index 10440f77e7277..6c3cf3942d527 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java @@ -21,7 +21,7 @@ import java.io.IOException; import java.util.List; -import com.amazonaws.services.s3.model.PartETag; +import software.amazon.awssdk.services.s3.model.CompletedPart; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.statistics.IOStatistics; @@ -76,7 +76,7 @@ public boolean outputImmediatelyVisible() { * @throws IOException I/O problem or validation failure. */ public boolean aboutToComplete(String uploadId, - List parts, + List parts, long bytesWritten, final IOStatistics iostatistics) throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java index 77c3fed11fb24..e4541ba4da370 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java @@ -31,9 +31,10 @@ import java.util.List; import java.util.Map; -import com.amazonaws.services.s3.model.PartETag; +import software.amazon.awssdk.services.s3.model.CompletedPart; import com.fasterxml.jackson.annotation.JsonProperty; + import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -215,13 +216,13 @@ public void touch(long millis) { * @param parts ordered list of etags. * @throws ValidationFailure if the data is invalid */ - public void bindCommitData(List parts) throws ValidationFailure { + public void bindCommitData(List parts) throws ValidationFailure { etags = new ArrayList<>(parts.size()); int counter = 1; - for (PartETag part : parts) { - verify(part.getPartNumber() == counter, - "Expected part number %s but got %s", counter, part.getPartNumber()); - etags.add(part.getETag()); + for (CompletedPart part : parts) { + verify(part.partNumber() == counter, + "Expected part number %s but got %s", counter, part.partNumber()); + etags.add(part.eTag()); counter++; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java index ef56d82978158..d1943fa47773f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java @@ -21,6 +21,8 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -32,13 +34,15 @@ import javax.annotation.Nullable; -import com.amazonaws.services.s3.model.MultipartUpload; -import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.UploadPartRequest; -import com.amazonaws.services.s3.model.UploadPartResult; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.services.s3.model.CompletedPart; +import software.amazon.awssdk.services.s3.model.MultipartUpload; +import software.amazon.awssdk.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.services.s3.model.UploadPartResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; @@ -155,9 +159,9 @@ public CommitOperations(S3AFileSystem fs, * @param tagIds list of tags * @return same list, now in numbered tuples */ - public static List toPartEtags(List tagIds) { + public static List toPartEtags(List tagIds) { return IntStream.range(0, tagIds.size()) - .mapToObj(i -> new PartETag(i + 1, tagIds.get(i))) + .mapToObj(i -> CompletedPart.builder().partNumber(i + 1).eTag(tagIds.get(i)).build()) .collect(Collectors.toList()); } @@ -566,26 +570,30 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile, numParts, length)); } - List parts = new ArrayList<>((int) numParts); + List parts = new ArrayList<>((int) numParts); LOG.debug("File size is {}, number of parts to upload = {}", length, numParts); - for (int partNumber = 1; partNumber <= numParts; partNumber += 1) { - progress.progress(); - long size = Math.min(length - offset, uploadPartSize); - UploadPartRequest part; - part = writeOperations.newUploadPartRequest( - destKey, - uploadId, - partNumber, - (int) size, - null, - localFile, - offset); - part.setLastPart(partNumber == numParts); - UploadPartResult partResult = writeOperations.uploadPart(part, statistics); - offset += uploadPartSize; - parts.add(partResult.getPartETag()); + + // Open the file to upload. + try (InputStream fileStream = Files.newInputStream(localFile.toPath())) { + for (int partNumber = 1; partNumber <= numParts; partNumber += 1) { + progress.progress(); + long size = Math.min(length - offset, uploadPartSize); + UploadPartRequest part = writeOperations.newUploadPartRequestBuilder( + destKey, + uploadId, + partNumber, + size).build(); + // Read from the file input stream at current position. + RequestBody body = RequestBody.fromInputStream(fileStream, size); + UploadPartResponse response = writeOperations.uploadPart(part, body, statistics); + offset += uploadPartSize; + parts.add(CompletedPart.builder() + .partNumber(partNumber) + .eTag(response.eTag()) + .build()); + } } commitData.bindCommitData(parts); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java index 1a5451df801dd..b2e703e1b088d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java @@ -20,19 +20,22 @@ import java.io.ByteArrayInputStream; import java.io.IOException; +import java.io.InputStream; import java.util.HashMap; import java.util.List; import java.util.Map; -import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.CompletedPart; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.S3ADataBlocks; import org.apache.hadoop.fs.s3a.WriteOperationHelper; import org.apache.hadoop.fs.s3a.commit.PutTracker; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; @@ -125,7 +128,7 @@ public boolean outputImmediatelyVisible() { */ @Override public boolean aboutToComplete(String uploadId, - List parts, + List parts, long bytesWritten, final IOStatistics iostatistics) throws IOException { @@ -144,10 +147,9 @@ public boolean aboutToComplete(String uploadId, headers.put(X_HEADER_MAGIC_MARKER, Long.toString(bytesWritten)); PutObjectRequest originalDestPut = writer.createPutObjectRequest( originalDestKey, - new ByteArrayInputStream(EMPTY), 0, - new PutObjectOptions(true, null, headers)); - upload(originalDestPut); + new PutObjectOptions(true, null, headers), false); + upload(originalDestPut, new ByteArrayInputStream(EMPTY)); // build the commit summary SinglePendingCommit commitData = new SinglePendingCommit(); @@ -170,22 +172,22 @@ public boolean aboutToComplete(String uploadId, path, pendingPartKey, commitData); PutObjectRequest put = writer.createPutObjectRequest( pendingPartKey, - new ByteArrayInputStream(bytes), - bytes.length, null); - upload(put); + bytes.length, null, false); + upload(put, new ByteArrayInputStream(bytes)); return false; } /** * PUT an object. * @param request the request + * @param inputStream input stream of data to be uploaded * @throws IOException on problems */ @Retries.RetryTranslated - private void upload(PutObjectRequest request) throws IOException { - trackDurationOfInvocation(trackerStatistics, - COMMITTER_MAGIC_MARKER_PUT.getSymbol(), () -> - writer.putObject(request, PutObjectOptions.keepingDirs(), null)); + private void upload(PutObjectRequest request, InputStream inputStream) throws IOException { + trackDurationOfInvocation(trackerStatistics, COMMITTER_MAGIC_MARKER_PUT.getSymbol(), + () -> writer.putObject(request, PutObjectOptions.keepingDirs(), + new S3ADataBlocks.BlockUploadData(inputStream), false, null)); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSCannedACL.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSCannedACL.java new file mode 100644 index 0000000000000..12a89d50f6b0d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSCannedACL.java @@ -0,0 +1,43 @@ +/* + * 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.impl; + +/** + * Enum to map AWS SDK V1 Acl values to SDK V2. + */ +public enum AWSCannedACL { + Private("private"), + PublicRead("public-read"), + PublicReadWrite("public-read-write"), + AuthenticatedRead("authenticated-read"), + AwsExecRead("aws-exec-read"), + BucketOwnerRead("bucket-owner-read"), + BucketOwnerFullControl("bucket-owner-full-control"), + LogDeliveryWrite("log-delivery-write"); + + private final String value; + + AWSCannedACL(String value){ + this.value = value; + } + + public String toString() { + return this.value; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSClientConfig.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSClientConfig.java new file mode 100644 index 0000000000000..3400daff509b2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSClientConfig.java @@ -0,0 +1,394 @@ +/* + * 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.impl; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.time.Duration; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.http.apache.ProxyConfiguration; +import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.S3AUtils; +import org.apache.hadoop.fs.s3a.auth.SignerFactory; +import org.apache.hadoop.util.VersionInfo; +import org.apache.http.client.utils.URIBuilder; + +import static org.apache.hadoop.fs.s3a.Constants.AWS_SERVICE_IDENTIFIER_S3; +import static org.apache.hadoop.fs.s3a.Constants.AWS_SERVICE_IDENTIFIER_STS; +import static org.apache.hadoop.fs.s3a.Constants.CONNECTION_TTL; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_CONNECTION_TTL; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ESTABLISH_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAXIMUM_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAX_ERROR_RETRIES; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_REQUEST_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_SOCKET_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.ESTABLISH_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.MAX_ERROR_RETRIES; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_DOMAIN; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_HOST; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_PASSWORD; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_PORT; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_SECURED; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_USERNAME; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_WORKSTATION; +import static org.apache.hadoop.fs.s3a.Constants.REQUEST_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.SIGNING_ALGORITHM; +import static org.apache.hadoop.fs.s3a.Constants.SIGNING_ALGORITHM_S3; +import static org.apache.hadoop.fs.s3a.Constants.SIGNING_ALGORITHM_STS; +import static org.apache.hadoop.fs.s3a.Constants.SOCKET_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.USER_AGENT_PREFIX; +import static org.apache.hadoop.fs.s3a.S3AUtils.longOption; + +/** + * Methods for configuring the S3 client. + * These methods are used when creating and configuring + * {@link software.amazon.awssdk.services.s3.S3Client} which communicates with the S3 service. + */ +public final class AWSClientConfig { + private static final Logger LOG = LoggerFactory.getLogger(AWSClientConfig.class); + + private AWSClientConfig() { + } + + public static ClientOverrideConfiguration.Builder createClientConfigBuilder(Configuration conf, + String awsServiceIdentifier) throws IOException { + ClientOverrideConfiguration.Builder overrideConfigBuilder = + ClientOverrideConfiguration.builder(); + + initRequestTimeout(conf, overrideConfigBuilder); + + initUserAgent(conf, overrideConfigBuilder); + + String signer = conf.getTrimmed(SIGNING_ALGORITHM, ""); + if (!signer.isEmpty()) { + LOG.debug("Signer override = {}", signer); + overrideConfigBuilder.putAdvancedOption(SdkAdvancedClientOption.SIGNER, + SignerFactory.createSigner(signer, SIGNING_ALGORITHM)); + } + + initSigner(conf, overrideConfigBuilder, awsServiceIdentifier); + + return overrideConfigBuilder; + } + + /** + * Configures the http client. + * + * @param conf The Hadoop configuration + * @return Http client builder + * @throws IOException on any problem + */ + public static ApacheHttpClient.Builder createHttpClientBuilder(Configuration conf) + throws IOException { + ApacheHttpClient.Builder httpClientBuilder = + ApacheHttpClient.builder(); + + httpClientBuilder.maxConnections(S3AUtils.intOption(conf, MAXIMUM_CONNECTIONS, + DEFAULT_MAXIMUM_CONNECTIONS, 1)); + + int connectionEstablishTimeout = + S3AUtils.intOption(conf, ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT, 0); + int socketTimeout = S3AUtils.intOption(conf, SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT, 0); + + httpClientBuilder.connectionTimeout(Duration.ofMillis(connectionEstablishTimeout)); + httpClientBuilder.socketTimeout(Duration.ofMillis(socketTimeout)); + + // set the connection TTL irrespective of whether the connection is in use or not. + // this can balance requests over different S3 servers, and avoid failed + // connections. See HADOOP-18845. + long ttl = longOption(conf, CONNECTION_TTL, DEFAULT_CONNECTION_TTL, -1); + httpClientBuilder.connectionTimeToLive(Duration.ofMillis(ttl)); + + NetworkBinding.bindSSLChannelMode(conf, httpClientBuilder); + + return httpClientBuilder; + } + + /** + * Configures the async http client. + * + * @param conf The Hadoop configuration + * @return Http client builder + */ + public static NettyNioAsyncHttpClient.Builder createAsyncHttpClientBuilder(Configuration conf) { + NettyNioAsyncHttpClient.Builder httpClientBuilder = + NettyNioAsyncHttpClient.builder(); + + httpClientBuilder.maxConcurrency(S3AUtils.intOption(conf, MAXIMUM_CONNECTIONS, + DEFAULT_MAXIMUM_CONNECTIONS, 1)); + + int connectionEstablishTimeout = + S3AUtils.intOption(conf, ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT, 0); + int socketTimeout = S3AUtils.intOption(conf, SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT, 0); + + httpClientBuilder.connectionTimeout(Duration.ofMillis(connectionEstablishTimeout)); + httpClientBuilder.readTimeout(Duration.ofMillis(socketTimeout)); + httpClientBuilder.writeTimeout(Duration.ofMillis(socketTimeout)); + + // set the connection TTL irrespective of whether the connection is in use or not. + // this can balance requests over different S3 servers, and avoid failed + // connections. See HADOOP-18845. + long ttl = longOption(conf, CONNECTION_TTL, DEFAULT_CONNECTION_TTL, -1); + httpClientBuilder.connectionTimeToLive(Duration.ofMillis(ttl)); + + // TODO: Don't think you can set a socket factory for the netty client. + // NetworkBinding.bindSSLChannelMode(conf, awsConf); + + return httpClientBuilder; + } + + /** + * Configures the retry policy. + * + * @param conf The Hadoop configuration + * @return Retry policy builder + */ + public static RetryPolicy.Builder createRetryPolicyBuilder(Configuration conf) { + + RetryPolicy.Builder retryPolicyBuilder = RetryPolicy.builder(); + + retryPolicyBuilder.numRetries(S3AUtils.intOption(conf, MAX_ERROR_RETRIES, + DEFAULT_MAX_ERROR_RETRIES, 0)); + + return retryPolicyBuilder; + } + + /** + * Configures the proxy. + * + * @param conf The Hadoop configuration + * @param bucket Optional bucket to use to look up per-bucket proxy secrets + * @return Proxy configuration + * @throws IOException on any IO problem + */ + public static ProxyConfiguration createProxyConfiguration(Configuration conf, + String bucket) throws IOException { + + ProxyConfiguration.Builder proxyConfigBuilder = ProxyConfiguration.builder(); + + String proxyHost = conf.getTrimmed(PROXY_HOST, ""); + int proxyPort = conf.getInt(PROXY_PORT, -1); + + if (!proxyHost.isEmpty()) { + if (proxyPort >= 0) { + String scheme = conf.getBoolean(PROXY_SECURED, false) ? "https" : "http"; + proxyConfigBuilder.endpoint(buildURI(scheme, proxyHost, proxyPort)); + } else { + if (conf.getBoolean(PROXY_SECURED, false)) { + LOG.warn("Proxy host set without port. Using HTTPS default 443"); + proxyConfigBuilder.endpoint(buildURI("https", proxyHost, 443)); + } else { + LOG.warn("Proxy host set without port. Using HTTP default 80"); + proxyConfigBuilder.endpoint(buildURI("http", proxyHost, 80)); + } + } + final String proxyUsername = S3AUtils.lookupPassword(bucket, conf, PROXY_USERNAME, + null, null); + final String proxyPassword = S3AUtils.lookupPassword(bucket, conf, PROXY_PASSWORD, + null, null); + if ((proxyUsername == null) != (proxyPassword == null)) { + String msg = "Proxy error: " + PROXY_USERNAME + " or " + + PROXY_PASSWORD + " set without the other."; + LOG.error(msg); + throw new IllegalArgumentException(msg); + } + proxyConfigBuilder.username(proxyUsername); + proxyConfigBuilder.password(proxyPassword); + proxyConfigBuilder.ntlmDomain(conf.getTrimmed(PROXY_DOMAIN)); + proxyConfigBuilder.ntlmWorkstation(conf.getTrimmed(PROXY_WORKSTATION)); + if (LOG.isDebugEnabled()) { + LOG.debug("Using proxy server {}:{} as user {} with password {} on " + + "domain {} as workstation {}", proxyHost, proxyPort, proxyUsername, proxyPassword, + PROXY_DOMAIN, PROXY_WORKSTATION); + } + } else if (proxyPort >= 0) { + String msg = + "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST; + LOG.error(msg); + throw new IllegalArgumentException(msg); + } + + return proxyConfigBuilder.build(); + } + + /** + * Configures the proxy for the async http client. + * + * @param conf The Hadoop configuration + * @param bucket Optional bucket to use to look up per-bucket proxy secrets + * @return Proxy configuration + * @throws IOException on any IO problem + */ + public static software.amazon.awssdk.http.nio.netty.ProxyConfiguration + createAsyncProxyConfiguration(Configuration conf, + String bucket) throws IOException { + + software.amazon.awssdk.http.nio.netty.ProxyConfiguration.Builder proxyConfigBuilder = + software.amazon.awssdk.http.nio.netty.ProxyConfiguration.builder(); + + String proxyHost = conf.getTrimmed(PROXY_HOST, ""); + int proxyPort = conf.getInt(PROXY_PORT, -1); + + if (!proxyHost.isEmpty()) { + if (proxyPort >= 0) { + String scheme = conf.getBoolean(PROXY_SECURED, false) ? "https" : "http"; + proxyConfigBuilder.host(proxyHost); + proxyConfigBuilder.port(proxyPort); + proxyConfigBuilder.scheme(scheme); + } else { + if (conf.getBoolean(PROXY_SECURED, false)) { + LOG.warn("Proxy host set without port. Using HTTPS default 443"); + proxyConfigBuilder.host(proxyHost); + proxyConfigBuilder.port(443); + proxyConfigBuilder.scheme("https"); + } else { + LOG.warn("Proxy host set without port. Using HTTP default 80"); + proxyConfigBuilder.host(proxyHost); + proxyConfigBuilder.port(80); + proxyConfigBuilder.scheme("http"); + } + } + final String proxyUsername = S3AUtils.lookupPassword(bucket, conf, PROXY_USERNAME, + null, null); + final String proxyPassword = S3AUtils.lookupPassword(bucket, conf, PROXY_PASSWORD, + null, null); + if ((proxyUsername == null) != (proxyPassword == null)) { + String msg = "Proxy error: " + PROXY_USERNAME + " or " + + PROXY_PASSWORD + " set without the other."; + LOG.error(msg); + throw new IllegalArgumentException(msg); + } + proxyConfigBuilder.username(proxyUsername); + proxyConfigBuilder.password(proxyPassword); + // TODO: check NTLM support + // proxyConfigBuilder.ntlmDomain(conf.getTrimmed(PROXY_DOMAIN)); + // proxyConfigBuilder.ntlmWorkstation(conf.getTrimmed(PROXY_WORKSTATION)); + if (LOG.isDebugEnabled()) { + LOG.debug("Using proxy server {}:{} as user {} with password {} on " + + "domain {} as workstation {}", proxyHost, proxyPort, proxyUsername, proxyPassword, + PROXY_DOMAIN, PROXY_WORKSTATION); + } + } else if (proxyPort >= 0) { + String msg = + "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST; + LOG.error(msg); + throw new IllegalArgumentException(msg); + } else { + return null; + } + + return proxyConfigBuilder.build(); + } + + /*** + * Builds a URI, throws an IllegalArgumentException in case of errors. + * + * @param host proxy host + * @param port proxy port + * @return uri with host and port + */ + private static URI buildURI(String scheme, String host, int port) { + try { + return new URIBuilder().setScheme(scheme).setHost(host).setPort(port).build(); + } catch (URISyntaxException e) { + String msg = + "Proxy error: incorrect " + PROXY_HOST + " or " + PROXY_PORT; + LOG.error(msg); + throw new IllegalArgumentException(msg); + } + } + + /** + * Initializes the User-Agent header to send in HTTP requests to AWS + * services. We always include the Hadoop version number. The user also + * may set an optional custom prefix to put in front of the Hadoop version + * number. The AWS SDK internally appends its own information, which seems + * to include the AWS SDK version, OS and JVM version. + * + * @param conf Hadoop configuration + * @param clientConfig AWS SDK configuration to update + */ + private static void initUserAgent(Configuration conf, + ClientOverrideConfiguration.Builder clientConfig) { + String userAgent = "Hadoop " + VersionInfo.getVersion(); + String userAgentPrefix = conf.getTrimmed(USER_AGENT_PREFIX, ""); + if (!userAgentPrefix.isEmpty()) { + userAgent = userAgentPrefix + ", " + userAgent; + } + LOG.debug("Using User-Agent: {}", userAgent); + clientConfig.putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX, userAgent); + } + + private static void initSigner(Configuration conf, + ClientOverrideConfiguration.Builder clientConfig, String awsServiceIdentifier) + throws IOException { + String configKey = null; + switch (awsServiceIdentifier) { + case AWS_SERVICE_IDENTIFIER_S3: + configKey = SIGNING_ALGORITHM_S3; + break; + case AWS_SERVICE_IDENTIFIER_STS: + configKey = SIGNING_ALGORITHM_STS; + break; + default: + // Nothing to do. The original signer override is already setup + } + if (configKey != null) { + String signerOverride = conf.getTrimmed(configKey, ""); + if (!signerOverride.isEmpty()) { + LOG.debug("Signer override for {} = {}", awsServiceIdentifier, signerOverride); + clientConfig.putAdvancedOption(SdkAdvancedClientOption.SIGNER, + SignerFactory.createSigner(signerOverride, configKey)); + } + } + } + + /** + * Configures request timeout. + * + * @param conf Hadoop configuration + * @param clientConfig AWS SDK configuration to update + */ + private static void initRequestTimeout(Configuration conf, + ClientOverrideConfiguration.Builder clientConfig) { + long requestTimeoutMillis = conf.getTimeDuration(REQUEST_TIMEOUT, + DEFAULT_REQUEST_TIMEOUT, TimeUnit.SECONDS, TimeUnit.MILLISECONDS); + + if (requestTimeoutMillis > Integer.MAX_VALUE) { + LOG.debug("Request timeout is too high({} ms). Setting to {} ms instead", + requestTimeoutMillis, Integer.MAX_VALUE); + requestTimeoutMillis = Integer.MAX_VALUE; + } + + if(requestTimeoutMillis > 0) { + clientConfig.apiCallAttemptTimeout(Duration.ofMillis(requestTimeoutMillis)); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSHeaders.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSHeaders.java new file mode 100644 index 0000000000000..e0d6fa5aecc0b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSHeaders.java @@ -0,0 +1,98 @@ +/* + * 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.impl; + +/** + * Common S3 HTTP header values used throughout the Amazon Web Services S3 Java client. + */ +public interface AWSHeaders { + + /* + * Standard HTTP Headers. + */ + + String CACHE_CONTROL = "Cache-Control"; + String CONTENT_DISPOSITION = "Content-Disposition"; + String CONTENT_ENCODING = "Content-Encoding"; + String CONTENT_LENGTH = "Content-Length"; + String CONTENT_RANGE = "Content-Range"; + String CONTENT_MD5 = "Content-MD5"; + String CONTENT_TYPE = "Content-Type"; + String CONTENT_LANGUAGE = "Content-Language"; + String DATE = "Date"; + String ETAG = "ETag"; + String LAST_MODIFIED = "Last-Modified"; + + /* + * Amazon HTTP Headers used by S3A. + */ + + /** S3's version ID header. */ + String S3_VERSION_ID = "x-amz-version-id"; + + /** Header describing what class of storage a user wants. */ + String STORAGE_CLASS = "x-amz-storage-class"; + + /** Header describing what archive tier the object is in, if any. */ + String ARCHIVE_STATUS = "x-amz-archive-status"; + + /** Header for optional server-side encryption algorithm. */ + String SERVER_SIDE_ENCRYPTION = "x-amz-server-side-encryption"; + + /** Range header for the get object request. */ + String RANGE = "Range"; + + /** + * Encrypted symmetric key header that is used in the Encryption Only (EO) envelope + * encryption mechanism. + */ + @Deprecated + String CRYPTO_KEY = "x-amz-key"; + + /** JSON-encoded description of encryption materials used during encryption. */ + String MATERIALS_DESCRIPTION = "x-amz-matdesc"; + + /** Header for the optional restore information of an object. */ + String RESTORE = "x-amz-restore"; + + /** + * Key wrapping algorithm such as "AESWrap" and "RSA/ECB/OAEPWithSHA-256AndMGF1Padding". + */ + String CRYPTO_KEYWRAP_ALGORITHM = "x-amz-wrap-alg"; + /** + * Content encryption algorithm, such as "AES/GCM/NoPadding". + */ + String CRYPTO_CEK_ALGORITHM = "x-amz-cek-alg"; + + /** + * Headers in request indicating that the requester must be charged for data + * transfer. + */ + String REQUESTER_PAYS_HEADER = "x-amz-request-payer"; + + /** Header for the replication status of an Amazon S3 Object.*/ + String OBJECT_REPLICATION_STATUS = "x-amz-replication-status"; + + String OBJECT_LOCK_MODE = "x-amz-object-lock-mode"; + + String OBJECT_LOCK_RETAIN_UNTIL_DATE = "x-amz-object-lock-retain-until-date"; + + String OBJECT_LOCK_LEGAL_HOLD_STATUS = "x-amz-object-lock-legal-hold"; + +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java index 4169a9899cb15..5808607762ea6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java @@ -20,8 +20,9 @@ import java.util.List; -import com.amazonaws.SdkClientException; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,6 +31,7 @@ import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; + import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException; import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS; import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED; @@ -113,15 +115,15 @@ public void bulkDeleteRetried( * @param deleteRequest request which failed. */ private void onDeleteThrottled(final DeleteObjectsRequest deleteRequest) { - final List keys = deleteRequest.getKeys(); + final List keys = deleteRequest.delete().objects(); final int size = keys.size(); incrementStatistic(STORE_IO_THROTTLED, size); instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, size); THROTTLE_LOG.info( "Bulk delete {} keys throttled -first key = {}; last = {}", size, - keys.get(0).getKey(), - keys.get(size - 1).getKey()); + keys.get(0).key(), + keys.get(size - 1).key()); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java index 0156207419210..e0580df08a76d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.List; +import java.util.Optional; import java.util.concurrent.Callable; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; @@ -155,19 +156,21 @@ public static void waitForCompletion( * Wait for a single of future to complete, extracting IOEs afterwards. * @param future future to wait for. * @param type + * @return the result * @throws IOException if one of the called futures raised an IOE. * @throws RuntimeException if one of the futures raised one. */ - public static void waitForCompletion( + public static T waitForCompletion( final CompletableFuture future) throws IOException { try (DurationInfo ignore = new DurationInfo(LOG, false, "Waiting for task completion")) { - future.join(); + return future.join(); } catch (CancellationException e) { throw new IOException(e); } catch (CompletionException e) { raiseInnerCause(e); + return null; } } @@ -175,31 +178,35 @@ public static void waitForCompletion( * Wait for a single of future to complete, ignoring exceptions raised. * @param future future to wait for. * @param type + * @return the outcome if successfully retrieved. */ - public static void waitForCompletionIgnoringExceptions( + public static Optional waitForCompletionIgnoringExceptions( @Nullable final CompletableFuture future) { - if (future != null) { - try (DurationInfo ignore = - new DurationInfo(LOG, false, "Waiting for task completion")) { - future.join(); - } catch (Exception e) { - LOG.debug("Ignoring exception raised in task completion: "); - } + + try { + return maybeAwaitCompletion(future); + } catch (Exception e) { + LOG.debug("Ignoring exception raised in task completion: ", e); + return Optional.empty(); } } /** * Block awaiting completion for any non-null future passed in; * No-op if a null arg was supplied. + * @param return type * @param future future + * @return the outcome; is empty if the future was null/had no return value * @throws IOException if one of the called futures raised an IOE. * @throws RuntimeException if one of the futures raised one. */ - public static void maybeAwaitCompletion( - @Nullable final CompletableFuture future) + public static Optional maybeAwaitCompletion( + @Nullable final CompletableFuture future) throws IOException { if (future != null) { - waitForCompletion(future); + return Optional.ofNullable(waitForCompletion(future)); + } else { + return Optional.empty(); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java index 205826dadae6f..65291aa7b2659 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java @@ -20,11 +20,12 @@ import java.util.Locale; -import com.amazonaws.services.s3.model.CopyObjectRequest; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.transfer.model.CopyResult; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.CopyObjectResponse; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -211,11 +212,24 @@ public String toString() { * null if the attribute is unavailable (such as when the policy says to use * versionId but object versioning is not enabled for the bucket). */ - public abstract String getRevisionId(ObjectMetadata objectMetadata, + public abstract String getRevisionId(HeadObjectResponse objectMetadata, String uri); /** - * Like {{@link #getRevisionId(ObjectMetadata, String)}}, but retrieves the + * Like {{@link #getRevisionId(HeadObjectResponse, String)}}, but retrieves the + * * revision identifier from {@link GetObjectResponse}. + * + * @param getObjectResponse the response instance + * @param uri the URI of the object + * @return the revisionId string as interpreted by this policy, or potentially + * null if the attribute is unavailable (such as when the policy says to use + * versionId but object versioning is not enabled for the bucket). + */ + public abstract String getRevisionId(GetObjectResponse getObjectResponse, + String uri); + + /** + * Like {{@link #getRevisionId(HeadObjectResponse, String)}}, but retrieves the * revision identifier from {@link S3ObjectAttributes}. * * @param s3Attributes the object attributes @@ -226,44 +240,44 @@ public abstract String getRevisionId(ObjectMetadata objectMetadata, public abstract String getRevisionId(S3ObjectAttributes s3Attributes); /** - * Like {{@link #getRevisionId(ObjectMetadata, String)}}, but retrieves the - * revision identifier from {@link CopyResult}. + * Like {{@link #getRevisionId(HeadObjectResponse, String)}}, but retrieves the + * revision identifier from {@link CopyObjectResponse}. * - * @param copyResult the copy result + * @param copyObjectResponse the copy response * @return the revisionId string as interpreted by this policy, or potentially * null if the attribute is unavailable (such as when the policy says to use * versionId but object versioning is not enabled for the bucket). */ - public abstract String getRevisionId(CopyResult copyResult); + public abstract String getRevisionId(CopyObjectResponse copyObjectResponse); /** - * Applies the given {@link #getRevisionId(ObjectMetadata, String) revisionId} - * as a server-side qualification on the {@code GetObjectRequest}. + * Applies the given {@link #getRevisionId(HeadObjectResponse, String) revisionId} + * as a server-side qualification on the {@code GetObjectRequest.Builder}. * * @param request the request * @param revisionId the revision id */ - public abstract void applyRevisionConstraint(GetObjectRequest request, + public abstract void applyRevisionConstraint(GetObjectRequest.Builder request, String revisionId); /** - * Applies the given {@link #getRevisionId(ObjectMetadata, String) revisionId} + * Applies the given {@link #getRevisionId(HeadObjectResponse, String) revisionId} * as a server-side qualification on the {@code CopyObjectRequest}. * - * @param request the request + * @param requestBuilder the copy object request builder * @param revisionId the revision id */ - public abstract void applyRevisionConstraint(CopyObjectRequest request, + public abstract void applyRevisionConstraint(CopyObjectRequest.Builder requestBuilder, String revisionId); /** - * Applies the given {@link #getRevisionId(ObjectMetadata, String) revisionId} + * Applies the given {@link #getRevisionId(HeadObjectResponse, String) revisionId} * as a server-side qualification on the {@code GetObjectMetadataRequest}. * - * @param request the request + * @param requestBuilder the head object request builder * @param revisionId the revision id */ - public abstract void applyRevisionConstraint(GetObjectMetadataRequest request, + public abstract void applyRevisionConstraint(HeadObjectRequest.Builder requestBuilder, String revisionId); /** @@ -323,7 +337,7 @@ public ImmutablePair onChangeDetected( } /** - * Change detection policy based on {@link ObjectMetadata#getETag() eTag}. + * Change detection policy based on {@link HeadObjectResponse#eTag()} () eTag}. */ static class ETagChangeDetectionPolicy extends ChangeDetectionPolicy { @@ -332,8 +346,13 @@ static class ETagChangeDetectionPolicy extends ChangeDetectionPolicy { } @Override - public String getRevisionId(ObjectMetadata objectMetadata, String uri) { - return objectMetadata.getETag(); + public String getRevisionId(GetObjectResponse objectMetadata, String uri) { + return objectMetadata.eTag(); + } + + @Override + public String getRevisionId(HeadObjectResponse objectMetadata, String uri) { + return objectMetadata.eTag(); } @Override @@ -342,34 +361,34 @@ public String getRevisionId(S3ObjectAttributes s3Attributes) { } @Override - public String getRevisionId(CopyResult copyResult) { - return copyResult.getETag(); + public String getRevisionId(CopyObjectResponse copyObjectResponse) { + return copyObjectResponse.copyObjectResult().eTag(); } @Override - public void applyRevisionConstraint(GetObjectRequest request, + public void applyRevisionConstraint(GetObjectRequest.Builder builder, String revisionId) { if (revisionId != null) { LOG.debug("Restricting get request to etag {}", revisionId); - request.withMatchingETagConstraint(revisionId); + builder.ifMatch(revisionId); } else { LOG.debug("No etag revision ID to use as a constraint"); } } @Override - public void applyRevisionConstraint(CopyObjectRequest request, + public void applyRevisionConstraint(CopyObjectRequest.Builder requestBuilder, String revisionId) { if (revisionId != null) { LOG.debug("Restricting copy request to etag {}", revisionId); - request.withMatchingETagConstraint(revisionId); + requestBuilder.copySourceIfMatch(revisionId); } else { LOG.debug("No etag revision ID to use as a constraint"); } } @Override - public void applyRevisionConstraint(GetObjectMetadataRequest request, + public void applyRevisionConstraint(HeadObjectRequest.Builder requestBuilder, String revisionId) { LOG.debug("Unable to restrict HEAD request to etag; will check later"); } @@ -388,7 +407,7 @@ public String toString() { /** * Change detection policy based on - * {@link ObjectMetadata#getVersionId() versionId}. + * {@link HeadObjectResponse#versionId()} () versionId}. */ static class VersionIdChangeDetectionPolicy extends ChangeDetectionPolicy { @@ -398,8 +417,16 @@ static class VersionIdChangeDetectionPolicy extends } @Override - public String getRevisionId(ObjectMetadata objectMetadata, String uri) { - String versionId = objectMetadata.getVersionId(); + public String getRevisionId(HeadObjectResponse objectMetadata, String uri) { + return logIfNull(objectMetadata.versionId(), uri); + } + + @Override + public String getRevisionId(GetObjectResponse getObjectResponse, String uri) { + return logIfNull(getObjectResponse.versionId(), uri); + } + + private String logIfNull(String versionId, String uri) { if (versionId == null) { // this policy doesn't work if the bucket doesn't have object versioning // enabled (which isn't by default) @@ -419,38 +446,38 @@ public String getRevisionId(S3ObjectAttributes s3Attributes) { } @Override - public String getRevisionId(CopyResult copyResult) { - return copyResult.getVersionId(); + public String getRevisionId(CopyObjectResponse copyObjectResponse) { + return copyObjectResponse.versionId(); } @Override - public void applyRevisionConstraint(GetObjectRequest request, + public void applyRevisionConstraint(GetObjectRequest.Builder builder, String revisionId) { if (revisionId != null) { LOG.debug("Restricting get request to version {}", revisionId); - request.withVersionId(revisionId); + builder.versionId(revisionId); } else { LOG.debug("No version ID to use as a constraint"); } } @Override - public void applyRevisionConstraint(CopyObjectRequest request, + public void applyRevisionConstraint(CopyObjectRequest.Builder requestBuilder, String revisionId) { if (revisionId != null) { LOG.debug("Restricting copy request to version {}", revisionId); - request.withSourceVersionId(revisionId); + requestBuilder.sourceVersionId(revisionId); } else { LOG.debug("No version ID to use as a constraint"); } } @Override - public void applyRevisionConstraint(GetObjectMetadataRequest request, + public void applyRevisionConstraint(HeadObjectRequest.Builder requestBuilder, String revisionId) { if (revisionId != null) { LOG.debug("Restricting metadata request to version {}", revisionId); - request.withVersionId(revisionId); + requestBuilder.versionId(revisionId); } else { LOG.debug("No version ID to use as a constraint"); } @@ -482,7 +509,13 @@ public Source getSource() { } @Override - public String getRevisionId(final ObjectMetadata objectMetadata, + public String getRevisionId(final GetObjectResponse objectMetadata, + final String uri) { + return null; + } + + @Override + public String getRevisionId(final HeadObjectResponse objectMetadata, final String uri) { return null; } @@ -493,24 +526,24 @@ public String getRevisionId(final S3ObjectAttributes s3ObjectAttributes) { } @Override - public String getRevisionId(CopyResult copyResult) { + public String getRevisionId(CopyObjectResponse copyObjectResponse) { return null; } @Override - public void applyRevisionConstraint(final GetObjectRequest request, + public void applyRevisionConstraint(final GetObjectRequest.Builder builder, final String revisionId) { } @Override - public void applyRevisionConstraint(CopyObjectRequest request, + public void applyRevisionConstraint(CopyObjectRequest.Builder requestBuilder, String revisionId) { } @Override - public void applyRevisionConstraint(GetObjectMetadataRequest request, + public void applyRevisionConstraint(HeadObjectRequest.Builder requestBuilder, String revisionId) { } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java index 2e144bbfcd04f..c7e9139f32d96 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java @@ -18,14 +18,14 @@ package org.apache.hadoop.fs.s3a.impl; -import com.amazonaws.AmazonServiceException; -import com.amazonaws.SdkBaseException; -import com.amazonaws.services.s3.model.CopyObjectRequest; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.transfer.model.CopyResult; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.CopyObjectResponse; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +39,7 @@ import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_412_PRECONDITION_FAILED; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; /** @@ -55,8 +56,6 @@ public class ChangeTracker { private static final Logger LOG = LoggerFactory.getLogger(ChangeTracker.class); - /** {@code 412 Precondition Failed} (HTTP/1.1 - RFC 2616) */ - public static final int SC_PRECONDITION_FAILED = 412; public static final String CHANGE_REPORTED_BY_S3 = "Change reported by S3"; /** Policy to use. */ @@ -117,15 +116,15 @@ public long getVersionMismatches() { /** * Apply any revision control set by the policy if it is to be * enforced on the server. - * @param request request to modify + * @param builder request builder to modify * @return true iff a constraint was added. */ public boolean maybeApplyConstraint( - final GetObjectRequest request) { + final GetObjectRequest.Builder builder) { if (policy.getMode() == ChangeDetectionPolicy.Mode.Server && revisionId != null) { - policy.applyRevisionConstraint(request, revisionId); + policy.applyRevisionConstraint(builder, revisionId); return true; } return false; @@ -134,26 +133,26 @@ public boolean maybeApplyConstraint( /** * Apply any revision control set by the policy if it is to be * enforced on the server. - * @param request request to modify + * @param requestBuilder copy object request builder to modify * @return true iff a constraint was added. */ public boolean maybeApplyConstraint( - final CopyObjectRequest request) { + final CopyObjectRequest.Builder requestBuilder) { if (policy.getMode() == ChangeDetectionPolicy.Mode.Server && revisionId != null) { - policy.applyRevisionConstraint(request, revisionId); + policy.applyRevisionConstraint(requestBuilder, revisionId); return true; } return false; } public boolean maybeApplyConstraint( - final GetObjectMetadataRequest request) { + final HeadObjectRequest.Builder requestBuilder) { if (policy.getMode() == ChangeDetectionPolicy.Mode.Server && revisionId != null) { - policy.applyRevisionConstraint(request, revisionId); + policy.applyRevisionConstraint(requestBuilder, revisionId); return true; } return false; @@ -168,7 +167,7 @@ public boolean maybeApplyConstraint( * @throws PathIOException raised on failure * @throws RemoteFileChangedException if the remote file has changed. */ - public void processResponse(final S3Object object, + public void processResponse(final GetObjectResponse object, final String operation, final long pos) throws PathIOException { if (object == null) { @@ -191,24 +190,24 @@ public void processResponse(final S3Object object, } } - processMetadata(object.getObjectMetadata(), operation); + processMetadata(object, operation); } /** * Process the response from the server for validation against the * change policy. - * @param copyResult result of a copy operation + * @param copyObjectResponse response of a copy operation * @throws PathIOException raised on failure * @throws RemoteFileChangedException if the remote file has changed. */ - public void processResponse(final CopyResult copyResult) + public void processResponse(final CopyObjectResponse copyObjectResponse) throws PathIOException { // ETag (sometimes, depending on encryption and/or multipart) is not the // same on the copied object as the original. Version Id seems to never // be the same on the copy. As such, there isn't really anything that // can be verified on the response, except that a revision ID is present // if required. - String newRevisionId = policy.getRevisionId(copyResult); + String newRevisionId = policy.getRevisionId(copyObjectResponse); LOG.debug("Copy result {}: {}", policy.getSource(), newRevisionId); if (newRevisionId == null && policy.isRequireVersion()) { throw new NoVersionAttributeException(uri, String.format( @@ -227,13 +226,11 @@ public void processResponse(final CopyResult copyResult) * generated (e.g. "copy", "read", "select"). * @throws RemoteFileChangedException if the remote file has changed. */ - public void processException(SdkBaseException e, String operation) throws + public void processException(SdkException e, String operation) throws RemoteFileChangedException { - if (e instanceof AmazonServiceException) { - AmazonServiceException serviceException = (AmazonServiceException) e; - // This isn't really going to be hit due to - // https://github.com/aws/aws-sdk-java/issues/1644 - if (serviceException.getStatusCode() == SC_PRECONDITION_FAILED) { + if (e instanceof AwsServiceException) { + AwsServiceException serviceException = (AwsServiceException)e; + if (serviceException.statusCode() == SC_412_PRECONDITION_FAILED) { versionMismatches.versionMismatchError(); throw new RemoteFileChangedException(uri, operation, String.format( RemoteFileChangedException.PRECONDITIONS_FAILED @@ -254,12 +251,26 @@ public void processException(SdkBaseException e, String operation) throws * @throws PathIOException raised on failure * @throws RemoteFileChangedException if the remote file has changed. */ - public void processMetadata(final ObjectMetadata metadata, + public void processMetadata(final HeadObjectResponse metadata, final String operation) throws PathIOException { final String newRevisionId = policy.getRevisionId(metadata, uri); processNewRevision(newRevisionId, operation, -1); } + /** + * Process the response from server for validation against the change + * policy. + * @param getObjectResponse response returned from server + * @param operation operation in progress + * @throws PathIOException raised on failure + * @throws RemoteFileChangedException if the remote file has changed. + */ + public void processMetadata(final GetObjectResponse getObjectResponse, + final String operation) throws PathIOException { + final String newRevisionId = policy.getRevisionId(getObjectResponse, uri); + processNewRevision(newRevisionId, operation, -1); + } + /** * Validate a revision from the server against our expectations. * @param newRevisionId new revision. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ConfigureShadedAWSSocketFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ConfigureShadedAWSSocketFactory.java index 8205668e8f354..ba21f6028a5f8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ConfigureShadedAWSSocketFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ConfigureShadedAWSSocketFactory.java @@ -21,8 +21,8 @@ import javax.net.ssl.HostnameVerifier; import java.io.IOException; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.thirdparty.org.apache.http.conn.ssl.SSLConnectionSocketFactory; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; @@ -35,13 +35,12 @@ public class ConfigureShadedAWSSocketFactory implements NetworkBinding.ConfigureAWSSocketFactory { @Override - public void configureSocketFactory(final ClientConfiguration awsConf, + public void configureSocketFactory(final ApacheHttpClient.Builder httpClientBuilder, final DelegatingSSLSocketFactory.SSLChannelMode channelMode) throws IOException { DelegatingSSLSocketFactory.initializeDefaultFactory(channelMode); - awsConf.getApacheHttpClientConfig().setSslSocketFactory( - new SSLConnectionSocketFactory( - DelegatingSSLSocketFactory.getDefaultFactory(), - (HostnameVerifier) null)); + httpClientBuilder.socketFactory(new SSLConnectionSocketFactory( + DelegatingSSLSocketFactory.getDefaultFactory(), + (HostnameVerifier) null)); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CopyOutcome.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CopyOutcome.java deleted file mode 100644 index 16459ac45b850..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CopyOutcome.java +++ /dev/null @@ -1,80 +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.impl; - -import com.amazonaws.SdkBaseException; -import com.amazonaws.services.s3.transfer.Copy; -import com.amazonaws.services.s3.transfer.model.CopyResult; - -/** - * Extracts the outcome of a TransferManager-executed copy operation. - */ -public final class CopyOutcome { - - /** - * Result of a successful copy. - */ - private final CopyResult copyResult; - - /** the copy was interrupted. */ - private final InterruptedException interruptedException; - - /** - * The copy raised an AWS Exception of some form. - */ - private final SdkBaseException awsException; - - public CopyOutcome(CopyResult copyResult, - InterruptedException interruptedException, - SdkBaseException awsException) { - this.copyResult = copyResult; - this.interruptedException = interruptedException; - this.awsException = awsException; - } - - public CopyResult getCopyResult() { - return copyResult; - } - - public InterruptedException getInterruptedException() { - return interruptedException; - } - - public SdkBaseException getAwsException() { - return awsException; - } - - /** - * Calls {@code Copy.waitForCopyResult()} to await the result, converts - * it to a copy outcome. - * Exceptions caught and - * @param copy the copy operation. - * @return the outcome. - */ - public static CopyOutcome waitForCopy(Copy copy) { - try { - CopyResult result = copy.waitForCopyResult(); - return new CopyOutcome(result, null, null); - } catch (SdkBaseException e) { - return new CopyOutcome(null, null, e); - } catch (InterruptedException e) { - return new CopyOutcome(null, e, null); - } - } -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java index 0797c36c52910..7cc94cf89203f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java @@ -21,10 +21,11 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors; import org.slf4j.Logger; @@ -40,6 +41,8 @@ import org.apache.hadoop.fs.s3a.Tristate; import org.apache.hadoop.util.DurationInfo; + +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions; import static org.apache.hadoop.fs.store.audit.AuditingFunctions.callableWithinAuditSpan; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.maybeAwaitCompletion; @@ -109,6 +112,16 @@ public class DeleteOperation extends ExecutingStoreOperation { */ private long filesDeleted; + /** + * Do directory operations purge pending uploads? + */ + private final boolean dirOperationsPurgeUploads; + + /** + * Count of uploads aborted. + */ + private Optional uploadsAborted = Optional.empty(); + /** * Constructor. * @param context store context @@ -116,12 +129,14 @@ public class DeleteOperation extends ExecutingStoreOperation { * @param recursive recursive delete? * @param callbacks callback provider * @param pageSize size of delete pages + * @param dirOperationsPurgeUploads Do directory operations purge pending uploads? */ public DeleteOperation(final StoreContext context, final S3AFileStatus status, final boolean recursive, final OperationCallbacks callbacks, - final int pageSize) { + final int pageSize, + final boolean dirOperationsPurgeUploads) { super(context); this.status = status; @@ -133,12 +148,22 @@ public DeleteOperation(final StoreContext context, this.pageSize = pageSize; executor = MoreExecutors.listeningDecorator( context.createThrottledExecutor(1)); + this.dirOperationsPurgeUploads = dirOperationsPurgeUploads; } public long getFilesDeleted() { return filesDeleted; } + /** + * Get the count of uploads aborted. + * Non-empty iff enabled, and the operations completed without errors. + * @return count of aborted uploads. + */ + public Optional getUploadsAborted() { + return uploadsAborted; + } + /** * Delete a file or directory tree. *

    @@ -235,6 +260,17 @@ protected void deleteDirectoryTree(final Path path, try (DurationInfo ignored = new DurationInfo(LOG, false, "deleting %s", dirKey)) { + final CompletableFuture abortUploads; + if (dirOperationsPurgeUploads) { + final StoreContext sc = getStoreContext(); + final String key = sc.pathToKey(path) + "/"; + LOG.debug("All uploads under {} will be deleted", key); + abortUploads = submit(sc.getExecutor(), sc.getActiveAuditSpan(), () -> + callbacks.abortMultipartUploadsUnderPrefix(key)); + } else { + abortUploads = null; + } + // init the lists of keys and paths to delete resetDeleteList(); deleteFuture = null; @@ -256,10 +292,10 @@ protected void deleteDirectoryTree(final Path path, LOG.debug("Deleting final batch of listed files"); submitNextBatch(); maybeAwaitCompletion(deleteFuture); - + uploadsAborted = waitForCompletionIgnoringExceptions(abortUploads); } - LOG.debug("Delete \"{}\" completed; deleted {} objects", path, - filesDeleted); + LOG.debug("Delete \"{}\" completed; deleted {} objects and aborted {} uploads", path, + filesDeleted, uploadsAborted.orElse(0L)); } /** @@ -312,7 +348,8 @@ private void submitNextBatch() throws IOException { // delete a single page of keys and the metadata. // block for any previous batch. - maybeAwaitCompletion(deleteFuture); + maybeAwaitCompletion(deleteFuture).ifPresent(count -> + LOG.debug("Deleted {} uploads", count)); // delete the current page of keys and paths deleteFuture = submitDelete(keys); @@ -386,9 +423,9 @@ private void asyncDeleteAction( "Delete page of %d keys", keyList.size())) { if (!keyList.isEmpty()) { // first delete the files. - List files = keyList.stream() + List files = keyList.stream() .filter(e -> !e.isDirMarker) - .map(e -> e.keyVersion) + .map(e -> e.objectIdentifier) .collect(Collectors.toList()); LOG.debug("Deleting of {} file objects", files.size()); Invoker.once("Remove S3 Files", @@ -398,9 +435,9 @@ private void asyncDeleteAction( false )); // now the dirs - List dirs = keyList.stream() + List dirs = keyList.stream() .filter(e -> e.isDirMarker) - .map(e -> e.keyVersion) + .map(e -> e.objectIdentifier) .collect(Collectors.toList()); LOG.debug("Deleting of {} directory markers", dirs.size()); // This is invoked with deleteFakeDir. @@ -422,17 +459,17 @@ private void asyncDeleteAction( * to choose which statistics to update. */ private static final class DeleteEntry { - private final DeleteObjectsRequest.KeyVersion keyVersion; + private final ObjectIdentifier objectIdentifier; private final boolean isDirMarker; private DeleteEntry(final String key, final boolean isDirMarker) { - this.keyVersion = new DeleteObjectsRequest.KeyVersion(key); + this.objectIdentifier = ObjectIdentifier.builder().key(key).build(); this.isDirMarker = isDirMarker; } public String getKey() { - return keyVersion.getKey(); + return objectIdentifier.key(); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java index f7e06413a3761..f8a1f907bb3b1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java @@ -18,9 +18,14 @@ package org.apache.hadoop.fs.s3a.impl; -import com.amazonaws.AmazonServiceException; +import java.io.IOException; +import java.lang.reflect.Constructor; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404; +import software.amazon.awssdk.awscore.exception.AwsServiceException; + +import org.apache.hadoop.fs.PathIOException; + +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404_NOT_FOUND; /** * Translate from AWS SDK-wrapped exceptions into IOExceptions with @@ -35,7 +40,7 @@ * The existing code las been left in S3AUtils it is to avoid cherry-picking * problems on backports. */ -public class ErrorTranslation { +public final class ErrorTranslation { /** * Private constructor for utility class. @@ -49,9 +54,9 @@ private ErrorTranslation() { * @return true if the status code and error code mean that the * remote bucket is unknown. */ - public static boolean isUnknownBucket(AmazonServiceException e) { - return e.getStatusCode() == SC_404 - && AwsErrorCodes.E_NO_SUCH_BUCKET.equals(e.getErrorCode()); + public static boolean isUnknownBucket(AwsServiceException e) { + return e.statusCode() == SC_404_NOT_FOUND + && AwsErrorCodes.E_NO_SUCH_BUCKET.equals(e.awsErrorDetails().errorCode()); } /** @@ -62,8 +67,73 @@ public static boolean isUnknownBucket(AmazonServiceException e) { * @return true if the status code and error code mean that the * HEAD request returned 404 but the bucket was there. */ - public static boolean isObjectNotFound(AmazonServiceException e) { - return e.getStatusCode() == SC_404 && !isUnknownBucket(e); + public static boolean isObjectNotFound(AwsServiceException e) { + return e.statusCode() == SC_404_NOT_FOUND && !isUnknownBucket(e); + } + + /** + * Translate an exception if it or its inner exception is an + * IOException. + * If this condition is not met, null is returned. + * @param path path of operation. + * @param thrown exception + * @return a translated exception or null. + */ + public static IOException maybeExtractIOException(String path, Throwable thrown) { + + if (thrown == null) { + return null; + } + + // look inside + Throwable cause = thrown.getCause(); + while (cause != null && cause.getCause() != null) { + cause = cause.getCause(); + } + if (!(cause instanceof IOException)) { + return null; + } + + // the cause can be extracted to an IOE. + // rather than just return it, we try to preserve the stack trace + // of the outer exception. + // as a new instance is created through reflection, the + // class of the returned instance will be that of the innermost, + // unless no suitable constructor is available. + final IOException ioe = (IOException) cause; + + return wrapWithInnerIOE(path, thrown, ioe); + + } + + /** + * Given an outer and an inner exception, create a new IOE + * of the inner type, with the outer exception as the cause. + * The message is derived from both. + * This only works if the inner exception has a constructor which + * takes a string; if not a PathIOException is created. + *

    + * See {@code NetUtils}. + * @param type of inner exception. + * @param path path of the failure. + * @param outer outermost exception. + * @param inner inner exception. + * @return the new exception. + */ + @SuppressWarnings("unchecked") + private static IOException wrapWithInnerIOE( + String path, + Throwable outer, + T inner) { + String msg = outer.toString() + ": " + inner.getMessage(); + Class clazz = inner.getClass(); + try { + Constructor ctor = clazz.getConstructor(String.class); + Throwable t = ctor.newInstance(msg); + return (T) (t.initCause(outer)); + } catch (Throwable e) { + return new PathIOException(path, msg, outer); + } } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java index f75066e049d3e..d42dda59caa5f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java @@ -23,13 +23,15 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Date; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.TreeMap; -import com.amazonaws.services.s3.Headers; -import com.amazonaws.services.s3.model.ObjectMetadata; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadBucketResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,50 +77,50 @@ public class HeaderProcessing extends AbstractStoreOperation { * Standard HTTP header found on some S3 objects: {@value}. */ public static final String XA_CACHE_CONTROL = - XA_HEADER_PREFIX + Headers.CACHE_CONTROL; + XA_HEADER_PREFIX + AWSHeaders.CACHE_CONTROL; /** * Standard HTTP header found on some S3 objects: {@value}. */ public static final String XA_CONTENT_DISPOSITION = - XA_HEADER_PREFIX + Headers.CONTENT_DISPOSITION; + XA_HEADER_PREFIX + AWSHeaders.CONTENT_DISPOSITION; /** * Content encoding; can be configured: {@value}. */ public static final String XA_CONTENT_ENCODING = - XA_HEADER_PREFIX + Headers.CONTENT_ENCODING; + XA_HEADER_PREFIX + AWSHeaders.CONTENT_ENCODING; /** * Standard HTTP header found on some S3 objects: {@value}. */ public static final String XA_CONTENT_LANGUAGE = - XA_HEADER_PREFIX + Headers.CONTENT_LANGUAGE; + XA_HEADER_PREFIX + AWSHeaders.CONTENT_LANGUAGE; /** * Length XAttr: {@value}. */ public static final String XA_CONTENT_LENGTH = - XA_HEADER_PREFIX + Headers.CONTENT_LENGTH; + XA_HEADER_PREFIX + AWSHeaders.CONTENT_LENGTH; /** * Standard HTTP header found on some S3 objects: {@value}. */ public static final String XA_CONTENT_MD5 = - XA_HEADER_PREFIX + Headers.CONTENT_MD5; + XA_HEADER_PREFIX + AWSHeaders.CONTENT_MD5; /** * Content range: {@value}. * This is returned on GET requests with ranges. */ public static final String XA_CONTENT_RANGE = - XA_HEADER_PREFIX + Headers.CONTENT_RANGE; + XA_HEADER_PREFIX + AWSHeaders.CONTENT_RANGE; /** * Content type: may be set when uploading. * {@value}. */ public static final String XA_CONTENT_TYPE = - XA_HEADER_PREFIX + Headers.CONTENT_TYPE; + XA_HEADER_PREFIX + AWSHeaders.CONTENT_TYPE; /** * Etag Header {@value}. @@ -126,14 +128,14 @@ public class HeaderProcessing extends AbstractStoreOperation { * it can be retrieved via {@code getFileChecksum(path)} if * the S3A connector is enabled. */ - public static final String XA_ETAG = XA_HEADER_PREFIX + Headers.ETAG; + public static final String XA_ETAG = XA_HEADER_PREFIX + AWSHeaders.ETAG; /** * last modified XAttr: {@value}. */ public static final String XA_LAST_MODIFIED = - XA_HEADER_PREFIX + Headers.LAST_MODIFIED; + XA_HEADER_PREFIX + AWSHeaders.LAST_MODIFIED; /* AWS Specific Headers. May not be found on other S3 endpoints. */ @@ -144,50 +146,50 @@ public class HeaderProcessing extends AbstractStoreOperation { * Value {@value}. */ public static final String XA_ARCHIVE_STATUS = - XA_HEADER_PREFIX + Headers.ARCHIVE_STATUS; + XA_HEADER_PREFIX + AWSHeaders.ARCHIVE_STATUS; /** * Object legal hold status. {@value}. */ public static final String XA_OBJECT_LOCK_LEGAL_HOLD_STATUS = - XA_HEADER_PREFIX + Headers.OBJECT_LOCK_LEGAL_HOLD_STATUS; + XA_HEADER_PREFIX + AWSHeaders.OBJECT_LOCK_LEGAL_HOLD_STATUS; /** * Object lock mode. {@value}. */ public static final String XA_OBJECT_LOCK_MODE = - XA_HEADER_PREFIX + Headers.OBJECT_LOCK_MODE; + XA_HEADER_PREFIX + AWSHeaders.OBJECT_LOCK_MODE; /** * ISO8601 expiry date of object lock hold. {@value}. */ public static final String XA_OBJECT_LOCK_RETAIN_UNTIL_DATE = - XA_HEADER_PREFIX + Headers.OBJECT_LOCK_RETAIN_UNTIL_DATE; + XA_HEADER_PREFIX + AWSHeaders.OBJECT_LOCK_RETAIN_UNTIL_DATE; /** * Replication status for cross-region replicated objects. {@value}. */ public static final String XA_OBJECT_REPLICATION_STATUS = - XA_HEADER_PREFIX + Headers.OBJECT_REPLICATION_STATUS; + XA_HEADER_PREFIX + AWSHeaders.OBJECT_REPLICATION_STATUS; /** * Version ID; empty for non-versioned buckets/data. {@value}. */ public static final String XA_S3_VERSION_ID = - XA_HEADER_PREFIX + Headers.S3_VERSION_ID; + XA_HEADER_PREFIX + AWSHeaders.S3_VERSION_ID; /** * The server-side encryption algorithm to use * with AWS-managed keys: {@value}. */ public static final String XA_SERVER_SIDE_ENCRYPTION = - XA_HEADER_PREFIX + Headers.SERVER_SIDE_ENCRYPTION; + XA_HEADER_PREFIX + AWSHeaders.SERVER_SIDE_ENCRYPTION; /** * Storage Class XAttr: {@value}. */ public static final String XA_STORAGE_CLASS = - XA_HEADER_PREFIX + Headers.STORAGE_CLASS; + XA_HEADER_PREFIX + AWSHeaders.STORAGE_CLASS; /** * HTTP Referrer for logs: {@value}. @@ -275,9 +277,28 @@ private Map retrieveHeaders( final Statistic statistic) throws IOException { StoreContext context = getStoreContext(); String objectKey = context.pathToKey(path); - ObjectMetadata md; String symbol = statistic.getSymbol(); S3AStatisticsContext instrumentation = context.getInstrumentation(); + Map headers = new TreeMap<>(); + HeadObjectResponse md; + + // Attempting to get metadata for the root, so use head bucket. + if (objectKey.isEmpty()) { + HeadBucketResponse headBucketResponse = + trackDuration(instrumentation, symbol, () -> callbacks.getBucketMetadata()); + + if (headBucketResponse.sdkHttpResponse() != null + && headBucketResponse.sdkHttpResponse().headers() != null + && headBucketResponse.sdkHttpResponse().headers().get(AWSHeaders.CONTENT_TYPE) != null) { + maybeSetHeader(headers, XA_CONTENT_TYPE, + headBucketResponse.sdkHttpResponse().headers().get(AWSHeaders.CONTENT_TYPE).get(0)); + } + + maybeSetHeader(headers, XA_CONTENT_LENGTH, 0); + + return headers; + } + try { md = trackDuration(instrumentation, symbol, () -> callbacks.getObjectMetadata(objectKey)); @@ -287,59 +308,64 @@ private Map retrieveHeaders( callbacks.getObjectMetadata(objectKey + "/")); } // all user metadata - Map rawHeaders = md.getUserMetadata(); - Map headers = new TreeMap<>(); + Map rawHeaders = md.metadata(); rawHeaders.forEach((key, value) -> headers.put(XA_HEADER_PREFIX + key, encodeBytes(value))); // and add the usual content length &c, if set maybeSetHeader(headers, XA_CACHE_CONTROL, - md.getCacheControl()); + md.cacheControl()); maybeSetHeader(headers, XA_CONTENT_DISPOSITION, - md.getContentDisposition()); + md.contentDisposition()); maybeSetHeader(headers, XA_CONTENT_ENCODING, - md.getContentEncoding()); + md.contentEncoding()); maybeSetHeader(headers, XA_CONTENT_LANGUAGE, - md.getContentLanguage()); + md.contentLanguage()); // If CSE is enabled, use the unencrypted content length. - if (md.getUserMetaDataOf(Headers.CRYPTO_CEK_ALGORITHM) != null - && md.getUserMetaDataOf(Headers.UNENCRYPTED_CONTENT_LENGTH) != null) { - maybeSetHeader(headers, XA_CONTENT_LENGTH, - md.getUserMetaDataOf(Headers.UNENCRYPTED_CONTENT_LENGTH)); - } else { - maybeSetHeader(headers, XA_CONTENT_LENGTH, - md.getContentLength()); + // TODO: CSE is not supported yet, add these headers in during CSE work. +// if (md.getUserMetaDataOf(Headers.CRYPTO_CEK_ALGORITHM) != null +// && md.getUserMetaDataOf(Headers.UNENCRYPTED_CONTENT_LENGTH) != null) { +// maybeSetHeader(headers, XA_CONTENT_LENGTH, +// md.getUserMetaDataOf(Headers.UNENCRYPTED_CONTENT_LENGTH)); +// } else { +// maybeSetHeader(headers, XA_CONTENT_LENGTH, +// md.contentLength()); +// } +// maybeSetHeader(headers, XA_CONTENT_MD5, +// md.getContentMD5()); + // TODO: Add back in else block during CSE work. + maybeSetHeader(headers, XA_CONTENT_LENGTH, + md.contentLength()); + if (md.sdkHttpResponse() != null && md.sdkHttpResponse().headers() != null + && md.sdkHttpResponse().headers().get("Content-Range") != null) { + maybeSetHeader(headers, XA_CONTENT_RANGE, + md.sdkHttpResponse().headers().get("Content-Range").get(0)); } - maybeSetHeader(headers, XA_CONTENT_MD5, - md.getContentMD5()); - maybeSetHeader(headers, XA_CONTENT_RANGE, - md.getContentRange()); maybeSetHeader(headers, XA_CONTENT_TYPE, - md.getContentType()); + md.contentType()); maybeSetHeader(headers, XA_ETAG, - md.getETag()); + md.eTag()); maybeSetHeader(headers, XA_LAST_MODIFIED, - md.getLastModified()); + Date.from(md.lastModified())); // AWS custom headers maybeSetHeader(headers, XA_ARCHIVE_STATUS, - md.getArchiveStatus()); + md.archiveStatus()); maybeSetHeader(headers, XA_OBJECT_LOCK_LEGAL_HOLD_STATUS, - md.getObjectLockLegalHoldStatus()); + md.objectLockLegalHoldStatus()); maybeSetHeader(headers, XA_OBJECT_LOCK_MODE, - md.getObjectLockMode()); + md.objectLockMode()); maybeSetHeader(headers, XA_OBJECT_LOCK_RETAIN_UNTIL_DATE, - md.getObjectLockRetainUntilDate()); + md.objectLockRetainUntilDate()); maybeSetHeader(headers, XA_OBJECT_REPLICATION_STATUS, - md.getReplicationStatus()); + md.replicationStatus()); maybeSetHeader(headers, XA_S3_VERSION_ID, - md.getVersionId()); + md.versionId()); maybeSetHeader(headers, XA_SERVER_SIDE_ENCRYPTION, - md.getSSEAlgorithm()); + md.serverSideEncryptionAsString()); maybeSetHeader(headers, XA_STORAGE_CLASS, - md.getStorageClass()); - maybeSetHeader(headers, XA_STORAGE_CLASS, - md.getReplicationStatus()); + md.storageClassAsString()); + return headers; } @@ -458,70 +484,51 @@ public static Optional extractXAttrLongValue(byte[] data) { } /** - * Creates a copy of the passed {@link ObjectMetadata}. - * Does so without using the {@link ObjectMetadata#clone()} method, - * to avoid copying unnecessary headers. + * Creates a copy of the passed metadata. * This operation does not copy the {@code X_HEADER_MAGIC_MARKER} * header to avoid confusion. If a marker file is renamed, * it loses information about any remapped file. * If new fields are added to ObjectMetadata which are not * present in the user metadata headers, they will not be picked * up or cloned unless this operation is updated. - * @param source the {@link ObjectMetadata} to copy + * @param source the source metadata to copy * @param dest the metadata to update; this is the return value. + * @param copyObjectRequestBuilder CopyObjectRequest builder */ - public static void cloneObjectMetadata(ObjectMetadata source, - ObjectMetadata dest) { + public static void cloneObjectMetadata(HeadObjectResponse source, + Map dest, CopyObjectRequest.Builder copyObjectRequestBuilder) { // Possibly null attributes // Allowing nulls to pass breaks it during later use - if (source.getCacheControl() != null) { - dest.setCacheControl(source.getCacheControl()); - } - if (source.getContentDisposition() != null) { - dest.setContentDisposition(source.getContentDisposition()); - } - if (source.getContentEncoding() != null) { - dest.setContentEncoding(source.getContentEncoding()); - } - if (source.getContentMD5() != null) { - dest.setContentMD5(source.getContentMD5()); - } - if (source.getContentType() != null) { - dest.setContentType(source.getContentType()); + if (source.cacheControl() != null) { + copyObjectRequestBuilder.cacheControl(source.cacheControl()); } - if (source.getExpirationTime() != null) { - dest.setExpirationTime(source.getExpirationTime()); + if (source.contentDisposition() != null) { + copyObjectRequestBuilder.contentDisposition(source.contentDisposition()); } - if (source.getExpirationTimeRuleId() != null) { - dest.setExpirationTimeRuleId(source.getExpirationTimeRuleId()); + if (source.contentEncoding() != null) { + copyObjectRequestBuilder.contentEncoding(source.contentEncoding()); } - if (source.getHttpExpiresDate() != null) { - dest.setHttpExpiresDate(source.getHttpExpiresDate()); - } - if (source.getLastModified() != null) { - dest.setLastModified(source.getLastModified()); - } - if (source.getOngoingRestore() != null) { - dest.setOngoingRestore(source.getOngoingRestore()); - } - if (source.getRestoreExpirationTime() != null) { - dest.setRestoreExpirationTime(source.getRestoreExpirationTime()); + + if (source.contentType() != null) { + copyObjectRequestBuilder.contentType(source.contentType()); } - if (source.getSSEAlgorithm() != null) { - dest.setSSEAlgorithm(source.getSSEAlgorithm()); + + if (source.serverSideEncryption() != null) { + copyObjectRequestBuilder.serverSideEncryption(source.serverSideEncryption()); } - if (source.getSSECustomerAlgorithm() != null) { - dest.setSSECustomerAlgorithm(source.getSSECustomerAlgorithm()); + + if (source.sseCustomerAlgorithm() != null) { + copyObjectRequestBuilder.copySourceSSECustomerAlgorithm(source.sseCustomerAlgorithm()); } - if (source.getSSECustomerKeyMd5() != null) { - dest.setSSECustomerKeyMd5(source.getSSECustomerKeyMd5()); + if (source.sseCustomerKeyMD5() != null) { + copyObjectRequestBuilder.copySourceSSECustomerKeyMD5(source.sseCustomerKeyMD5()); } // copy user metadata except the magic marker header. - source.getUserMetadata().entrySet().stream() + source.metadata().entrySet().stream() .filter(e -> !e.getKey().equals(X_HEADER_MAGIC_MARKER)) - .forEach(e -> dest.addUserMetadata(e.getKey(), e.getValue())); + .forEach(e -> dest.put(e.getKey(), e.getValue())); } public interface HeaderProcessingCallbacks { @@ -534,6 +541,15 @@ public interface HeaderProcessingCallbacks { * @throws IOException IO and object access problems. */ @Retries.RetryTranslated - ObjectMetadata getObjectMetadata(String key) throws IOException; + HeadObjectResponse getObjectMetadata(String key) throws IOException; + + /** + * Retrieve the bucket metadata. + * + * @return metadata + * @throws IOException IO and object access problems. + */ + @Retries.RetryTranslated + HeadBucketResponse getBucketMetadata() throws IOException; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InstantiationIOException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InstantiationIOException.java new file mode 100644 index 0000000000000..e09571fc338d6 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InstantiationIOException.java @@ -0,0 +1,182 @@ +/* + * 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.impl; + +import java.net.URI; + +import javax.annotation.Nullable; + +import org.apache.hadoop.fs.PathIOException; + +/** + * An instantiation exception raised during reflection-based creation + * of classes. + * Uses an enum of kind so tests/code can examine it, without + * creating a full hierarchy of exception classes. + */ +public class InstantiationIOException extends PathIOException { + + public static final String ABSTRACT_PROVIDER = + "is abstract and therefore cannot be created"; + + public static final String CONSTRUCTOR_EXCEPTION = "constructor exception"; + + public static final String INSTANTIATION_EXCEPTION + = "instantiation exception"; + + public static final String DOES_NOT_IMPLEMENT + = "does not implement"; + + /** + * Exception kind. + */ + private final Kind kind; + + /** + * Class being instantiated. + */ + private final String classname; + + /** + * key used. + */ + private final String key; + + /** + * An (extensible) enum of kinds of instantiation failure. + */ + public enum Kind { + Forbidden, + InstantiationFailure, + IsAbstract, + IsNotImplementation, + Other, + Unavailable, + UnsupportedConstructor, + } + + public InstantiationIOException( + Kind kind, + @Nullable URI uri, + @Nullable String classname, + @Nullable String key, + String message, + @Nullable Throwable cause) { + super(uri!= null ? uri.toString() : "", + (classname != null ? ("Class " + classname + " ") : "") + + message + + (key != null ? (" (configuration key " + key + ")") : ""), + cause); + this.kind = kind; + this.classname = classname; + this.key = key; + } + + public String getClassname() { + return classname; + } + + public Kind getKind() { + return kind; + } + + public String getKey() { + return key; + } + + /** + * Class is abstract. + * @param uri URI of filesystem + * @param classname classname. + * @param key configuration key + * @return an exception. + */ + public static InstantiationIOException isAbstract(URI uri, String classname, String key) { + return new InstantiationIOException(Kind.IsAbstract, + uri, classname, key, ABSTRACT_PROVIDER, null); + } + + /** + * Class does not implement the desired interface. + * @param uri URI of filesystem + * @param classname classname. + * @param interfaceName required interface + * @param key configuration key + * @return an exception. + */ + public static InstantiationIOException isNotInstanceOf( + @Nullable URI uri, + String classname, + String interfaceName, + String key) { + return new InstantiationIOException(Kind.IsNotImplementation, uri, classname, + key, DOES_NOT_IMPLEMENT + " " + interfaceName, null); + } + + /** + * Class is unavailable for some reason, probably a missing dependency. + * @param uri URI of filesystem + * @param classname classname. + * @param key configuration key + * @param text text to include + * @return an exception. + */ + public static InstantiationIOException unavailable( + @Nullable URI uri, + @Nullable String classname, + @Nullable String key, + String text) { + return new InstantiationIOException(Kind.Unavailable, + uri, classname, key, text, null); + } + + /** + * Failure to find a valid constructor (signature, visibility) or + * factory method. + * @param uri URI of filesystem + * @param classname classname. + * @param key configuration key + * @return an exception. + */ + public static InstantiationIOException unsupportedConstructor( + @Nullable URI uri, + String classname, + String key) { + return new InstantiationIOException(Kind.UnsupportedConstructor, + uri, classname, key, CONSTRUCTOR_EXCEPTION, null); + } + + /** + * General instantiation failure. + * @param uri URI of filesystem + * @param classname classname. + * @param key configuration key + * @param t thrown + * @return an exception. + */ + public static InstantiationIOException instantiationException( + @Nullable URI uri, + String classname, + String key, + Throwable t) { + return new InstantiationIOException(Kind.InstantiationFailure, + uri, classname, key, INSTANTIATION_EXCEPTION + " " + t, t); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index f894479646946..3409bd3cc79e1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -110,11 +110,67 @@ private InternalConstants() { S3A_OPENFILE_KEYS = Collections.unmodifiableSet(keys); } + /** 200 status code: OK. */ + public static final int SC_200_OK = 200; + + /** 301 status code: Moved Permanently. */ + public static final int SC_301_MOVED_PERMANENTLY = 301; + + /** 307 status code: Temporary Redirect. */ + public static final int SC_307_TEMPORARY_REDIRECT = 307; + + /** 400 status code: Bad Request. */ + public static final int SC_400_BAD_REQUEST = 400; + + /** 401 status code: Unauthorized. */ + public static final int SC_401_UNAUTHORIZED = 401; + + /** 403 status code: Forbidden. */ + public static final int SC_403_FORBIDDEN = 403; + /** 403 error code. */ - public static final int SC_403 = 403; + @Deprecated + public static final int SC_403 = SC_403_FORBIDDEN; + + /** 404 status code: Not Found. */ + public static final int SC_404_NOT_FOUND = 404; /** 404 error code. */ - public static final int SC_404 = 404; + @Deprecated + public static final int SC_404 = SC_404_NOT_FOUND; + + /** 405 status code: Method Not Allowed. */ + public static final int SC_405_METHOD_NOT_ALLOWED = 405; + + /** 410 status code: Gone. */ + public static final int SC_410_GONE = 410; + + /** 412 status code: Precondition Failed. */ + public static final int SC_412_PRECONDITION_FAILED = 412; + + /** 415 status code: Content type unsupported by this store. */ + public static final int SC_415_UNSUPPORTED_MEDIA_TYPE = 415; + + /** 416 status code: Range Not Satisfiable. */ + public static final int SC_416_RANGE_NOT_SATISFIABLE = 416; + + /** 429 status code: This is the google GCS throttle message. */ + public static final int SC_429_TOO_MANY_REQUESTS_GCS = 429; + + /** 443 status code: No Response (unofficial). */ + public static final int SC_443_NO_RESPONSE = 443; + + /** 444 status code: No Response (unofficial). */ + public static final int SC_444_NO_RESPONSE = 444; + + /** 500 status code: Internal Server Error. */ + public static final int SC_500_INTERNAL_SERVER_ERROR = 500; + + /** 501 status code: method not implemented. */ + public static final int SC_501_NOT_IMPLEMENTED = 501; + + /** 503 status code: Service Unavailable. on AWS S3: throttle response. */ + public static final int SC_503_SERVICE_UNAVAILABLE = 503; /** Name of the log for throttling events. Value: {@value}. */ public static final String THROTTLE_LOG_NAME = diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteException.java similarity index 50% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteException.java index 96e32f362dfd9..72ead1fb151fc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteException.java @@ -22,26 +22,34 @@ import java.nio.file.AccessDeniedException; import java.util.List; -import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import software.amazon.awssdk.awscore.exception.AwsErrorDetails; +import software.amazon.awssdk.http.SdkHttpResponse; +import software.amazon.awssdk.services.s3.model.S3Error; +import software.amazon.awssdk.services.s3.model.S3Exception; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.s3a.AWSS3IOException; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_200_OK; /** - * Support for Multi Object Deletion. - * This is used to be a complex piece of code as it was required to - * update s3guard. - * Now all that is left is the exception extraction for better - * reporting, + * Exception raised in {@link S3AFileSystem#deleteObjects} when + * one or more of the keys could not be deleted. + * + * Used to reproduce the behaviour of SDK v1 for partial failures + * on DeleteObjects. In SDK v2, the errors are returned as part of + * the response objects. */ -public final class MultiObjectDeleteSupport { +@InterfaceAudience.Public +@InterfaceStability.Unstable +public class MultiObjectDeleteException extends S3Exception { private static final Logger LOG = LoggerFactory.getLogger( - MultiObjectDeleteSupport.class); - - private MultiObjectDeleteSupport() { - } + MultiObjectDeleteException.class); /** * This is the exception exit code if access was denied on a delete. @@ -49,6 +57,46 @@ private MultiObjectDeleteSupport() { */ public static final String ACCESS_DENIED = "AccessDenied"; + /** + * Field value for the superclass builder: {@value}. + */ + private static final int STATUS_CODE = SC_200_OK; + + /** + * Field value for the superclass builder: {@value}. + */ + private static final String ERROR_CODE = "MultiObjectDeleteException"; + + /** + * Field value for the superclass builder: {@value}. + */ + private static final String SERVICE_NAME = "Amazon S3"; + + /** + * Extracted error list. + */ + private final List errors; + + public MultiObjectDeleteException(List errors) { + super(builder() + .message(errors.toString()) + .awsErrorDetails( + AwsErrorDetails.builder() + .errorCode(ERROR_CODE) + .errorMessage(ERROR_CODE) + .serviceName(SERVICE_NAME) + .sdkHttpResponse(SdkHttpResponse.builder() + .statusCode(STATUS_CODE) + .build()) + .build()) + .statusCode(STATUS_CODE)); + this.errors = errors; + } + + public List errors() { + return errors; + } + /** * A {@code MultiObjectDeleteException} is raised if one or more * paths listed in a bulk DELETE operation failed. @@ -58,29 +106,23 @@ private MultiObjectDeleteSupport() { * the causes, otherwise grabs the status code and uses it in the * returned exception. * @param message text for the exception - * @param deleteException the delete exception. to translate * @return an IOE with more detail. */ - public static IOException translateDeleteException( - final String message, - final MultiObjectDeleteException deleteException) { - List errors - = deleteException.getErrors(); + public IOException translateException(final String message) { LOG.info("Bulk delete operation failed to delete all objects;" + " failure count = {}", - errors.size()); + errors().size()); final StringBuilder result = new StringBuilder( - errors.size() * 256); + errors().size() * 256); result.append(message).append(": "); String exitCode = ""; - for (MultiObjectDeleteException.DeleteError error : - deleteException.getErrors()) { - String code = error.getCode(); - String item = String.format("%s: %s%s: %s%n", code, error.getKey(), - (error.getVersionId() != null - ? (" (" + error.getVersionId() + ")") + for (S3Error error : errors()) { + String code = error.code(); + String item = String.format("%s: %s%s: %s%n", code, error.key(), + (error.versionId() != null + ? (" (" + error.versionId() + ")") : ""), - error.getMessage()); + error.message()); LOG.info(item); result.append(item); if (exitCode == null || exitCode.isEmpty() || ACCESS_DENIED.equals(code)) { @@ -89,9 +131,9 @@ public static IOException translateDeleteException( } if (ACCESS_DENIED.equals(exitCode)) { return (IOException) new AccessDeniedException(result.toString()) - .initCause(deleteException); + .initCause(this); } else { - return new AWSS3IOException(result.toString(), deleteException); + return new AWSS3IOException(result.toString(), this); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java index 575a3d1b2de81..34b4049b06123 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java @@ -23,9 +23,9 @@ import java.net.URI; import java.net.URISyntaxException; -import com.amazonaws.ClientConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.http.apache.ApacheHttpClient; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.net.NetUtils; @@ -56,13 +56,12 @@ private NetworkBinding() { * so as to avoid * @param conf the {@link Configuration} used to get the client specified * value of {@code SSL_CHANNEL_MODE} - * @param awsConf the {@code ClientConfiguration} to set the - * SSLConnectionSocketFactory for. + * @param httpClientBuilder the http client builder. * @throws IOException if there is an error while initializing the * {@code SSLSocketFactory} other than classloader problems. */ public static void bindSSLChannelMode(Configuration conf, - ClientConfiguration awsConf) throws IOException { + ApacheHttpClient.Builder httpClientBuilder) throws IOException { // Validate that SSL_CHANNEL_MODE is set to a valid value. String channelModeString = conf.getTrimmed( @@ -89,7 +88,7 @@ public static void bindSSLChannelMode(Configuration conf, (Class) Class.forName(BINDING_CLASSNAME); clazz.getConstructor() .newInstance() - .configureSocketFactory(awsConf, channelMode); + .configureSocketFactory(httpClientBuilder, channelMode); } catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException | InstantiationException | InvocationTargetException | LinkageError e) { @@ -103,7 +102,7 @@ public static void bindSSLChannelMode(Configuration conf, * works with the shaded AWS libraries to exist in their own class. */ interface ConfigureAWSSocketFactory { - void configureSocketFactory(ClientConfiguration awsConf, + void configureSocketFactory(ApacheHttpClient.Builder httpClientBuilder, DelegatingSSLSocketFactory.SSLChannelMode channelMode) throws IOException; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java index 5d17ae91b81e7..9c88870633a35 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java @@ -22,10 +22,9 @@ import java.io.InterruptedIOException; import java.util.List; -import com.amazonaws.AmazonClientException; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.MultiObjectDeleteException; -import com.amazonaws.services.s3.transfer.model.CopyResult; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.s3.model.CopyObjectResponse; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.InvalidRequestException; @@ -127,7 +126,7 @@ RemoteIterator listFilesAndDirectoryMarkers( * @throws IOException Other IO problems */ @Retries.RetryTranslated - CopyResult copyFile(String srcKey, + CopyObjectResponse copyFile(String srcKey, String destKey, S3ObjectAttributes srcAttributes, S3AReadOpContext readContext) @@ -142,14 +141,14 @@ CopyResult copyFile(String srcKey, * a mistaken attempt to delete the root directory. * @throws MultiObjectDeleteException one or more of the keys could not * be deleted in a multiple object delete operation. - * @throws AmazonClientException amazon-layer failure. + * @throws AwsServiceException amazon-layer failure. * @throws IOException other IO Exception. */ @Retries.RetryRaw void removeKeys( - List keysToDelete, + List keysToDelete, boolean deleteFakeDir) - throws MultiObjectDeleteException, AmazonClientException, + throws MultiObjectDeleteException, AwsServiceException, IOException; /** @@ -165,4 +164,16 @@ RemoteIterator listObjects( Path path, String key) throws IOException; + + /** + * Abort multipart uploads under a path; paged. + * @param prefix prefix for uploads to abort + * @return a count of aborts + * @throws IOException trouble; FileNotFoundExceptions are swallowed. + */ + @Retries.RetryTranslated + default long abortMultipartUploadsUnderPrefix(String prefix) + throws IOException { + return 0; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListener.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListener.java new file mode 100644 index 0000000000000..5e4c3cf37e5ce --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListener.java @@ -0,0 +1,26 @@ +/* + * 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.impl; + +/** + * Interface for progress listeners to implement. + */ +public interface ProgressListener { + default void progressChanged(ProgressListenerEvent eventType, long bytesTransferred) {}; +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java new file mode 100644 index 0000000000000..f3f9fb61e434d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java @@ -0,0 +1,29 @@ +/* + * 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.impl; + +/** + * Enum for progress listener events. + */ +public enum ProgressListenerEvent { + REQUEST_BYTE_TRANSFER_EVENT, + TRANSFER_PART_STARTED_EVENT, + TRANSFER_PART_COMPLETED_EVENT, + TRANSFER_PART_FAILED_EVENT; +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java index ae4d2fe7a3431..288b3c0aae585 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java @@ -22,12 +22,12 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; -import com.amazonaws.AmazonClientException; -import com.amazonaws.SdkBaseException; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,6 +45,7 @@ import org.apache.hadoop.util.OperationDuration; import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions; import static org.apache.hadoop.fs.store.audit.AuditingFunctions.callableWithinAuditSpan; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; @@ -122,12 +123,21 @@ public class RenameOperation extends ExecutingStoreOperation { /** * list of keys to delete on the next (bulk) delete call. */ - private final List keysToDelete = + private final List keysToDelete = new ArrayList<>(); + /** + * Do directory operations purge pending uploads? + */ + private final boolean dirOperationsPurgeUploads; + + /** + * Count of uploads aborted. + */ + private Optional uploadsAborted = Optional.empty(); + /** * Initiate the rename. - * * @param storeContext store context * @param sourcePath source path * @param sourceKey key of source @@ -137,6 +147,7 @@ public class RenameOperation extends ExecutingStoreOperation { * @param destStatus destination status. * @param callbacks callback provider * @param pageSize size of delete requests + * @param dirOperationsPurgeUploads Do directory operations purge pending uploads? */ public RenameOperation( final StoreContext storeContext, @@ -147,7 +158,8 @@ public RenameOperation( final String destKey, final S3AFileStatus destStatus, final OperationCallbacks callbacks, - final int pageSize) { + final int pageSize, + final boolean dirOperationsPurgeUploads) { super(storeContext); this.sourcePath = sourcePath; this.sourceKey = sourceKey; @@ -160,6 +172,16 @@ public RenameOperation( && pageSize <= InternalConstants.MAX_ENTRIES_TO_DELETE, "page size out of range: %s", pageSize); this.pageSize = pageSize; + this.dirOperationsPurgeUploads = dirOperationsPurgeUploads; + } + + /** + * Get the count of uploads aborted. + * Non-empty iff enabled, and the operations completed without errors. + * @return count of aborted uploads. + */ + public Optional getUploadsAborted() { + return uploadsAborted; } /** @@ -199,7 +221,7 @@ private void completeActiveCopies(String reason) throws IOException { */ private void queueToDelete(Path path, String key) { LOG.debug("Queueing to delete {}", path); - keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key)); + keysToDelete.add(ObjectIdentifier.builder().key(key).build()); } /** @@ -268,7 +290,7 @@ public Long execute() throws IOException { } else { recursiveDirectoryRename(); } - } catch (AmazonClientException | IOException ex) { + } catch (SdkException | IOException ex) { // rename failed. // block for all ongoing copies to complete, successfully or not try { @@ -342,6 +364,16 @@ protected void recursiveDirectoryRename() throws IOException { throw new RenameFailedException(srcKey, dstKey, "cannot rename a directory to a subdirectory of itself "); } + // start the async dir cleanup + final CompletableFuture abortUploads; + if (dirOperationsPurgeUploads) { + final String key = srcKey; + LOG.debug("All uploads under {} will be deleted", key); + abortUploads = submit(getStoreContext().getExecutor(), () -> + callbacks.abortMultipartUploadsUnderPrefix(key)); + } else { + abortUploads = null; + } if (destStatus != null && destStatus.isEmptyDirectory() == Tristate.TRUE) { @@ -423,6 +455,8 @@ protected void recursiveDirectoryRename() throws IOException { // have been deleted. completeActiveCopiesAndDeleteSources("final copy and delete"); + // and if uploads were being aborted, wait for that to finish + uploadsAborted = waitForCompletionIgnoringExceptions(abortUploads); } /** @@ -572,7 +606,7 @@ private Path copySource( */ @Retries.RetryTranslated private void removeSourceObjects( - final List keys) + final List keys) throws IOException { // remove the keys @@ -580,9 +614,9 @@ private void removeSourceObjects( // who is trying to debug why objects are no longer there. if (LOG.isDebugEnabled()) { LOG.debug("Initiating delete operation for {} objects", keys.size()); - for (DeleteObjectsRequest.KeyVersion key : keys) { - LOG.debug(" {} {}", key.getKey(), - key.getVersion() != null ? key.getVersion() : ""); + for (ObjectIdentifier objectIdentifier : keys) { + LOG.debug(" {} {}", objectIdentifier.key(), + objectIdentifier.versionId() != null ? objectIdentifier.versionId() : ""); } } @@ -619,10 +653,10 @@ private String maybeAddTrailingSlash(String key) { protected IOException convertToIOException(final Exception ex) { if (ex instanceof IOException) { return (IOException) ex; - } else if (ex instanceof SdkBaseException) { + } else if (ex instanceof SdkException) { return translateException("rename " + sourcePath + " to " + destPath, sourcePath.toString(), - (SdkBaseException) ex); + (SdkException) ex); } else { // should never happen, but for completeness return new IOException(ex); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java index 614ed32cc7543..7c952aa15d2a6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java @@ -18,38 +18,36 @@ package org.apache.hadoop.fs.s3a.impl; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; +import java.util.Base64; +import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import javax.annotation.Nullable; -import com.amazonaws.AmazonWebServiceRequest; -import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; -import com.amazonaws.services.s3.model.CannedAccessControlList; -import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; -import com.amazonaws.services.s3.model.CopyObjectRequest; -import com.amazonaws.services.s3.model.DeleteObjectRequest; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; -import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; -import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsV2Request; -import com.amazonaws.services.s3.model.ObjectListing; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; -import com.amazonaws.services.s3.model.SSECustomerKey; -import com.amazonaws.services.s3.model.SelectObjectContentRequest; -import com.amazonaws.services.s3.model.StorageClass; -import com.amazonaws.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.core.SdkRequest; +import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CompletedMultipartUpload; +import software.amazon.awssdk.services.s3.model.CompletedPart; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.HeadBucketRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; +import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.MetadataDirective; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; +import software.amazon.awssdk.services.s3.model.ServerSideEncryption; +import software.amazon.awssdk.services.s3.model.StorageClass; +import software.amazon.awssdk.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.utils.Md5Utils; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,9 +60,11 @@ import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.fs.s3a.S3AEncryptionMethods.UNKNOWN_ALGORITHM; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkArgument; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; +import static software.amazon.awssdk.services.s3.model.StorageClass.UNKNOWN_TO_SDK_VERSION; /** * The standard implementation of the request factory. @@ -79,8 +79,8 @@ * This is where audit span information is added to the requests, * until it is done in the AWS SDK itself. * - * All created requests will be passed through - * {@link PrepareRequest#prepareRequest(AmazonWebServiceRequest)} before + * All created request builders will be passed to + * {@link PrepareRequest#prepareRequest(SdkRequest.Builder)} before * being returned to the caller. */ public class RequestFactoryImpl implements RequestFactory { @@ -101,7 +101,7 @@ public class RequestFactoryImpl implements RequestFactory { /** * ACL For new objects. */ - private final CannedAccessControlList cannedACL; + private final String cannedACL; /** * Max number of multipart entries allowed in a large @@ -147,14 +147,15 @@ protected RequestFactoryImpl( /** * Preflight preparation of AWS request. - * @param web service request - * @return prepared entry. + * @param web service request builder + * @return prepared builder. */ @Retries.OnceRaw - private T prepareRequest(T t) { - return requestPreparer != null - ? requestPreparer.prepareRequest(t) - : t; + private T prepareRequest(T t) { + if (requestPreparer != null) { + requestPreparer.prepareRequest(t); + } + return t; } /** @@ -162,7 +163,7 @@ private T prepareRequest(T t) { * @return an ACL, if any */ @Override - public CannedAccessControlList getCannedACL() { + public String getCannedACL() { return cannedACL; } @@ -174,29 +175,6 @@ protected String getBucket() { return bucket; } - /** - * 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 - */ - @Override - public Optional generateSSEAwsKeyParams() { - return EncryptionSecretOperations.createSSEAwsKeyManagementParams( - encryptionSecrets); - } - - /** - * 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 an optional customer key. - */ - @Override - public Optional generateSSECustomerKey() { - return EncryptionSecretOperations.createSSECustomerKey( - encryptionSecrets); - } - /** * Get the encryption algorithm of this endpoint. * @return the encryption algorithm. @@ -227,309 +205,367 @@ public StorageClass getStorageClass() { /** * Sets server side encryption parameters to the part upload * request when encryption is enabled. - * @param request upload part request + * @param builder upload part request builder */ - protected void setOptionalUploadPartRequestParameters( - UploadPartRequest request) { - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + protected void uploadPartEncryptionParameters( + UploadPartRequest.Builder builder) { + // need to set key to get objects encrypted with SSE_C + EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets).ifPresent(base64customerKey -> { + builder.sseCustomerAlgorithm(ServerSideEncryption.AES256.name()) + .sseCustomerKey(base64customerKey) + .sseCustomerKeyMD5(Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey))); + }); } - /** - * Sets server side encryption parameters to the GET reuquest. - * request when encryption is enabled. - * @param request upload part request - */ - protected void setOptionalGetObjectMetadataParameters( - GetObjectMetadataRequest request) { - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); - } + private CopyObjectRequest.Builder buildCopyObjectRequest() { - /** - * Set the optional parameters when initiating the request (encryption, - * headers, storage, etc). - * @param request request to patch. - */ - protected void setOptionalMultipartUploadRequestParameters( - InitiateMultipartUploadRequest request) { - generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams); - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); - } + CopyObjectRequest.Builder copyObjectRequestBuilder = CopyObjectRequest.builder(); - /** - * Set the optional parameters for a PUT request. - * @param request request to patch. - */ - protected void setOptionalPutRequestParameters(PutObjectRequest request) { - generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams); - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); - } - - /** - * Set the optional metadata for an object being created or copied. - * @param metadata to update. - * @param isDirectoryMarker is this for a directory marker? - */ - protected void setOptionalObjectMetadata(ObjectMetadata metadata, - boolean isDirectoryMarker) { - final S3AEncryptionMethods algorithm - = getServerSideEncryptionAlgorithm(); - if (S3AEncryptionMethods.SSE_S3 == algorithm) { - metadata.setSSEAlgorithm(algorithm.getMethod()); - } - if (contentEncoding != null && !isDirectoryMarker) { - metadata.setContentEncoding(contentEncoding); + if (contentEncoding != null) { + copyObjectRequestBuilder.contentEncoding(contentEncoding); } + + return copyObjectRequestBuilder; } - /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * - * @param length length of data to set in header; Ignored if negative - * @return a new metadata instance - */ @Override - public ObjectMetadata newObjectMetadata(long length) { - return createObjectMetadata(length, false); - } + public CopyObjectRequest.Builder newCopyObjectRequestBuilder(String srcKey, + String dstKey, + HeadObjectResponse srcom) { - /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * - * @param length length of data to set in header; Ignored if negative - * @param isDirectoryMarker is this for a directory marker? - * @return a new metadata instance - */ - private ObjectMetadata createObjectMetadata(long length, boolean isDirectoryMarker) { - final ObjectMetadata om = new ObjectMetadata(); - setOptionalObjectMetadata(om, isDirectoryMarker); - if (length >= 0) { - om.setContentLength(length); + CopyObjectRequest.Builder copyObjectRequestBuilder = buildCopyObjectRequest(); + + Map dstom = new HashMap<>(); + HeaderProcessing.cloneObjectMetadata(srcom, dstom, copyObjectRequestBuilder); + copyEncryptionParameters(srcom, copyObjectRequestBuilder); + + copyObjectRequestBuilder + .metadata(dstom) + .metadataDirective(MetadataDirective.REPLACE) + .acl(cannedACL); + + if (srcom.storageClass() != null && srcom.storageClass() != UNKNOWN_TO_SDK_VERSION) { + copyObjectRequestBuilder.storageClass(srcom.storageClass()); } - return om; - } - @Override - public CopyObjectRequest newCopyObjectRequest(String srcKey, - String dstKey, - ObjectMetadata srcom) { - CopyObjectRequest copyObjectRequest = - new CopyObjectRequest(getBucket(), srcKey, getBucket(), dstKey); - ObjectMetadata dstom = newObjectMetadata(srcom.getContentLength()); - HeaderProcessing.cloneObjectMetadata(srcom, dstom); - setOptionalObjectMetadata(dstom, false); - copyEncryptionParameters(srcom, copyObjectRequest); - copyObjectRequest.setCannedAccessControlList(cannedACL); - copyObjectRequest.setNewObjectMetadata(dstom); - Optional.ofNullable(srcom.getStorageClass()) - .ifPresent(copyObjectRequest::setStorageClass); - return prepareRequest(copyObjectRequest); + copyObjectRequestBuilder.destinationBucket(getBucket()) + .destinationKey(dstKey).sourceBucket(getBucket()).sourceKey(srcKey); + + return prepareRequest(copyObjectRequestBuilder); } /** * Propagate encryption parameters from source file if set else use the * current filesystem encryption settings. + * @param copyObjectRequestBuilder copy object request builder. * @param srcom source object metadata. - * @param copyObjectRequest copy object request body. */ - protected void copyEncryptionParameters( - ObjectMetadata srcom, - CopyObjectRequest copyObjectRequest) { - String sourceKMSId = srcom.getSSEAwsKmsKeyId(); + protected void copyEncryptionParameters(HeadObjectResponse srcom, + CopyObjectRequest.Builder copyObjectRequestBuilder) { + + final S3AEncryptionMethods algorithm = getServerSideEncryptionAlgorithm(); + + String sourceKMSId = srcom.ssekmsKeyId(); if (isNotEmpty(sourceKMSId)) { // source KMS ID is propagated LOG.debug("Propagating SSE-KMS settings from source {}", sourceKMSId); - copyObjectRequest.setSSEAwsKeyManagementParams( - new SSEAwsKeyManagementParams(sourceKMSId)); + copyObjectRequestBuilder.ssekmsKeyId(sourceKMSId); + return; } - switch (getServerSideEncryptionAlgorithm()) { + + switch (algorithm) { case SSE_S3: - /* no-op; this is set in destination object metadata */ + copyObjectRequestBuilder.serverSideEncryption(algorithm.getMethod()); + break; + case SSE_KMS: + copyObjectRequestBuilder.serverSideEncryption(ServerSideEncryption.AWS_KMS); + // Set the KMS key if present, else S3 uses AWS managed key. + EncryptionSecretOperations.getSSEAwsKMSKey(encryptionSecrets) + .ifPresent(copyObjectRequestBuilder::ssekmsKeyId); + break; + case DSSE_KMS: + copyObjectRequestBuilder.serverSideEncryption(ServerSideEncryption.AWS_KMS_DSSE); + EncryptionSecretOperations.getSSEAwsKMSKey(encryptionSecrets) + .ifPresent(copyObjectRequestBuilder::ssekmsKeyId); break; - case SSE_C: - generateSSECustomerKey().ifPresent(customerKey -> { - copyObjectRequest.setSourceSSECustomerKey(customerKey); - copyObjectRequest.setDestinationSSECustomerKey(customerKey); - }); + EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets) + .ifPresent(base64customerKey -> copyObjectRequestBuilder + .copySourceSSECustomerAlgorithm(ServerSideEncryption.AES256.name()) + .copySourceSSECustomerKey(base64customerKey) + .copySourceSSECustomerKeyMD5( + Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey))) + .sseCustomerAlgorithm(ServerSideEncryption.AES256.name()) + .sseCustomerKey(base64customerKey).sseCustomerKeyMD5( + Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey)))); break; - - case SSE_KMS: - generateSSEAwsKeyParams().ifPresent( - copyObjectRequest::setSSEAwsKeyManagementParams); + case CSE_KMS: + case CSE_CUSTOM: + case NONE: break; default: + LOG.warn(UNKNOWN_ALGORITHM + ": " + algorithm); } } /** * Create a putObject request. * Adds the ACL, storage class and metadata * @param key key of object - * @param metadata metadata header * @param options options for the request, including headers - * @param srcfile source file - * @return the request + * @param length length of object to be uploaded + * @param isDirectoryMarker true if object to be uploaded is a directory marker + * @return the request builder */ @Override - public PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, + public PutObjectRequest.Builder newPutObjectRequestBuilder(String key, final PutObjectOptions options, - File srcfile) { - Preconditions.checkNotNull(srcfile); - PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, - srcfile); - maybeSetMetadata(options, metadata); - setOptionalPutRequestParameters(putObjectRequest); - putObjectRequest.setCannedAcl(cannedACL); + long length, + boolean isDirectoryMarker) { + + Preconditions.checkArgument(isNotEmpty(key), "Null/empty key"); + + PutObjectRequest.Builder putObjectRequestBuilder = + buildPutObjectRequest(length, isDirectoryMarker); + putObjectRequestBuilder.bucket(getBucket()).key(key); + + if (options != null) { + putObjectRequestBuilder.metadata(options.getHeaders()); + } + + putEncryptionParameters(putObjectRequestBuilder); + if (storageClass != null) { - putObjectRequest.setStorageClass(storageClass); + putObjectRequestBuilder.storageClass(storageClass); } - putObjectRequest.setMetadata(metadata); - return prepareRequest(putObjectRequest); + + return prepareRequest(putObjectRequestBuilder); } - /** - * Create a {@link PutObjectRequest} request. - * The metadata is assumed to have been configured with the size of the - * operation. - * @param key key of object - * @param metadata metadata header - * @param options options for the request - * @param inputStream source data. - * @return the request - */ - @Override - public PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, - @Nullable final PutObjectOptions options, - InputStream inputStream) { - Preconditions.checkNotNull(inputStream); - Preconditions.checkArgument(isNotEmpty(key), "Null/empty key"); - maybeSetMetadata(options, metadata); - PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, - inputStream, metadata); - setOptionalPutRequestParameters(putObjectRequest); - putObjectRequest.setCannedAcl(cannedACL); - if (storageClass != null) { - putObjectRequest.setStorageClass(storageClass); + private PutObjectRequest.Builder buildPutObjectRequest(long length, boolean isDirectoryMarker) { + + PutObjectRequest.Builder putObjectRequestBuilder = PutObjectRequest.builder(); + + putObjectRequestBuilder.acl(cannedACL); + + if (length >= 0) { + putObjectRequestBuilder.contentLength(length); + } + + if (contentEncoding != null && !isDirectoryMarker) { + putObjectRequestBuilder.contentEncoding(contentEncoding); + } + + return putObjectRequestBuilder; + } + + private void putEncryptionParameters(PutObjectRequest.Builder putObjectRequestBuilder) { + final S3AEncryptionMethods algorithm + = getServerSideEncryptionAlgorithm(); + + switch (algorithm) { + case SSE_S3: + putObjectRequestBuilder.serverSideEncryption(algorithm.getMethod()); + break; + case SSE_KMS: + putObjectRequestBuilder.serverSideEncryption(ServerSideEncryption.AWS_KMS); + // Set the KMS key if present, else S3 uses AWS managed key. + EncryptionSecretOperations.getSSEAwsKMSKey(encryptionSecrets) + .ifPresent(putObjectRequestBuilder::ssekmsKeyId); + break; + case DSSE_KMS: + putObjectRequestBuilder.serverSideEncryption(ServerSideEncryption.AWS_KMS_DSSE); + EncryptionSecretOperations.getSSEAwsKMSKey(encryptionSecrets) + .ifPresent(putObjectRequestBuilder::ssekmsKeyId); + break; + case SSE_C: + EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets) + .ifPresent(base64customerKey -> putObjectRequestBuilder + .sseCustomerAlgorithm(ServerSideEncryption.AES256.name()) + .sseCustomerKey(base64customerKey) + .sseCustomerKeyMD5(Md5Utils.md5AsBase64( + Base64.getDecoder().decode(base64customerKey)))); + break; + case CSE_KMS: + case CSE_CUSTOM: + case NONE: + break; + default: + LOG.warn(UNKNOWN_ALGORITHM + ": " + algorithm); } - return prepareRequest(putObjectRequest); } @Override - public PutObjectRequest newDirectoryMarkerRequest(String directory) { + public PutObjectRequest.Builder newDirectoryMarkerRequest(String directory) { String key = directory.endsWith("/") ? directory : (directory + "/"); - // an input stream which is always empty - final InputStream inputStream = new InputStream() { - @Override - public int read() throws IOException { - return -1; - } - }; + // preparation happens in here - final ObjectMetadata metadata = createObjectMetadata(0L, true); - metadata.setContentType(HeaderProcessing.CONTENT_TYPE_X_DIRECTORY); + PutObjectRequest.Builder putObjectRequestBuilder = buildPutObjectRequest(0L, true); - PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, - inputStream, metadata); - setOptionalPutRequestParameters(putObjectRequest); - putObjectRequest.setCannedAcl(cannedACL); - return prepareRequest(putObjectRequest); + putObjectRequestBuilder.bucket(getBucket()).key(key) + .contentType(HeaderProcessing.CONTENT_TYPE_X_DIRECTORY); + + putEncryptionParameters(putObjectRequestBuilder); + + return prepareRequest(putObjectRequestBuilder); } @Override - public ListMultipartUploadsRequest - newListMultipartUploadsRequest(String prefix) { - ListMultipartUploadsRequest request = new ListMultipartUploadsRequest( - getBucket()); + public ListMultipartUploadsRequest.Builder + newListMultipartUploadsRequestBuilder(String prefix) { + + ListMultipartUploadsRequest.Builder requestBuilder = ListMultipartUploadsRequest.builder(); + + requestBuilder.bucket(getBucket()); if (prefix != null) { - request.setPrefix(prefix); + requestBuilder.prefix(prefix); } - return prepareRequest(request); + return prepareRequest(requestBuilder); } @Override - public AbortMultipartUploadRequest newAbortMultipartUploadRequest( + public AbortMultipartUploadRequest.Builder newAbortMultipartUploadRequestBuilder( String destKey, String uploadId) { - return prepareRequest(new AbortMultipartUploadRequest(getBucket(), - destKey, - uploadId)); + AbortMultipartUploadRequest.Builder requestBuilder = + AbortMultipartUploadRequest.builder().bucket(getBucket()).key(destKey).uploadId(uploadId); + + return prepareRequest(requestBuilder); + } + + private void multipartUploadEncryptionParameters( + CreateMultipartUploadRequest.Builder mpuRequestBuilder) { + final S3AEncryptionMethods algorithm = getServerSideEncryptionAlgorithm(); + + switch (algorithm) { + case SSE_S3: + mpuRequestBuilder.serverSideEncryption(algorithm.getMethod()); + break; + case SSE_KMS: + mpuRequestBuilder.serverSideEncryption(ServerSideEncryption.AWS_KMS); + // Set the KMS key if present, else S3 uses AWS managed key. + EncryptionSecretOperations.getSSEAwsKMSKey(encryptionSecrets) + .ifPresent(mpuRequestBuilder::ssekmsKeyId); + break; + case DSSE_KMS: + mpuRequestBuilder.serverSideEncryption(ServerSideEncryption.AWS_KMS_DSSE); + EncryptionSecretOperations.getSSEAwsKMSKey(encryptionSecrets) + .ifPresent(mpuRequestBuilder::ssekmsKeyId); + break; + case SSE_C: + EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets) + .ifPresent(base64customerKey -> mpuRequestBuilder + .sseCustomerAlgorithm(ServerSideEncryption.AES256.name()) + .sseCustomerKey(base64customerKey) + .sseCustomerKeyMD5( + Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey)))); + break; + case CSE_KMS: + case CSE_CUSTOM: + case NONE: + break; + default: + LOG.warn(UNKNOWN_ALGORITHM + ": " + algorithm); + } } @Override - public InitiateMultipartUploadRequest newMultipartUploadRequest( + public CreateMultipartUploadRequest.Builder newMultipartUploadRequestBuilder( final String destKey, @Nullable final PutObjectOptions options) throws PathIOException { if (!isMultipartUploadEnabled) { throw new PathIOException(destKey, "Multipart uploads are disabled."); } - final ObjectMetadata objectMetadata = newObjectMetadata(-1); - maybeSetMetadata(options, objectMetadata); - final InitiateMultipartUploadRequest initiateMPURequest = - new InitiateMultipartUploadRequest(getBucket(), - destKey, - objectMetadata); - initiateMPURequest.setCannedACL(getCannedACL()); - if (getStorageClass() != null) { - initiateMPURequest.withStorageClass(getStorageClass()); + + CreateMultipartUploadRequest.Builder requestBuilder = CreateMultipartUploadRequest.builder(); + + if (contentEncoding != null) { + requestBuilder.contentEncoding(contentEncoding); + } + + if (options != null) { + requestBuilder.metadata(options.getHeaders()); + } + + requestBuilder.bucket(getBucket()).key(destKey).acl(cannedACL); + + multipartUploadEncryptionParameters(requestBuilder); + + if (storageClass != null) { + requestBuilder.storageClass(storageClass); } - setOptionalMultipartUploadRequestParameters(initiateMPURequest); - return prepareRequest(initiateMPURequest); + + return prepareRequest(requestBuilder); } @Override - public CompleteMultipartUploadRequest newCompleteMultipartUploadRequest( + public CompleteMultipartUploadRequest.Builder newCompleteMultipartUploadRequestBuilder( String destKey, String uploadId, - List partETags) { + List partETags) { // a copy of the list is required, so that the AWS SDK doesn't // attempt to sort an unmodifiable list. - return prepareRequest(new CompleteMultipartUploadRequest(bucket, - destKey, uploadId, new ArrayList<>(partETags))); + CompleteMultipartUploadRequest.Builder requestBuilder = + CompleteMultipartUploadRequest.builder().bucket(bucket).key(destKey).uploadId(uploadId) + .multipartUpload(CompletedMultipartUpload.builder().parts(partETags).build()); + return prepareRequest(requestBuilder); } @Override - public GetObjectMetadataRequest newGetObjectMetadataRequest(String key) { - GetObjectMetadataRequest request = - new GetObjectMetadataRequest(getBucket(), key); - //SSE-C requires to be filled in if enabled for object metadata - setOptionalGetObjectMetadataParameters(request); - return prepareRequest(request); + public HeadObjectRequest.Builder newHeadObjectRequestBuilder(String key) { + + HeadObjectRequest.Builder headObjectRequestBuilder = + HeadObjectRequest.builder().bucket(getBucket()).key(key); + + // need to set key to get metadata for objects encrypted with SSE_C + EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets).ifPresent(base64customerKey -> { + headObjectRequestBuilder.sseCustomerAlgorithm(ServerSideEncryption.AES256.name()) + .sseCustomerKey(base64customerKey) + .sseCustomerKeyMD5(Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey))); + }); + + return prepareRequest(headObjectRequestBuilder); + } + + @Override + public HeadBucketRequest.Builder newHeadBucketRequestBuilder(String bucketName) { + + HeadBucketRequest.Builder headBucketRequestBuilder = + HeadBucketRequest.builder().bucket(bucketName); + + return prepareRequest(headBucketRequestBuilder); } @Override - public GetObjectRequest newGetObjectRequest(String key) { - GetObjectRequest request = new GetObjectRequest(bucket, key); - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + public GetObjectRequest.Builder newGetObjectRequestBuilder(String key) { + GetObjectRequest.Builder builder = GetObjectRequest.builder() + .bucket(bucket) + .key(key); + + // need to set key to get objects encrypted with SSE_C + EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets).ifPresent(base64customerKey -> { + builder.sseCustomerAlgorithm(ServerSideEncryption.AES256.name()) + .sseCustomerKey(base64customerKey) + .sseCustomerKeyMD5(Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey))); + }); - return prepareRequest(request); + return prepareRequest(builder); } @Override - public UploadPartRequest newUploadPartRequest( + public UploadPartRequest.Builder newUploadPartRequestBuilder( String destKey, String uploadId, int partNumber, - long size, - InputStream uploadStream, - File sourceFile, - long offset) throws PathIOException { + long size) throws PathIOException { checkNotNull(uploadId); - // exactly one source must be set; xor verifies this - checkArgument((uploadStream != null) ^ (sourceFile != null), - "Data source"); checkArgument(size >= 0, "Invalid partition size %s", size); checkArgument(partNumber > 0, "partNumber must be between 1 and %s inclusive, but is %s", - DEFAULT_UPLOAD_PART_COUNT_LIMIT, partNumber); + multipartPartCountLimit, partNumber); LOG.debug("Creating part upload request for {} #{} size {}", uploadId, partNumber, size); @@ -539,88 +575,76 @@ public UploadPartRequest newUploadPartRequest( throw new PathIOException(destKey, String.format(pathErrorMsg, partNumber, multipartPartCountLimit)); } - UploadPartRequest request = new UploadPartRequest() - .withBucketName(getBucket()) - .withKey(destKey) - .withUploadId(uploadId) - .withPartNumber(partNumber) - .withPartSize(size); - if (uploadStream != null) { - // there's an upload stream. Bind to it. - request.setInputStream(uploadStream); - } else { - checkArgument(sourceFile.exists(), - "Source file does not exist: %s", sourceFile); - checkArgument(sourceFile.isFile(), - "Source is not a file: %s", sourceFile); - checkArgument(offset >= 0, "Invalid offset %s", offset); - long length = sourceFile.length(); - checkArgument(offset == 0 || offset < length, - "Offset %s beyond length of file %s", offset, length); - request.setFile(sourceFile); - request.setFileOffset(offset); - } - setOptionalUploadPartRequestParameters(request); - return prepareRequest(request); + UploadPartRequest.Builder builder = UploadPartRequest.builder() + .bucket(getBucket()) + .key(destKey) + .uploadId(uploadId) + .partNumber(partNumber) + .contentLength(size); + uploadPartEncryptionParameters(builder); + return prepareRequest(builder); } @Override - public SelectObjectContentRequest newSelectRequest(String key) { - SelectObjectContentRequest request = new SelectObjectContentRequest(); - request.setBucketName(bucket); - request.setKey(key); - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); - return prepareRequest(request); + public SelectObjectContentRequest.Builder newSelectRequestBuilder(String key) { + SelectObjectContentRequest.Builder requestBuilder = + SelectObjectContentRequest.builder().bucket(bucket).key(key); + + EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets).ifPresent(base64customerKey -> { + requestBuilder.sseCustomerAlgorithm(ServerSideEncryption.AES256.name()) + .sseCustomerKey(base64customerKey) + .sseCustomerKeyMD5(Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey))); + }); + + return prepareRequest(requestBuilder); } @Override - public ListObjectsRequest newListObjectsV1Request( + public ListObjectsRequest.Builder newListObjectsV1RequestBuilder( final String key, final String delimiter, final int maxKeys) { - ListObjectsRequest request = new ListObjectsRequest() - .withBucketName(bucket) - .withMaxKeys(maxKeys) - .withPrefix(key); + + ListObjectsRequest.Builder requestBuilder = + ListObjectsRequest.builder().bucket(bucket).maxKeys(maxKeys).prefix(key); + if (delimiter != null) { - request.setDelimiter(delimiter); + requestBuilder.delimiter(delimiter); } - return prepareRequest(request); - } - @Override - public ListNextBatchOfObjectsRequest newListNextBatchOfObjectsRequest( - ObjectListing prev) { - return prepareRequest(new ListNextBatchOfObjectsRequest(prev)); + return prepareRequest(requestBuilder); } @Override - public ListObjectsV2Request newListObjectsV2Request( + public ListObjectsV2Request.Builder newListObjectsV2RequestBuilder( final String key, final String delimiter, final int maxKeys) { - final ListObjectsV2Request request = new ListObjectsV2Request() - .withBucketName(bucket) - .withMaxKeys(maxKeys) - .withPrefix(key); + + final ListObjectsV2Request.Builder requestBuilder = ListObjectsV2Request.builder() + .bucket(bucket) + .maxKeys(maxKeys) + .prefix(key); + if (delimiter != null) { - request.setDelimiter(delimiter); + requestBuilder.delimiter(delimiter); } - return prepareRequest(request); + + return prepareRequest(requestBuilder); } @Override - public DeleteObjectRequest newDeleteObjectRequest(String key) { - return prepareRequest(new DeleteObjectRequest(bucket, key)); + public DeleteObjectRequest.Builder newDeleteObjectRequestBuilder(String key) { + return prepareRequest(DeleteObjectRequest.builder().bucket(bucket).key(key)); } @Override - public DeleteObjectsRequest newBulkDeleteRequest( - List keysToDelete) { - return prepareRequest( - new DeleteObjectsRequest(bucket) - .withKeys(keysToDelete) - .withQuiet(true)); + public DeleteObjectsRequest.Builder newBulkDeleteRequestBuilder( + List keysToDelete) { + return prepareRequest(DeleteObjectsRequest + .builder() + .bucket(bucket) + .delete(d -> d.objects(keysToDelete).quiet(true))); } @Override @@ -628,23 +652,6 @@ public void setEncryptionSecrets(final EncryptionSecrets secrets) { encryptionSecrets = secrets; } - /** - * Set the metadata from the options if the options are not - * null and the metadata contains headers. - * @param options options for the request - * @param objectMetadata metadata to patch - */ - private void maybeSetMetadata( - @Nullable PutObjectOptions options, - final ObjectMetadata objectMetadata) { - if (options != null) { - Map headers = options.getHeaders(); - if (headers != null) { - objectMetadata.setUserMetadata(headers); - } - } - } - /** * Create a builder. * @return new builder. @@ -671,7 +678,7 @@ public static final class RequestFactoryBuilder { /** * ACL For new objects. */ - private CannedAccessControlList cannedACL = null; + private String cannedACL = null; /** Content Encoding. */ private String contentEncoding; @@ -754,7 +761,7 @@ public RequestFactoryBuilder withEncryptionSecrets( * @return the builder */ public RequestFactoryBuilder withCannedACL( - final CannedAccessControlList value) { + final String value) { cannedACL = value; return this; } @@ -806,11 +813,9 @@ public interface PrepareRequest { /** * Post-creation preparation of AWS request. - * @param t request - * @param request type. - * @return prepared entry. + * @param t request builder */ @Retries.OnceRaw - T prepareRequest(T t); + void prepareRequest(SdkRequest.Builder t); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java index 4ab5bc6a99245..b7eae8ead7096 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java @@ -34,10 +34,12 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; -import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; -import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.UploadPartRequest; -import com.amazonaws.services.s3.model.UploadPartResult; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse; +import software.amazon.awssdk.services.s3.model.CompletedPart; +import software.amazon.awssdk.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.services.s3.model.UploadPartResponse; + import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.commons.lang3.StringUtils; @@ -152,18 +154,18 @@ public CompletableFuture putPart( Charsets.UTF_8); return context.submit(new CompletableFuture<>(), () -> { - UploadPartRequest request = writeOperations.newUploadPartRequest(key, - uploadIdString, partNumber, (int) lengthInBytes, inputStream, - null, 0L); - UploadPartResult result = writeOperations.uploadPart(request, statistics); + UploadPartRequest request = writeOperations.newUploadPartRequestBuilder(key, + uploadIdString, partNumber, lengthInBytes).build(); + RequestBody body = RequestBody.fromInputStream(inputStream, lengthInBytes); + UploadPartResponse response = writeOperations.uploadPart(request, body, statistics); statistics.partPut(lengthInBytes); - String eTag = result.getETag(); + String eTag = response.eTag(); return BBPartHandle.from( ByteBuffer.wrap( buildPartHandlePayload( filePath.toUri().toString(), uploadIdString, - result.getPartNumber(), + partNumber, eTag, lengthInBytes))); }); @@ -188,7 +190,7 @@ public CompletableFuture complete( String uploadIdStr = new String(uploadIdBytes, 0, uploadIdBytes.length, Charsets.UTF_8); - ArrayList eTags = new ArrayList<>(); + ArrayList eTags = new ArrayList<>(); eTags.ensureCapacity(handles.size()); long totalLength = 0; // built up to identify duplicates -if the size of this set is @@ -201,7 +203,8 @@ public CompletableFuture complete( payload.validate(uploadIdStr, filePath); ids.add(payload.getPartNumber()); totalLength += payload.getLen(); - eTags.add(new PartETag(handle.getKey(), payload.getEtag())); + eTags.add( + CompletedPart.builder().partNumber(handle.getKey()).eTag(payload.getEtag()).build()); } Preconditions.checkArgument(ids.size() == count, "Duplicate PartHandles"); @@ -210,7 +213,7 @@ public CompletableFuture complete( long finalLen = totalLength; return context.submit(new CompletableFuture<>(), trackDurationOfCallable(statistics, MULTIPART_UPLOAD_COMPLETED.getSymbol(), () -> { - CompleteMultipartUploadResult result = + CompleteMultipartUploadResponse result = writeOperations.commitUpload( key, uploadIdStr, @@ -218,7 +221,7 @@ public CompletableFuture complete( finalLen ); - byte[] eTag = result.getETag().getBytes(Charsets.UTF_8); + byte[] eTag = result.eTag().getBytes(Charsets.UTF_8); statistics.uploadCompleted(); return (PathHandle) () -> ByteBuffer.wrap(eTag); })); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/SDKStreamDrainer.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/SDKStreamDrainer.java index b566f9ad42765..49c2fb8947dce 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/SDKStreamDrainer.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/SDKStreamDrainer.java @@ -18,12 +18,10 @@ package org.apache.hadoop.fs.s3a.impl; -import java.io.Closeable; +import java.io.InputStream; import java.util.concurrent.atomic.AtomicBoolean; -import javax.annotation.Nullable; - -import com.amazonaws.internal.SdkFilterInputStream; +import software.amazon.awssdk.http.Abortable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,23 +29,18 @@ import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import org.apache.hadoop.util.functional.CallableRaisingIOE; + import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DRAIN_BUFFER_SIZE; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration; -import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; /** * Drains/aborts s3 or other AWS SDK streams. * It is callable so can be passed directly to a submitter * for async invocation. - * A request object may be passed in; it will be implicitly - * cached until this object is GCd. - * This is because in some versions of the AWS SDK, the S3Object - * has a finalize() method which releases the http connection, - * even when the stream is still open. - * See HADOOP-17338 for details. */ -public class SDKStreamDrainer implements CallableRaisingIOE { +public class SDKStreamDrainer + implements CallableRaisingIOE { private static final Logger LOG = LoggerFactory.getLogger( SDKStreamDrainer.class); @@ -58,17 +51,9 @@ public class SDKStreamDrainer implements CallableRaisingIOE { private final String uri; /** - * Request object; usually S3Object - * Never used, but needed to keep the http connection - * open long enough for draining to take place. + * Stream from the getObject response for draining and closing. */ - @Nullable - private final Closeable requestObject; - - /** - * Stream from the {@link #requestObject} for draining and closing. - */ - private final SdkFilterInputStream sdkStream; + private final TStream sdkStream; /** * Should the request be aborted? @@ -118,7 +103,6 @@ public class SDKStreamDrainer implements CallableRaisingIOE { /** * Prepare to drain the stream. * @param uri URI for messages - * @param requestObject http request object; needed to avoid GC issues. * @param sdkStream stream to close. * @param shouldAbort force an abort; used if explicitly requested. * @param streamStatistics stats to update @@ -126,14 +110,12 @@ public class SDKStreamDrainer implements CallableRaisingIOE { * @param remaining remaining bytes */ public SDKStreamDrainer(final String uri, - @Nullable final Closeable requestObject, - final SdkFilterInputStream sdkStream, + final TStream sdkStream, final boolean shouldAbort, final int remaining, final S3AInputStreamStatistics streamStatistics, final String reason) { this.uri = uri; - this.requestObject = requestObject; this.sdkStream = requireNonNull(sdkStream); this.shouldAbort = shouldAbort; this.remaining = remaining; @@ -233,7 +215,6 @@ private boolean drainOrAbortHttpStream() { LOG.debug("Closing stream"); sdkStream.close(); - cleanupWithLogger(LOG, requestObject); // this MUST come after the close, so that if the IO operations fail // and an abort is triggered, the initial attempt's statistics // aren't collected. @@ -255,8 +236,6 @@ private boolean drainOrAbortHttpStream() { LOG.warn("When aborting {} stream after failing to close it for {}", uri, reason, e); thrown = e; - } finally { - cleanupWithLogger(LOG, requestObject); } streamStatistics.streamClose(true, remaining); @@ -269,11 +248,7 @@ public String getUri() { return uri; } - public Object getRequestObject() { - return requestObject; - } - - public SdkFilterInputStream getSdkStream() { + public TStream getSdkStream() { return sdkStream; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/V2Migration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/V2Migration.java index 3aa8ad270eedd..bc9b0e49a37b0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/V2Migration.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/V2Migration.java @@ -23,11 +23,22 @@ import org.apache.hadoop.fs.store.LogExactlyOnce; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SDK_V2_UPGRADE_LOG_NAME; /** * This class provides utility methods required for migrating S3A to AWS Java SDK V2. * For more information on the upgrade, see HADOOP-18073. + * + *

    in HADOOP-18382. Upgrade AWS SDK to V2 - Prerequisites, + * this class contained a series of `LogExactlyOnce` loggers to warn on + * the first use of a feature which would change incompatibly; this shipped in Hadoop 3.3.5. + *

    + * With the move to v2 completed, attempts to use the v1 classes, will fail + * -except for the special case of support for v1 credential providers. + *

    + * The warning methods are still present, where appropriate, but downgraded to debug + * and only retained for debugging migration issues. */ public final class V2Migration { @@ -35,64 +46,17 @@ private V2Migration() { } public static final Logger SDK_V2_UPGRADE_LOG = LoggerFactory.getLogger(SDK_V2_UPGRADE_LOG_NAME); - private static final LogExactlyOnce WARN_ON_DELEGATION_TOKENS = - new LogExactlyOnce(SDK_V2_UPGRADE_LOG); - - private static final LogExactlyOnce WARN_ON_GET_S3_CLIENT = - new LogExactlyOnce(SDK_V2_UPGRADE_LOG); - - private static final LogExactlyOnce WARN_OF_DIRECTLY_REFERENCED_CREDENTIAL_PROVIDER = - new LogExactlyOnce(SDK_V2_UPGRADE_LOG); - - private static final LogExactlyOnce WARN_OF_CUSTOM_SIGNER = + private static final LogExactlyOnce WARN_OF_REQUEST_HANDLERS = new LogExactlyOnce(SDK_V2_UPGRADE_LOG); - private static final LogExactlyOnce WARN_ON_GET_OBJECT_METADATA = - new LogExactlyOnce(SDK_V2_UPGRADE_LOG); - - /** - * Warns on an AWS V1 credential provider being referenced directly. - * @param name name of the credential provider - */ - public static void v1ProviderReferenced(String name) { - WARN_OF_DIRECTLY_REFERENCED_CREDENTIAL_PROVIDER.warn( - "Directly referencing AWS SDK V1 credential provider {}. AWS SDK V1 credential " - + "providers will be removed once S3A is upgraded to SDK V2", name); - } - - /** - * Warns on the v1 s3 client being requested. - */ - public static void v1S3ClientRequested() { - WARN_ON_GET_S3_CLIENT.warn( - "getAmazonS3ClientForTesting() will be removed as part of upgrading S3A to AWS SDK V2"); - } - - /** - * Warns when v1 credential providers are used with delegation tokens. - */ - public static void v1DelegationTokenCredentialProvidersUsed() { - WARN_ON_DELEGATION_TOKENS.warn( - "The credential provider interface has changed in AWS SDK V2, custom credential " - + "providers used in delegation tokens binding classes will need to be updated once " - + "S3A is upgraded to SDK V2"); - } - - /** - * Warns on use of custom signers. - */ - public static void v1CustomSignerUsed() { - WARN_OF_CUSTOM_SIGNER.warn( - "The signer interface has changed in AWS SDK V2, custom signers will need to be updated " - + "once S3A is upgraded to SDK V2"); - } - /** - * Warns on use of getObjectMetadata. + * Notes use of request handlers. + * @param handlers handlers declared */ - public static void v1GetObjectMetadataCalled() { - WARN_ON_GET_OBJECT_METADATA.warn("getObjectMetadata() called. This operation and it's response " - + "will be changed as part of upgrading S3A to AWS SDK V2"); + public static void v1RequestHandlersUsed(final String handlers) { + WARN_OF_REQUEST_HANDLERS.warn( + "Ignoring V1 SDK request handlers set in {}: {}", + AUDIT_REQUEST_HANDLERS, handlers); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java index 3ab0022bb082e..ec6e3700226e0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java @@ -19,15 +19,11 @@ package org.apache.hadoop.fs.s3a.prefetch; - import java.io.IOException; -import java.io.InputStream; -import java.util.IdentityHashMap; -import java.util.Map; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.S3ObjectInputStream; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,12 +31,14 @@ import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.S3AReadOpContext; +import org.apache.hadoop.fs.s3a.S3AUtils; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; import org.apache.hadoop.fs.s3a.impl.SDKStreamDrainer; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import org.apache.hadoop.fs.statistics.DurationTracker; + /** * Encapsulates low level interactions with S3 object on AWS. */ @@ -74,12 +72,6 @@ public class S3ARemoteObject { */ private final ChangeTracker changeTracker; - /** - * Maps a stream returned by openForRead() to the associated S3 object. - * That allows us to close the object when closing the stream. - */ - private final Map s3Objects; - /** * uri of the object being read. */ @@ -123,7 +115,6 @@ public S3ARemoteObject( this.client = client; this.streamStatistics = streamStatistics; this.changeTracker = changeTracker; - this.s3Objects = new IdentityHashMap<>(); this.uri = this.getPath(); } @@ -187,21 +178,23 @@ public long size() { * @throws IllegalArgumentException if offset is greater than or equal to file size. * @throws IllegalArgumentException if size is greater than the remaining bytes. */ - public InputStream openForRead(long offset, int size) throws IOException { + public ResponseInputStream openForRead(long offset, int size) + throws IOException { Validate.checkNotNegative(offset, "offset"); Validate.checkLessOrEqual(offset, "offset", size(), "size()"); Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset"); streamStatistics.streamOpened(); - final GetObjectRequest request = - client.newGetRequest(s3Attributes.getKey()) - .withRange(offset, offset + size - 1); - changeTracker.maybeApplyConstraint(request); + final GetObjectRequest request = client + .newGetRequestBuilder(s3Attributes.getKey()) + .range(S3AUtils.formatRange(offset, offset + size - 1)) + .applyMutation(changeTracker::maybeApplyConstraint) + .build(); String operation = String.format( "%s %s at %d", S3AInputStream.OPERATION_OPEN, uri, offset); DurationTracker tracker = streamStatistics.initiateGetRequest(); - S3Object object = null; + ResponseInputStream object = null; try { object = Invoker.once(operation, uri, () -> client.getObject(request)); @@ -212,27 +205,14 @@ public InputStream openForRead(long offset, int size) throws IOException { tracker.close(); } - changeTracker.processResponse(object, operation, offset); - InputStream stream = object.getObjectContent(); - synchronized (s3Objects) { - s3Objects.put(stream, object); - } - - return stream; + changeTracker.processResponse(object.response(), operation, offset); + return object; } - void close(InputStream inputStream, int numRemainingBytes) { - S3Object obj; - synchronized (s3Objects) { - obj = s3Objects.remove(inputStream); - if (obj == null) { - throw new IllegalArgumentException("inputStream not found"); - } - } + void close(ResponseInputStream inputStream, int numRemainingBytes) { SDKStreamDrainer drainer = new SDKStreamDrainer( uri, - obj, - (S3ObjectInputStream)inputStream, + inputStream, false, numRemainingBytes, streamStatistics, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java index 89ea77d6d0ebb..b49b2699f916b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java @@ -22,7 +22,6 @@ import java.io.Closeable; import java.io.EOFException; import java.io.IOException; -import java.io.InputStream; import java.net.SocketTimeoutException; import java.nio.ByteBuffer; @@ -33,6 +32,9 @@ import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; + import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_REMOTE_BLOCK_READ; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation; @@ -144,7 +146,8 @@ private void readOneBlock(ByteBuffer buffer, long offset, int size) return; } - InputStream inputStream = remoteObject.openForRead(offset, readSize); + ResponseInputStream inputStream = + remoteObject.openForRead(offset, readSize); int numRemainingBytes = readSize; byte[] bytes = new byte[READ_BUFFER_SIZE]; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index 608f9168c24cc..ea1ea908486e2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -24,7 +24,6 @@ import java.io.PrintStream; import java.net.URI; import java.net.URISyntaxException; -import java.nio.file.AccessDeniedException; import java.util.Arrays; import java.util.Collection; import java.util.Date; @@ -33,7 +32,7 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import com.amazonaws.services.s3.model.MultipartUpload; +import software.amazon.awssdk.services.s3.model.MultipartUpload; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,8 +47,8 @@ import org.apache.hadoop.fs.FilterFileSystem; import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.Constants; -import org.apache.hadoop.fs.s3a.MultipartUtils; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.WriteOperationHelper; import org.apache.hadoop.fs.s3a.auth.RolePolicies; @@ -400,12 +399,20 @@ public int run(String[] args, PrintStream out) println(out, "Filesystem %s", fsUri); try { println(out, "Location: %s", fs.getBucketLocation()); - } catch (AccessDeniedException e) { + } catch (IOException e) { // Caller cannot get the location of this bucket due to permissions - // in their role or the bucket itself. + // in their role or the bucket itself, or it is not an operation + // supported by this store. // Note and continue. LOG.debug("failed to get bucket location", e); println(out, LOCATION_UNKNOWN); + + // it may be the bucket is not found; we can't differentiate + // that and handle third party store issues where the API may + // not work. + // Fallback to looking for bucket root attributes. + println(out, "Probing for bucket existence"); + fs.listXAttrs(new Path("/")); } // print any auth paths for directory marker info @@ -676,7 +683,7 @@ private void promptBeforeAbort(PrintStream out) throws IOException { private void processUploads(PrintStream out) throws IOException { final S3AFileSystem fs = getFilesystem(); - MultipartUtils.UploadIterator uploads = fs.listUploads(prefix); + RemoteIterator uploads = fs.listUploads(prefix); // create a span so that the write operation helper // is within one AuditSpan span = @@ -694,11 +701,11 @@ private void processUploads(PrintStream out) throws IOException { count++; if (mode == Mode.ABORT || mode == Mode.LIST || verbose) { println(out, "%s%s %s", mode == Mode.ABORT ? "Deleting: " : "", - upload.getKey(), upload.getUploadId()); + upload.key(), upload.uploadId()); } if (mode == Mode.ABORT) { writeOperationHelper - .abortMultipartUpload(upload.getKey(), upload.getUploadId(), + .abortMultipartUpload(upload.key(), upload.uploadId(), true, LOG_EVENT); } } @@ -726,7 +733,7 @@ private boolean olderThan(MultipartUpload u, long msec) { return true; } Date ageDate = new Date(System.currentTimeMillis() - msec); - return ageDate.compareTo(u.getInitiated()) >= 0; + return ageDate.compareTo(Date.from(u.initiated())) >= 0; } private void processArgs(List args, PrintStream out) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/BlockingEnumeration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/BlockingEnumeration.java new file mode 100644 index 0000000000000..42000f1017259 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/BlockingEnumeration.java @@ -0,0 +1,156 @@ +/* + * 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.select; + +import java.util.Enumeration; +import java.util.NoSuchElementException; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; + +import org.reactivestreams.Subscriber; +import org.reactivestreams.Subscription; + +import software.amazon.awssdk.core.async.SdkPublisher; +import software.amazon.awssdk.core.exception.SdkException; + +/** + * Implements the {@link Enumeration} interface by subscribing to a + * {@link SdkPublisher} instance. The enumeration will buffer a fixed + * number of elements and only request new ones from the publisher + * when they are consumed. Calls to {@link #hasMoreElements()} and + * {@link #nextElement()} may block while waiting for new elements. + * @param the type of element. + */ +public final class BlockingEnumeration implements Enumeration { + private static final class Signal { + private final T element; + private final Throwable error; + + Signal(T element) { + this.element = element; + this.error = null; + } + + Signal(Throwable error) { + this.element = null; + this.error = error; + } + } + + private final Signal endSignal = new Signal<>((Throwable)null); + private final CompletableFuture subscription = new CompletableFuture<>(); + private final BlockingQueue> signalQueue; + private final int bufferSize; + private Signal current = null; + + /** + * Create an enumeration with a fixed buffer size and an + * optional injected first element. + * @param publisher the publisher feeding the enumeration. + * @param bufferSize the buffer size. + * @param firstElement (optional) first element the enumeration will return. + */ + public BlockingEnumeration(SdkPublisher publisher, + final int bufferSize, + final T firstElement) { + this.signalQueue = new LinkedBlockingQueue<>(); + this.bufferSize = bufferSize; + if (firstElement != null) { + this.current = new Signal<>(firstElement); + } + publisher.subscribe(new EnumerationSubscriber()); + } + + /** + * Create an enumeration with a fixed buffer size. + * @param publisher the publisher feeding the enumeration. + * @param bufferSize the buffer size. + */ + public BlockingEnumeration(SdkPublisher publisher, + final int bufferSize) { + this(publisher, bufferSize, null); + } + + @Override + public boolean hasMoreElements() { + if (current == null) { + try { + current = signalQueue.take(); + } catch (InterruptedException e) { + current = new Signal<>(e); + subscription.thenAccept(Subscription::cancel); + Thread.currentThread().interrupt(); + } + } + if (current.error != null) { + Throwable error = current.error; + current = endSignal; + if (error instanceof Error) { + throw (Error)error; + } else if (error instanceof SdkException) { + throw (SdkException)error; + } else { + throw SdkException.create("Unexpected error", error); + } + } + return current != endSignal; + } + + @Override + public T nextElement() { + if (!hasMoreElements()) { + throw new NoSuchElementException(); + } + T element = current.element; + current = null; + subscription.thenAccept(s -> s.request(1)); + return element; + } + + private final class EnumerationSubscriber implements Subscriber { + + @Override + public void onSubscribe(Subscription s) { + long request = bufferSize; + if (current != null) { + request--; + } + if (request > 0) { + s.request(request); + } + subscription.complete(s); + } + + @Override + public void onNext(T t) { + signalQueue.add(new Signal<>(t)); + } + + @Override + public void onError(Throwable t) { + signalQueue.add(new Signal<>(t)); + } + + @Override + public void onComplete() { + signalQueue.add(endSignal); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java index 150043aea9f1a..e626fb11afd7f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java @@ -21,13 +21,13 @@ import java.io.IOException; import java.util.Locale; -import com.amazonaws.services.s3.model.CSVInput; -import com.amazonaws.services.s3.model.CSVOutput; -import com.amazonaws.services.s3.model.ExpressionType; -import com.amazonaws.services.s3.model.InputSerialization; -import com.amazonaws.services.s3.model.OutputSerialization; -import com.amazonaws.services.s3.model.QuoteFields; -import com.amazonaws.services.s3.model.SelectObjectContentRequest; +import software.amazon.awssdk.services.s3.model.CSVInput; +import software.amazon.awssdk.services.s3.model.CSVOutput; +import software.amazon.awssdk.services.s3.model.ExpressionType; +import software.amazon.awssdk.services.s3.model.InputSerialization; +import software.amazon.awssdk.services.s3.model.OutputSerialization; +import software.amazon.awssdk.services.s3.model.QuoteFields; +import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -145,9 +145,9 @@ public SelectObjectContentRequest buildSelectRequest( Preconditions.checkState(isEnabled(), "S3 Select is not enabled for %s", path); - SelectObjectContentRequest request = operations.newSelectRequest(path); + SelectObjectContentRequest.Builder request = operations.newSelectRequestBuilder(path); buildRequest(request, expression, builderOptions); - return request; + return request.build(); } /** @@ -175,14 +175,14 @@ private SelectInputStream executeSelect( } boolean sqlInErrors = builderOptions.getBoolean(SELECT_ERRORS_INCLUDE_SQL, errorsIncludeSql); - String expression = request.getExpression(); + String expression = request.expression(); final String errorText = sqlInErrors ? expression : "Select"; if (sqlInErrors) { LOG.info("Issuing SQL request {}", expression); } + SelectEventStreamPublisher selectPublisher = operations.select(path, request, errorText); return new SelectInputStream(readContext, - objectAttributes, - operations.select(path, request, errorText)); + objectAttributes, selectPublisher); } /** @@ -197,14 +197,14 @@ private SelectInputStream executeSelect( *

  • The default values in {@link SelectConstants}
  • * * - * @param request request to build up + * @param requestBuilder request to build up * @param expression SQL expression * @param builderOptions the options which came in from the openFile builder. * @throws IllegalArgumentException if an option is somehow invalid. * @throws IOException if an option is somehow invalid. */ void buildRequest( - final SelectObjectContentRequest request, + final SelectObjectContentRequest.Builder requestBuilder, final String expression, final Configuration builderOptions) throws IllegalArgumentException, IOException { @@ -213,7 +213,6 @@ void buildRequest( final Configuration ownerConf = operations.getConf(); - String inputFormat = builderOptions.get(SELECT_INPUT_FORMAT, SELECT_FORMAT_CSV).toLowerCase(Locale.ENGLISH); Preconditions.checkArgument(SELECT_FORMAT_CSV.equals(inputFormat), @@ -224,34 +223,24 @@ void buildRequest( Preconditions.checkArgument(SELECT_FORMAT_CSV.equals(outputFormat), "Unsupported output format %s", outputFormat); - request.setExpressionType(ExpressionType.SQL); - request.setExpression(expandBackslashChars(expression)); - - InputSerialization inputSerialization = buildCsvInputRequest(ownerConf, - builderOptions); - String compression = opt(builderOptions, - ownerConf, - SELECT_INPUT_COMPRESSION, - COMPRESSION_OPT_NONE, - true).toUpperCase(Locale.ENGLISH); - if (isNotEmpty(compression)) { - inputSerialization.setCompressionType(compression); - } - request.setInputSerialization(inputSerialization); - - request.setOutputSerialization(buildCSVOutput(ownerConf, builderOptions)); + requestBuilder.expressionType(ExpressionType.SQL); + requestBuilder.expression(expandBackslashChars(expression)); + requestBuilder.inputSerialization( + buildCsvInput(ownerConf, builderOptions)); + requestBuilder.outputSerialization( + buildCSVOutput(ownerConf, builderOptions)); } /** - * Build the CSV input request. + * Build the CSV input format for a request. * @param ownerConf FS owner configuration * @param builderOptions options on the specific request - * @return the constructed request + * @return the input format * @throws IllegalArgumentException argument failure * @throws IOException validation failure */ - public InputSerialization buildCsvInputRequest( + public InputSerialization buildCsvInput( final Configuration ownerConf, final Configuration builderOptions) throws IllegalArgumentException, IOException { @@ -283,28 +272,35 @@ public InputSerialization buildCsvInputRequest( CSV_INPUT_QUOTE_ESCAPE_CHARACTER_DEFAULT); // CSV input - CSVInput csv = new CSVInput(); - csv.setFieldDelimiter(fieldDelimiter); - csv.setRecordDelimiter(recordDelimiter); - csv.setComments(commentMarker); - csv.setQuoteCharacter(quoteCharacter); + CSVInput.Builder csvBuilder = CSVInput.builder() + .fieldDelimiter(fieldDelimiter) + .recordDelimiter(recordDelimiter) + .comments(commentMarker) + .quoteCharacter(quoteCharacter); if (StringUtils.isNotEmpty(quoteEscapeCharacter)) { - csv.setQuoteEscapeCharacter(quoteEscapeCharacter); + csvBuilder.quoteEscapeCharacter(quoteEscapeCharacter); } - csv.setFileHeaderInfo(headerInfo); - - InputSerialization inputSerialization = new InputSerialization(); - inputSerialization.setCsv(csv); - - return inputSerialization; + csvBuilder.fileHeaderInfo(headerInfo); + InputSerialization.Builder inputSerialization = + InputSerialization.builder() + .csv(csvBuilder.build()); + String compression = opt(builderOptions, + ownerConf, + SELECT_INPUT_COMPRESSION, + COMPRESSION_OPT_NONE, + true).toUpperCase(Locale.ENGLISH); + if (isNotEmpty(compression)) { + inputSerialization.compressionType(compression); + } + return inputSerialization.build(); } /** - * Build CSV output for a request. + * Build CSV output format for a request. * @param ownerConf FS owner configuration * @param builderOptions options on the specific request - * @return the constructed request + * @return the output format * @throws IllegalArgumentException argument failure * @throws IOException validation failure */ @@ -333,21 +329,19 @@ public OutputSerialization buildCSVOutput( CSV_OUTPUT_QUOTE_FIELDS, CSV_OUTPUT_QUOTE_FIELDS_ALWAYS).toUpperCase(Locale.ENGLISH); - // output is CSV, always - OutputSerialization outputSerialization - = new OutputSerialization(); - CSVOutput csvOut = new CSVOutput(); - csvOut.setQuoteCharacter(quoteCharacter); - csvOut.setQuoteFields( - QuoteFields.fromValue(quoteFields)); - csvOut.setFieldDelimiter(fieldDelimiter); - csvOut.setRecordDelimiter(recordDelimiter); + CSVOutput.Builder csvOutputBuilder = CSVOutput.builder() + .quoteCharacter(quoteCharacter) + .quoteFields(QuoteFields.fromValue(quoteFields)) + .fieldDelimiter(fieldDelimiter) + .recordDelimiter(recordDelimiter); if (!quoteEscapeCharacter.isEmpty()) { - csvOut.setQuoteEscapeCharacter(quoteEscapeCharacter); + csvOutputBuilder.quoteEscapeCharacter(quoteEscapeCharacter); } - outputSerialization.setCsv(csvOut); - return outputSerialization; + // output is CSV, always + return OutputSerialization.builder() + .csv(csvOutputBuilder.build()) + .build(); } /** @@ -359,18 +353,18 @@ public OutputSerialization buildCSVOutput( public static String toString(final SelectObjectContentRequest request) { StringBuilder sb = new StringBuilder(); sb.append("SelectObjectContentRequest{") - .append("bucket name=").append(request.getBucketName()) - .append("; key=").append(request.getKey()) - .append("; expressionType=").append(request.getExpressionType()) - .append("; expression=").append(request.getExpression()); - InputSerialization input = request.getInputSerialization(); + .append("bucket name=").append(request.bucket()) + .append("; key=").append(request.key()) + .append("; expressionType=").append(request.expressionType()) + .append("; expression=").append(request.expression()); + InputSerialization input = request.inputSerialization(); if (input != null) { sb.append("; Input") .append(input.toString()); } else { sb.append("; Input Serialization: none"); } - OutputSerialization out = request.getOutputSerialization(); + OutputSerialization out = request.outputSerialization(); if (out != null) { sb.append("; Output") .append(out.toString()); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectEventStreamPublisher.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectEventStreamPublisher.java new file mode 100644 index 0000000000000..c71ea5f1623a1 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectEventStreamPublisher.java @@ -0,0 +1,124 @@ +/* + * 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.select; + +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.io.SequenceInputStream; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; + +import org.reactivestreams.Subscriber; + +import software.amazon.awssdk.core.async.SdkPublisher; +import software.amazon.awssdk.http.AbortableInputStream; +import software.amazon.awssdk.services.s3.model.EndEvent; +import software.amazon.awssdk.services.s3.model.RecordsEvent; +import software.amazon.awssdk.services.s3.model.SelectObjectContentEventStream; +import software.amazon.awssdk.services.s3.model.SelectObjectContentResponse; +import software.amazon.awssdk.utils.ToString; + +/** + * Async publisher of {@link SelectObjectContentEventStream}s returned + * from a SelectObjectContent call. + */ +public final class SelectEventStreamPublisher implements + SdkPublisher { + + private final CompletableFuture selectOperationFuture; + private final SelectObjectContentResponse response; + private final SdkPublisher publisher; + + /** + * Create the publisher. + * @param selectOperationFuture SelectObjectContent future + * @param response SelectObjectContent response + * @param publisher SelectObjectContentEventStream publisher to wrap + */ + public SelectEventStreamPublisher( + CompletableFuture selectOperationFuture, + SelectObjectContentResponse response, + SdkPublisher publisher) { + this.selectOperationFuture = selectOperationFuture; + this.response = response; + this.publisher = publisher; + } + + /** + * Retrieve an input stream to the subset of the S3 object that matched the select query. + * This is equivalent to loading the content of all RecordsEvents into an InputStream. + * This will lazily-load the content from S3, minimizing the amount of memory used. + * @param onEndEvent callback on the end event + * @return the input stream + */ + public AbortableInputStream toRecordsInputStream(Consumer onEndEvent) { + SdkPublisher recordInputStreams = this.publisher + .filter(e -> { + if (e instanceof RecordsEvent) { + return true; + } else if (e instanceof EndEvent) { + onEndEvent.accept((EndEvent) e); + } + return false; + }) + .map(e -> ((RecordsEvent) e).payload().asInputStream()); + + // Subscribe to the async publisher using an enumeration that will + // buffer a single chunk (RecordsEvent's payload) at a time and + // block until it is consumed. + // Also inject an empty stream as the first element that + // SequenceInputStream will request on construction. + BlockingEnumeration enumeration = + new BlockingEnumeration(recordInputStreams, 1, EMPTY_STREAM); + return AbortableInputStream.create( + new SequenceInputStream(enumeration), + this::cancel); + } + + /** + * The response from the SelectObjectContent call. + * @return the response object + */ + public SelectObjectContentResponse response() { + return response; + } + + @Override + public void subscribe(Subscriber subscriber) { + publisher.subscribe(subscriber); + } + + /** + * Cancel the operation. + */ + public void cancel() { + selectOperationFuture.cancel(true); + } + + @Override + public String toString() { + return ToString.builder("SelectObjectContentEventStream") + .add("response", response) + .add("publisher", publisher) + .build(); + } + + private static final InputStream EMPTY_STREAM = + new ByteArrayInputStream(new byte[0]); +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java index e13aa23db3ce6..ada46d2d305cf 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java @@ -23,11 +23,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import com.amazonaws.AbortedException; -import com.amazonaws.services.s3.model.SelectObjectContentEvent; -import com.amazonaws.services.s3.model.SelectObjectContentEventVisitor; -import com.amazonaws.services.s3.model.SelectObjectContentResult; -import com.amazonaws.services.s3.model.SelectRecordsInputStream; +import software.amazon.awssdk.core.exception.AbortedException; +import software.amazon.awssdk.http.AbortableInputStream; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,7 +90,7 @@ public class SelectInputStream extends FSInputStream implements * Abortable response stream. * This is guaranteed to never be null. */ - private final SelectRecordsInputStream wrappedStream; + private final AbortableInputStream wrappedStream; private final String bucket; @@ -112,14 +109,14 @@ public class SelectInputStream extends FSInputStream implements * The read attempt is initiated immediately. * @param readContext read context * @param objectAttributes object attributes from a HEAD request - * @param selectResponse response from the already executed call + * @param selectPublisher event stream publisher from the already executed call * @throws IOException failure */ @Retries.OnceTranslated public SelectInputStream( final S3AReadOpContext readContext, final S3ObjectAttributes objectAttributes, - final SelectObjectContentResult selectResponse) throws IOException { + final SelectEventStreamPublisher selectPublisher) throws IOException { Preconditions.checkArgument(isNotEmpty(objectAttributes.getBucket()), "No Bucket"); Preconditions.checkArgument(isNotEmpty(objectAttributes.getKey()), @@ -132,17 +129,17 @@ public SelectInputStream( this.readahead = readContext.getReadahead(); this.streamStatistics = readContext.getS3AStatisticsContext() .newInputStreamStatistics(); - SelectRecordsInputStream stream = once( + + AbortableInputStream stream = once( "S3 Select", uri, - () -> selectResponse.getPayload() - .getRecordsInputStream(new SelectObjectContentEventVisitor() { - @Override - public void visit(final SelectObjectContentEvent.EndEvent event) { - LOG.debug("Completed successful S3 select read from {}", uri); - completedSuccessfully.set(true); - } - })); + () -> { + return selectPublisher.toRecordsInputStream(e -> { + LOG.debug("Completed successful S3 select read from {}", uri); + completedSuccessfully.set(true); + }); + }); + this.wrappedStream = checkNotNull(stream); // this stream is already opened, so mark as such in the statistics. streamStatistics.streamOpened(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectObjectContentHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectObjectContentHelper.java new file mode 100644 index 0000000000000..8233e67eea0a5 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectObjectContentHelper.java @@ -0,0 +1,114 @@ +/* + * 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.select; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +import software.amazon.awssdk.core.async.SdkPublisher; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.s3.model.SelectObjectContentEventStream; +import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; +import software.amazon.awssdk.services.s3.model.SelectObjectContentResponse; +import software.amazon.awssdk.services.s3.model.SelectObjectContentResponseHandler; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AUtils; + +import static org.apache.hadoop.fs.s3a.WriteOperationHelper.WriteOperationHelperCallbacks; + +/** + * Helper for SelectObjectContent queries against an S3 Bucket. + */ +public final class SelectObjectContentHelper { + + private SelectObjectContentHelper() { + } + + /** + * Execute an S3 Select operation. + * @param writeOperationHelperCallbacks helper callbacks + * @param source source for selection + * @param request Select request to issue. + * @param action the action for use in exception creation + * @return the select response event stream publisher + * @throws IOException on failure + */ + public static SelectEventStreamPublisher select( + WriteOperationHelperCallbacks writeOperationHelperCallbacks, + Path source, + SelectObjectContentRequest request, + String action) + throws IOException { + try { + Handler handler = new Handler(); + CompletableFuture selectOperationFuture = + writeOperationHelperCallbacks.selectObjectContent(request, handler); + return handler.eventPublisher(selectOperationFuture).join(); + } catch (Throwable e) { + if (e instanceof CompletionException) { + e = e.getCause(); + } + IOException translated; + if (e instanceof SdkException) { + translated = S3AUtils.translateException(action, source, + (SdkException)e); + } else { + translated = new IOException(e); + } + throw translated; + } + } + + private static class Handler implements SelectObjectContentResponseHandler { + private volatile CompletableFuture>> responseAndPublisherFuture = + new CompletableFuture<>(); + + private volatile SelectObjectContentResponse response; + + public CompletableFuture eventPublisher( + CompletableFuture selectOperationFuture) { + return responseAndPublisherFuture.thenApply(p -> + new SelectEventStreamPublisher(selectOperationFuture, + p.getLeft(), p.getRight())); + } + + @Override + public void responseReceived(SelectObjectContentResponse selectObjectContentResponse) { + this.response = selectObjectContentResponse; + } + + @Override + public void onEventStream(SdkPublisher publisher) { + responseAndPublisherFuture.complete(Pair.of(response, publisher)); + } + + @Override + public void exceptionOccurred(Throwable error) { + responseAndPublisherFuture.completeExceptionally(error); + } + + @Override + public void complete() { + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java index c002a4a6dee1d..711b582300200 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java @@ -21,23 +21,18 @@ import java.time.Duration; import java.util.function.Consumer; import java.util.function.LongConsumer; +import java.util.stream.Collectors; +import java.util.stream.Stream; -import com.amazonaws.Request; -import com.amazonaws.Response; -import com.amazonaws.metrics.RequestMetricCollector; -import com.amazonaws.util.TimingInfo; +import software.amazon.awssdk.core.metrics.CoreMetric; +import software.amazon.awssdk.http.HttpMetric; +import software.amazon.awssdk.http.HttpStatusCode; +import software.amazon.awssdk.metrics.MetricCollection; +import software.amazon.awssdk.metrics.MetricPublisher; +import software.amazon.awssdk.metrics.SdkMetric; import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; -import static com.amazonaws.util.AWSRequestMetrics.Field.ClientExecuteTime; -import static com.amazonaws.util.AWSRequestMetrics.Field.HttpClientRetryCount; -import static com.amazonaws.util.AWSRequestMetrics.Field.HttpRequestTime; -import static com.amazonaws.util.AWSRequestMetrics.Field.RequestCount; -import static com.amazonaws.util.AWSRequestMetrics.Field.RequestMarshallTime; -import static com.amazonaws.util.AWSRequestMetrics.Field.RequestSigningTime; -import static com.amazonaws.util.AWSRequestMetrics.Field.ResponseProcessingTime; -import static com.amazonaws.util.AWSRequestMetrics.Field.ThrottleException; - /** * Collect statistics from the AWS SDK and forward to an instance of * {@link StatisticsFromAwsSdk} and thence into the S3A statistics. @@ -45,9 +40,9 @@ * See {@code com.facebook.presto.hive.s3.PrestoS3FileSystemMetricCollector} * for the inspiration for this. *

    - * See {@code com.amazonaws.util.AWSRequestMetrics} for metric names. + * See {@code software.amazon.awssdk.core.metrics.CoreMetric} for metric names. */ -public class AwsStatisticsCollector extends RequestMetricCollector { +public class AwsStatisticsCollector implements MetricPublisher { /** * final destination of updates. @@ -65,65 +60,122 @@ public AwsStatisticsCollector(final StatisticsFromAwsSdk collector) { /** * This is the callback from the AWS SDK where metrics * can be collected. - * @param request AWS request - * @param response AWS response + * @param metricCollection metrics collection */ @Override - public void collectMetrics( - final Request request, - final Response response) { - - TimingInfo timingInfo = request.getAWSRequestMetrics().getTimingInfo(); - - counter(timingInfo, HttpClientRetryCount.name(), - collector::updateAwsRetryCount); - counter(timingInfo, RequestCount.name(), - collector::updateAwsRequestCount); - counter(timingInfo, ThrottleException.name(), - collector::updateAwsThrottleExceptionsCount); - - timing(timingInfo, ClientExecuteTime.name(), - collector::noteAwsClientExecuteTime); - timing(timingInfo, HttpRequestTime.name(), - collector::noteAwsRequestTime); - timing(timingInfo, RequestMarshallTime.name(), - collector::noteRequestMarshallTime); - timing(timingInfo, RequestSigningTime.name(), - collector::noteRequestSigningTime); - timing(timingInfo, ResponseProcessingTime.name(), - collector::noteResponseProcessingTime); + public void publish(MetricCollection metricCollection) { + // MetricCollections are nested, so we need to traverse through their + // "children" to collect the desired metrics. E.g.: + // + // ApiCall + // ┌─────────────────────────────────────────┐ + // │ MarshallingDuration=PT0.002808333S │ + // │ RetryCount=0 │ + // │ ApiCallSuccessful=true │ + // │ OperationName=DeleteObject │ + // │ ApiCallDuration=PT0.079801458S │ + // │ CredentialsFetchDuration=PT0.000007083S │ + // │ ServiceId=S3 │ + // └─────────────────────────────────────────┘ + // ApiCallAttempt + // ┌─────────────────────────────────────────────────────────────────┐ + // │ SigningDuration=PT0.000319375S │ + // │ ServiceCallDuration=PT0.078908584S │ + // │ AwsExtendedRequestId=Kmvb2Sz8NuDgIFJPKzLLBhuHgQGmpAjVYBMrSHDvy= │ + // │ HttpStatusCode=204 │ + // │ BackoffDelayDuration=PT0S │ + // │ AwsRequestId=KR0XZCSX │ + // └─────────────────────────────────────────────────────────────────┘ + // HttpClient + // ┌─────────────────────────────────┐ + // │ AvailableConcurrency=1 │ + // │ LeasedConcurrency=0 │ + // │ ConcurrencyAcquireDuration=PT0S │ + // │ PendingConcurrencyAcquires=0 │ + // │ MaxConcurrency=96 │ + // │ HttpClientName=Apache │ + // └─────────────────────────────────┘ + + final long[] throttling = {0}; + recurseThroughChildren(metricCollection) + .collect(Collectors.toList()) + .forEach(m -> { + counter(m, CoreMetric.RETRY_COUNT, retries -> { + collector.updateAwsRetryCount(retries); + collector.updateAwsRequestCount(retries + 1); + }); + + counter(m, HttpMetric.HTTP_STATUS_CODE, statusCode -> { + if (statusCode == HttpStatusCode.THROTTLING) { + throttling[0] += 1; + } + }); + + timing(m, CoreMetric.API_CALL_DURATION, + collector::noteAwsClientExecuteTime); + + timing(m, CoreMetric.SERVICE_CALL_DURATION, + collector::noteAwsRequestTime); + + timing(m, CoreMetric.MARSHALLING_DURATION, + collector::noteRequestMarshallTime); + + timing(m, CoreMetric.SIGNING_DURATION, + collector::noteRequestSigningTime); + + timing(m, CoreMetric.UNMARSHALLING_DURATION, + collector::noteResponseProcessingTime); + }); + + collector.updateAwsThrottleExceptionsCount(throttling[0]); + } + + @Override + public void close() { + } /** * Process a timing. - * @param timingInfo timing info - * @param subMeasurementName sub measurement + * @param collection metric collection + * @param metric metric * @param durationConsumer consumer */ private void timing( - TimingInfo timingInfo, - String subMeasurementName, + MetricCollection collection, + SdkMetric metric, Consumer durationConsumer) { - TimingInfo t1 = timingInfo.getSubMeasurement(subMeasurementName); - if (t1 != null && t1.getTimeTakenMillisIfKnown() != null) { - durationConsumer.accept(Duration.ofMillis( - t1.getTimeTakenMillisIfKnown().longValue())); - } + collection + .metricValues(metric) + .forEach(v -> durationConsumer.accept(v)); } /** * Process a counter. - * @param timingInfo timing info - * @param subMeasurementName sub measurement + * @param collection metric collection + * @param metric metric * @param consumer consumer */ private void counter( - TimingInfo timingInfo, - String subMeasurementName, + MetricCollection collection, + SdkMetric metric, LongConsumer consumer) { - Number n = timingInfo.getCounter(subMeasurementName); - if (n != null) { - consumer.accept(n.longValue()); - } + collection + .metricValues(metric) + .forEach(v -> consumer.accept(v.longValue())); + } + + /** + * Metric collections can be nested. Exposes a stream of the given + * collection and its nested children. + * @param metrics initial collection + * @return a stream of all nested metric collections + */ + private static Stream recurseThroughChildren( + MetricCollection metrics) { + return Stream.concat( + Stream.of(metrics), + metrics.children().stream() + .flatMap(c -> recurseThroughChildren(c))); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index b09a3d3fdefee..fc60f4b623dd3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -32,10 +32,10 @@ import java.util.Map; import java.util.stream.Collectors; -import com.amazonaws.AmazonClientException; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.fs.s3a.AWSBadRequestException; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,12 +55,14 @@ import org.apache.hadoop.fs.s3a.impl.DirMarkerTracker; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicyImpl; +import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteException; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import org.apache.hadoop.fs.shell.CommandFormat; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.ExitUtil; + import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE; import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT; @@ -678,8 +680,30 @@ private boolean scanDirectoryTree( int count = 0; boolean result = true; - RemoteIterator listing = operations - .listObjects(path, storeContext.pathToKey(path)); + + // the path/key stuff loses any trailing / passed in. + // but this may actually be needed. + RemoteIterator listing = null; + String listkey = storeContext.pathToKey(path); + if (listkey.isEmpty()) { + // root. always give it a path to keep ranger happy. + listkey = "/"; + } + + try { + listing = operations.listObjects(path, listkey); + } catch (AWSBadRequestException e) { + // endpoint was unhappy. this is generally unrecoverable, but some + // third party stores do insist on a / here. + LOG.debug("Failed to list \"{}\"", listkey, e); + // now retry with a trailing / in case that works + if (listkey.endsWith("/")) { + // already has a trailing /, so fail + throw e; + } + // try again. + listing = operations.listObjects(path, listkey + "/"); + } while (listing.hasNext()) { count++; S3AFileStatus status = listing.next(); @@ -784,7 +808,7 @@ long getTotalDeleteRequestDuration() { private MarkerPurgeSummary purgeMarkers( final DirMarkerTracker tracker, final int deletePageSize) - throws MultiObjectDeleteException, AmazonClientException, IOException { + throws MultiObjectDeleteException, AwsServiceException, IOException { MarkerPurgeSummary summary = new MarkerPurgeSummary(); // we get a map of surplus markers to delete. @@ -792,13 +816,13 @@ private MarkerPurgeSummary purgeMarkers( = tracker.getSurplusMarkers(); int size = markers.size(); // build a list from the strings in the map - List collect = + List collect = markers.values().stream() - .map(p -> new DeleteObjectsRequest.KeyVersion(p.getKey())) + .map(p -> ObjectIdentifier.builder().key(p.getKey()).build()) .collect(Collectors.toList()); // build an array list for ease of creating the lists of // keys in each page through the subList() method. - List markerKeys = + List markerKeys = new ArrayList<>(collect); // now randomize. Why so? if the list spans multiple S3 partitions, @@ -819,7 +843,7 @@ pages, suffix(pages), while (start < size) { // end is one past the end of the page int end = Math.min(start + deletePageSize, size); - List page = markerKeys.subList(start, + List page = markerKeys.subList(start, end); once("Remove S3 Keys", tracker.getBasePath().toString(), () -> diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java index a701f86f7b0c3..7aaec40a86805 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java @@ -21,15 +21,16 @@ import java.io.IOException; import java.util.List; -import com.amazonaws.AmazonClientException; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import org.apache.hadoop.fs.InvalidRequestException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.Retries; import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteException; + /** * Operations which must be offered by the store for {@link MarkerTool}. @@ -62,14 +63,14 @@ RemoteIterator listObjects( * a mistaken attempt to delete the root directory. * @throws MultiObjectDeleteException one or more of the keys could not * be deleted in a multiple object delete operation. - * @throws AmazonClientException amazon-layer failure. + * @throws AwsServiceException amazon-layer failure. * @throws IOException other IO Exception. */ @Retries.RetryMixed void removeKeys( - List keysToDelete, + List keysToDelete, boolean deleteFakeDir) - throws MultiObjectDeleteException, AmazonClientException, + throws MultiObjectDeleteException, AwsServiceException, IOException; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java index ccf80e1dde00e..d7c77feed1083 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java @@ -21,15 +21,16 @@ import java.io.IOException; import java.util.List; -import com.amazonaws.AmazonClientException; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteException; import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; + /** * Implement the marker tool operations by forwarding to the * {@link OperationCallbacks} instance provided in the constructor. @@ -55,9 +56,9 @@ public RemoteIterator listObjects(final Path path, @Override public void removeKeys( - final List keysToDelete, + final List keysToDelete, final boolean deleteFakeDir) - throws MultiObjectDeleteException, AmazonClientException, IOException { + throws MultiObjectDeleteException, AwsServiceException, IOException { operationCallbacks.removeKeys(keysToDelete, deleteFakeDir ); } diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md index 094ea5668c05e..7da23c9fe7cf2 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/assumed_roles.md @@ -195,7 +195,7 @@ Here are the full set of configuration options. fs.s3a.assumed.role.credentials.provider org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider, - com.amazonaws.auth.EnvironmentVariableCredentialsProvider + software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider List of credential providers to authenticate with the STS endpoint and @@ -371,16 +371,7 @@ The Assumed Role Credential Provider is enabled, but `fs.s3a.assumed.role.arn` i ``` java.io.IOException: Unset property fs.s3a.assumed.role.arn at org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider.(AssumedRoleCredentialProvider.java:76) - 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) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520) - at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354) - at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474) + ``` ### "Not authorized to perform sts:AssumeRole" @@ -390,17 +381,9 @@ or one to which the caller has no access. ``` java.nio.file.AccessDeniedException: : Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider - on : com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException: Not authorized to perform sts:AssumeRole (Service: AWSSecurityTokenService; Status Code: 403; Error Code: AccessDenied; Request ID: aad4e59a-f4b0-11e7-8c78-f36aaa9457f6):AccessDenied - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:215) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520) - at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354) - at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474) - at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361) + ``` ### "Roles may not be assumed by root accounts" @@ -411,31 +394,15 @@ the role. ``` java.nio.file.AccessDeniedException: : Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider - on : com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException: + Roles may not be assumed by root accounts. (Service: AWSSecurityTokenService; Status Code: 403; Error Code: AccessDenied; Request ID: e86dfd8f-e758-11e7-88e7-ad127c04b5e2): No AWS Credentials provided by AssumedRoleCredentialProvider : - com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException: + software.amazon.awssdk.services.sts.model.StsException: Roles may not be assumed by root accounts. (Service: AWSSecurityTokenService; Status Code: 403; Error Code: AccessDenied; Request ID: e86dfd8f-e758-11e7-88e7-ad127c04b5e2) - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:215) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520) - at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354) - at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474) - at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361) - ... 22 more -Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException: - Roles may not be assumed by root accounts. - (Service: AWSSecurityTokenService; Status Code: 403; Error Code: AccessDenied; - Request ID: e86dfd8f-e758-11e7-88e7-ad127c04b5e2) - 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) + + ``` ### `Member must have value greater than or equal to 900` @@ -444,7 +411,7 @@ The value of `fs.s3a.assumed.role.session.duration` is too low. ``` org.apache.hadoop.fs.s3a.AWSBadRequestException: request role credentials: -com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException: +software.amazon.awssdk.services.sts.model.StsException: 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; @@ -459,7 +426,7 @@ 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: + software.amazon.awssdk.services.sts.model.StsException: 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) @@ -478,7 +445,7 @@ any role for up to 12h; attempting to use a larger number will fail. ``` -Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException: +Caused by: software.amazon.awssdk.services.sts.model.StsException: 1 validation error detected: Value '345600' at 'durationSeconds' failed to satisfy constraint: Member must have value less than or equal to 43200 @@ -492,7 +459,7 @@ 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: +software.amazon.awssdk.services.sts.model.StsException: 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; @@ -510,26 +477,12 @@ AWS specification of Role Policies. ``` org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on : - com.amazonaws.services.securitytoken.model.MalformedPolicyDocumentException: + software.amazon.awssdk.services.sts.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): MalformedPolicyDocument: The policy is not in the valid JSON format. (Service: AWSSecurityTokenService; Status Code: 400; Error Code: MalformedPolicyDocument; Request ID: baf8cb62-f552-11e7-9768-9df3b384e40c) - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:209) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520) - at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354) - at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474) - at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361) -Caused by: 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) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303) ``` ### `MalformedPolicyDocumentException` "Syntax errors in policy" @@ -539,27 +492,13 @@ The policy set in `fs.s3a.assumed.role.policy` is not valid JSON. ``` org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on : - com.amazonaws.services.securitytoken.model.MalformedPolicyDocumentException: + software.amazon.awssdk.services.sts.model.MalformedPolicyDocumentException: Syntax errors in policy. (Service: AWSSecurityTokenService; Status Code: 400; Error Code: MalformedPolicyDocument; Request ID: 24a281e8-f553-11e7-aa91-a96becfb4d45): MalformedPolicyDocument: Syntax errors in policy. (Service: AWSSecurityTokenService; Status Code: 400; Error Code: MalformedPolicyDocument; Request ID: 24a281e8-f553-11e7-aa91-a96becfb4d45) - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:209) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520) - at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354) - at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474) - at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361) - (Service: AWSSecurityTokenService; Status Code: 400; Error Code: MalformedPolicyDocument; - Request ID: 24a281e8-f553-11e7-aa91-a96becfb4d45) - 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) - ... 19 more ``` ### `IOException`: "AssumedRoleCredentialProvider cannot be in fs.s3a.assumed.role.credentials.provider" @@ -591,7 +530,7 @@ inner authentication which is breaking signature creation. ``` org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider - on : com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException: + on : software.amazon.awssdk.services.sts.model.StsException: 'valid/20180109/us-east-1/sts/aws4_request' not a valid key=value pair (missing equal-sign) in Authorization header: 'AWS4-HMAC-SHA256 Credential=not valid/20180109/us-east-1/sts/aws4_request, SignedHeaders=amz-sdk-invocation-id;amz-sdk-retry;host;user-agent;x-amz-date. @@ -601,21 +540,7 @@ inner authentication which is breaking signature creation. in Authorization header: 'AWS4-HMAC-SHA256 Credential=not valid/20180109/us-east-1/sts/aws4_request, SignedHeaders=amz-sdk-invocation-id;amz-sdk-retry;host;user-agent;x-amz-date, (Service: AWSSecurityTokenService; Status Code: 400; Error Code: IncompleteSignature; - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:209) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520) - at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354) - at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474) - at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361) -Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException: - 'valid/20180109/us-east-1/sts/aws4_request' not a valid key=value pair (missing equal-sign) - in Authorization header: 'AWS4-HMAC-SHA256 Credential=not valid/20180109/us-east-1/sts/aws4_request, - 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) ``` ### `AccessDeniedException/InvalidClientTokenId`: "The security token included in the request is invalid" @@ -625,27 +550,11 @@ The credentials used to authenticate with the AWS Security Token Service are inv ``` [ERROR] Failures: [ERROR] java.nio.file.AccessDeniedException: : Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on : - com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException: + software.amazon.awssdk.services.sts.model.StsException: The security token included in the request is invalid. (Service: AWSSecurityTokenService; Status Code: 403; Error Code: InvalidClientTokenId; Request ID: 74aa7f8a-f557-11e7-850c-33d05b3658d7):InvalidClientTokenId - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:215) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520) - at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354) - at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474) - -Caused by: com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException: -The security token included in the request is invalid. - (Service: AWSSecurityTokenService; Status Code: 403; Error Code: InvalidClientTokenId; - Request ID: 74aa7f8a-f557-11e7-850c-33d05b3658d7) - 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) - ... 25 more -``` + ``` ### `AWSSecurityTokenServiceExceptiond`: "Member must satisfy regular expression pattern: `[\w+=,.@-]*`" @@ -659,7 +568,7 @@ If set explicitly, it must be valid. ``` org.apache.hadoop.fs.s3a.AWSBadRequestException: Instantiate org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider on - com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException: + software.amazon.awssdk.services.sts.model.StsException: 1 validation error detected: Value 'Session Names cannot Hava Spaces!' at 'roleSessionName' failed to satisfy constraint: Member must satisfy regular expression pattern: [\w+=,.@-]* (Service: AWSSecurityTokenService; Status Code: 400; Error Code: ValidationError; @@ -667,23 +576,7 @@ org.apache.hadoop.fs.s3a.AWSBadRequestException: 1 validation error detected: Value 'Session Names cannot Hava Spaces!' at 'roleSessionName' failed to satisfy constraint: Member must satisfy regular expression pattern: [\w+=,.@-]* (Service: AWSSecurityTokenService; Status Code: 400; Error Code: ValidationError; - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:209) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:616) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:520) - at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:52) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:252) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354) - at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:474) - at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361) - -Caused by: 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+=,.@-]* - (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) -``` + ``` ### `java.nio.file.AccessDeniedException` within a FileSystem API call @@ -692,23 +585,11 @@ If an operation fails with an `AccessDeniedException`, then the role does not ha the permission for the S3 Operation invoked during the call. ``` -java.nio.file.AccessDeniedException: s3a://bucket/readonlyDir: - rename(s3a://bucket/readonlyDir, s3a://bucket/renameDest) - on s3a://bucket/readonlyDir: - com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied - (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 2805F2ABF5246BB1; - S3 Extended Request ID: iEXDVzjIyRbnkAc40MS8Sjv+uUQNvERRcqLsJsy9B0oyrjHLdkRKwJ/phFfA17Kjn483KSlyJNw=), - S3 Extended Request ID: iEXDVzjIyRbnkAc40MS8Sjv+uUQNvERRcqLsJsy9B0oyrjHLdkRKwJ/phFfA17Kjn483KSlyJNw=:AccessDenied - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:216) - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:143) - at org.apache.hadoop.fs.s3a.S3AFileSystem.rename(S3AFileSystem.java:853) - ... -Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied - (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 2805F2ABF5246BB1; - S3 Extended Request ID: iEXDVzjIyRbnkAc40MS8Sjv+uUQNvERRcqLsJsy9B0oyrjHLdkRKwJ/phFfA17Kjn483KSlyJNw=), - 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) +> hadoop fs -touch s3a://landsat-pds/a + +java.nio.file.AccessDeniedException: a: Writing Object on a: + software.amazon.awssdk.services.s3.model.S3Exception: Access Denied + (Service: S3, Status Code: 403, Request ID: F08EV50F85AYKF1V, Extended Request ID: 75vMz9xWNP5/lYplPSZfm/i4yQ5q0G32SIwOwfaj6a8gNCRj9tLBAqcLaaexT/aNg2DhWZQPvDU=):AccessDenied ``` This is the policy restriction behaving as intended: the caller is trying to @@ -743,28 +624,9 @@ If the client does have write access to the bucket, verify that the caller has ``` java.nio.file.AccessDeniedException: test/testDTFileSystemClient: Writing Object on test/testDTFileSystemClient: - com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403; + software.amazon.awssdk.services.s3.model.S3Exception: Access Denied (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: E86544FF1D029857) - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:243) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111) - at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$4(Invoker.java:314) - at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:406) - at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:310) - at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:285) - at org.apache.hadoop.fs.s3a.WriteOperationHelper.retry(WriteOperationHelper.java:150) - at org.apache.hadoop.fs.s3a.WriteOperationHelper.putObject(WriteOperationHelper.java:460) - at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.lambda$putObject$0(S3ABlockOutputStream.java:438) - at org.apache.hadoop.util.SemaphoredDelegatingExecutor$CallableWithPermitRelease.call(SemaphoredDelegatingExecutor.java:219) - at org.apache.hadoop.util.SemaphoredDelegatingExecutor$CallableWithPermitRelease.call(SemaphoredDelegatingExecutor.java:219) - at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125) - at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:57) - at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) - at java.lang.Thread.run(Thread.java:748) -Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403; - Error Code: AccessDenied; Request ID: E86544FF1D029857) ``` Note: the ability to read encrypted data in the store does not guarantee that the caller can encrypt new data. @@ -779,14 +641,9 @@ This is a low-level networking error. Possible causes include: ``` 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) + ``` ### Error "Credential should be scoped to a valid region" @@ -800,7 +657,7 @@ Variant 1: `Credential should be scoped to a valid region, not 'us-west-1'` (or ``` java.nio.file.AccessDeniedException: : request session credentials: -com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException: +software.amazon.awssdk.services.sts.model.StsException: 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 ``` @@ -817,7 +674,7 @@ Variant 2: `Credential should be scoped to a valid region, not ''` ``` java.nio.file.AccessDeniedException: : request session credentials: -com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException: +software.amazon.awssdk.services.sts.model.StsException: 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 diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md index 7a95907217789..dbb9c0b3d97c9 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md @@ -22,7 +22,7 @@ and inside the AWS S3 SDK, immediately before the request is executed. The full architecture is covered in [Auditing Architecture](auditing_architecture.html); this document covers its use. -## Important: Auditing is disabled by default +## Important: Auditing is currently enabled Due to a memory leak from the use of `ThreadLocal` fields, this auditing feature leaked memory as S3A filesystem instances were created and deleted. @@ -32,7 +32,7 @@ See [HADOOP-18091](https://issues.apache.org/jira/browse/HADOOP-18091) _S3A audi To avoid these memory leaks, auditing was disabled by default in the hadoop 3.3.2 release. -As these memory leaks have now been fixed, auditing has been re-enabled. +As these memory leaks have now been fixed, auditing has been re-enabled in Hadoop 3.3.5+ To disable it, set `fs.s3a.audit.enabled` to `false`. @@ -77,7 +77,7 @@ ideally even identifying the process/job generating load. ## Using Auditing -Auditing is disabled by default. +Auditing is enabled by default. When auditing enabled, a Logging Auditor will annotate the S3 logs through a custom HTTP Referrer header in requests made to S3. Other auditor classes may be used instead. @@ -88,7 +88,7 @@ Other auditor classes may be used instead. |--------|---------|---------------| | `fs.s3a.audit.enabled` | Is auditing enabled? | `true` | | `fs.s3a.audit.service.classname` | Auditor classname | `org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor` | -| `fs.s3a.audit.request.handlers` | List of extra subclasses of AWS SDK RequestHandler2 to include in handler chain | `""` | +| `fs.s3a.audit.execution.interceptors` | Implementations of AWS v2 SDK `ExecutionInterceptor` to include in handler chain | `""` | | `fs.s3a.audit.referrer.enabled` | Logging auditor to publish the audit information in the HTTP Referrer header | `true` | | `fs.s3a.audit.referrer.filter` | List of audit fields to filter | `""` | | `fs.s3a.audit.reject.out.of.span.operations` | Auditor to reject operations "outside of a span" | `false` | @@ -96,14 +96,14 @@ Other auditor classes may be used instead. ### Disabling Auditing. -In this release of Hadoop, auditing is disabled. +In this release of Hadoop, auditing is enabled by default. This can be explicitly set globally or for specific buckets ```xml fs.s3a.audit.enabled - false + true ``` @@ -162,6 +162,26 @@ correlate access by S3 clients to the actual operations taking place. Note: this logging is described as "Best Effort". There's no guarantee as to when logs arrive. +### Integration with AWS SDK request processing + +The auditing component inserts itself into the AWS SDK request processing +code, so it can attach the referrer header. + +It is possible to declare extra classes to add to the processing chain, +all of which must implement the interface `software.amazon.awssdk.core.interceptor.ExecutionInterceptor`. + +The list of classes is set in the configuration option `fs.s3a.audit.execution.interceptors`. + +Any class in the list which implements `org.apache.hadoop.conf.Configurable` will have +`Configurable.setConf()` called with the filesystem configuration passed down. + +Before the upgrade to the V2 SDK, a list of extra subclasses of the AWS SDK `com.amazonaws.handlers.RequestHandler2` +class could be declared in the option `fs.s3a.audit.request.handlers`; +these would be wired up into the V1 request processing pipeline. + +This option is now ignored completely, other than printing a warning message the first time a filesystem is created with a non-empty value. + + ### Rejecting out-of-span operations The logging auditor can be configured to raise an exception whenever @@ -201,7 +221,7 @@ The HTTP referrer header is attached by the logging auditor. If the S3 Bucket is configured to log requests to another bucket, then these logs entries will include the audit information _as the referrer_. -This can be parsed (consult AWS documentation for a regular expression) +The S3 Server log entries can be parsed (consult AWS documentation for a regular expression) and the http referrer header extracted. ``` @@ -242,13 +262,15 @@ If any of the field values were `null`, the field is omitted. _Notes_ -* Thread IDs are from the current thread in the JVM, so can be compared to those in````````` +* Thread IDs are from the current thread in the JVM, so can be compared to those in Log4J logs. They are never unique. * Task Attempt/Job IDs are only ever set during operations involving the S3A committers, specifically - all operations excecuted by the committer. + all operations executed by the committer. Operations executed in the same thread as the committer's instantiation _may_ also report the IDs, even if they are unrelated to the actual task. Consider them "best effort". +Thread IDs are generated as follows: + ```java Long.toString(Thread.currentThread().getId()) ``` @@ -269,6 +291,8 @@ This is why the span ID is always passed in as part of the URL, rather than just an HTTP query parameter: even if the header is chopped, the span ID will always be present. +As of August 2023, this header is not collected in AWS CloudTrail -only S3 Server logs. + ## Privacy Implications of HTTP Referrer auditing When the S3A client makes requests of an S3 bucket, the auditor @@ -423,6 +447,12 @@ log4j.logger.org.apache.hadoop.fs.s3a.audit=TRACE This is very noisy and not recommended in normal operation. +If logging of HTTP IO is enabled then the "referer" header is printed as part of every request: +``` +log4j.logger.org.apache.http=DEBUG +log4j.logger.software.amazon.awssdk.thirdparty.org.apache.http.client.HttpClient=DEBUG +``` + ## Integration with S3A Committers Work submitted through the S3A committer will have the job (query) ID associated diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/aws_sdk_upgrade.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/aws_sdk_upgrade.md index e649a8d76d539..138e060155c0e 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/aws_sdk_upgrade.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/aws_sdk_upgrade.md @@ -20,28 +20,42 @@ This work is tracked in [HADOOP-18073](https://issues.apache.org/jira/browse/HAD ## Why the upgrade? - Moving to SDK V2 will provide performance benefits. -For example, the [transfer manager for SDKV2](https://aws.amazon.com/blogs/developer/introducing-amazon-s3-transfer-manager-in-the-aws-sdk-for-java-2-x/) +For example, the [transfer manager for SDK V2](https://aws.amazon.com/blogs/developer/introducing-amazon-s3-transfer-manager-in-the-aws-sdk-for-java-2-x/) is built using java bindings of the AWS Common Runtime S3 client (https://github.com/awslabs/aws-crt-java) (CRT). CRT is a set of packages written in C, designed for maximising performance when interacting with AWS services such as S3. +- The V1 SDK is essentially in maintenance mode. - New features such as [additional checksum algorithms](https://aws.amazon.com/blogs/aws/new-additional-checksum-algorithms-for-amazon-s3/) -which S3A will benefit from are not available in SDKV1. +which S3A will benefit from are not available in SDK V1. ## What's changing? The [SDK V2](https://github.com/aws/aws-sdk-java-v2) for S3 is very different from [SDK V1](https://github.com/aws/aws-sdk-java), and brings breaking changes for S3A. -A complete list of the changes can be found in the [Changelog](https://github.com/aws/aws-sdk-java-v2/blob/master/docs/LaunchChangelog.md#41-s3-changes). +A complete list of the changes can be found in the +[Changelog](https://github.com/aws/aws-sdk-java-v2/blob/master/docs/LaunchChangelog.md#41-s3-changes). -The major changes and how this affects S3A are listed below. +## Packaging: `aws-java-sdk-bundle-1.12.x.jar` becomes `bundle-2.x.y.jar` -### Package Change +As the module name is lost, in hadoop releases a large JAR file with +the name "bundle" is now part of the distribution. +This is the AWS V2 SDK shaded artifact. -Package names have changed, all classes in SDK V2 are under `software.amazon.awssdk`, SDK V1 classes -were under `com.amazonaws`. +The new and old SDKs can co-exist; the only place that the hadoop code +may still use the original SDK is when a non-standard V1 AWS credential +provider is declared. + +Any deployment of the S3A connector must include this JAR or +the subset of non-shaded aws- JARs needed for communication +with S3 and any other services used. +As before: the exact set of dependencies used by the S3A connector +is neither defined nor comes with any commitments of stability +or compatibility of dependent libraries. -### Credential Providers + + +## Credential Provider changes and migration - Interface change: [com.amazonaws.auth.AWSCredentialsProvider](https://github.com/aws/aws-sdk-java/blob/master/aws-java-sdk-core/src/main/java/com/amazonaws/auth/AWSCredentialsProvider.java) has been replaced by [software.amazon.awssdk.auth.credentials.AwsCredentialsProvider](https://github.com/aws/aws-sdk-java-v2/blob/master/core/auth/src/main/java/software/amazon/awssdk/auth/credentials/AwsCredentialsProvider.java). @@ -49,23 +63,295 @@ has been replaced by [software.amazon.awssdk.auth.credentials.AwsCredentialsProv changed. The change in interface will mean that custom credential providers will need to be updated to now -implement `AwsCredentialsProvider` instead of `AWSCredentialProvider`. +implement `software.amazon.awssdk.auth.credentials.AwsCredentialsProvider` instead of +`com.amazonaws.auth.AWSCredentialsProvider`. -Due to change in class names, references to SDK V1 credential providers -in `fs.s3a.aws.credentials.provider` will need to be updated to reference V2 providers. +### Original V1 `AWSCredentialsProvider` interface -### Delegation Tokens +Note how the interface begins with the capitalized "AWS" acronym. +The V2 interface starts with "Aws". This is a very subtle change +for developers to spot. +Compilers _will_ detect and report the type mismatch. + + +```java +package com.amazonaws.auth; + +public interface AWSCredentialsProvider { + + public AWSCredentials getCredentials(); + + public void refresh(); + +} + +``` +The interface binding also supported a factory method, `AWSCredentialsProvider instance()` which, +if available, would be invoked in preference to using any constructor. + +If the interface implemented `Closeable` or `AutoCloseable`, these would +be invoked when the provider chain was being shut down. + +### New V2 `AwsCredentialsProvider` interface + +```java +package software.amazon.awssdk.auth.credentials; + +public interface AwsCredentialsProvider { + + AwsCredentials resolveCredentials(); + +} +``` + +1. There is no `refresh()` method any more. +2. `getCredentials()` has become `resolveCredentials()`. +3. There is now the expectation in the SDK that credential resolution/lookup etc will be + performed in `resolveCredentials()`. +4. If the interface implements `Closeable` or `AutoCloseable`, these will + be invoked when the provider chain is being shut down. +5. A static method `create()` which returns an `AwsCredentialsProvider` or subclass; this will be used + in preference to a constructor + +### S3A `AWSCredentialProviderList` is now a V2 credential provider + +The class `org.apache.hadoop.fs.s3a.AWSCredentialProviderList` has moved from +being a V1 to a V2 credential provider; even if an instance can be created with +existing code, the V1 methods will not resolve: + +``` +java.lang.NoSuchMethodError: org.apache.hadoop.fs.s3a.AWSCredentialProviderList.getCredentials()Lcom/amazonaws/auth/AWSCredentials; + at org.apache.hadoop.fs.store.diag.S3ADiagnosticsInfo.validateFilesystem(S3ADiagnosticsInfo.java:903) +``` + +### Migration of Credential Providers listed in `fs.s3a.aws.credentials.provider` -Custom credential providers used in delegation token binding classes will also need to be updated. -### AmazonS3 replaced by S3Client +Before: `fs.s3a.aws.credentials.provider` took a list of v1 credential providers, +This took a list containing +1. V1 credential providers implemented in the `hadoop-aws` module. +2. V1 credential providers implemented in the `aws-sdk-bundle` library. +3. Custom V1 credential providers placed onto the classpath. +4. Custom subclasses of hadoop-aws credential providers. -The s3 client is an instance of `S3Client` in V2 rather than `AmazonS3`. +And here is how they change +1. All `hadoop-aws` credential providers migrated to V2. +2. Well-known `aws-sdk-bundle` credential providers _automatically remapped_ to their V2 equivalents. +3. Custom v1 providers supported if the original `aws-sdk-bundle` JAR is on the classpath. +4. Custom subclasses of hadoop-aws credential providers need manual migration. -For this reason, the `S3ClientFactory` will be deprecated and replaced by one that creates a V2 -`S3Client`. +Because of (1) and (2), As result, standard `fs.s3a.aws.credentials.provider` configurations +should seamlessly upgrade. This also means that the same provider list, if restricted to +those classes, will work across versions. + + +### `hadoop-aws` credential providers migration to V2 + +All the fs.s3a credential providers have the same name and functionality as before. + +| Hadoop module credential provider | Authentication Mechanism | +|----------------------------------------------------------------|--------------------------------------------------| +| `org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider` | Session Credentials in configuration | +| `org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider` | Simple name/secret credentials in configuration | +| `org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider` | Anonymous Login | +| `org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider` | [Assumed Role credentials](./assumed_roles.html) | +| `org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` | EC2/k8s instance credentials | + +### Automatic `aws-sdk-bundle` credential provider remapping + +The commonly-used set of V1 credential providers are automatically remapped to V2 equivalents. + + + +| V1 Credential Provider | Remapped V2 substitute | +|-------------------------------------------------------------|----------------------------------------------------------------------------------| +| `com.amazonaws.auth.AnonymousAWSCredentials` | `org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider` | +| `com.amazonaws.auth.EnvironmentVariableCredentialsProvider` | `software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider` | +| `com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper` | `org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` | +| `com.amazonaws.auth.InstanceProfileCredentialsProvider` | `org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` | +| `com.amazonaws.auth.profile.ProfileCredentialsProvider` | `software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider` | + +There are still a number of troublespots here: + +#### Less widely used`com.amazonaws.auth.` AWS providers + +There should be equivalents in the new SDK, but as well as being renamed +they are likely to have moved different factory/builder mechanisms. +Identify the changed classes and use their +names in the `fs.s3a.aws.credentials.provider` option. + +If a V2 equivalent is not found; provided the V1 SDK is added to the classpath, +it should still be possible to use the existing classes. + + +#### Private/third-party credential providers + +Provided the V1 SDK is added to the classpath, +it should still be possible to use the existing classes. + +Adding a V2 equivalent is the recommended long-term solution. + +#### Custom subclasses of the Hadoop credential providers + +Because all the standard hadoop credential providers have been upgraded, +any subclasses of these are not going to link or work. + +These will need to be manually migrated to being V2 Credential providers. + + +## Source code/binary integration changes + +The major changes and how this affects S3A are listed below. + +### SDK API Package Change + +* Package names have changed, all classes in SDK V2 are under `software.amazon.awssdk`, SDK V1 classes +were under `com.amazonaws`. +* There is no interoperability between the old and new classes. +* All classnames are different, often in very subtle ways. It is possible to use both in the same + class, as is done in the package `org.apache.hadoop.fs.s3a.adapter`. +* All the core message classes are now automatically generated from a JSON protocol description. +* All getter methods have been renamed. +* All classes are constructed via builder methods +* Message classes are no longer Java `Serializable`. + +Most of these changes simply create what will feel to be gratuitous migration effort; +the removable of the `Serializable` nature from all message response classes can +potentially break applications -such as anything passing them between Spark workers. +See AWS SDK V2 issue [Simplify Modeled Message Marshalling #82](https://github.com/aws/aws-sdk-java-v2/issues/82), +note that it was filed in 2017, then implement your own workaround pending that issue +being resolved. + +### Compilation/Linkage Errors + +Any code making use of V1 sdk classes will fail if they +* Expect the V1 sdk classes to be on the classpath when `hadoop-aws` is declared as a dependency +* Use V1-SDK-compatible methods previously exported by the `S3AFileSystem` class and associated classes. +* Try to pass s3a classes to V1 SDK classes (e.g. credential providers). + +The sole solution to these problems is "move to the V2 SDK". + +Some `S3AUtils` methods are deleted +``` +cannot find symbol +[ERROR] symbol: method createAwsConf(org.apache.hadoop.conf.Configuration,java.lang.String) +[ERROR] location: class org.apache.hadoop.fs.s3a.S3AUtils +``` + +The signature and superclass of `AWSCredentialProviderList` has changed, which can surface in different +ways + +Signature mismatch +``` + cannot find symbol +[ERROR] symbol: method getCredentials() +[ERROR] location: variable credentials of type org.apache.hadoop.fs.s3a.AWSCredentialProviderList +``` + +It is no longer a V1 credential provider, cannot be used to pass credentials to a V1 SDK class +``` +incompatible types: org.apache.hadoop.fs.s3a.AWSCredentialProviderList cannot be converted to com.amazonaws.auth.AWSCredentialsProvider +``` + +### `AmazonS3` replaced by `S3Client`; factory and accessor changed. + +The V1 s3 client class `com.amazonaws.services.s3.AmazonS3` has been superseded by +`software.amazon.awssdk.services.s3.S3Client` + +The `S3ClientFactory` interface has been replaced by one that creates a V2 `S3Client`. +* Custom implementations will need to be updated. +* The `InconsistentS3ClientFactory` class has been deleted. + +### `S3AFileSystem` method changes: `S3AInternals`. + +The low-level s3 operations/client accessors have been moved into a new interface, +`org.apache.hadoop.fs.s3a.S3AInternals`, which must be accessed via the +`S3AFileSystem.getS3AInternals()` method. +They have also been updated to return V2 SDK classes. + +```java +@InterfaceStability.Unstable +@InterfaceAudience.LimitedPrivate("testing/diagnostics") +public interface S3AInternals { + S3Client getAmazonS3V2Client(String reason); + + @Retries.RetryTranslated + @AuditEntryPoint + String getBucketLocation() throws IOException; + + @AuditEntryPoint + @Retries.RetryTranslated + String getBucketLocation(String bucketName) throws IOException; + + @AuditEntryPoint + @Retries.RetryTranslated + HeadObjectResponse getObjectMetadata(Path path) throws IOException; + + AWSCredentialProviderList shareCredentials(final String purpose); +} +``` + + +#### `S3AFileSystem.getAmazonS3ClientForTesting(String)` moved and return type changed + +The `S3AFileSystem.getAmazonS3ClientForTesting()` method has been been deleted. + +Compilation +``` +cannot find symbol +[ERROR] symbol: method getAmazonS3ClientForTesting(java.lang.String) +[ERROR] location: variable fs of type org.apache.hadoop.fs.s3a.S3AFileSystem +``` + +It has been replaced by an `S3AInternals` equivalent which returns the V2 `S3Client` +of the filesystem instance. + +```java +((S3AFilesystem)fs).getAmazonS3ClientForTesting("testing") +``` + +```java +((S3AFilesystem)fs).getS3AInternals().getAmazonS3Client("testing") +``` + +##### `S3AFileSystem.getObjectMetadata(Path path)` moved to `S3AInternals`; return type changed + +The `getObjectMetadata(Path)` call has been moved to the `S3AInternals` interface +and an instance of the `software.amazon.awssdk.services.s3.model.HeadObjectResponse` class +returned. +The original `S3AFileSystem` method has been deleted + +Before: +```java +((S3AFilesystem)fs).getObjectMetadata(path) +``` + +After: +```java +((S3AFilesystem)fs).getS3AInternals().getObjectMetadata(path) +``` + +##### `AWSCredentialProviderList shareCredentials(String)` moved to `S3AInternals` + +The operation to share a reference-counted access to the AWS credentials used +by the S3A FS has been moved to `S3AInternals`. + +This is very much an implementation method, used to allow extension modules to share +an authentication chain into other AWS SDK client services (dynamoDB, etc.). + +### Delegation Tokens + +1. Custom credential providers used in delegation token binding classes will need to be updated +2. The return type from delegation token binding has changed to support more class + instances being returned in the future. + +`AWSCredentialProviderList` has been upgraded to the V2 API. +* It still retains a `refresh()` method but this is now a deprecated no-op. +* It is still `Closeable`; its `close()` method iterates through all entries in +the list; if they are `Closeable` or `AutoCloseable` then their `close()` method is invoked. +* Accordingly, providers may still perform background refreshes in separate threads; + the S3A client will close its provider list when the filesystem itself is closed. -The `getAmazonS3ClientForTesting()` method will also be updated to return the `S3Client`. ### Signers @@ -74,3 +360,21 @@ has been replaced by [software.amazon.awssdk.core.signer.Signer](https://github. The change in signers will mean the custom signers will need to be updated to implement the new interface. + +There is no support to assist in this migration. + +### S3A Auditing Extensions. + +The callbacks from the SDK have all changed, as has +the interface `org.apache.hadoop.fs.s3a.audit.AWSAuditEventCallbacks` + +Examine the interface and associated implementations to +see how to migrate. + +The option `fs.s3a.audit.request.handlers` to declare a list of v1 SDK +`com.amazonaws.handlers.RequestHandler2` implementations to include +in the AWS request chain is no longer supported: a warning is printed +and the value ignored. + +The V2 SDK equivalent, classes implementing `software.amazon.awssdk.core.interceptor.ExecutionInterceptor` +can be declared in the configuration option `fs.s3a.audit.execution.interceptors`. diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/aws_sdk_v2_changelog.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/aws_sdk_v2_changelog.md new file mode 100644 index 0000000000000..162f15951f5ca --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/aws_sdk_v2_changelog.md @@ -0,0 +1,340 @@ + + +# Upgrade S3A to AWS SDK V2: Changelog + +Note: This document is not meant to be committed as part of the final merge, and instead just serves +as a guide to help with reviewing the PR. + +This document tracks changes to S3A during the upgrade to AWS SDK V2. Once the upgrade +is complete, some of its content will be added to the existing document +[Upcoming upgrade to AWS Java SDK V2](./aws_sdk_upgrade.html). + +This work is tracked in [HADOOP-18073](https://issues.apache.org/jira/browse/HADOOP-18073). + +## Contents + +* [Client Configuration](#client-configuration) +* [Endpoint and region configuration](#endpoint-and-region-configuration) +* [List Object](#list-object) +* [EncryptionSecretOperations](#encryptionsecretoperations) +* [GetObjectMetadata](#getobjectmetadata) +* [PutObject](#putobject) +* [CopyObject](#copyobject) +* [MultipartUpload](#multipartupload) +* [GetObject](#getObject) +* [DeleteObject](#deleteobject) +* [Select](#select) +* [CredentialsProvider](#credentialsprovider) +* [Auditing](#auditing) +* [Metric Collection](#metric-collection) +* [Exception Handling](#exception-handling) +* [Failure Injection](#failure-injection) + +### Client Configuration: + +* We now have two clients, a sync S3 Client and an async S3 Client. The async s3 client is required + as the select operation is currently only supported on the async client. Once we are confident in + the current set of changes, we will also be exploring moving other operations to the async client + as this could provide potential performance benefits. However those changes are not in the scope + of this PR, and will be done separately. +* The [createAwsConf](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java#L1190) +method is now split into: + ``` + createClientConfigBuilder // sets request timeout, user agent* + createHttpClientBuilder* // sets max connections, connection timeout, socket timeout + createProxyConfigurationBuilder // sets proxy config, defined in table below + ``` + +The table below lists the configurations S3A was using and what they now map to. + +|SDK V1 |SDK V2 | +|---|---| +|setMaxConnections |httpClientBuilder.maxConnections | +|setProtocol |The protocol is now HTTPS by default, and can only be modified by setting an HTTP endpoint on the client builder. This is done when setting the endpoint in getS3Endpoint() | +|setMaxErrorRetry |createRetryPolicyBuilder | +|setConnectionTimeout |httpClientBuilder.connectionTimeout | +|setSocketTimeout |httpClientBuilder.socketTimeout | +|setRequestTimeout |overrideConfigBuilder.apiCallAttemptTimeout | +|setSocketBufferSizeHints |Not supported | +|setSignerOverride |Not done yet | +|setProxyHost |proxyConfigBuilder.endpoint | +|setProxyPort |set when setting proxy host with .endpoint | +|setProxyUsername |proxyConfigBuilder.username | +|setProxyPassword |proxyConfigBuilder.password | +|setProxyDomain |proxyConfigBuilder.ntlmDomain, not supported in async client | +|setProxyWorkstation |proxyConfigBuilder.ntlmWorkstation, not supported in async client | +|setUserAgentPrefix |overrideConfigBuilder.putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX, userAgent); | +|addHeader |overrideConfigBuilder.putHeader | +|setUseThrottleRetries |not supported | + +### Endpoint and region configuration + +Previously, if no endpoint and region was configured, fall back to using us-east-1. Set +withForceGlobalBucketAccessEnabled(true) which will allow access to buckets not in this region too. +Since the SDK V2 no longer supports cross region access, we need to set the region and endpoint of +the bucket. The behaviour has now been changed to: + +* If no endpoint is specified, use s3.amazonaws.com. +* When setting the endpoint, also set the protocol (HTTP or HTTPS) +* When setting the region, first initiate a default S3 Client with region eu-west-2. Call headBucket + using this client. If the bucket is also in eu-west-2, then this will return a successful + response. Otherwise it will throw an error with status code 301 permanently moved. This error + contains the region of the bucket in its header, which we can then use to configure the client. + +### List Object: + +There is no way to paginate the listObject V1 result, we are +doing [this](https://github.com/ahmarsuhail/hadoop/pull/23/files#diff-4050f95b7e3912145415b6e2f9cd3b0760fcf2ce96bf0980c6c30a6edad2d0fbR2745) +instead. We are trying to get pagination to listObject V1 in the SDK, but will have to use this +workaround for now. + +### EncryptionSecretOperations: + +Two new methods have been added, `getSSECustomerKey` and `getSSEAwsKMSKey`. Previously SDK V1 had +specific classes for these keys `SSECustomerKey` and `SSEAwsKeyManagementParams` . There are no such +classes with V2, and things need to be set manually. For this reason, we simply just return keys as +strings now. And will have to calculate and set md5’s ourselves when building the request. + + +### GetObjectMetadata: + +* `RequestFactory.newGetObjectMetadataRequest` is now `RequestFactory.newHeadObjectRequestBuilder`. +* In `HeaderProcessing.retrieveHeaders()`, called by `getXAttrs()`, + removed `maybeSetHeader(headers, XA_CONTENT_MD5, md.getContentMD5())` as S3 doesn’t ever actually + return an md5 header, regardless of whether you set it during a putObject. It does return + an `etag` which may or may not be an md5 depending on certain conditions. `getContentMD5()` is + always empty, there does not seem to be a need to set this header. +* `RequestFactoryImpl.setOptionalGetObjectMetadataParameters` : Method has been removed and this + logic has been moved to `RequestFactoryImpl.newHeadObjectRequestBuilder()` +* `RequestFactoryImpl.generateSSECustomerKey()` has been removed, and instead + call `EncryptionSecretOperations.createSSECustomerKey` directly in `newHeadObjectRequestBuilder()` + + + +### PutObject + +* Previously, when creating the `putObjectRequest`, you would also give it the data to be uploaded. + So it would be of the form `PutObjectRequest(bucket, key, file/inputstream)`, this is no longer + the case. Instead, the data now needs to be passed in while making the `s3Client.putObject()` + call. For this reason, the data is now part of + the `S3AFileSystem.putObject(putObjectRequest, file, listener)` + and `S3AFileSystem.putObjectDirect(putObjectRequest, putOptions, uploadData, isFile)`. +* `S3ADataBlocks`: Need to make this class public as it’s now used to pass in data + to `putObjectDirect()`, sometimes from outside the package (`MagicCommitTracker` + , `ITestS3AMiscOperations`). +* `ProgressableProgressListener`: You can no longer pass in the `Upload` while initialising the + listener + as `ProgressableProgressListener listener = new ProgressableProgressListener(this, key, upload, progress);` + The upload is now only available after initialising the listener, since the listener needs to be + initialised during creation of the Transfer Manager upload. Previously, you could create the + listener after the starting the TM upload, and attach it. +* The `Upload` is now passed into the progress listener later, + in `listener.uploadCompleted(uploadInfo.getFileUpload());`. +* `UploadInfo`: Previously, since the data to be uploaded was part of `putObjectRequest`, the + transfer manager only returned a single `Upload` type, which could be used to track the upload. + Now, depending on the upload type (eg: File or InputStream), it returns different types. This + class has been updated to return FileUpload info, as it’s only ever used for file uploads + currently. It can be extended to store different transfer types in the future. +* `WriteOperationHelper.createPutObjectRequest() `: Previously the data to be uploaded was part + of `PutObjectRequest`, and so we required two methods to create the request. One for input streams + and one for files. Since the data to be uploaded is no longer part of the request, but instead an + argument in `putObject` , we only need one method now. +* `WriteOperationHelper.newObjectMetadata()`: This method has been removed, as standard metadata, + instead of being part of the `ObjectMetadata`, is now just added while building the request, for + example `putObjectRequestBuilder.serverSideEncryption().` +* `RequestFactory`: Similar to WriteOperationHelper, there is now a single putObjectRequest, + and `newObjectMetadata` has been removed. Instead, all standard metadata is now set in the new + method `buildPutObjectRequest`. +* `RequestFactoryImpl.newObjectMetadata()`: Previously, object metadata was created + using `newObjectMetadata()` and passed into the `newPutObjectRequest()` call. This method has been + removed, as standard metadata, instead of being part of the `ObjectMetadata`, is now just added + while building the request, in `putObjectRequestBuilder.serverSideEncryption().` Content length + and content encoding set in this method is now set in `buildPutObjectRequest()` , and SSE is set + in `putEncryptionParameters()`. +* `RequestFactoryImpl.maybeSetMetadata()` : was a generic method to set user metadata on object + metadata. user metadata now gets set on the request builder, so method has been removed. +* `RequestFactoryImpl.setOptionalPutRequestParameters()` : Method has been removed, and this logic + has been moved to `putEncryptionParameters()` . + +### CopyObject + +* `RequestFactoryImpl.buildPutObjectRequest` : Destination metadata is no longer built + using `newObjectMetadata()` and instead set on the request builder. The logic has a couple of + differences: + * content encoding is set in `buildCopyObjectRequest`, + the `if (contentEncoding != null && !isDirectoryMarker)` can just + be `if (contentEncoding != null)` for copy, as for this `isDirectoryMarker` was always false. + * contentLength is not set, as this is a system defined header, and copied over automatically by + S3 during copy. +* `HeaderProcessing.cloneObjectMetadata`: This was previously also setting a lot of system defined + metadata, eg: `setHttpExpiresDate` and `setLastModified`. These have been removed as they are set + by S3 during the copy. Have tested, and can see they are set automatically regardless of the + metadataDirective (copy or replace). +* `RequestFactoryImpl. copyEncryptionParameters()` : Due to the changes + in `EncryptionSecretOperations`, source and destination encryption params have to be set manually. + +### MultipartUpload + +* `RequestFactoryImpl.newObjectMetdata()` : Metadata is now set on the request builder. For MPU, only +content encoding needs to be set, as per per previous behaviour. Encryption params are set +in ` multipartUploadEncryptionParameters`. + +### GetObject + +* Previously, GetObject returned a `S3Object` response which exposed its content in a + `S3ObjectInputStream` through the `getObjectContent()` method. In SDK v2, the response is + directly a `ResponseInputStream` with the content, while the + `GetObjectResponse` instance can be retrieved by calling `response()` on it. +* The above change simplifies managing the lifetime of the response input stream. In v1, + `S3AInputStream` had to keep a reference to the `S3Object` while holding the wrapped + `S3ObjectInputStream`. When upgraded to SDK v2, it can simply wrap the new + `ResponseInputStream`, which handles lifetime correctly. Same applies + to `SDKStreamDrainer`. Furthermore, the map in `S3ARemoteObject` associating input streams and + `S3Object` instances is no longer needed. +* The range header on a `GetObject` request is now specified as a string, rather than a + `start`-`end` pair. `S3AUtils.formatRange` was introduced to format it. + +### DeleteObject + +In SDK v1, bulk delete would throw a `com.amazonaws.services.s3.model.MultiObjectDeleteException` +in case of partial failure. In v2, instead, it returns a `DeleteObjectsResponse` containing a +list of errors. A new `MultiObjectDeleteException` class was introduced in +`org.apache.hadoop.fs.s3a` and is thrown when appropriate to reproduce the previous behaviour. +* `MultiObjectDeleteSupport.translateDeleteException` was moved into `MultiObjectDeleteException`. +* `ObjectIdentifier` replaces DeleteObjectsRequest.KeyVersion. + +### Select + +In SDK v2, Handling of select requests has changes significantly since SelectObjectContent is +only supported on the new async S3 client. In previous versions, the response to a +SelectObjectContent request exposed the results in a `SelectRecordsInputStream`, which S3A +could wrap in `SelectInputStream`. In v2, instead, the response needs to be handled by an object +implementing `SelectObjectContentResponseHandler`, which can receive an async publisher of +the "events" returned by the service (`SdkPublisher`). + +In order to adapt the new API in S3A, three new classes have been introduced in +`org.apache.hadoop.fs.s3a.select`: + +* `SelectObjectContentHelper`: wraps the `selectObjectContent()` call, provides a custom + response handler to receive the response, and exposes a `SelectEventStreamPublisher`. +* `SelectEventStreamPublisher`: a publisher of select event stream events, which handles the + future returned by the select call and wraps the original publisher. This class provides + a `toRecordsInputStream()` method which returns an input stream containing the results, + reproducing the behaviour of the old `SelectRecordsInputStream`. +* `BlockingEnumeration`: an adapter which lazily requests new elements from the publisher and + exposes them through an `Enumeration` interface. Used in + `SelectEventStreamPublisher.toRecordsInputStream()` to adapt the event publisher into + an enumeration of input streams, eventually passed to a `SequenceInputStream`. + Note that the "lazy" behaviour means that new elements are requested only on `read()` calls on + the input stream. + + + +### CredentialsProvider + +* All credential provider classes implemented in Hadoop now implement V2's `AwsCredentialProvider` +* New adapter class `org.apache.hadoop.fs.s3a.adapter.V1ToV2AwsCredentialProviderAdapter` has been + added. This converts SDK V1 credential providers to SDK V2’s which + implement `AwsCredentialsProvider`. +* `AWSCredentialProviderList` also implements `AwsCredentialProvider`. But keeps existing + constructors and add methods for V1 credential providers, and wraps V1 cred providers in the + adapter here. This means that custom binding classes in delegation tokens, as well as any custom + credential providers will continue to work. +* Added a new `getCredentials()` method in `AWSCredentialProviderList`, which ensured that custom + binding classes which are calling `AWSCredentialProviderList.getCredentials()`, continue to work. +* The following values `fs.s3a.aws.credentials.provider` are mapped: + as `com.amazonaws.auth.EnvironmentVariableCredentialsProvider`, then map it to V2’s + +|`fs.s3a.aws.credentials.provider` value |Mapped to | +|---|---| +|`com.amazonaws.auth.EnvironmentVariableCredentialsProvider` |`software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider` | +|`com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper` |`org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` | +|`com.amazonaws.auth.`InstanceProfileCredentialsProvider`` |`org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` | + + +### Auditing + +The SDK v2 offers a new `ExecutionInterceptor` +[interface](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/core/interceptor/ExecutionInterceptor.html) +which broadly replaces the `RequestHandler2` abstract class from v1. +Switching to the new mechanism in S3A brings: + +* Simplification in `AWSAuditEventCallbacks` (and implementors) which can now extend + `ExecutionInterceptor` +* "Registering" a Span with a request has moved from `requestCreated` to `beforeExecution` + (where an `ExecutionAttributes` instance is first available) +* The ReferrerHeader is built and added to the http request in `modifyHttpRequest`, + rather than in `beforeExecution`, where no http request is yet available +* Dynamic loading of interceptors has been implemented to reproduce previous behaviour + with `RequestHandler2`s. The AWS SDK v2 offers an alternative mechanism, described + [here](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/core/interceptor/ExecutionInterceptor.html) + under "Interceptor Registration", which could make it redundant. + +In the Transfer Manager, `TransferListener` replaces `TransferStateChangeListener`. S3A code +has been updated and `AuditManagerS3A` implementations now provide an instance of the former to +switch to the active span, but registration of the new listeners is currently commented out because +it causes an incompatibility issue with the internal logger, resulting in `NoSuchMethodError`s, +at least in the current TransferManager Preview release. + + +### Metric Collection + +`AwsStatisticsCollector` has been updated to implement the new `MetricPublisher` interface +and collect the metrics from a `MetricCollection` object. +The following table maps SDK v2 metrics to their equivalent in v1: + +| v2 Metrics| com.amazonaws.util.AWSRequestMetrics.Field| Comment| +|-------------------------------------------------------------|---------------------------------------------|--------------------------------| +| CoreMetric.RETRY_COUNT| HttpClientRetryCount|| +| CoreMetric.RETRY_COUNT| RequestCount| always HttpClientRetryCount+1| +| HttpMetric.HTTP_STATUS_CODE with HttpStatusCode.THROTTLING| ThrottleException| to be confirmed| +| CoreMetric.API_CALL_DURATION| ClientExecuteTime|| +| CoreMetric.SERVICE_CALL_DURATION| HttpRequestTime|| +| CoreMetric.MARSHALLING_DURATION| RequestMarshallTime|| +| CoreMetric.SIGNING_DURATION| RequestSigningTime|| +| CoreMetric.UNMARSHALLING_DURATION| ResponseProcessingTime| to be confirmed| + +Note that none of the timing metrics (`*_DURATION`) are currently collected in S3A. + +### Exception Handling + +The code to handle exceptions thrown by the SDK has been updated to reflect the changes in v2: + +* `com.amazonaws.SdkBaseException` and `com.amazonaws.AmazonClientException` changes: + * These classes have combined and replaced with + `software.amazon.awssdk.core.exception.SdkException`. +* `com.amazonaws.SdkClientException` changes: + * This class has been replaced with `software.amazon.awssdk.core.exception.SdkClientException`. + * This class now extends `software.amazon.awssdk.core.exception.SdkException`. +* `com.amazonaws.AmazonServiceException` changes: + * This class has been replaced with + `software.amazon.awssdk.awscore.exception.AwsServiceException`. + * This class now extends `software.amazon.awssdk.core.exception.SdkServiceException`, + a new exception type that extends `software.amazon.awssdk.core.exception.SdkException`. + +See also the +[SDK changelog](https://github.com/aws/aws-sdk-java-v2/blob/master/docs/LaunchChangelog.md#3-exception-changes). + + +### Failure Injection + +While using the SDK v1, failure injection was implemented in `InconsistentAmazonS3CClient`, +which extended the S3 client. In SDK v2, reproducing this approach would not be straightforward, +since the default S3 client is an internal final class. Instead, the same fault injection strategy +is now performed by a `FailureInjectionInterceptor` (see +[ExecutionInterceptor](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/core/interceptor/ExecutionInterceptor.html)) +registered on the default client by `InconsistentS3CClientFactory`. +`InconsistentAmazonS3CClient` has been removed. No changes to the user configuration are required. + diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/delegation_tokens.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/delegation_tokens.md index 7aaa1b8b5ce79..6ed6365a9a9ae 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/delegation_tokens.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/delegation_tokens.md @@ -339,7 +339,7 @@ Here is the effective list of providers if none are declared: org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider, org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider, - com.amazonaws.auth.EnvironmentVariableCredentialsProvider, + software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider, org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md index ce1286c414a60..11bb2937db994 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md @@ -66,7 +66,7 @@ The server-side "SSE" encryption is performed with symmetric AES256 encryption; S3 offers different mechanisms for actually defining the key to use. -There are four key management mechanisms, which in order of simplicity of use, +There are five key management mechanisms, which in order of simplicity of use, are: * S3 Default Encryption @@ -75,6 +75,9 @@ are: by Amazon's Key Management Service, a key referenced by name in the uploading client. * SSE-C : the client specifies an actual base64 encoded AES-256 key to be used to encrypt and decrypt the data. +* DSSE-KMS: Two independent layers of encryption at server side. An AES256 key is +generated in S3, and encrypted with a secret key provided by Amazon's Key Management +Service. Encryption options @@ -84,6 +87,7 @@ Encryption options | `SSE-KMS` | server side, KMS key | key used to encrypt/decrypt | none | | `SSE-C` | server side, custom key | encryption algorithm and secret | encryption algorithm and secret | | `CSE-KMS` | client side, KMS key | encryption algorithm and key ID | encryption algorithm | +| `DSSE-KMS` | server side, KMS key | key used to encrypt/decrypt | none | With server-side encryption, the data is uploaded to S3 unencrypted (but wrapped by the HTTPS encryption channel). @@ -91,7 +95,7 @@ The data is encrypted in the S3 store and decrypted when it's being retrieved. A server side algorithm can be enabled by default for a bucket, so that whenever data is uploaded unencrypted a default encryption algorithm is added. -When data is encrypted with S3-SSE or SSE-KMS it is transparent to all clients +When data is encrypted with S3-SSE, SSE-KMS or DSSE-KMS it is transparent to all clients downloading the data. SSE-C is different in that every client must know the secret key needed to decypt the data. @@ -132,7 +136,7 @@ not explicitly declare an encryption algorithm. [S3 Default Encryption for S3 Buckets](https://docs.aws.amazon.com/AmazonS3/latest/dev/bucket-encryption.html) -This supports SSE-S3 and SSE-KMS. +This supports SSE-S3, SSE-KMS and DSSE-KMS. There is no need to set anything up in the client: do it in the AWS console. @@ -316,6 +320,82 @@ metadata. Since only one encryption key can be provided at a time, S3A will not pass the correct encryption key to decrypt the data. +### DSSE-KMS: Dual-layer Server-Encryption with KMS Managed Encryption Keys + +By providing a dual-layer server-side encryption mechanism using AWS Key Management Service +(AWS KMS) keys, known as DSSE-KMS, two layers of encryption are applied to objects upon their +upload to Amazon S3. DSSE-KMS simplifies the process of meeting compliance requirements that +mandate the implementation of multiple layers of encryption for data while maintaining complete +control over the encryption keys. + + +When uploading data encrypted with SSE-KMS, the sequence is as follows: + +1. The S3A client must declare a specific CMK in the property `fs.s3a.encryption.key`, or leave + it blank to use the default configured for that region. + +2. The S3A client uploads all the data as normal, now including encryption information. + +3. The S3 service encrypts the data with a symmetric key unique to the new object. + +4. The S3 service retrieves the chosen CMK key from the KMS service, and, if the user has + the right to use it, uses it to provide dual-layer encryption for the data. + + +When downloading DSSE-KMS encrypted data, the sequence is as follows + +1. The S3A client issues an HTTP GET request to read the data. + +2. S3 sees that the data was encrypted with DSSE-KMS, and looks up the specific key in the + KMS service. + +3. If and only if the requesting user has been granted permission to use the CMS key does + the KMS service provide S3 with the key. + +4. As a result, S3 will only decode the data if the user has been granted access to the key. + +Further reading on DSSE-KMS [here](https://docs.aws.amazon.com/AmazonS3/latest/userguide/UsingDSSEncryption.html) + +AWS Blog post [here](https://aws.amazon.com/blogs/aws/new-amazon-s3-dual-layer-server-side-encryption-with-keys-stored-in-aws-key-management-service-dsse-kms/) + +### Enabling DSSE-KMS + +To enable DSSE-KMS, the property `fs.s3a.encryption.algorithm` must be set to `DSSE-KMS` in `core-site`: + +```xml + + fs.s3a.encryption.algorithm + DSSE-KMS + +``` + +The ID of the specific key used to encrypt the data should also be set in the property `fs.s3a.encryption.key`: + +```xml + + fs.s3a.encryption.key + arn:aws:kms:us-west-2:360379543683:key/071a86ff-8881-4ba0-9230-95af6d01ca01 + +``` + +Organizations may define a default key in the Amazon KMS; if a default key is set, +then it will be used whenever SSE-KMS encryption is chosen and the value of `fs.s3a.encryption.key` is empty. + +### the S3A `fs.s3a.encryption.key` key only affects created files + +With SSE-KMS, the S3A client option `fs.s3a.encryption.key` sets the +key to be used when new files are created. When reading files, this key, +and indeed the value of `fs.s3a.encryption.algorithm` is ignored: +S3 will attempt to retrieve the key and decrypt the file based on the create-time settings. + +This means that + +* There's no need to configure any client simply reading data. +* It is possible for a client to read data encrypted with one KMS key, and + write it with another. + + + ## Encryption best practises diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 0c787de46768f..076ea0fd5b8a7 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -35,19 +35,20 @@ full details. * [Encryption](./encryption.html) * [Performance](./performance.html) -* [S3Guard](./s3guard.html) +* [The upgrade to AWS Java SDK V2](./aws_sdk_upgrade.html) +* [Working with Third-party S3 Stores](./third_party_stores.html) * [Troubleshooting](./troubleshooting_s3a.html) +* [Prefetching](./prefetching.html) * [Controlling the S3A Directory Marker Behavior](directory_markers.html). +* [Auditing](./auditing.html). * [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). -* [Auditing](./auditing.html). * [Auditing Architecture](./auditing_architecture.html). * [Testing](./testing.html) -* [Prefetching](./prefetching.html) -* [Upcoming upgrade to AWS Java SDK V2](./aws_sdk_upgrade.html) +* [S3Guard](./s3guard.html) ## Overview @@ -79,16 +80,15 @@ and compatible implementations. * Supports partitioned uploads for many-GB objects. * Offers a high-performance random IO mode for working with columnar data such as Apache ORC and Apache Parquet files. -* Uses Amazon's Java S3 SDK with support for latest S3 features and authentication +* Uses Amazon's Java V2 SDK with support for latest S3 features and authentication schemes. * Supports authentication via: environment variables, Hadoop configuration properties, the Hadoop key management store and IAM roles. * Supports per-bucket configuration. * Supports S3 "Server Side Encryption" for both reading and writing: SSE-S3, SSE-KMS and SSE-C. +* Supports S3-CSE client side encryption. * Instrumented with Hadoop metrics. -* Before S3 was consistent, provided a consistent view of inconsistent storage - through [S3Guard](./s3guard.html). * Actively maintained by the open source community. @@ -97,19 +97,17 @@ properties, the Hadoop key management store and IAM roles. There other Hadoop connectors to S3. Only S3A is actively maintained by the Hadoop project itself. -1. Apache's Hadoop's original `s3://` client. This is no longer included in Hadoop. 1. Amazon EMR's `s3://` client. This is from the Amazon EMR team, who actively maintain it. -1. Apache's Hadoop's [`s3n:` filesystem client](./s3n.html). - This connector is no longer available: users must migrate to the newer `s3a:` client. + ## Getting Started S3A depends upon two JARs, alongside `hadoop-common` and its dependencies. -* `hadoop-aws` JAR. -* `aws-java-sdk-bundle` JAR. +* `hadoop-aws` JAR. This contains the S3A connector. +* `bundle` JAR. This contains the full shaded AWS V2 SDK. The versions of `hadoop-common` and `hadoop-aws` must be identical. @@ -189,7 +187,8 @@ of work, as opposed to HDFS or other "real" filesystem. The [S3A committers](./committers.html) are the sole mechanism available to safely save the output of queries directly into S3 object stores -through the S3A filesystem. +through the S3A filesystem when the filesystem structure is +how the table is represented. ### Warning #2: Object stores have different authorization models @@ -199,10 +198,7 @@ authorization model of HDFS and traditional file systems. The S3A client simply reports stub information from APIs that would query this metadata: * File owner is reported as the current user. -* File group also is reported as the current user. Prior to Apache Hadoop -2.8.0, file group was reported as empty (no group associated), which is a -potential incompatibility problem for scripts that perform positional parsing of -shell output and other clients that expect to find a well-defined group. +* File group also is reported as the current user. * Directory permissions are reported as 777. * File permissions are reported as 666. @@ -239,66 +235,44 @@ However, with the upcoming upgrade to AWS Java SDK V2, these classes will need t updated to implement `software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`. For more information see [Upcoming upgrade to AWS Java SDK V2](./aws_sdk_upgrade.html). -*Important*: The S3A connector no longer supports username and secrets -in URLs of the form `s3a://key:secret@bucket/`. -It is near-impossible to stop those secrets being logged —which is why -a warning has been printed since Hadoop 2.8 whenever such a URL was used. - ### Authentication properties ```xml fs.s3a.access.key - AWS access key ID. - Omit for IAM role-based or provider-based authentication. + AWS access key ID used by S3A file system. Omit for IAM role-based or provider-based authentication. fs.s3a.secret.key - AWS secret key. - Omit for IAM role-based or provider-based authentication. + AWS secret key used by S3A file system. Omit for IAM role-based or provider-based authentication. - fs.s3a.aws.credentials.provider - - Comma-separated class names of credential provider classes which implement - com.amazonaws.auth.AWSCredentialsProvider. - - These are loaded and queried in sequence for a valid set of credentials. - Each listed class must implement one of the following means of - construction, which are attempted in order: - 1. a public constructor accepting java.net.URI and - org.apache.hadoop.conf.Configuration, - 2. 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. - - 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: - 1. 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 - 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 - of instance profile credentials if running in an EC2 VM. + fs.s3a.session.token + Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider + as one of the providers. - fs.s3a.session.token + fs.s3a.aws.credentials.provider + + org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider, + org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider, + software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider, + org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider + - Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider - as one of the providers. + Comma-separated class names of credential provider classes which implement + software.amazon.awssdk.auth.credentials.AwsCredentialsProvider. + + When S3A delegation tokens are not enabled, this list will be used + to directly authenticate with S3 and other AWS 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. ``` @@ -350,13 +324,19 @@ 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: 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 +1. An attempt is made to query the Amazon EC2 Instance/k8s container Metadata Service to retrieve credentials published to EC2 VMs. S3A can be configured to obtain client authentication providers from classes -which integrate with the AWS SDK by implementing the `com.amazonaws.auth.AWSCredentialsProvider` -Interface. This is done by listing the implementation classes, in order of +which integrate with the AWS SDK by implementing the +`software.amazon.awssdk.auth.credentials.AwsCredentialsProvider` +interface. +This is done by listing the implementation classes, in order of preference, in the configuration option `fs.s3a.aws.credentials.provider`. +In previous hadoop releases, providers were required to +implement the AWS V1 SDK interface `com.amazonaws.auth.AWSCredentialsProvider`. +Consult the [Upgrading S3A to AWS SDK V2](./aws_sdk_upgrade.html) documentation +to see how to migrate credential providers. *Important*: AWS Credential Providers are distinct from _Hadoop Credential Providers_. As will be covered later, Hadoop Credential Providers allow passwords and other secrets @@ -371,21 +351,23 @@ 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) | +| Hadoop module credential provider | Authentication Mechanism | +|----------------------------------------------------------------|--------------------------------------------------| +| `org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider` | Session Credentials in configuration | +| `org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider` | Simple name/secret credentials in configuration | +| `org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider` | Anonymous Login | +| `org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider` | [Assumed Role credentials](./assumed_roles.html) | +| `org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` | EC2/k8s instance credentials | -There are also many in the Amazon SDKs, in particular two which are automatically -set up in the authentication chain: +There are also many in the Amazon SDKs, with the common ones being as follows + +| classname | description | +|----------------------------------------------------------------------------------|------------------------------| +| `software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider` | AWS Environment Variables | +| `software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider` | EC2 Metadata Credentials | +| `software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider` | EC2/k8s Metadata Credentials | -| classname | description | -|-----------|-------------| -| `com.amazonaws.auth.InstanceProfileCredentialsProvider`| EC2 Metadata Credentials | -| `com.amazonaws.auth.EnvironmentVariableCredentialsProvider`| AWS Environment Variables | ### EC2 IAM Metadata Authentication with `InstanceProfileCredentialsProvider` @@ -402,7 +384,7 @@ You can configure Hadoop to authenticate to AWS using a [named profile](https:// To authenticate with a named profile: -1. Declare `com.amazonaws.auth.profile.ProfileCredentialsProvider` as the provider. +1. Declare `software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider` as the provider. 1. Set your profile via the `AWS_PROFILE` environment variable. 1. Due to a [bug in version 1 of the AWS Java SDK](https://github.com/aws/aws-sdk-java/issues/803), you'll need to remove the `profile` prefix from the AWS configuration section heading. @@ -525,50 +507,9 @@ This means that the default S3A authentication chain can be defined as org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider, org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider, - com.amazonaws.auth.EnvironmentVariableCredentialsProvider, + software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider - - 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 other AWS services. - When S3A Delegation tokens are enabled, depending upon the delegation - token binding it may be used - to communicate with 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, AWS_SECRET_ACCESS_KEY, - and AWS_SESSION_TOKEN as documented in the AWS SDK. - * org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider: picks up - IAM credentials of any EC2 VM or AWS container in which the process is running. - ``` @@ -766,16 +707,145 @@ Here are the S3A properties for use in production; some testing-related options are covered in [Testing](./testing.md). ```xml + + + fs.s3a.aws.credentials.provider + + org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider, + org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider, + software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider, + org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider + + + Comma-separated class names of credential provider classes which implement + software.amazon.awssdk.auth.credentials.AwsCredentialsProvider. + + When S3A delegation tokens are not enabled, this list will be used + to directly authenticate with S3 and other AWS 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. + + + + + fs.s3a.security.credential.provider.path + + + Optional comma separated list of credential providers, a list + which is prepended to that set in hadoop.security.credential.provider.path + + + + + fs.s3a.assumed.role.arn + + + AWS ARN for the role to be assumed. + Required if the fs.s3a.aws.credentials.provider contains + org.apache.hadoop.fs.s3a.AssumedRoleCredentialProvider + + + + + fs.s3a.assumed.role.session.name + + + Session name for the assumed role, must be valid characters according to + the AWS APIs. + Only used if AssumedRoleCredentialProvider is the AWS credential provider. + If not set, one is generated from the current Hadoop/Kerberos username. + + + + + fs.s3a.assumed.role.policy + + + JSON policy to apply to the role. + Only used if AssumedRoleCredentialProvider is the AWS credential provider. + + + + + fs.s3a.assumed.role.session.duration + 30m + + Duration of assumed roles before a refresh is attempted. + Used when session tokens are requested. + Range: 15m to 1h + + + + + fs.s3a.assumed.role.sts.endpoint + + + 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. + + + + + fs.s3a.assumed.role.sts.endpoint.region + + + 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. + Used by the AssumedRoleCredentialProvider and in Session and Role delegation + tokens. + + + + + fs.s3a.assumed.role.credentials.provider + org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider + + 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". + + + + + fs.s3a.delegation.token.binding + + + 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. + + Bindings available in hadoop-aws are: + org.apache.hadoop.fs.s3a.auth.delegation.SessionTokenBinding + org.apache.hadoop.fs.s3a.auth.delegation.FullCredentialsTokenBinding + org.apache.hadoop.fs.s3a.auth.delegation.RoleTokenBinding + + + fs.s3a.connection.maximum - 15 - Controls the maximum number of simultaneous connections to S3. + 96 + Controls the maximum number of simultaneous connections to S3. + This must be bigger than the value of fs.s3a.threads.max so as to stop + threads being blocked waiting for new HTTPS connections. + Why not equal? The AWS SDK transfer manager also uses these connections. + fs.s3a.connection.ssl.enabled true - Enables or disables SSL connections to S3. + Enables or disables SSL connections to AWS services. + Also sets the default port to use for the s3a proxy settings, + when not explicitly set in fs.s3a.proxy.port. @@ -786,16 +856,6 @@ options are covered in [Testing](./testing.md). - - fs.s3a.endpoint.region - AWS S3 region for a bucket, which bypasses the parsing of - fs.s3a.endpoint to know the region. Would be helpful in avoiding errors - while using privateLink URL and explicitly set the bucket region. - If set to a blank string (or 1+ space), falls back to the - (potentially brittle) SDK region resolution process. - - - fs.s3a.path.style.access false @@ -838,8 +898,14 @@ options are covered in [Testing](./testing.md). fs.s3a.attempts.maximum - 20 - How many times we should retry commands on transient errors. + 5 + + Number of times the AWS client library should retry errors before + escalating to the S3A code: {@value}. + The S3A connector does its own selective retries; the only time the AWS + SDK operations are not wrapped is during multipart copy via the AWS SDK + transfer manager. + @@ -854,20 +920,6 @@ options are covered in [Testing](./testing.md). Socket connection timeout in milliseconds. - - fs.s3a.paging.maximum - 5000 - How many keys to request from S3 when doing - directory listings at a time. - - - - fs.s3a.threads.max - 10 - Maximum number of concurrent active (part)uploads, - which each use a thread from the threadpool. - - fs.s3a.socket.send.buffer 8192 @@ -880,6 +932,20 @@ options are covered in [Testing](./testing.md). Socket receive buffer hint to amazon connector. Represented in bytes. + + fs.s3a.paging.maximum + 5000 + How many keys to request from S3 when doing + directory listings at a time. + + + + fs.s3a.threads.max + 64 + The total number of threads available in the filesystem for data + uploads *or any other queued filesystem operation*. + + fs.s3a.threads.keepalivetime 60 @@ -889,9 +955,25 @@ options are covered in [Testing](./testing.md). fs.s3a.max.total.tasks - 5 - Number of (part)uploads allowed to the queue before - blocking additional uploads. + 32 + The number of operations which can be queued for execution. + This is in addition to the number of active threads in fs.s3a.threads.max. + + + + + fs.s3a.executor.capacity + 16 + The maximum number of submitted tasks which is a single + operation (e.g. rename(), delete()) may submit simultaneously for + execution -excluding the IO-heavy block uploads, whose capacity + is set in "fs.s3a.fast.upload.active.blocks" + + All tasks are submitted to the shared thread pool whose size is + set in "fs.s3a.threads.max"; the value of capacity should be less than that + of the thread pool itself, as the goal is to stop a single operation + from overloading that thread pool. + @@ -904,7 +986,7 @@ options are covered in [Testing](./testing.md). fs.s3a.multipart.threshold - 128MB + 128M How big (in bytes) to split upload or copy operations up into. This also controls the partition size in renamed files, as rename() involves copying the source file(s). @@ -924,23 +1006,52 @@ options are covered in [Testing](./testing.md). fs.s3a.acl.default Set a canned ACL for newly created and copied objects. Value may be Private, - PublicRead, PublicReadWrite, AuthenticatedRead, LogDeliveryWrite, BucketOwnerRead, - or BucketOwnerFullControl. + PublicRead, PublicReadWrite, AuthenticatedRead, LogDeliveryWrite, BucketOwnerRead, + or BucketOwnerFullControl. If set, caller IAM role must have "s3:PutObjectAcl" permission on the bucket. - + fs.s3a.multipart.purge false True if you want to purge existing multipart uploads that may not have been - completed/aborted correctly + completed/aborted correctly. The corresponding purge age is defined in + fs.s3a.multipart.purge.age. + If set, when the filesystem is instantiated then all outstanding uploads + older than the purge age will be terminated -across the entire bucket. + This will impact multipart uploads by other applications and users. so should + be used sparingly, with an age value chosen to stop failed uploads, without + breaking ongoing operations. + fs.s3a.multipart.purge.age 86400 - Minimum age in seconds of multipart uploads to purge + Minimum age in seconds of multipart uploads to purge + on startup if "fs.s3a.multipart.purge" is true + + + + + fs.s3a.encryption.algorithm + Specify a server-side encryption or client-side + encryption algorithm for s3a: file system. Unset by default. It supports the + following values: 'AES256' (for SSE-S3), 'SSE-KMS', 'SSE-C', and 'CSE-KMS' + + + + + fs.s3a.encryption.key + Specific encryption key to use if fs.s3a.encryption.algorithm + has been set to 'SSE-KMS', 'SSE-C' or 'CSE-KMS'. In the case of SSE-C + , the value of this property should be the Base64 encoded key. If you are + using SSE-KMS and leave this property empty, you'll be using your default's + S3 KMS key, otherwise you should set this property to the specific KMS key + id. In case of 'CSE-KMS' this value needs to be the AWS-KMS Key ID + generated from AWS console. + @@ -950,23 +1061,19 @@ options are covered in [Testing](./testing.md). - fs.s3a.encryption.algorithm - Specify a server-side encryption or client-side - encryption algorithm for s3a: file system. Unset by default. It supports the - following values: 'AES256' (for SSE-S3), 'SSE-KMS', 'SSE-C', and 'CSE-KMS' - + fs.s3a.accesspoint.required + false + Require that all S3 access is made through Access Points and not through + buckets directly. If enabled, use per-bucket overrides to allow bucket access to a specific set + of buckets. - fs.s3a.encryption.key - Specific encryption key to use if fs.s3a.encryption.algorithm - has been set to 'SSE-KMS', 'SSE-C' or 'CSE-KMS'. In the case of SSE-C - , the value of this property should be the Base64 encoded key. If you are - using SSE-KMS and leave this property empty, you'll be using your default's - S3 KMS key, otherwise you should set this property to the specific KMS key - id. In case of 'CSE-KMS' this value needs to be the AWS-KMS Key ID - generated from AWS console. - + fs.s3a.block.size + 32M + Block size to use when reading files using s3a: file system. + A suffix from the set {K,M,G,T,P} may be used to scale the numeric value. + @@ -980,9 +1087,51 @@ options are covered in [Testing](./testing.md). - fs.s3a.block.size - 32M - Block size to use when reading files using s3a: file system. + fs.s3a.fast.upload.buffer + disk + + The buffering mechanism to for data being written. + Values: disk, array, bytebuffer. + + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory in a single stream up to the number + of blocks set by: + + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks. + + If using either of these mechanisms, keep this value low + + The total number of threads performing work across all threads is set by + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued + work items. + + + + + fs.s3a.fast.upload.active.blocks + 4 + + Maximum Number of blocks a single output stream can have + active (uploading, or queued to the central FileSystem + instance's pool of queued operations. + + This stops a single stream overloading the shared thread pool. + + + + + fs.s3a.readahead.range + 64K + Bytes to read ahead during a seek() before closing and + re-opening the S3 HTTP connection. This option will be overridden if + any call to setReadahead() is made to an open stream. + A suffix from the set {K,M,G,T,P} may be used to scale the numeric value. @@ -1008,118 +1157,232 @@ options are covered in [Testing](./testing.md). - fs.AbstractFileSystem.s3a.impl - org.apache.hadoop.fs.s3a.S3A - The implementation class of the S3A AbstractFileSystem. + fs.s3a.retry.limit + 7 + + Number of times to retry any repeatable S3 client request on failure, + excluding throttling requests. + - fs.s3a.readahead.range - 64K - Bytes to read ahead during a seek() before closing and - re-opening the S3 HTTP connection. This option will be overridden if - any call to setReadahead() is made to an open stream. + fs.s3a.retry.interval + 500ms + + Initial retry interval when retrying operations for any reason other + than S3 throttle errors. + - fs.s3a.input.async.drain.threshold - 64K - Bytes to read ahead during a seek() before closing and - re-opening the S3 HTTP connection. This option will be overridden if - any call to setReadahead() is made to an open stream. + fs.s3a.retry.throttle.limit + 20 + + Number of times to retry any throttled request. + + + + + fs.s3a.retry.throttle.interval + 100ms + + Initial between retry attempts on throttled requests, +/- 50%. chosen at random. + i.e. for an intial value of 3000ms, the initial delay would be in the range 1500ms to 4500ms. + Backoffs are exponential; again randomness is used to avoid the thundering heard problem. + 500ms is the default value used by the AWS S3 Retry policy. + + + + + fs.s3a.committer.name + file + + Committer to create for output to S3A, one of: + "file", "directory", "partitioned", "magic". + + + + + fs.s3a.committer.magic.enabled + true + + Enable support in the S3A filesystem for the "Magic" committer. + + + + + fs.s3a.committer.threads + 8 + + Number of threads in committers for parallel operations on files + (upload, commit, abort, delete...) + + + + + fs.s3a.committer.staging.tmp.path + tmp/staging + + Path in the cluster filesystem for temporary data. + This is for HDFS, not the local filesystem. + It is only for the summary data of each file, not the actual + data being committed. + Using an unqualified path guarantees that the full path will be + generated relative to the home directory of the user creating the job, + hence private (assuming home directory permissions are secure). + + + + + fs.s3a.committer.staging.unique-filenames + true + + Option for final files to have a unique name through job attempt info, + or the value of fs.s3a.committer.staging.uuid + When writing data with the "append" conflict option, this guarantees + that new data will not overwrite any existing data. + + + + + fs.s3a.committer.staging.conflict-mode + append + + Staging committer conflict resolution policy. + Supported: "fail", "append", "replace". + + + + + fs.s3a.committer.abort.pending.uploads + true + + Should the committers abort all pending uploads to the destination + directory? + + Set to false if more than one job is writing to the same directory tree. + fs.s3a.list.version 2 - Select which version of the S3 SDK's List Objects API to use. - Currently support 2 (default) and 1 (older API). + + Select which version of the S3 SDK's List Objects API to use. Currently + support 2 (default) and 1 (older API). + fs.s3a.connection.request.timeout 0 - Time out on HTTP requests to the AWS service; 0 means no timeout. - Measured in seconds; the usual time suffixes are all supported + Time out on HTTP requests to the AWS service; 0 means no timeout. + Measured in seconds; the usual time suffixes are all supported - Important: this is the maximum duration of any AWS service call, - including upload and copy operations. If non-zero, it must be larger - than the time to upload multi-megabyte blocks to S3 from the client, - and to rename many-GB files. Use with care. + Important: this is the maximum duration of any AWS service call, + including upload and copy operations. If non-zero, it must be larger + than the time to upload multi-megabyte blocks to S3 from the client, + and to rename many-GB files. Use with care. - Values that are larger than Integer.MAX_VALUE milliseconds are - converged to Integer.MAX_VALUE milliseconds + Values that are larger than Integer.MAX_VALUE milliseconds are + converged to Integer.MAX_VALUE milliseconds - fs.s3a.bucket.probe - 0 + fs.s3a.etag.checksum.enabled + false + + Should calls to getFileChecksum() return the etag value of the remote + object. + WARNING: if enabled, distcp operations between HDFS and S3 will fail unless + -skipcrccheck is set. + + + + + fs.s3a.change.detection.source + etag - The value can be 0 (default), 1 or 2. - When set to 0, bucket existence checks won't be done - during initialization thus making it faster. - Though it should be noted that when the bucket is not available in S3, - or if fs.s3a.endpoint points to the wrong instance of a private S3 store - consecutive calls like listing, read, write etc. will fail with - an UnknownStoreException. - When set to 1, the bucket existence check will be done using the - V1 API of the S3 protocol which doesn't verify the client's permissions - to list or read data in the bucket. - When set to 2, the bucket existence check will be done using the - V2 API of the S3 protocol which does verify that the - client has permission to read the bucket. + Select which S3 object attribute to use for change detection. + Currently support 'etag' for S3 object eTags and 'versionid' for + S3 object version IDs. Use of version IDs requires object versioning to be + enabled for each S3 bucket utilized. Object versioning is disabled on + buckets by default. When version ID is used, the buckets utilized should + have versioning enabled before any data is written. - fs.s3a.object.content.encoding - + fs.s3a.change.detection.mode + server - Content encoding: gzip, deflate, compress, br, etc. - This will be set in the "Content-Encoding" header of the object, - and returned in HTTP HEAD/GET requests. + Determines how change detection is applied to alert to inconsistent S3 + objects read during or after an overwrite. Value 'server' indicates to apply + the attribute constraint directly on GetObject requests to S3. Value 'client' + means to do a client-side comparison of the attribute value returned in the + response. Value 'server' would not work with third-party S3 implementations + that do not support these constraints on GetObject. Values 'server' and + 'client' generate RemoteObjectChangedException when a mismatch is detected. + Value 'warn' works like 'client' but generates only a warning. Value 'none' + will ignore change detection completely. - fs.s3a.create.storage.class - + fs.s3a.change.detection.version.required + true - Storage class: standard, reduced_redundancy, intelligent_tiering, etc. - Specify the storage class for S3A PUT object requests. - If not set the storage class will be null - and mapped to default standard class on S3. + Determines if S3 object version attribute defined by + fs.s3a.change.detection.source should be treated as required. If true and the + referred attribute is unavailable in an S3 GetObject response, + NoVersionAttributeException is thrown. Setting to 'true' is encouraged to + avoid potential for inconsistent reads with third-party S3 implementations or + against S3 buckets that have object versioning disabled. - fs.s3a.prefetch.enabled - false + fs.s3a.ssl.channel.mode + default_jsse - Enables prefetching and caching when reading from input stream. + If secure connections to S3 are enabled, configures the SSL + implementation used to encrypt connections to S3. Supported values are: + "default_jsse", "default_jsse_with_gcm", "default", and "openssl". + "default_jsse" uses the Java Secure Socket Extension package (JSSE). + However, when running on Java 8, the GCM cipher is removed from the list + of enabled ciphers. This is due to performance issues with GCM in Java 8. + "default_jsse_with_gcm" uses the JSSE with the default list of cipher + suites. "default_jsse_with_gcm" is equivalent to the behavior prior to + this feature being introduced. "default" attempts to use OpenSSL rather + than the JSSE for SSL encryption, if OpenSSL libraries cannot be loaded, + it falls back to the "default_jsse" behavior. "openssl" attempts to use + OpenSSL as well, but fails if OpenSSL libraries cannot be loaded. - fs.s3a.prefetch.block.size - 8MB + fs.s3a.downgrade.syncable.exceptions + true - The size of a single prefetched block of data. - Decreasing this will increase the number of prefetches required, and may negatively impact performance. + Warn but continue when applications use Syncable.hsync when writing + to S3A. + - fs.s3a.prefetch.block.count - 8 + fs.s3a.audit.enabled + true - Maximum number of blocks prefetched concurrently at any given time. + Should auditing of S3A requests be enabled? -``` +``` ## Retry and Recovery The S3A client makes a best-effort attempt at recovering from network failures; @@ -1139,8 +1402,10 @@ not the failing operation is idempotent. * Interruptions: `InterruptedIOException`, `InterruptedException`. * Rejected HTTP requests: `InvalidRequestException` -These are all considered unrecoverable: S3A will make no attempt to recover -from them. +These and others are all considered unrecoverable: S3A will make no attempt to recover +from them. The AWS SDK itself may retry before the S3A connector sees the exception. +As an example, the SDK will retry on `UnknownHostException` in case it is a transient +DNS error. ### Possibly Recoverable Problems: Retry @@ -1191,17 +1456,9 @@ only succeed if the first `delete()` call has already succeeded. Because S3 is eventually consistent *and* doesn't support an atomic create-no-overwrite operation, the choice is more ambiguous. -Currently S3A considers delete to be -idempotent because it is convenient for many workflows, including the -commit protocols. Just be aware that in the presence of transient failures, -more things may be deleted than expected. (For anyone who considers this to -be the wrong decision: rebuild the `hadoop-aws` module with the constant -`S3AFileSystem.DELETE_CONSIDERED_IDEMPOTENT` set to `false`). - - - - - +S3A considers delete to be idempotent because it is convenient for many workflows, +including the commit protocols. Just be aware that in the presence of transient failures, +more things may be deleted than expected. ### Throttled requests from S3 @@ -1415,7 +1672,7 @@ role information available when deployed in Amazon EC2. ```xml fs.s3a.aws.credentials.provider - com.amazonaws.auth.InstanceProfileCredentialsProvider + org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider ``` @@ -1707,13 +1964,17 @@ the storage class you want. ``` Please note that S3A does not support reading from archive storage classes at the moment. -`AccessDeniedException` with InvalidObjectState will be thrown if you're trying to do so. +`AccessDeniedException` with `InvalidObjectState` will be thrown if you're trying to do so. + +When a file is "renamed" through the s3a connector it is copied then deleted. +Storage Classes will normally be propagated. + -## Configuring S3A for S3 on Outposts +## Configuring S3A for S3 on Outposts S3A now supports [S3 on Outposts](https://docs.aws.amazon.com/AmazonS3/latest/userguide/S3onOutposts.html). Accessing data through an access point is done by using its Amazon Resource Name (ARN), as opposed to just the bucket name. -The only supported storage class on Outposts is **OUTPOSTS**, and by default objects are encrypted with [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/userguide/s3-outposts-data-encryption.html). +The only supported storage class on Outposts is `OUTPOSTS`, and by default objects are encrypted with [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/userguide/s3-outposts-data-encryption.html). You can set the Access Point ARN property using the following per bucket configuration property: ```xml @@ -2155,7 +2416,7 @@ If no custom signers are being used - this value does not need to be set. `SignerName:SignerClassName` - register a new signer with the specified name, and the class for this signer. -The Signer Class must implement `com.amazonaws.auth.Signer`. +The Signer Class must implement `software.amazon.awssdk.core.signer.Signer`. `SignerName:SignerClassName:SignerInitializerClassName` - similar time above except also allows for a custom SignerInitializer diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md index 88e6e8a0b2115..6eec639b7fd90 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md @@ -447,7 +447,7 @@ and rate of requests. Spreading data across different buckets, and/or using a more balanced directory structure may be beneficial. Consult [the AWS documentation](http://docs.aws.amazon.com/AmazonS3/latest/dev/request-rate-perf-considerations.html). -Reading or writing data encrypted with SSE-KMS forces S3 to make calls of +Reading or writing data encrypted with SSE-KMS or DSSE-KMS forces S3 to make calls of the AWS KMS Key Management Service, which comes with its own [Request Rate Limits](http://docs.aws.amazon.com/kms/latest/developerguide/limits.html). These default to 1200/second for an account, across all keys and all uses of diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3_select.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3_select.md index 649c80a22d4f7..1361b8d9f63f3 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3_select.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3_select.md @@ -14,7 +14,7 @@ # S3 Select -**Experimental Feature** +**Deprecated Feature** @@ -60,6 +60,20 @@ Record Readers. It's better here to directly use the Apache Spark, Hive, Impala, Flink or similar, which all use the latest ASF-supported libraries. +## Dependencies: eventstream JAR + +To use S3 Select through the S3A connector, an extra JAR MUST be added to the classpath of your application, +`eventstream-1.0.1.jar`.a +For command line tool use, this should be done by adding it to `share/hadoop/common/lib/` + +```xml + + software.amazon.eventstream + eventstream + 1.0.1 + +``` + ## Enabling/Disabling S3 Select S3 Select is enabled by default: @@ -288,10 +302,12 @@ hadoop s3guard \ ``` -## Use in MR/Analytics queries: Work in Progress +## Use in MR/Analytics queries: Partially Supported -S3 Select support in analytics queries is a work in progress. It does -not work reliably with large source files where the work is split up. +S3 Select support in analytics queries is only partially supported. +It does not work reliably with large source files where the work is split up, +and as the various query engines all assume that .csv and .json formats are splittable, +things go very wrong, fast. As a proof of concept *only*, S3 Select queries can be made through MapReduce jobs which use any Hadoop `RecordReader` @@ -663,6 +679,24 @@ to the `get()` call: do it. ## Troubleshooting +### `NoClassDefFoundError: software/amazon/eventstream/MessageDecoder` + +Select operation failing with a missing eventstream class. + +``` +java.io.IOException: java.lang.NoClassDefFoundError: software/amazon/eventstream/MessageDecoder +at org.apache.hadoop.fs.s3a.select.SelectObjectContentHelper.select(SelectObjectContentHelper.java:75) +at org.apache.hadoop.fs.s3a.WriteOperationHelper.lambda$select$10(WriteOperationHelper.java:660) +at org.apache.hadoop.fs.store.audit.AuditingFunctions.lambda$withinAuditSpan$0(AuditingFunctions.java:62) +at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:122) +``` + +The eventstream JAR is not on the classpath/not in sync with the version of the full "bundle.jar" JDK + +Fix: get a compatible version of the JAR on the classpath. + +### SQL errors + Getting S3 Select code to work is hard, though those knowledgeable in SQL will find it easier. @@ -673,7 +707,6 @@ Problems can be split into: 1. Datatype casting issues 1. Bad records/data in source files. 1. Failure to configure MR jobs to work correctly. -1. Failure of MR jobs due to The exceptions here are all based on the experience during writing tests; more may surface with broader use. diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md index dc060e9d8f52e..e49aa0a0304d8 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md @@ -377,6 +377,19 @@ This adds extra overhead to every operation, but helps verify that the connector not keeping markers where it needs to be deleting them -and hence backwards compatibility is maintained. +## Enabling prefetch for all tests + +The tests are run with prefetch if the `prefetch` property is set in the +maven build. This can be combined with the scale tests as well. + +```bash +mvn verify -Dprefetch + +mvn verify -Dparallel-tests -Dprefetch -DtestsThreadCount=8 + +mvn verify -Dparallel-tests -Dprefetch -Dscale -DtestsThreadCount=8 +``` + ## Scale Tests There are a set of tests designed to measure the scalability and performance @@ -519,7 +532,7 @@ Otherwise, set a large timeout in `fs.s3a.scale.test.timeout` The tests are executed in an order to only clean up created files after the end of all the tests. If the tests are interrupted, the test data will remain. -## Load tests. +## Load tests. Some are designed to overload AWS services with more requests per second than an AWS account is permitted. @@ -545,6 +558,32 @@ which address issues. In particular, we encourage testing of Hadoop release candidates, as these third-party endpoints get even less testing than the S3 endpoint itself. +The core XML settings to turn off tests of features unavailable +on third party stores. + +```xml + + test.fs.s3a.encryption.enabled + false + + + test.fs.s3a.create.storage.class.enabled + false + + + fs.s3a.select.enabled + false + + + test.fs.s3a.sts.enabled + false + + + test.fs.s3a.create.create.acl.enabled + false + < /property> +``` + ### Public datasets used in tests Some tests rely on the presence of existing public datasets available on Amazon S3. @@ -585,7 +624,7 @@ S3 storage class, these tests might fail. They can be disabled. ``` -### Configuring the CSV file read tests** +### Configuring the CSV file read tests To test on alternate infrastructures supporting the same APIs, the option `fs.s3a.scale.test.csvfile` must either be @@ -618,19 +657,20 @@ your `core-site.xml` file, so that trying to use S3 select fails fast with a meaningful error ("S3 Select not supported") rather than a generic Bad Request exception. -### Enabling prefetch for all tests +### Disabling V1 List API tests -The tests are run with prefetch if the `prefetch` property is set in the -maven build. This can be combined with the scale tests as well. - -```bash -mvn verify -Dprefetch -mvn verify -Dparallel-tests -Dprefetch -DtestsThreadCount=8 - -mvn verify -Dparallel-tests -Dprefetch -Dscale -DtestsThreadCount=8 +If `ITestS3AContractGetFileStatusV1List` fails with any error about unsupported API. +```xml + + test.fs.s3a.list.v1.enabled + false + ``` +Note: there's no equivalent for turning off v2 listing API, which all stores are now +expected to support. + ### Testing Requester Pays @@ -697,6 +737,20 @@ 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. +### Disabling Content Encoding tests + +Tests in `ITestS3AContentEncoding` may need disabling +```xml + + test.fs.s3a.content.encoding.enabled + false + +``` +### Tests which may fail (and which you can ignore) + +* `ITestS3AContractMultipartUploader` tests `testMultipartUploadAbort` and `testSingleUpload` raising `FileNotFoundException` +* `ITestS3AMiscOperations.testEmptyFileChecksums`: if the FS encrypts data always. + ## Debugging Test failures Logging at debug level is the standard way to provide more diagnostics output; @@ -943,7 +997,7 @@ sequential one afterwards. The IO heavy ones must also be subclasses of `S3AScaleTestBase` and so only run if the system/maven property `fs.s3a.scale.test.enabled` is true. -## Individual test cases can be run in an IDE +### Individual test cases can be run in an IDE This is invaluable for debugging test failures. @@ -1002,71 +1056,19 @@ using an absolute XInclude reference to it. ## Failure Injection -**Warning do not enable any type of failure injection in production. The -following settings are for testing only.** - -One of the challenges with S3A integration tests was the fact that S3 was an -eventually-consistent storage system. To simulate inconsistencies more -frequently than they would normally surface, S3A supports a shim layer on top of the `AmazonS3Client` -class which artificially delays certain paths from appearing in listings. -This is implemented in the class `InconsistentAmazonS3Client`. +S3A provides an "Inconsistent S3 Client Factory" that can be used to +simulate throttling by injecting random failures on S3 client requests. -Now that S3 is consistent, injecting inconsistency is no longer needed -during testing. -However, it is stil useful to use the other feature of the client: -throttling simulation. -## Simulating List Inconsistencies +**Note** -### Enabling the InconsistentAmazonS3CClient +In previous releases, this factory could also be used to simulate +inconsistencies during testing of S3Guard. Now that S3 is consistent, +injecting inconsistency is no longer needed during testing. -To enable the fault-injecting client via configuration, switch the -S3A client to use the "Inconsistent S3 Client Factory" when connecting to -S3: - -```xml - - fs.s3a.s3.client.factory.impl - org.apache.hadoop.fs.s3a.InconsistentS3ClientFactory - -``` -The inconsistent client will, on every AWS SDK request, -generate a random number, and if less than the probability, -raise a 503 exception. - -```xml - - - fs.s3a.failinject.throttle.probability - 0.05 - -``` - -These exceptions are returned to S3; they do not test the -AWS SDK retry logic. - - -### Using the `InconsistentAmazonS3CClient` in downstream integration tests - -The inconsistent client is shipped in the `hadoop-aws` JAR, so it can -be used in integration tests. - -## Testing S3Guard - -As part of the removal of S3Guard from the production code, the tests have been updated -so that - -* All S3Guard-specific tests have been deleted. -* All tests parameterized on S3Guard settings have had those test configurations removed. -* The maven profiles option to run tests with S3Guard have been removed. - -There is no need to test S3Guard -and so tests are lot faster. -(We developers are all happy) - - -## Testing Assumed Roles +## Testing Assumed Roles Tests for the AWS Assumed Role credential provider require an assumed role to request. @@ -1083,7 +1085,7 @@ The specific tests an Assumed Role ARN is required for are 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 KMS for any SSE-KMS tests. +the S3 bucket used in the tests, and KMS for any SSE-KMS or DSSE-KMS tests. 1. Your IAM User to have the permissions to "assume" that role. @@ -1287,10 +1289,13 @@ time bin/hadoop fs -copyToLocal -t 10 $BUCKET/\*aws\* tmp # --------------------------------------------------- # S3 Select on Landsat +# this will fail with a ClassNotFoundException unless +# eventstore JAR is added to the classpath # --------------------------------------------------- export LANDSATGZ=s3a://landsat-pds/scene_list.gz + bin/hadoop s3guard select -header use -compression gzip $LANDSATGZ \ "SELECT s.entityId,s.cloudCover FROM S3OBJECT s WHERE s.cloudCover < '0.0' LIMIT 100" diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md new file mode 100644 index 0000000000000..0216e46014c7e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md @@ -0,0 +1,468 @@ + + +# Working with Third-party S3 Stores + +The S3A connector works well with third-party S3 stores if the following requirements are met: + +* It correctly implements the core S3 REST API, including support for uploads and the V2 listing API. +* The store supports the AWS V4 signing API *or* a custom signer is switched to. + This release does not support the legacy v2 signing API. +* Errors are reported with the same HTTPS status codes as the S3 store. Error messages do not + need to be consistent. +* The store is consistent. + +There are also specific deployment requirements: +* The clock on the store and the client are close enough that signing works. +* The client is correctly configured to connect to the store *and not use unavailable features* +* If HTTPS authentication is used, the client/JVM TLS configurations allows it to authenticate the endpoint. + +The features which may be unavailable include: + +* Checksum-based server-side change detection during copy/read (`fs.s3a.change.detection.mode=server`) +* Object versioning and version-based change detection (`fs.s3a.change.detection.source=versionid` and `fs.s3a.versioned.store=true`) +* Bulk delete (`fs.s3a.multiobjectdelete.enable=true`) +* Encryption. (`fs.s3a.encryption.algorithm`) +* Storage class set in `fs.s3a.create.storage.class` +* Content encodings as set with `fs.s3a.object.content.encoding`. +* Optional Bucket Probes at startup (`fs.s3a.bucket.probe = 0`). + This is now the default -do not change it. +* List API to use (`fs.s3a.list.version = 1`) +* Bucket lifecycle rules to clean up pending uploads. + +## Configuring s3a to connect to a third party store + + +## Connecting to a third party object store over HTTPS + +The core setting for a third party store is to change the endpoint in `fs.s3a.endpoint`. + +This can be a URL or a hostname/hostname prefix +For third-party stores without virtual hostname support, providing the URL is straightforward; +path style access must also be enabled in `fs.s3a.path.style.access`. + +The v4 signing algorithm requires a region to be set in `fs.s3a.endpoint.region`. +A non-empty value is generally sufficient, though some deployments may require +a specific value. + +Finally, assuming the credential source is the normal access/secret key +then these must be set, either in XML or (preferred) in a JCEKS file. + +```xml + + + fs.s3a.endpoint + https://storeendpoint.example.com + + + + fs.s3a.path.style.access + true + + + + fs.s3a.endpoint.region + anything + + + + fs.s3a.access.key + 13324445 + + + + fs.s3a.secret.key + 4C6B906D-233E-4E56-BCEA-304CC73A14F8 + + +``` + +If per-bucket settings are used here, then third-party stores and credentials may be used alongside an AWS store. + + + +## Other issues + +### Coping without bucket lifecycle rules + +Not all third-party stores support bucket lifecycle rules to clean up buckets +of incomplete uploads. + +This can be addressed in two ways +* Command line: `hadoop s3guard uploads -abort -force \`. +* With `fs.s3a.multipart.purge` and a purge age set in `fs.s3a.multipart.purge.age` +* In rename/delete `fs.s3a.directory.operations.purge.uploads = true`. + +#### S3Guard uploads command + +This can be executed on a schedule, or manually + +``` +hadoop s3guard uploads -abort -force s3a://bucket/ +``` + +Consult the [S3Guard documentation](s3guard.html) for the full set of parameters. + +#### In startup: `fs.s3a.multipart.purge` + +This lists all uploads in a bucket when a filesystem is created and deletes +all of those above a certain age. + +This can hurt performance on a large bucket, as the purge scans the entire tree, +and is executed whenever a filesystem is created -which can happen many times during +hive, spark, distcp jobs. + +For this reason, this option may be deleted in future, however it has long been +available in the S3A client and so guaranteed to work across versions. + +#### During rename and delete: `fs.s3a.directory.operations.purge.uploads` + +When `fs.s3a.directory.operations.purge.uploads` is set, when a directory is renamed +or deleted, then in parallel with the delete an attempt is made to list +all pending uploads. +If there are any, they are aborted (sequentially). + +* This is disabled by default: it adds overhead and extra cost. +* Because it only applies to the directories being processed, directories which + are not renamed or deleted will retain all incomplete uploads. +* There is no age checking: all uploads will be aborted. +* If any other process is writing to the same directory tree, their operations +will be cancelled. + + +# Troubleshooting + +The most common problem when talking to third-party stores are + +1. The S3A client is still configured to talk to the AWS S3 endpoint. This leads to authentication failures and/or reports that the bucket is unknown. +2. Path access has not been enabled, the client is generating a host name for the target bucket and it does not exist. +3. Invalid authentication credentials. +4. JVM HTTPS settings include the certificates needed to negotiate a TLS connection with the store. + + +## How to improve troubleshooting + +### log more network info + +There are some very low level logs. +```properties +# Log all HTTP requests made; includes S3 interaction. This may +# include sensitive information such as account IDs in HTTP headers. +log4j.logger.software.amazon.awssdk.request=DEBUG + +# Turn on low level HTTP protocol debugging +log4j.logger.org.apache.http.wire=DEBUG + +# async client +log4j.logger.io.netty.handler.logging=DEBUG +log4j.logger.io.netty.handler.codec.http2.Http2FrameLogger=DEBUG +``` + +### Cut back on retries, shorten timeouts + +By default, there's a lot of retries going on in the AWS connector (which even retries on DNS failures) +and in the S3A code which invokes it. + +Normally this helps prevent long-lived jobs from failing due to a transient network problem, however +it means that when trying to debug connectivity problems, the commands can hang for a long time +as they keep trying to reconnect to ports which are never going to be available. + +```xml + + + fs.iostatistics.logging.level + info + + + + fs.s3a.bucket.nonexistent-bucket-example.attempts.maximum + 0 + + + + fs.s3a.bucket.nonexistent-bucket-example.retry.limit + 1 + + + + fs.s3a.bucket.nonexistent-bucket-example.connection.timeout + 500 + + + + fs.s3a.bucket.nonexistent-bucket-example.connection.establish.timeout + 500 + +``` +## Cloudstore's Storediag + +There's an external utility, [cloudstore](https://github.com/steveloughran/cloudstore) whose [storediag](https://github.com/steveloughran/cloudstore#command-storediag) exists to debug the connection settings to hadoop cloud storage. + +```bash +hadoop jar cloudstore-1.0.jar storediag s3a://nonexistent-bucket-example/ +``` + +The main reason it's not an ASF release is that it allows for a rapid release cycle, sometimes hours; if anyone doesn't trust +third-party code then they can download and build it themselves. + + +# Problems + +## S3A client still pointing at AWS endpoint + +This is the most common initial problem, as it happens by default. + +To fix, set `fs.s3a.endpoint` to the URL of the internal store. + +### `org.apache.hadoop.fs.s3a.UnknownStoreException: `s3a://nonexistent-bucket-example/': Bucket does not exist` + +Either the bucket doesn't exist, or the bucket does exist but the endpoint is still set to an AWS endpoint. + +``` +stat: `s3a://nonexistent-bucket-example/': Bucket does not exist +``` +The hadoop filesystem commands don't log stack traces on failure -adding this adds too much risk +of breaking scripts, and the output is very uninformative + +``` +stat: nonexistent-bucket-example: getS3Region on nonexistent-bucket-example: +software.amazon.awssdk.services.s3.model.S3Exception: null +(Service: S3, Status Code: 403, Request ID: X26NWV0RJ1697SXF, Extended Request ID: bqq0rRm5Bdwt1oHSfmWaDXTfSOXoYvNhQxkhjjNAOpxhRaDvWArKCFAdL2hDIzgec6nJk1BVpJE=):null +``` + +It is possible to turn on debugging + +``` +log4j.logger.org.apache.hadoop.fs.shell=DEBUG +``` + +After which useful stack traces are logged. + +``` +org.apache.hadoop.fs.s3a.UnknownStoreException: `s3a://nonexistent-bucket-example/': Bucket does not exist + at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$null$3(S3AFileSystem.java:1075) + at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:122) + at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$4(Invoker.java:376) + at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:468) + at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:372) + at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:347) + at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$getS3Region$4(S3AFileSystem.java:1039) + at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration(IOStatisticsBinding.java:543) + at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:524) + at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:445) + at org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2631) + at org.apache.hadoop.fs.s3a.S3AFileSystem.getS3Region(S3AFileSystem.java:1038) + at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:982) + at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:622) + at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3452) +``` + +### `S3Exception: null (Service: S3, Status Code: 403...` or `AccessDeniedException` + +* Endpoint is default +* Credentials were not issued by AWS . +* `fs.s3a.endpoint.region` unset. + +If the client doesn't have any AWS credentials (from hadoop settings, environment variables or elsewhere) then +the binding will fail even before the existence of the bucket can be probed for. + +```bash +hadoop fs -stat s3a://nonexistent-bucket-example +``` + + +``` +stat: nonexistent-bucket-example: getS3Region on nonexistent-bucket-example: +software.amazon.awssdk.services.s3.model.S3Exception: null (Service: S3, Status Code: 403, + Request ID: X26NWV0RJ1697SXF, Extended Request ID: bqq0rRm5Bdwt1oHSfmWaDXTfSOXoYvNhQxkhjjNAOpxhRaDvWArKCFAdL2hDIzgec6nJk1BVpJE=):null +``` + +Or with a more detailed stack trace: + +``` +java.nio.file.AccessDeniedException: nonexistent-bucket-example: getS3Region on nonexistent-bucket-example: software.amazon.awssdk.services.s3.model.S3Exception: null (Service: S3, Status Code: 403, Request ID: X26NWV0RJ1697SXF, Extended Request ID: bqq0rRm5Bdwt1oHSfmWaDXTfSOXoYvNhQxkhjjNAOpxhRaDvWArKCFAdL2hDIzgec6nJk1BVpJE=):null + at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:235) + at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:124) + at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$4(Invoker.java:376) + at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:468) + at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:372) + at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:347) + at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$getS3Region$4(S3AFileSystem.java:1039) + at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration(IOStatisticsBinding.java:543) + at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:524) + at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:445) + at org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2631) + at org.apache.hadoop.fs.s3a.S3AFileSystem.getS3Region(S3AFileSystem.java:1038) + at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:982) + at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:622) + at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3452) +``` + +## `Received an UnknownHostException when attempting to interact with a service` + + +### Hypothesis 1: Region set, but not endpoint + +The bucket `fs.s3a.endpoint.region` region setting is valid internally, but as the endpoint +is still AWS, this region is not recognised. +The S3A client's creation of an endpoint URL generates an unknown host. + +```xml + + fs.s3a.bucket.nonexistent-bucket-example.endpoint.region + internal + +``` + + +``` +ls: software.amazon.awssdk.core.exception.SdkClientException: + Received an UnknownHostException when attempting to interact with a service. + See cause for the exact endpoint that is failing to resolve. + If this is happening on an endpoint that previously worked, there may be + a network connectivity issue or your DNS cache could be storing endpoints for too long.: + nonexistent-bucket-example.s3.internal.amazonaws.com: nodename nor servname provided, or not known + + +``` + +### Hypothesis 2: region set, endpoint set, but `fs.s3a.path.style.access` is still set to `false` + +* The bucket `fs.s3a.endpoint.region` region setting is valid internally, +* and `fs.s3a.endpoint` is set to a hostname (not a URL). +* `fs.s3a.path.style.access` set to `false` + +``` +ls: software.amazon.awssdk.core.exception.SdkClientException: + Received an UnknownHostException when attempting to interact with a service. + See cause for the exact endpoint that is failing to resolve. + If this is happening on an endpoint that previously worked, there may be + a network connectivity issue or your DNS cache could be storing endpoints for too long.: + nonexistent-bucket-example.localhost: nodename nor servname provided, or not known +``` + +Fix: path style access + +```xml + + fs.s3a.bucket.nonexistent-bucket-example.path.style.access + true + +``` + +# Connecting to Google Cloud Storage through the S3A connector + +It *is* possible to connect to google cloud storage through the S3A connector. +However, Google provide their own [Cloud Storage connector](https://cloud.google.com/dataproc/docs/concepts/connectors/cloud-storage). +That is a well maintained Hadoop filesystem client which uses their XML API, +And except for some very unusual cases, that is the connector to use. + +When interacting with a GCS container through the S3A connector may make sense +* The installation doesn't have the gcs-connector JAR. +* The different credential mechanism may be convenient. +* There's a desired to use S3A Delegation Tokens to pass secrets with a job. +* There's a desire to use an external S3A extension (delegation tokens etc.) + +The S3A connector binding works through the Google Cloud [S3 Storage API](https://cloud.google.com/distributed-cloud/hosted/docs/ga/gdch/apis/storage-s3-rest-api), +which is a subset of the AWS API. + + +To get a compatible access and secret key, follow the instructions of +[Simple migration from Amazon S3 to Cloud Storage](https://cloud.google.com/storage/docs/aws-simple-migration#defaultproj). + +Here are the per-bucket setings for an example bucket "gcs-container" +in Google Cloud Storage. Note the multiobject delete option must be disabled; +this makes renaming and deleting significantly slower. + + +```xml + + + + fs.s3a.bucket.gcs-container.access.key + GOOG1EZ.... + + + + fs.s3a.bucket.gcs-container.secret.key + SECRETS + + + + fs.s3a.bucket.gcs-container.endpoint + https://storage.googleapis.com + + + + fs.s3a.bucket.gcs-container.bucket.probe + 0 + + + + fs.s3a.bucket.gcs-container.list.version + 1 + + + + fs.s3a.bucket.gcs-container.multiobjectdelete.enable + false + + + + fs.s3a.bucket.gcs-container.select.enabled + false + + + + fs.s3a.bucket.gcs-container.path.style.access + true + + + + fs.s3a.bucket.gcs-container.endpoint.region + dummy + + + +``` + +This is a very rarely used configuration -however, it can be done, possibly as a way to interact with Google Cloud Storage in a deployment +which lacks the GCS connector. + +It is also a way to regression test foundational S3A third-party store compatibility if you lack access to to any alternative. + +```xml + + + test.fs.s3a.encryption.enabled + false + + + fs.s3a.scale.test.csvfile + + + + test.fs.s3a.sts.enabled + false + + + test.fs.s3a.content.encoding.enabled + false + + +``` + +_Note_ If anyone is set up to test this reguarly, please let the hadoop developer team know if regressions do surface, +as it is not a common test configuration. +[] \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md index d3e7c7d806dcf..8c57db1faf4c1 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md @@ -70,14 +70,55 @@ These are Hadoop filesystem client classes, found in the `hadoop-aws` JAR. An exception reporting this class as missing means that this JAR is not on the classpath. -### `ClassNotFoundException: com.amazonaws.services.s3.AmazonS3Client` + +### `java.lang.NoClassDefFoundError: software/amazon/awssdk/services/s3/model/S3Exception` + +This is one of the first stack traces which can surface when trying to instantiate +an S3A filesystem instance without having the AWS V2 SDK `bundle.jar` on the classpath + +``` +java.lang.NoClassDefFoundError: software/amazon/awssdk/services/s3/model/S3Exception + at java.lang.ClassLoader.defineClass1(Native Method) + at java.lang.ClassLoader.defineClass(ClassLoader.java:756) + at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) + at java.net.URLClassLoader.defineClass(URLClassLoader.java:473) + at java.net.URLClassLoader.access$100(URLClassLoader.java:74) + at java.net.URLClassLoader$1.run(URLClassLoader.java:369) + at java.net.URLClassLoader$1.run(URLClassLoader.java:363) + at java.security.AccessController.doPrivileged(Native Method) + at java.net.URLClassLoader.findClass(URLClassLoader.java:362) + at java.lang.ClassLoader.loadClass(ClassLoader.java:418) + at java.lang.ClassLoader.loadClass(ClassLoader.java:351) + at java.lang.Class.forName0(Native Method) + at java.lang.Class.forName(Class.java:348) + at org.apache.hadoop.conf.Configuration.getClassByNameOrNull(Configuration.java:2639) + at org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2604) + at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2700) + at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:3414) + at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3449) + at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:162) +``` + +Fix: add it to classpath. + +Maven/Ivy/SBT/Gradle builds which import `hadoop-aws` or +`hadoop-cloud-storage` artifacts should get the artifact automatically. + +### `ClassNotFoundException: com.amazonaws.auth.AWSCredentials` (or other `com.amazonaws` class.) -This means that the `aws-java-sdk-bundle.jar` JAR is not on the classpath: -add it. +With the move to the [V2 AWS SDK](../aws_sdk_upgrade.html), +the v1 SDK classes are no longer on the classpath. -### `java.lang.NoSuchMethodError` referencing a `com.amazonaws` class +If this happens when trying to use a custom credential provider defined +in `fs.s3a.aws.credentials.provider`, then add the `aws-sdk-bundle.jar` +JAR to the classpath. + +If this happens in your own/third-party code, then again, add the JAR, +and/or consider moving to the v2 sdk yourself. + +### `java.lang.NoSuchMethodError` referencing a `software.amazon` class This can be triggered by incompatibilities between the AWS SDK on the classpath and the version which Hadoop was compiled with. @@ -86,12 +127,14 @@ The AWS SDK JARs change their signature enough between releases that the only way to safely update the AWS SDK version is to recompile Hadoop against the later version. -The sole fix is to use the same version of the AWS SDK with which Hadoop +The fix is to use the same version of the AWS SDK with which Hadoop was built. This can also be caused by having more than one version of an AWS SDK -JAR on the classpath. If the full `aws-java-sdk-bundle<` JAR is on the -classpath, do not add any of the `aws-sdk-` JARs. +JAR on the classpath. If the full `bundle.jar` JAR is on the +classpath, do not add any of the `aws-sdk-` JARs *except* for +`aws-crt.jar` (which is required) and +`eventstream.jar` which is required when using S3 Select. ### `java.lang.NoSuchMethodError` referencing an `org.apache.hadoop` class @@ -119,7 +162,6 @@ the client retries a number of times before eventually failing. When it finally gives up, it will report a message about signature mismatch: ``` -com.amazonaws.services.s3.model.AmazonS3Exception: The request signature we calculated does not match the signature you provided. Check your key and signing method. (Service: Amazon S3; Status Code: 403; Error Code: SignatureDoesNotMatch, @@ -196,53 +238,18 @@ read requests are allowed, but operations which write to the bucket are denied. Check the system clock. -### "Bad Request" exception when working with AWS S3 Frankfurt, Seoul, or other "V4" endpoint +### "Bad Request" exception when working with data stores in an AWS region other than us-eaast -S3 Frankfurt and Seoul *only* support -[the V4 authentication API](http://docs.aws.amazon.com/AmazonS3/latest/API/sig-v4-authenticating-requests.html). - -Requests using the V2 API will be rejected with 400 `Bad Request` ``` $ bin/hadoop fs -ls s3a://frankfurt/ WARN s3a.S3AFileSystem: Client: Amazon S3 error 400: 400 Bad Request; Bad Request (retryable) -com.amazonaws.services.s3.model.AmazonS3Exception: Bad Request (Service: Amazon S3; - Status Code: 400; Error Code: 400 Bad Request; Request ID: 923C5D9E75E44C06), - S3 Extended Request ID: HDwje6k+ANEeDsM6aJ8+D5gUmNAMguOk2BvZ8PH3g9z0gpH+IuwT7N19oQOnIr5CIx7Vqb/uThE= - at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182) - at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770) - at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) - at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) - at com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1107) - at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1070) - at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:307) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:284) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2793) - at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:101) - at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2830) - at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2812) - at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:389) - at org.apache.hadoop.fs.Path.getFileSystem(Path.java:356) - at org.apache.hadoop.fs.shell.PathData.expandAsGlob(PathData.java:325) - at org.apache.hadoop.fs.shell.Command.expandArgument(Command.java:235) - at org.apache.hadoop.fs.shell.Command.expandArguments(Command.java:218) - at org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:103) - at org.apache.hadoop.fs.shell.Command.run(Command.java:165) - at org.apache.hadoop.fs.FsShell.run(FsShell.java:315) - at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) - at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90) - at org.apache.hadoop.fs.FsShell.main(FsShell.java:373) -ls: doesBucketExist on frankfurt-new: com.amazonaws.services.s3.model.AmazonS3Exception: +ls: doesBucketExist on frankfurt-new: S3Exception: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; ``` -This happens when trying to work with any S3 service which only supports the -"V4" signing API —but the client is configured to use the default S3 service -endpoint. - The S3A client needs to be given the endpoint to use via the `fs.s3a.endpoint` property. @@ -258,10 +265,10 @@ As an example, the endpoint for S3 Frankfurt is `s3.eu-central-1.amazonaws.com`: When [PrivateLink](https://docs.aws.amazon.com/AmazonS3/latest/userguide/privatelink-interface-endpoints.html) URL is used instead of standard s3a endpoint, it returns "authorization -header is malformed" exception. So, if we set fs.s3a.endpoint=bucket.vpce --.s3.ca-central-1.vpce.amazonaws.com and make s3 calls we get: +header is malformed" exception. So, if we set `fs.s3a.endpoint=bucket.vpce +-.s3.ca-central-1.vpce.amazonaws.com` and make s3 calls we get: ``` -com.amazonaws.services.s3.model.AmazonS3Exception: The authorization header is malformed; the region 'vpce' is wrong; expecting 'ca-central-1' +S3Exception: The authorization header is malformed; the region 'vpce' is wrong; expecting 'ca-central-1' (Service: Amazon S3; Status Code: 400; Error Code: AuthorizationHeaderMalformed; Request ID: req-id; S3 Extended Request ID: req-id-2), S3 Extended Request ID: req-id-2:AuthorizationHeaderMalformed: The authorization header is malformed; the region 'vpce' is wrong; expecting 'ca-central-1' (Service: Amazon S3; Status Code: 400; Error Code: AuthorizationHeaderMalformed; Request ID: req-id; ``` @@ -281,35 +288,27 @@ S3 region as `ca-central-1`. ``` -### `Class does not implement AWSCredentialsProvider` +### `Classdoes not implement software.amazon.awssdk.auth.credentials.AwsCredentialsProvider` A credential provider listed in `fs.s3a.aws.credentials.provider` does not implement -the interface `com.amazonaws.auth.AWSCredentialsProvider`. - -``` - Cause: java.lang.RuntimeException: java.io.IOException: Class class com.amazonaws.auth.EnvironmentVariableCredentialsProvider does not implement AWSCredentialsProvider - at org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:686) - at org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:621) - at org.apache.spark.sql.hive.client.HiveClientImpl.newState(HiveClientImpl.scala:219) - at org.apache.spark.sql.hive.client.HiveClientImpl.(HiveClientImpl.scala:126) - 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.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:306) - at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:433) - ... - Cause: java.io.IOException: Class class com.amazonaws.auth.EnvironmentVariableCredentialsProvider does not implement AWSCredentialsProvider - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProvider(S3AUtils.java:722) - at org.apache.hadoop.fs.s3a.S3AUtils.buildAWSProviderList(S3AUtils.java:687) - at org.apache.hadoop.fs.s3a.S3AUtils.createAWSCredentialProviderSet(S3AUtils.java:620) - at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:673) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:414) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3462) - at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:171) - at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3522) - at org.apache.hadoop.fs.FileSystem$Cache.getUnique(FileSystem.java:3496) - at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:591) +the interface `software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`. + +``` +InstantiationIOException: `s3a://stevel-gcs/': Class org.apache.hadoop.fs.s3a.S3ARetryPolicy does not implement software.amazon.awssdk.auth.credentials.AwsCredentialsProvider (configuration key fs.s3a.aws.credentials.provider) + at org.apache.hadoop.fs.s3a.impl.InstantiationIOException.isNotInstanceOf(InstantiationIOException.java:128) + at org.apache.hadoop.fs.s3a.S3AUtils.getInstanceFromReflection(S3AUtils.java:604) + at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSV2CredentialProvider(CredentialProviderListFactory.java:299) + at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.buildAWSProviderList(CredentialProviderListFactory.java:245) + at org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSCredentialProviderList(CredentialProviderListFactory.java:144) + at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:971) + at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:624) + at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3601) + at org.apache.hadoop.fs.FileSystem.access$300(FileSystem.java:171) + at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3702) + at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3653) + at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:555) + at org.apache.hadoop.fs.Path.getFileSystem(Path.java:366) + ``` There's two main causes @@ -324,12 +323,12 @@ There's two main causes If you see this and you are trying to use the S3A connector with Spark, then the cause can be that the isolated classloader used to load Hive classes is interfering with the S3A -connector's dynamic loading of `com.amazonaws` classes. To fix this, declare that that +connector's dynamic loading of `software.amazon.awssdk` classes. To fix this, declare that the classes in the aws SDK are loaded from the same classloader which instantiated the S3A FileSystem instance: ``` -spark.sql.hive.metastore.sharedPrefixes com.amazonaws. +spark.sql.hive.metastore.sharedPrefixes software.amazon.awssdk. ``` ## "The security token included in the request is invalid" @@ -350,59 +349,8 @@ It may be mistyped, or the access key may have been deleted by one of the accoun ``` java.nio.file.AccessDeniedException: bucket: doesBucketExist on bucket: - com.amazonaws.services.s3.model.AmazonS3Exception: + S3Exception: The AWS Access Key Id you provided does not exist in our records. - (Service: Amazon S3; Status Code: 403; Error Code: InvalidAccessKeyId; - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:214) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111) - at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:260) - at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:314) - at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:256) - at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:231) - at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:366) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:302) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354) - at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:124) - at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3403) - at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3371) - at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:477) - at org.apache.hadoop.fs.contract.AbstractBondedFSContract.init(AbstractBondedFSContract.java:72) - at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.setup(AbstractFSContractTestBase.java:177) - at org.apache.hadoop.fs.s3a.commit.AbstractCommitITest.setup(AbstractCommitITest.java:163) - at org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob.setup(AbstractITCommitMRJob.java:129) - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) - at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) - at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.lang.reflect.Method.invoke(Method.java:498) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:24) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48) - at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55) - at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74) -Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: - The AWS Access Key Id you provided does not exist in our records. - (Service: Amazon S3; Status Code: 403; Error Code: InvalidAccessKeyId; - 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.getAcl(AmazonS3Client.java:3381) - at com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1160) - at com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1150) - at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1266) - at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$verifyBucketExists$1(S3AFileSystem.java:367) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109) - ... 27 more ``` @@ -412,47 +360,12 @@ Caller has no permission to access the bucket at all. ``` doesBucketExist on fdsd: java.nio.file.AccessDeniedException: fdsd: doesBucketExist on fdsd: - com.amazonaws.services.s3.model.AmazonS3Exception: All access to this object has been disabled + S3Exception: All access to this object has been disabled (Service: Amazon S3; Status Code: 403; Error Code: AllAccessDisabled; Request ID: E6229D7F8134E64F; S3 Extended Request ID: 6SzVz2t4qa8J2Wxo/oc8yBuB13Mgrn9uMKnxVY0hsBd2kU/YdHzW1IaujpJdDXRDCQRX3f1RYn0=), S3 Extended Request ID: 6SzVz2t4qa8J2Wxo/oc8yBuB13Mgrn9uMKnxVY0hsBd2kU/YdHzW1IaujpJdDXRDCQRX3f1RYn0=:AllAccessDisabled All access to this object has been disabled (Service: Amazon S3; Status Code: 403; - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:205) - at org.apache.hadoop.fs.s3a.S3ALambda.once(S3ALambda.java:122) - at org.apache.hadoop.fs.s3a.S3ALambda.lambda$retry$2(S3ALambda.java:233) - at org.apache.hadoop.fs.s3a.S3ALambda.retryUntranslated(S3ALambda.java:288) - at org.apache.hadoop.fs.s3a.S3ALambda.retry(S3ALambda.java:228) - at org.apache.hadoop.fs.s3a.S3ALambda.retry(S3ALambda.java:203) - at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:357) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:293) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3288) - at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:123) - at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3337) - at org.apache.hadoop.fs.FileSystem$Cache.getUnique(FileSystem.java:3311) - at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:529) - at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool$BucketInfo.run(S3GuardTool.java:997) - at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.run(S3GuardTool.java:309) - at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) - at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.run(S3GuardTool.java:1218) - at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.main(S3GuardTool.java:1227) -Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: All access to this object has been disabled - 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.getAcl(AmazonS3Client.java:3381) - at com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1160) - at com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1150) - at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1266) - at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$verifyBucketExists$1(S3AFileSystem.java:360) - at org.apache.hadoop.fs.s3a.S3ALambda.once(S3ALambda.java:120) + ``` Check the name of the bucket is correct, and validate permissions for the active user/role. @@ -467,19 +380,9 @@ or the caller does not have the right to access the data. ``` java.nio.file.AccessDeniedException: test/: PUT 0-byte object on test/: - com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403; + S3Exception: Access Denied (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: EDC662AD2EEEA33C; - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:210) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:110) - at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:259) - at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:313) - at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:255) - at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:230) - at org.apache.hadoop.fs.s3a.S3AFileSystem.createEmptyObject(S3AFileSystem.java:2691) - at org.apache.hadoop.fs.s3a.S3AFileSystem.createFakeDirectory(S3AFileSystem.java:2666) - at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:2030) - at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1965) - at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2305) + ``` In the AWS S3 management console, select the "permissions" tab for the bucket, then "bucket policy". @@ -506,7 +409,7 @@ _all_ the rights which the caller has. ``` mv: rename s3a://london/dest to s3a://london/src on s3a://london/dest: - com.amazonaws.services.s3.model.MultiObjectDeleteException: One or more objects + MultiObjectDeleteException: One or more objects could not be deleted (Service: null; Status Code: 200; Error Code: null; Request ID: 5C9018EF245F02C5; S3 Extended Request ID: 5fQ2RVCPF0rdvADRv2XY3U4yb2J0gHRID/4jm1eqCXp7RxpU0dH9DliChYsCUD1aVCFtbwfWJWY=), @@ -528,7 +431,7 @@ directory will _only_ be in the destination. And files for which the rename oper had yet to commence -they will only be in the source tree. The user has to recover from this themselves. Be assured: no data will have been deleted, it -is just that the data may now be scattered across two directories. +is just that the data may now be scattered across two directories. Note: this is one reason why any application which tries to atomically commit work via rename (classic Hadoop output committers, distcp with the `-atomic` option) are not safe to use with S3. It is not a file system. @@ -556,7 +459,7 @@ If you don't enable this acknowledgement within S3A, then you will see a message ``` java.nio.file.AccessDeniedException: s3a://my-bucket/my-object: getFileStatus on s3a://my-bucket/my-object: -com.amazonaws.services.s3.model.AmazonS3Exception: Forbidden (Service: Amazon S3; Status Code: 403; +S3Exception: Forbidden (Service: Amazon S3; Status Code: 403; Error Code: 403 Forbidden; Request ID: myshortreqid; S3 Extended Request ID: mylongreqid):403 Forbidden ``` @@ -565,9 +468,9 @@ To enable requester pays, set `fs.s3a.requester.pays.enabled` property to `true` ### `AccessDeniedException` "InvalidObjectState" when trying to read files ``` -java.nio.file.AccessDeniedException: file1: copyFile(file1, file2) on file1: com.amazonaws.services.s3.model.AmazonS3Exception: Operation is not valid for the source object's storage class (Service: Amazon S3; Status Code: 403; Error Code: InvalidObjectState; Request ID: SK9EMPC1YRX75VZR; S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw=; Proxy: null), S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw=:InvalidObjectState +java.nio.file.AccessDeniedException: file1: copyFile(file1, file2) on file1: S3Exception: Operation is not valid for the source object's storage class (Service: Amazon S3; Status Code: 403; Error Code: InvalidObjectState; Request ID: SK9EMPC1YRX75VZR; S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw=; Proxy: null), S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw=:InvalidObjectState -Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Operation is not valid for the source object's storage class (Service: Amazon S3; Status Code: 403; Error Code: InvalidObjectState; Request ID: SK9EMPC1YRX75VZR; S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw=; Proxy: null), S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw= +Caused by: S3Exception: Operation is not valid for the source object's storage class (Service: Amazon S3; Status Code: 403; Error Code: InvalidObjectState; Request ID: SK9EMPC1YRX75VZR; S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw=; Proxy: null), S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw= ``` This happens when you're trying to read or copy files that have archive storage class such as @@ -581,7 +484,7 @@ Region must be provided when requesting session credentials, or an exception wil message: ``` -com.amazonaws.SdkClientException: Unable to find a region via the region provider + Unable to find a region via the region provider chain. Must provide an explicit region in the builder or setup environment to supply a region. ``` @@ -591,12 +494,13 @@ endpoint and region like the following: ```xml - fs.s3a.assumed.role.sts.endpoint - ${sts.endpoint} + fs.s3a.assumed.role.sts.endpoint + ${sts.endpoint} + -fs.s3a.assumed.role.sts.endpoint.region -${sts.region} + fs.s3a.assumed.role.sts.endpoint.region + ${sts.region} ``` @@ -639,8 +543,7 @@ can be used: ``` -Using the explicit endpoint for the region is recommended for speed and -to use the V4 signing API. +Using the explicit endpoint for the region is recommended for speed. ### `Unable to find a region via the region provider chain` @@ -657,15 +560,9 @@ This failure surfaces when _all_ the following conditions are met: Stack trace (Hadoop 3.3.1): ``` -Caused by: com.amazonaws.SdkClientException: Unable to find a region via the region provider chain. +Unable to find a region via the region provider chain. Must provide an explicit region in the builder or setup environment to supply a region. - at com.amazonaws.client.builder.AwsClientBuilder.setRegion(AwsClientBuilder.java:462) - at com.amazonaws.client.builder.AwsClientBuilder.configureMutableProperties(AwsClientBuilder.java:424) - at com.amazonaws.client.builder.AwsSyncClientBuilder.build(AwsSyncClientBuilder.java:46) - at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.buildAmazonS3Client(DefaultS3ClientFactory.java:145) - at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:97) - at org.apache.hadoop.fs.s3a.S3AFileSystem.bindAWSClient(S3AFileSystem.java:788) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:478) + ``` Log and stack trace on later releases, with @@ -692,14 +589,7 @@ warning that the SDK resolution chain is in use: at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3565) at org.apache.hadoop.fs.FileSystem$Cache.getUnique(FileSystem.java:3518) at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:592) -Caused by: com.amazonaws.SdkClientException: Unable to find a region via the region provider chain. - Must provide an explicit region in the builder or setup environment to supply a region. - at com.amazonaws.client.builder.AwsClientBuilder.setRegion(AwsClientBuilder.java:462) - at com.amazonaws.client.builder.AwsClientBuilder.configureMutableProperties(AwsClientBuilder.java:424) - at com.amazonaws.client.builder.AwsSyncClientBuilder.build(AwsSyncClientBuilder.java:46) - at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.buildAmazonS3Client(DefaultS3ClientFactory.java:185) - at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:117) - ... 21 more + ``` Due to changes in S3 client construction in Hadoop 3.3.1 this option surfaces in @@ -745,33 +635,9 @@ This happens when using the output stream thread pool runs out of capacity. ``` [s3a-transfer-shared-pool1-t20] INFO http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: - Timeout waiting for connection from poolorg.apache.http.conn.ConnectionPoolTimeoutException: + org.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool - at org.apache.http.impl.conn.PoolingClientConnectionManager.leaseConnection(PoolingClientConnectionManager.java:230) - at org.apache.http.impl.conn.PoolingClientConnectionManager$1.getConnection(PoolingClientConnectionManager.java:199) - at sun.reflect.GeneratedMethodAccessor13.invoke(Unknown Source) - at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.lang.reflect.Method.invoke(Method.java:498) - at com.amazonaws.http.conn.ClientConnectionRequestFactory$Handler.invoke(ClientConnectionRequestFactory.java:70) - at com.amazonaws.http.conn.$Proxy10.getConnection(Unknown Source) - at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:424) - at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884) - at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82) - at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55) - at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728) - at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) - at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) - at com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:2921) - at com.amazonaws.services.s3.AmazonS3Client.uploadPart(AmazonS3Client.java:2906) - at org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:1025) - at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:360) - at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:355) - at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) - at java.lang.Thread.run(Thread.java:745) + ``` Make sure that `fs.s3a.connection.maximum` is at least larger @@ -780,12 +646,12 @@ than `fs.s3a.threads.max`. ```xml fs.s3a.threads.max - 20 + 64 fs.s3a.connection.maximum - 30 + 64 ``` @@ -803,39 +669,15 @@ Set `fs.s3a.connection.maximum` to a larger value (and at least as large as The HTTP Server did not respond. ``` -2017-02-07 10:01:07,950 INFO [s3a-transfer-shared-pool1-t7] com.amazonaws.http.AmazonHttpClient: - Unable to execute HTTP request: bucket.s3.amazonaws.com:443 failed to respond +2017-02-07 10:01:07,950 INFO [s3a-transfer-shared-pool1-t7] Unable to execute HTTP request: bucket.s3.amazonaws.com:443 failed to respond org.apache.http.NoHttpResponseException: bucket.s3.amazonaws.com:443 failed to respond - at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:143) - at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:57) - at org.apache.http.impl.io.AbstractMessageParser.parse(AbstractMessageParser.java:261) - at org.apache.http.impl.AbstractHttpClientConnection.receiveResponseHeader(AbstractHttpClientConnection.java:283) - at org.apache.http.impl.conn.DefaultClientConnection.receiveResponseHeader(DefaultClientConnection.java:259) - at org.apache.http.impl.conn.ManagedClientConnectionImpl.receiveResponseHeader(ManagedClientConnectionImpl.java:209) - at org.apache.http.protocol.HttpRequestExecutor.doReceiveResponse(HttpRequestExecutor.java:272) - at com.amazonaws.http.protocol.SdkHttpRequestExecutor.doReceiveResponse(SdkHttpRequestExecutor.java:66) - at org.apache.http.protocol.HttpRequestExecutor.execute(HttpRequestExecutor.java:124) - at org.apache.http.impl.client.DefaultRequestDirector.tryExecute(DefaultRequestDirector.java:686) - at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:488) - at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884) - at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82) - at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55) - at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728) - at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) - at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) - at com.amazonaws.services.s3.AmazonS3Client.copyPart(AmazonS3Client.java:1731) - at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:41) - at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:28) - at org.apache.hadoop.fs.s3a.SemaphoredDelegatingExecutor$CallableWithPermitRelease.call(SemaphoredDelegatingExecutor.java:222) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) - at java.lang.Thread.run(Thread.java:745) + ``` Probably network problems, unless it really is an outage of S3. +If you are working with a third party store, check its network configuration. + ### Out of heap memory when writing with via Fast Upload @@ -889,19 +731,8 @@ for up to date advice. ``` org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on test/testname/streaming/: - com.amazonaws.AmazonClientException: Failed to sanitize XML document - destined for handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler: - Failed to sanitize XML document destined for handler class - com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:105) - at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1462) - at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListStatus(S3AFileSystem.java:1227) - at org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:1203) - at org.apache.hadoop.fs.s3a.S3AGlobber.listStatus(S3AGlobber.java:69) - at org.apache.hadoop.fs.s3a.S3AGlobber.doGlob(S3AGlobber.java:210) - at org.apache.hadoop.fs.s3a.S3AGlobber.glob(S3AGlobber.java:125) - at org.apache.hadoop.fs.s3a.S3AFileSystem.globStatus(S3AFileSystem.java:1853) - at org.apache.hadoop.fs.s3a.S3AFileSystem.globStatus(S3AFileSystem.java:1841) + Failed to sanitize XML document + ``` We believe this is caused by the connection to S3 being broken. @@ -911,7 +742,7 @@ It may go away if the operation is retried. ### JSON Parse Error from AWS SDK -Sometimes a JSON Parse error is reported with the stack trace in the `com.amazonaws`, +Sometimes a JSON Parse error is reported with the stack trace from `software.amazon.awssdk`, Again, we believe this is caused by the connection to S3 being broken. @@ -998,18 +829,7 @@ from breaking. org.apache.hadoop.fs.s3a.RemoteFileChangedException: re-open `s3a://my-bucket/test/file.txt': Change reported by S3 while reading at position 1949. ETag f9c186d787d4de9657e99f280ba26555 was unavailable - at org.apache.hadoop.fs.s3a.impl.ChangeTracker.processResponse(ChangeTracker.java:137) - at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:200) - at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$lazySeek$1(S3AInputStream.java:346) - at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$2(Invoker.java:195) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109) - at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265) - at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:322) - at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:261) - at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:193) - at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:215) - at org.apache.hadoop.fs.s3a.S3AInputStream.lazySeek(S3AInputStream.java:339) - at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:372) + ``` If an S3 object is updated while an S3A filesystem reader has an open @@ -1032,17 +852,7 @@ the following error. org.apache.hadoop.fs.s3a.NoVersionAttributeException: `s3a://my-bucket/test/file.txt': Change detection policy requires ETag at org.apache.hadoop.fs.s3a.impl.ChangeTracker.processResponse(ChangeTracker.java:153) - at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:200) - at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$lazySeek$1(S3AInputStream.java:346) - at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$2(Invoker.java:195) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109) - at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265) - at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:322) - at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:261) - at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:193) - at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:215) - at org.apache.hadoop.fs.s3a.S3AInputStream.lazySeek(S3AInputStream.java:339) - at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:372) + ``` If the change policy is `versionid` there are a number of possible causes @@ -1100,51 +910,11 @@ key arn is invalid. ``` org.apache.hadoop.fs.s3a.AWSS3IOException: innerMkdirs on /test: - com.amazonaws.services.s3.model.AmazonS3Exception: + S3Exception: Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: CA89F276B3394565), S3 Extended Request ID: ncz0LWn8zor1cUO2fQ7gc5eyqOk3YfyQLDn2OQNoe5Zj/GqDLggUYz9QY7JhdZHdBaDTh+TL5ZQ=: Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: CA89F276B3394565) - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:194) - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:117) - at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1541) - at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2230) - at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.mkdirs(AbstractFSContractTestBase.java:338) - at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.setup(AbstractFSContractTestBase.java:193) - at org.apache.hadoop.fs.s3a.scale.S3AScaleTestBase.setup(S3AScaleTestBase.java:90) - at org.apache.hadoop.fs.s3a.scale.AbstractSTestS3AHugeFiles.setup(AbstractSTestS3AHugeFiles.java:77) - at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source) - at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) - at java.lang.reflect.Method.invoke(Method.java:498) - at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47) - at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) - at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44) - at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:24) - at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) - at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55) - at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74) -Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: - Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: CA89F276B3394565) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1588) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1258) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1030) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:742) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:716) - 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:4221) - at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4168) - at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1718) - at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:133) - at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:125) - at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:143) - at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:48) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) - at java.lang.Thread.run(Thread.java:745) ``` Possible causes: @@ -1153,14 +923,13 @@ Possible causes: * the KMS key referenced by the ARN is in a different region than the S3 bucket being used. - ### Using SSE-C "Bad Request" When performing file operations the user may run into an unexpected 400/403 error such as ``` org.apache.hadoop.fs.s3a.AWSS3IOException: getFileStatus on fork-4/: - com.amazonaws.services.s3.model.AmazonS3Exception: + S3Exception: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: 42F9A1987CB49A99), S3 Extended Request ID: jU2kcwaXnWj5APB14Cgb1IKkc449gu2+dhIsW/+7x9J4D+VUkKvu78mBo03oh9jnOT2eoTLdECU=: @@ -1181,40 +950,7 @@ file using configured SSE-C keyB into that structure. Reading an unencrypted file would fail when read through CSE enabled client. ``` java.lang.SecurityException: Instruction file not found for S3 object with bucket name: ap-south-cse, key: unencryptedData.txt - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleAE.decipher(S3CryptoModuleAE.java:190) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleAE.getObjectSecurely(S3CryptoModuleAE.java:136) - at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.getObject(AmazonS3EncryptionClientV2.java:241) - at org.apache.hadoop.fs.s3a.S3AFileSystem$InputStreamCallbacksImpl.getObject(S3AFileSystem.java:1462) - at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$reopen$0(S3AInputStream.java:217) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117) - at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:216) - at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$lazySeek$1(S3AInputStream.java:382) - at org.apache.hadoop.fs.s3a.Invoker.lambda$maybeRetry$3(Invoker.java:230) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117) - at org.apache.hadoop.fs.s3a.Invoker.lambda$maybeRetry$5(Invoker.java:354) - at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:414) - at org.apache.hadoop.fs.s3a.Invoker.maybeRetry(Invoker.java:350) - at org.apache.hadoop.fs.s3a.Invoker.maybeRetry(Invoker.java:228) - at org.apache.hadoop.fs.s3a.Invoker.maybeRetry(Invoker.java:272) - at org.apache.hadoop.fs.s3a.S3AInputStream.lazySeek(S3AInputStream.java:374) - at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:493) - at java.io.DataInputStream.read(DataInputStream.java:100) - at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:94) - at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:68) - at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:129) - at org.apache.hadoop.fs.shell.Display$Cat.printToStdout(Display.java:101) - at org.apache.hadoop.fs.shell.Display$Cat.processPath(Display.java:96) - at org.apache.hadoop.fs.shell.Command.processPathInternal(Command.java:370) - at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:333) - at org.apache.hadoop.fs.shell.Command.processPathArgument(Command.java:306) - at org.apache.hadoop.fs.shell.Command.processArgument(Command.java:288) - at org.apache.hadoop.fs.shell.Command.processArguments(Command.java:272) - at org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:121) - at org.apache.hadoop.fs.shell.Command.run(Command.java:179) - at org.apache.hadoop.fs.FsShell.run(FsShell.java:327) - at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:81) - at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:95) - at org.apache.hadoop.fs.FsShell.main(FsShell.java:390) + ``` CSE enabled client should read encrypted data only. @@ -1233,14 +969,14 @@ method requires KMS key ID. Use fs.s3a.encryption.key property to set it. set `fs.s3a.encryption.key=` generated through AWS console. -### `com.amazonaws.services.kms.model.IncorrectKeyException` The key ID in the request does not identify a CMK that can perform this operation. +### `software.amazon.awssdk.services.kms.model.IncorrectKeyException` The key ID in the request does not identify a CMK that can perform this operation. KMS key ID used to PUT(encrypt) the data, must be the one used to GET the data. ``` cat: open s3a://ap-south-cse/encryptedData.txt at 0 on s3a://ap-south-cse/encryptedData.txt: -com.amazonaws.services.kms.model.IncorrectKeyException: The key ID in the +software.amazon.awssdk.services.kms.model.IncorrectKeyException: The key ID in the request does not identify a CMK that can perform this operation. (Service: AWSKMS; Status Code: 400; ErrorCode: IncorrectKeyException; Request ID: da21aa8a-f00d-467c-94a0-32b627d32bc0; Proxy: null):IncorrectKeyException: @@ -1250,14 +986,14 @@ Request ID: da21aa8a-f00d-467c-94a0-32b627d32bc0; Proxy: null) ``` Use the same KMS key ID used to upload data to download and read it as well. -### `com.amazonaws.services.kms.model.NotFoundException` key/ does not exist +### `software.amazon.awssdk.services.kms.model.NotFoundException` key/ does not exist Using a KMS key ID from a different region than the bucket used to store data would lead to failure while uploading. ``` mkdir: PUT 0-byte object on testmkdir: -com.amazonaws.services.kms.model.NotFoundException: Key +software.amazon.awssdk.services.kms.model.NotFoundException: Key 'arn:aws:kms:ap-south-1:152813717728:key/' does not exist (Service: AWSKMS; Status Code: 400; Error Code: NotFoundException; Request ID: 279db85d-864d-4a38-9acd-d892adb504c0; Proxy: null):NotFoundException: @@ -1274,24 +1010,6 @@ If Range get is not supported for a CSE algorithm or is disabled: ``` java.lang.SecurityException: Unable to perform range get request: Range get support has been disabled. See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryptography.html - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleAE.assertCanGetPartialObject(S3CryptoModuleAE.java:446) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleAE.getObjectSecurely(S3CryptoModuleAE.java:117) - at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.getObject(AmazonS3EncryptionClientV2.java:241) - at org.apache.hadoop.fs.s3a.S3AFileSystem$InputStreamCallbacksImpl.getObject(S3AFileSystem.java:1462) - at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$reopen$0(S3AInputStream.java:217) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117) - at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:216) - at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$lazySeek$1(S3AInputStream.java:382) - at org.apache.hadoop.fs.s3a.Invoker.lambda$maybeRetry$3(Invoker.java:230) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117) - at org.apache.hadoop.fs.s3a.Invoker.lambda$maybeRetry$5(Invoker.java:354) - at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:414) - at org.apache.hadoop.fs.s3a.Invoker.maybeRetry(Invoker.java:350) - at org.apache.hadoop.fs.s3a.Invoker.maybeRetry(Invoker.java:228) - at org.apache.hadoop.fs.s3a.Invoker.maybeRetry(Invoker.java:272) - at org.apache.hadoop.fs.s3a.S3AInputStream.lazySeek(S3AInputStream.java:374) - at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:408) - at java.io.DataInputStream.readByte(DataInputStream.java:265) ``` Range gets must be enabled for CSE to work. @@ -1327,124 +1045,43 @@ enhance security. See https://docs.aws.amazon.com/general/latest/gr/aws_sdk_cryp We can ignore this, since this CryptoMode setting(CryptoMode.AuthenticatedEncryption) is required for range gets to work. -### com.amazonaws.services.kms.model.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK +### `software.amazon.awssdk.services.kms.mode.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK` If you generated an Asymmetric CMK from AWS console then CSE-KMS won't be able to generate unique data key for encryption. ``` -Caused by: com.amazonaws.services.kms.model.InvalidKeyUsageException: +Caused by: software.amazon.awssdk.services.kms.mode.InvalidKeyUsageException: You cannot generate a data key with an asymmetric CMK (Service: AWSKMS; Status Code: 400; Error Code: InvalidKeyUsageException; Request ID: 93609c15-e490-4035-8390-f4396f0d90bf; Proxy: null) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704) - at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530) - at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223) - at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190) - at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179) - at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482) - at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156) - at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236) - at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792) - at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604) - at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789) - at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117) - ... 49 more ``` Generate a Symmetric Key in the same region as your S3 storage for CSE-KMS to work. -### com.amazonaws.services.kms.model.NotFoundException: Invalid keyId +### software.amazon.awssdk.services.kms.mode.NotFoundException: Invalid keyId If the value in `fs.s3a.encryption.key` property, does not exist /valid in AWS KMS CMK(Customer managed keys), then this error would be seen. ``` -Caused by: com.amazonaws.services.kms.model.NotFoundException: Invalid keyId abc -(Service: AWSKMS; Status Code: 400; Error Code: NotFoundException; Request ID: 9d53552a-3d1b-47c8-984c-9a599d5c2391; Proxy: null) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704) - at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530) - at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223) - at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190) - at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179) - at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482) - at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156) - at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236) - at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792) - at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604) - at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789) - at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117) - ... 49 more +Caused by: software.amazon.awssdk.services.kms.model.NotFoundException: Invalid keyId abc +(Service: AWSKMS; Status Code: 400; Error Code: NotFoundException; Request ID: + 9d53552a-3d1b-47c8-984c-9a599d5c2391; Proxy: null) ``` Check if `fs.s3a.encryption.key` is set correctly and matches the same on AWS console. -### com.amazonaws.services.kms.model.AWSKMSException: User: is not authorized to perform : kms :GenerateDataKey on resource: +### software.amazon.awssdk.services.kms.model.KmsException: User: is not authorized to perform : kms :GenerateDataKey on resource: User doesn't have authorization to the specific AWS KMS Key ID. ``` -Caused by: com.amazonaws.services.kms.model.AWSKMSException: -User: arn:aws:iam::152813717728:user/ is not authorized to perform: kms:GenerateDataKey on resource: -(Service: AWSKMS; Status Code: 400; Error Code: AccessDeniedException; Request ID: 4ded9f1f-b245-4213-87fc-16cba7a1c4b9; Proxy: null) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704) - at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530) - at com.amazonaws.services.kms.AWSKMSClient.doInvoke(AWSKMSClient.java:7223) - at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7190) - at com.amazonaws.services.kms.AWSKMSClient.invoke(AWSKMSClient.java:7179) - at com.amazonaws.services.kms.AWSKMSClient.executeGenerateDataKey(AWSKMSClient.java:3482) - at com.amazonaws.services.kms.AWSKMSClient.generateDataKey(AWSKMSClient.java:3451) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.buildContentCryptoMaterial(S3CryptoModuleBase.java:533) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.newContentCryptoMaterial(S3CryptoModuleBase.java:481) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.createContentCryptoMaterial(S3CryptoModuleBase.java:447) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectUsingMetadata(S3CryptoModuleBase.java:160) - at com.amazonaws.services.s3.internal.crypto.v2.S3CryptoModuleBase.putObjectSecurely(S3CryptoModuleBase.java:156) - at com.amazonaws.services.s3.AmazonS3EncryptionClientV2.putObject(AmazonS3EncryptionClientV2.java:236) - at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2792) - at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604) - at org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2789) - at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$33(S3AFileSystem.java:4440) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117) - ... 49 more +Caused by: software.amazon.awssdk.services.kms.model.KmsException: +User: arn:aws:iam::152813717728:user/ is not authorized to perform: + kms:GenerateDataKey on resource: +(Service: AWSKMS; Status Code: 400; Error Code: AccessDeniedException; + Request ID: 4ded9f1f-b245-4213-87fc-16cba7a1c4b9; Proxy: null) ``` The user trying to use the KMS Key ID should have the right permissions to access @@ -1495,13 +1132,13 @@ is more than 10000 (specified by aws SDK). You can configure The bucket does not exist. ``` -org.apache.hadoop.fs.s3a.UnknownStoreException: - Bucket random-bucket-33013fb8-f7f7-4edb-9c26-16a6ed019184 does not exist - at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:537) - at org.apache.hadoop.fs.s3a.S3AFileSystem.doBucketProbing(S3AFileSystem.java:471) - at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:387) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3422) - at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:502) +org.apache.hadoop.fs.s3a.UnknownStoreException: `s3a://random-bucket-7d9217b0-b426-4344-82ea-25d6cbb316f1/': + Bucket does not exist: software.amazon.awssdk.services.s3.model.NoSuchBucketException: null + (Service: S3, Status Code: 404, Request ID: RD254TC8EVDV98AK, + Extended Request ID: 49F5CO1IKavFsz+VBecf2uwZeNVar3InHkdIrONvAK5yQ73gqZ1hFoAEMo8/x5wRNe3OXO3aebvZkev2bS81kw==) + (Service: S3, Status Code: 404, Request ID: RD254TC8EVDV98AK): null + (Service: S3, Status Code: 404, Request ID: RD254TC8EVDV98AK, Extended Request ID: 49F5CO1IKavFsz+VBecf2uwZeNVar3InHkdIrONvAK5yQ73gqZ1hFoAEMo8/x5wRNe3OXO3aebvZkev2bS81kw==) + (Service: S3, Status Code: 404, Request ID: RD254TC8EVDV98AK) ``` Check the URI is correct, and that the bucket actually exists. @@ -1514,20 +1151,6 @@ for a bucket is not an unusual occurrence. This can surface during filesystem API calls if the bucket is deleted while you are using it, -or the startup check for bucket existence has been disabled by setting `fs.s3a.bucket.probe` to 0. -``` -org.apache.hadoop.fs.s3a.UnknownStoreException: s3a://random-bucket-7d9217b0-b426-4344-82ea-25d6cbb316f1/ - - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:254) - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:167) - at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListFiles(S3AFileSystem.java:4149) - at org.apache.hadoop.fs.s3a.S3AFileSystem.listFiles(S3AFileSystem.java:3983) -Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: -The specified bucket does not exist - (Service: Amazon S3; Status Code: 404; Error Code: NoSuchBucket - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1712) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1367) -``` - ## S3Guard Errors @@ -1630,73 +1253,16 @@ Possible causes for this This is a very, very rare occurrence. -If the problem is a signing one, try changing the signature algorithm. - -```xml - - fs.s3a.signing-algorithm - S3SignerType - -``` - -We cannot make any promises that it will work, only that it has been known to -make the problem go away "once" ### `AWSS3IOException` The Content-MD5 you specified did not match what we received Reads work, but writes, even `mkdir`, fail: -``` -org.apache.hadoop.fs.s3a.AWSS3IOException: copyFromLocalFile(file:/tmp/hello.txt, s3a://bucket/hello.txt) - on file:/tmp/hello.txt: - The Content-MD5 you specified did not match what we received. - (Service: Amazon S3; Status Code: 400; Error Code: BadDigest; Request ID: 4018131225), - S3 Extended Request ID: null - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:127) - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:69) - at org.apache.hadoop.fs.s3a.S3AFileSystem.copyFromLocalFile(S3AFileSystem.java:1494) - at org.apache.hadoop.tools.cloudup.Cloudup.uploadOneFile(Cloudup.java:466) - at org.apache.hadoop.tools.cloudup.Cloudup.access$000(Cloudup.java:63) - at org.apache.hadoop.tools.cloudup.Cloudup$1.call(Cloudup.java:353) - at org.apache.hadoop.tools.cloudup.Cloudup$1.call(Cloudup.java:350) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) - at java.util.concurrent.FutureTask.run(FutureTask.java:266) - at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) - at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) - at java.lang.Thread.run(Thread.java:748) -Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: - The Content-MD5 you specified did not match what we received. - (Service: Amazon S3; Status Code: 400; Error Code: BadDigest; Request ID: 4018131225), - S3 Extended Request ID: null - at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1307) - at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:894) - at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:597) - at com.amazonaws.http.AmazonHttpClient.doExecute(AmazonHttpClient.java:363) - at com.amazonaws.http.AmazonHttpClient.executeWithTimer(AmazonHttpClient.java:329) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:308) - at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3659) - at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1422) - at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:131) - at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:123) - at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:139) - at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:47) - at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239) - ... 4 more -``` - -This stack trace was seen when interacting with a third-party S3 store whose -expectations of headers related to the AWS V4 signing mechanism was not -compatible with that of the specific AWS SDK Hadoop was using. +This has been seen with third party stores. -Workaround: revert to V2 signing. +If the store is configured to require content-MD5 headers with data uploaded: disable it. -```xml - - fs.s3a.signing-algorithm - S3SignerType - -``` +If the store requires the use of the v2 signing algorithm, know that it is unsupported on this release. ### When writing data: "java.io.FileNotFoundException: Completing multi-part upload" @@ -1705,20 +1271,9 @@ with that ID. ``` java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multipart/1c397ca6-9dfb-4ac1-9cf7-db666673246b: - com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist. + S3Exception: The specified upload does not exist. The upload ID may be invalid, or the upload may have been aborted or completed. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload; - at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182) - at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770) - at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) - at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) - at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) - at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2705) - at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:473) - at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:382) - at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:272) - at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) - at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) ``` This can happen when all outstanding uploads have been aborted, including the @@ -1806,37 +1361,16 @@ connections more frequently. ``` Cause: org.apache.hadoop.fs.s3a.AWSBadRequestException: put on : - com.amazonaws.services.s3.model.AmazonS3Exception: + S3Exception: The unspecified location constraint is incompatible for the region specific endpoint this request was sent to. (Service: Amazon S3; Status Code: 400; Error Code: IllegalLocationConstraintException; - - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:178) - at org.apache.hadoop.fs.s3a.S3ALambda.execute(S3ALambda.java:64) - at org.apache.hadoop.fs.s3a.WriteOperationHelper.uploadObject(WriteOperationHelper.java:451) - at org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker.aboutToComplete(MagicCommitTracker.java:128) - at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:373) - at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) - at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101) - at org.apache.hadoop.hive.ql.io.orc.WriterImpl.close(WriterImpl.java:2429) - at org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat$OrcRecordWriter.close(OrcOutputFormat.java:106) - at org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat$OrcRecordWriter.close(OrcOutputFormat.java:91) ... - Cause: com.amazonaws.services.s3.model.AmazonS3Exception: + Cause: S3Exception: The unspecified location constraint is incompatible for the region specific endpoint this request was sent to. (Service: Amazon S3; Status Code: 400; Error Code: IllegalLocationConstraintException; Request ID: EEBC5A08BCB3A645) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1588) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1258) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1030) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:742) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:716) - 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:4221) - ... + ``` Something has been trying to write data to "/". @@ -1850,8 +1384,7 @@ more detail, as can S3A itself. ```properties log4j.logger.org.apache.hadoop.fs.s3a=DEBUG -log4j.logger.com.amazonaws.request=DEBUG -log4j.logger.com.amazonaws.thirdparty.apache.http=DEBUG +log4j.logger.software.amazon.awssdk.request=DEBUG ``` If using the "unshaded" JAR, then the Apache HttpClient can be directly configured: @@ -1947,8 +1480,9 @@ The number of retries and interval between each retry can be configured: ``` -Not all failures are retried. Specifically excluded are those considered -unrecoverable: +Not all failures are retried *in the S3A Code* -though some may be retried within the +AWS SDK. +Specifically excluded are those considered unrecoverable: * Low-level networking: `UnknownHostException`, `NoRouteToHostException`. * 302 redirects. @@ -1998,7 +1532,7 @@ If this value is configured too low, user may encounter `SdkClientException`s du timing-out. ``` -com.amazonaws.SdkClientException: Unable to execute HTTP request: +software.amazon.awssdk.core.exception.SdkClientException: Unable to execute HTTP request: Request did not complete before the request timeout configuration.: Unable to execute HTTP request: Request did not complete before the request timeout configuration. at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:205) @@ -2010,51 +1544,3 @@ com.amazonaws.SdkClientException: Unable to execute HTTP request: When this happens, try to set `fs.s3a.connection.request.timeout` to a larger value or disable it completely by setting it to `0`. - -## SDK Upgrade Warnings - -S3A will soon be upgraded to [AWS's Java SDK V2](https://github.com/aws/aws-sdk-java-v2). -For more information on the upgrade and what's changing, see -[Upcoming upgrade to AWS Java SDK V2](./aws_sdk_upgrade.html). - -S3A logs the following warnings for things that will be changing in the upgrade. To disable these -logs, comment out `log4j.logger.org.apache.hadoop.fs.s3a.SDKV2Upgrade` in log4j.properties. - -### `Directly referencing AWS SDK V1 credential provider` - -This will be logged when an AWS credential provider is referenced directly in -`fs.s3a.aws.credentials.provider`. -For example, `com.amazonaws.auth.AWSSessionCredentialsProvider` - -To stop this warning, remove any AWS credential providers from `fs.s3a.aws.credentials.provider`. -Instead, use S3A's credential providers. - -### `getAmazonS3ClientForTesting() will be removed` - -This will be logged when `getAmazonS3ClientForTesting()` is called to get the S3 Client. With V2, -the S3 client will change from type `com.amazonaws.services.s3.AmazonS3` to -`software.amazon.awssdk.services.s3.S3Client`, and so this method will be removed. - -### -### `Custom credential providers used in delegation tokens binding classes will need to be updated` - -This will be logged when delegation tokens are used. -Delegation tokens allow the use of custom binding classes which can implement custom credential -providers. -These credential providers will currently be implementing -`com.amazonaws.auth.AWSCredentialsProvider` and will need to be updated to implement -`software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`. - -### -### `The signer interface has changed in AWS SDK V2, custom signers will need to be updated` - -This will be logged when a custom signer is used. -Custom signers will currently be implementing `com.amazonaws.auth.Signer` and will need to be -updated to implement `software.amazon.awssdk.core.signer.Signer`. - -### -### `getObjectMetadata() called. This operation and it's response will be changed` - -This will be logged when `getObjectMetadata` is called. In SDK V2, this operation has changed to -`headObject()` and will return a response of the type `HeadObjectResponse`. - diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java index a46303f339678..734bcfd9c5d30 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java @@ -20,8 +20,9 @@ import static org.apache.hadoop.fs.s3a.Constants.*; -import com.amazonaws.AmazonServiceException; -import com.amazonaws.services.s3.AmazonS3; +import software.amazon.awssdk.awscore.exception.AwsErrorDetails; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.s3.S3Client; import java.net.URI; @@ -32,6 +33,7 @@ import org.junit.Rule; import org.junit.rules.ExpectedException; + /** * Abstract base class for S3A unit tests using a mock S3 client and a null * metadata store. @@ -39,17 +41,20 @@ public abstract class AbstractS3AMockTest { protected static final String BUCKET = "mock-bucket"; - protected static final AmazonServiceException NOT_FOUND; - static { - NOT_FOUND = new AmazonServiceException("Not Found"); - NOT_FOUND.setStatusCode(404); - } + protected static final AwsServiceException NOT_FOUND = + AwsServiceException.builder() + .message("Not Found") + .statusCode(404) + .awsErrorDetails(AwsErrorDetails.builder() + .errorCode("") + .build()) + .build(); @Rule public ExpectedException exception = ExpectedException.none(); protected S3AFileSystem fs; - protected AmazonS3 s3; + protected S3Client s3; @Before public void setup() throws Exception { @@ -59,10 +64,9 @@ public void setup() throws Exception { // unset S3CSE property from config to avoid pathIOE. conf.unset(Constants.S3_ENCRYPTION_ALGORITHM); fs.initialize(uri, conf); - s3 = fs.getAmazonS3ClientForTesting("mocking"); + s3 = fs.getS3AInternals().getAmazonS3Client("mocking"); } - @SuppressWarnings("deprecation") public Configuration createConfiguration() { Configuration conf = new Configuration(); conf.setClass(S3_CLIENT_FACTORY_IMPL, MockS3ClientFactory.class, @@ -75,9 +79,15 @@ public Configuration createConfiguration() { // assertions to be safely made without worrying // about any race conditions conf.setInt(ASYNC_DRAIN_THRESHOLD, Integer.MAX_VALUE); + // set the region to avoid the getBucketLocation on FS init. + conf.set(AWS_REGION, "eu-west-1"); return conf; } + public S3Client getS3Client() { + return s3; + } + @After public void teardown() throws Exception { if (fs != null) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index e90ad8b73efae..93f41cfaa81bb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -210,6 +210,14 @@ public S3AFileSystem getFileSystem() { return (S3AFileSystem) super.getFileSystem(); } + /** + * Get the {@link S3AInternals} internal access for the + * test filesystem. + * @return internals. + */ + public S3AInternals getS3AInternals() { + return getFileSystem().getS3AInternals(); + } /** * Describe a test in the logs. * @param text text to print diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/EncryptionTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/EncryptionTestUtils.java index 4013e9db29a3e..7b2b1c639e3cc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/EncryptionTestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/EncryptionTestUtils.java @@ -20,7 +20,7 @@ import java.io.IOException; -import com.amazonaws.services.s3.model.ObjectMetadata; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.net.util.Base64; @@ -28,8 +28,7 @@ import org.apache.hadoop.fs.Path; import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_KEY; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; +import static org.assertj.core.api.Assertions.assertThat; public final class EncryptionTestUtils { @@ -39,6 +38,8 @@ private EncryptionTestUtils() { public static final String AWS_KMS_SSE_ALGORITHM = "aws:kms"; + public static final String AWS_KMS_DSSE_ALGORITHM = "aws:kms:dsse"; + public static final String SSE_C_ALGORITHM = "AES256"; /** @@ -69,33 +70,44 @@ public static void assertEncrypted(S3AFileSystem fs, final S3AEncryptionMethods algorithm, final String kmsKeyArn) throws IOException { - ObjectMetadata md = fs.getObjectMetadata(path); + HeadObjectResponse md = fs.getS3AInternals().getObjectMetadata(path); String details = String.format( "file %s with encryption algorithm %s and key %s", path, - md.getSSEAlgorithm(), - md.getSSEAwsKmsKeyId()); + md.serverSideEncryptionAsString(), + md.ssekmsKeyId()); switch(algorithm) { case SSE_C: - assertNull("Metadata algorithm should have been null in " - + details, - md.getSSEAlgorithm()); - assertEquals("Wrong SSE-C algorithm in " - + details, - SSE_C_ALGORITHM, md.getSSECustomerAlgorithm()); + assertThat(md.serverSideEncryptionAsString()) + .describedAs("Details of the server-side encryption algorithm used: %s", details) + .isNull(); + assertThat(md.sseCustomerAlgorithm()) + .describedAs("Details of SSE-C algorithm: %s", details) + .isEqualTo(SSE_C_ALGORITHM); String md5Key = convertKeyToMd5(fs); - assertEquals("getSSECustomerKeyMd5() wrong in " + details, - md5Key, md.getSSECustomerKeyMd5()); + assertThat(md.sseCustomerKeyMD5()) + .describedAs("Details of the customer provided encryption key: %s", details) + .isEqualTo(md5Key); break; case SSE_KMS: - assertEquals("Wrong algorithm in " + details, - AWS_KMS_SSE_ALGORITHM, md.getSSEAlgorithm()); - assertEquals("Wrong KMS key in " + details, - kmsKeyArn, - md.getSSEAwsKmsKeyId()); + assertThat(md.serverSideEncryptionAsString()) + .describedAs("Details of the server-side encryption algorithm used: %s", details) + .isEqualTo(AWS_KMS_SSE_ALGORITHM); + assertThat(md.ssekmsKeyId()) + .describedAs("Details of the KMS key: %s", details) + .isEqualTo(kmsKeyArn); + break; + case DSSE_KMS: + assertThat(md.serverSideEncryptionAsString()) + .describedAs("Details of the server-side encryption algorithm used: %s", details) + .isEqualTo(AWS_KMS_DSSE_ALGORITHM); + assertThat(md.ssekmsKeyId()) + .describedAs("Details of the KMS key: %s", details) + .isEqualTo(kmsKeyArn); break; default: - assertEquals("AES256", md.getSSEAlgorithm()); + assertThat(md.serverSideEncryptionAsString()) + .isEqualTo("AES256"); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java index c13c3f48b8466..bccbe79c2a48b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java @@ -26,27 +26,30 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.fs.s3a.impl.InstantiationIOException; +import org.assertj.core.api.Assertions; import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; -import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.BasicAWSCredentials; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getCSVTestPath; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; -import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_BINDING; +import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.CONSTRUCTOR_EXCEPTION; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.junit.Assert.*; /** - * Integration tests for {@link Constants#AWS_CREDENTIALS_PROVIDER} logic. + * Integration tests for {@link Constants#AWS_CREDENTIALS_PROVIDER} logic + * through the S3A Filesystem instantiation process. */ public class ITestS3AAWSCredentialsProvider { private static final Logger LOG = @@ -55,17 +58,21 @@ public class ITestS3AAWSCredentialsProvider { @Rule public Timeout testTimeout = new Timeout(60_1000, TimeUnit.MILLISECONDS); + /** + * Expecting a wrapped ClassNotFoundException. + */ @Test - public void testBadConfiguration() throws IOException { - Configuration conf = createConf(); - conf.set(AWS_CREDENTIALS_PROVIDER, "no.such.class"); - try { - createFailingFS(conf); - } catch (IOException e) { - if (!(e.getCause() instanceof ClassNotFoundException)) { - LOG.error("Unexpected nested cause: {} in {}", e.getCause(), e, e); - throw e; - } + public void testProviderClassNotFound() throws Exception { + Configuration conf = createConf("no.such.class"); + final InstantiationIOException e = + intercept(InstantiationIOException.class, "java.lang.ClassNotFoundException", () -> + createFailingFS(conf)); + if (InstantiationIOException.Kind.InstantiationFailure != e.getKind()) { + throw e; + } + if (!(e.getCause() instanceof ClassNotFoundException)) { + LOG.error("Unexpected nested cause: {} in {}", e.getCause(), e, e); + throw e; } } @@ -73,105 +80,144 @@ public void testBadConfiguration() throws IOException { * A bad CredentialsProvider which has no suitable constructor. * * This class does not provide a public constructor accepting Configuration, - * or a public factory method named getInstance that accepts no arguments, + * or a public factory method named create() that accepts no arguments, * or a public default constructor. */ - static class BadCredentialsProviderConstructor - implements AWSCredentialsProvider { + public static class BadCredentialsProviderConstructor + implements AwsCredentialsProvider { @SuppressWarnings("unused") public BadCredentialsProviderConstructor(String fsUri, Configuration conf) { } @Override - public AWSCredentials getCredentials() { - return new BasicAWSCredentials("dummy_key", "dummy_secret"); + public AwsCredentials resolveCredentials() { + return AwsBasicCredentials.create("dummy_key", "dummy_secret"); } - @Override - public void refresh() { - } } @Test public void testBadCredentialsConstructor() throws Exception { - Configuration conf = createConf(); - conf.set(AWS_CREDENTIALS_PROVIDER, - BadCredentialsProviderConstructor.class.getName()); - try { - createFailingFS(conf); - } catch (IOException e) { - GenericTestUtils.assertExceptionContains(CONSTRUCTOR_EXCEPTION, e); + Configuration conf = createConf(BadCredentialsProviderConstructor.class); + final InstantiationIOException ex = + intercept(InstantiationIOException.class, CONSTRUCTOR_EXCEPTION, () -> + createFailingFS(conf)); + if (InstantiationIOException.Kind.UnsupportedConstructor != ex.getKind()) { + throw ex; } } - protected Configuration createConf() { + /** + * Create a configuration bonded to the given provider classname. + * @param provider provider to bond to + * @return a configuration + */ + protected Configuration createConf(String provider) { Configuration conf = new Configuration(); removeBaseAndBucketOverrides(conf, DELEGATION_TOKEN_BINDING, AWS_CREDENTIALS_PROVIDER); + conf.set(AWS_CREDENTIALS_PROVIDER, provider); + conf.set(DELEGATION_TOKEN_BINDING, ""); return conf; } + /** + * Create a configuration bonded to the given provider class. + * @param provider provider to bond to + * @return a configuration + */ + protected Configuration createConf(Class provider) { + return createConf(provider.getName()); + } + /** * Create a filesystem, expect it to fail by raising an IOException. * Raises an assertion exception if in fact the FS does get instantiated. + * The FS is always deleted. * @param conf configuration * @throws IOException an expected exception. */ private void createFailingFS(Configuration conf) throws IOException { - S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf); - fs.listStatus(new Path("/")); - fail("Expected exception - got " + fs); + try(S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) { + fs.listStatus(new Path("/")); + fail("Expected exception - got " + fs); + } } - static class BadCredentialsProvider implements AWSCredentialsProvider { + /** + * Returns an invalid set of credentials. + */ + public static class BadCredentialsProvider implements AwsCredentialsProvider { @SuppressWarnings("unused") public BadCredentialsProvider(Configuration conf) { } @Override - public AWSCredentials getCredentials() { - return new BasicAWSCredentials("bad_key", "bad_secret"); + public AwsCredentials resolveCredentials() { + return AwsBasicCredentials.create("bad_key", "bad_secret"); } - @Override - public void refresh() { - } } @Test public void testBadCredentials() throws Exception { - Configuration conf = new Configuration(); - conf.set(AWS_CREDENTIALS_PROVIDER, BadCredentialsProvider.class.getName()); - try { - createFailingFS(conf); - } catch (AccessDeniedException e) { - // expected - } catch (AWSServiceIOException e) { - GenericTestUtils.assertExceptionContains( - "UnrecognizedClientException", e); - // expected - } + Configuration conf = createConf(BadCredentialsProvider.class); + intercept(AccessDeniedException.class, "", () -> + createFailingFS(conf)); } + /** + * Test using the anonymous credential provider with the public csv + * test file; if the test file path is unset then it will be skipped. + */ @Test - @SuppressWarnings("deprecation") public void testAnonymousProvider() throws Exception { - Configuration conf = new Configuration(); - conf.set(AWS_CREDENTIALS_PROVIDER, - AnonymousAWSCredentialsProvider.class.getName()); + Configuration conf = createConf(AnonymousAWSCredentialsProvider.class); Path testFile = getCSVTestPath(conf); try (FileSystem fs = FileSystem.newInstance(testFile.toUri(), conf)) { - assertNotNull("S3AFileSystem instance must not be null", fs); - assertTrue("FileSystem must be the instance of S3AFileSystem", fs instanceof S3AFileSystem); + Assertions.assertThat(fs) + .describedAs("Filesystem") + .isNotNull(); FileStatus stat = fs.getFileStatus(testFile); - assertNotNull("FileStatus with qualified path must not be null", stat); assertEquals( "The qualified path returned by getFileStatus should be same as the original file", testFile, stat.getPath()); } } + /** + * Create credentials via the create() method. + * They are invalid credentials, so IO will fail as access denied. + */ + @Test + public void testCredentialsWithCreateMethod() throws Exception { + Configuration conf = createConf(CredentialsProviderWithCreateMethod.class); + intercept(AccessDeniedException.class, "", () -> + createFailingFS(conf)); + } + + /** + * Credentials via the create() method. + */ + public static final class CredentialsProviderWithCreateMethod implements AwsCredentialsProvider { + + public static AwsCredentialsProvider create() { + LOG.info("creating CredentialsProviderWithCreateMethod"); + return new CredentialsProviderWithCreateMethod(); + } + + /** Private: cannot be created directly. */ + private CredentialsProviderWithCreateMethod() { + } + + @Override + public AwsCredentials resolveCredentials() { + return AwsBasicCredentials.create("bad_key", "bad_secret"); + } + + } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java index 9485202f64cb4..ded2f0b885079 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a; import java.net.URI; +import java.nio.file.AccessDeniedException; import java.util.UUID; import java.util.concurrent.Callable; @@ -33,11 +34,15 @@ import org.apache.hadoop.test.LambdaTestUtils; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION; import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_ACCESSPOINT_REQUIRED; +import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT; import static org.apache.hadoop.fs.s3a.Constants.FS_S3A; +import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS; import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** @@ -67,8 +72,14 @@ public void testNoBucketProbing() throws Exception { assertTrue("getFileStatus on root should always return a directory", fs.getFileStatus(root).isDirectory()); - expectUnknownStore( - () -> fs.listStatus(root)); + try { + expectUnknownStore( + () -> fs.listStatus(root)); + } catch (AccessDeniedException e) { + // this is a sign that there's tests with a third-party bucket and + // interacting with aws is not going to authenticate + skip("no aws credentials"); + } Path src = new Path(root, "testfile"); Path dest = new Path(root, "dst"); @@ -124,12 +135,18 @@ public static void expectUnknownStore( private Configuration createConfigurationWithProbe(final int probe) { Configuration conf = new Configuration(getFileSystem().getConf()); S3ATestUtils.disableFilesystemCaching(conf); + removeBaseAndBucketOverrides(conf, + S3A_BUCKET_PROBE, + ENDPOINT, + AWS_REGION, + PATH_STYLE_ACCESS); conf.setInt(S3A_BUCKET_PROBE, probe); + conf.set(AWS_REGION, EU_WEST_1); return conf; } @Test - public void testBucketProbingV1() throws Exception { + public void testBucketProbing() throws Exception { describe("Test the V1 bucket probe"); Configuration configuration = createConfigurationWithProbe(1); expectUnknownStore( @@ -137,18 +154,24 @@ public void testBucketProbingV1() throws Exception { } @Test - public void testBucketProbingV2() throws Exception { - describe("Test the V2 bucket probe"); + public void testBucketProbing2() throws Exception { + describe("Test the bucket probe with probe value set to 2"); Configuration configuration = createConfigurationWithProbe(2); + expectUnknownStore( () -> FileSystem.get(uri, configuration)); - /* - * Bucket probing should also be done when value of - * S3A_BUCKET_PROBE is greater than 2. - */ - configuration.setInt(S3A_BUCKET_PROBE, 3); - expectUnknownStore( - () -> FileSystem.get(uri, configuration)); + } + + @Test + public void testBucketProbing3() throws Exception { + describe("Test the bucket probe with probe value set to 3"); + Configuration configuration = createConfigurationWithProbe(3); + fs = FileSystem.get(uri, configuration); + Path root = new Path(uri); + + assertTrue("root path should always exist", fs.exists(root)); + assertTrue("getFileStatus on root should always return a directory", + fs.getFileStatus(root).isDirectory()); } @Test @@ -162,8 +185,8 @@ public void testBucketProbingParameterValidation() throws Exception { } @Test - public void testAccessPointProbingV2() throws Exception { - describe("Test V2 bucket probing using an AccessPoint ARN"); + public void testAccessPointProbing2() throws Exception { + describe("Test bucket probing using probe value 2, and an AccessPoint ARN"); Configuration configuration = createArnConfiguration(); String accessPointArn = "arn:aws:s3:eu-west-1:123456789012:accesspoint/" + randomBucket; configuration.set(String.format(InternalConstants.ARN_BUCKET_OPTION, randomBucket), @@ -175,7 +198,7 @@ public void testAccessPointProbingV2() throws Exception { @Test public void testAccessPointRequired() throws Exception { - describe("Test V2 bucket probing with 'fs.s3a.accesspoint.required' property."); + describe("Test bucket probing with 'fs.s3a.accesspoint.required' property."); Configuration configuration = createArnConfiguration(); configuration.set(AWS_S3_ACCESSPOINT_REQUIRED, "true"); intercept(PathIOException.class, @@ -197,7 +220,7 @@ public void testAccessPointRequired() throws Exception { */ private Configuration createArnConfiguration() { Configuration configuration = createConfigurationWithProbe(2); - configuration.set(AWS_REGION, "eu-west-1"); + configuration.set(AWS_REGION, EU_WEST_1); return configuration; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACannedACLs.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACannedACLs.java index 1071582cc67d2..e8dcca6df4baa 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACannedACLs.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACannedACLs.java @@ -20,11 +20,13 @@ import java.util.List; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.AccessControlList; -import com.amazonaws.services.s3.model.Grant; -import com.amazonaws.services.s3.model.GroupGrantee; -import com.amazonaws.services.s3.model.Permission; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.GetObjectAclRequest; +import software.amazon.awssdk.services.s3.model.GetObjectAclResponse; +import software.amazon.awssdk.services.s3.model.Grant; +import software.amazon.awssdk.services.s3.model.Grantee; +import software.amazon.awssdk.services.s3.model.Permission; +import software.amazon.awssdk.services.s3.model.Type; import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; @@ -38,7 +40,9 @@ import org.apache.hadoop.fs.store.audit.AuditSpan; import static org.apache.hadoop.fs.s3a.Constants.CANNED_ACL; +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.skipIfACLTestsDisabled; /** * Tests of ACL handling in the FS. @@ -53,9 +57,10 @@ public class ITestS3ACannedACLs extends AbstractS3ATestBase { @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); + skipIfACLTestsDisabled(conf); + disableFilesystemCaching(conf); removeBaseAndBucketOverrides(conf, CANNED_ACL); - conf.set(CANNED_ACL, LOG_DELIVERY_WRITE); // needed because of direct calls made conf.setBoolean(S3AAuditConstants.REJECT_OUT_OF_SPAN_OPERATIONS, false); @@ -89,18 +94,26 @@ private void assertObjectHasLoggingGrant(Path path, boolean isFile) { S3AFileSystem fs = getFileSystem(); StoreContext storeContext = fs.createStoreContext(); - AmazonS3 s3 = fs.getAmazonS3ClientForTesting("acls"); + S3Client s3 = getS3AInternals().getAmazonS3Client("acls"); String key = storeContext.pathToKey(path); if (!isFile) { key = key + "/"; } - AccessControlList acl = s3.getObjectAcl(storeContext.getBucket(), - key); - List grants = acl.getGrantsAsList(); + GetObjectAclResponse acl = s3.getObjectAcl(GetObjectAclRequest.builder() + .bucket(storeContext.getBucket()) + .key(key) + .build()); + List grants = acl.grants(); for (Grant grant : grants) { LOG.info("{}", grant.toString()); } - Grant loggingGrant = new Grant(GroupGrantee.LogDelivery, Permission.Write); + Grant loggingGrant = Grant.builder() + .grantee(Grantee.builder() + .type(Type.GROUP) + .uri("http://acs.amazonaws.com/groups/s3/LogDelivery") + .build()) + .permission(Permission.WRITE) + .build(); Assertions.assertThat(grants) .describedAs("ACL grants of object %s", path) .contains(loggingGrant); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryptionKms.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryptionKms.java index bcc37c8bfbbba..4f1dcdfd5238b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryptionKms.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClientSideEncryptionKms.java @@ -21,11 +21,11 @@ import java.io.IOException; import java.util.Map; -import com.amazonaws.services.s3.Headers; import org.assertj.core.api.Assertions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.AWSHeaders; import org.apache.hadoop.fs.s3a.impl.HeaderProcessing; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; @@ -69,14 +69,14 @@ protected void assertEncrypted(Path path) throws IOException { // Assert KeyWrap Algo assertEquals("Key wrap algo isn't same as expected", KMS_KEY_WRAP_ALGO, processHeader(fsXAttrs, - xAttrPrefix + Headers.CRYPTO_KEYWRAP_ALGORITHM)); + xAttrPrefix + AWSHeaders.CRYPTO_KEYWRAP_ALGORITHM)); // Assert content encryption algo for KMS, is present in the // materials description and KMS key ID isn't. String keyId = getS3EncryptionKey(getTestBucketName(getConfiguration()), getConfiguration()); Assertions.assertThat(processHeader(fsXAttrs, - xAttrPrefix + Headers.MATERIALS_DESCRIPTION)) + xAttrPrefix + AWSHeaders.MATERIALS_DESCRIPTION)) .describedAs("Materials Description should contain the content " + "encryption algo and should not contain the KMS keyID.") .contains(KMS_CONTENT_ENCRYPTION_ALGO) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java index ff75f6e26138d..d9c847d4b2b0a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -18,10 +18,29 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.S3ClientOptions; +import java.io.File; +import java.io.IOException; +import java.net.ConnectException; +import java.net.URI; +import java.security.PrivilegedExceptionAction; +import org.assertj.core.api.Assertions; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.client.config.SdkClientConfiguration; +import software.amazon.awssdk.core.client.config.SdkClientOption; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.signer.Signer; +import software.amazon.awssdk.http.SdkHttpFullRequest; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.S3Configuration; +import software.amazon.awssdk.services.s3.model.HeadBucketRequest; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.model.StsException; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.reflect.FieldUtils; @@ -29,31 +48,20 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.auth.STSClientFactory; import org.apache.hadoop.fs.s3native.S3xLoginHelper; -import org.apache.hadoop.test.GenericTestUtils; - -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.io.File; -import java.net.URI; -import java.security.PrivilegedExceptionAction; - import org.apache.hadoop.security.ProviderUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.alias.CredentialProvider; import org.apache.hadoop.security.alias.CredentialProviderFactory; +import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.VersionInfo; import org.apache.http.HttpStatus; -import org.junit.rules.TemporaryFolder; +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.s3a.S3ATestConstants.EU_WEST_1; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -84,6 +92,23 @@ public class ITestS3AConfiguration { @Rule public final TemporaryFolder tempDir = new TemporaryFolder(); + /** + * Get the S3 client of the active filesystem. + * @param reason why? + * @return the client + */ + private S3Client getS3Client(String reason) { + return requireNonNull(getS3AInternals().getAmazonS3Client(reason)); + } + + /** + * Get the internals of the active filesystem. + * @return the internals + */ + private S3AInternals getS3AInternals() { + return fs.getS3AInternals(); + } + /** * Test if custom endpoint is picked up. *

    @@ -111,7 +136,6 @@ public void testEndpoint() throws Exception { } else { conf.set(Constants.ENDPOINT, endpoint); fs = S3ATestUtils.createTestFileSystem(conf); - AmazonS3 s3 = fs.getAmazonS3ClientForTesting("test endpoint"); String endPointRegion = ""; // Differentiate handling of "s3-" and "s3." based endpoint identifiers String[] endpointParts = StringUtils.split(endpoint, '.'); @@ -122,8 +146,9 @@ public void testEndpoint() throws Exception { } else { fail("Unexpected endpoint"); } + String region = getS3AInternals().getBucketLocation(); assertEquals("Endpoint config setting and bucket location differ: ", - endPointRegion, s3.getBucketLocation(fs.getUri().getHost())); + endPointRegion, region); } } @@ -134,7 +159,7 @@ public void testProxyConnection() throws Exception { conf.setInt(Constants.PROXY_PORT, 1); String proxy = conf.get(Constants.PROXY_HOST) + ":" + conf.get(Constants.PROXY_PORT); - expectFSCreateFailure(AWSClientIOException.class, + expectFSCreateFailure(ConnectException.class, conf, "when using proxy " + proxy); } @@ -149,7 +174,7 @@ protected void useFailFastConfiguration() { } /** - * Expect a filesystem to not be created from a configuration + * Expect a filesystem to not be created from a configuration. * @return the exception intercepted * @throws Exception any other exception */ @@ -188,10 +213,10 @@ public void testAutomaticProxyPortSelection() throws Exception { conf.unset(Constants.PROXY_PORT); conf.set(Constants.PROXY_HOST, "127.0.0.1"); conf.set(Constants.SECURE_CONNECTIONS, "true"); - expectFSCreateFailure(AWSClientIOException.class, + expectFSCreateFailure(ConnectException.class, conf, "Expected a connection error for proxy server"); conf.set(Constants.SECURE_CONNECTIONS, "false"); - expectFSCreateFailure(AWSClientIOException.class, + expectFSCreateFailure(ConnectException.class, conf, "Expected a connection error for proxy server"); } @@ -342,28 +367,31 @@ public void shouldBeAbleToSwitchOnS3PathStyleAccessViaConfigProperty() throws Exception { conf = new Configuration(); + skipIfCrossRegionClient(conf); conf.set(Constants.PATH_STYLE_ACCESS, Boolean.toString(true)); assertTrue(conf.getBoolean(Constants.PATH_STYLE_ACCESS, false)); try { fs = S3ATestUtils.createTestFileSystem(conf); assertNotNull(fs); - AmazonS3 s3 = fs.getAmazonS3ClientForTesting("configuration"); - assertNotNull(s3); - S3ClientOptions clientOptions = getField(s3, S3ClientOptions.class, - "clientOptions"); + S3Client s3 = getS3Client("configuration"); + + SdkClientConfiguration clientConfiguration = getField(s3, SdkClientConfiguration.class, + "clientConfiguration"); + S3Configuration s3Configuration = + (S3Configuration)clientConfiguration.option(SdkClientOption.SERVICE_CONFIGURATION); assertTrue("Expected to find path style access to be switched on!", - clientOptions.isPathStyleAccess()); + s3Configuration.pathStyleAccessEnabled()); byte[] file = ContractTestUtils.toAsciiByteArray("test file"); ContractTestUtils.writeAndRead(fs, new Path("/path/style/access/testFile"), file, file.length, (int) conf.getLongBytes(Constants.FS_S3A_BLOCK_SIZE, file.length), false, true); - } catch (final AWSS3IOException e) { + } catch (final AWSRedirectException e) { LOG.error("Caught exception: ", e); // Catch/pass standard path style access behaviour when live bucket // isn't in the same region as the s3 client default. See // http://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html - assertEquals(HttpStatus.SC_MOVED_PERMANENTLY, e.getStatusCode()); + assertEquals(HttpStatus.SC_MOVED_PERMANENTLY, e.statusCode()); } catch (final IllegalArgumentException e) { // Path style addressing does not work with AP ARNs if (!fs.getBucket().contains("arn:")) { @@ -378,50 +406,53 @@ public void shouldBeAbleToSwitchOnS3PathStyleAccessViaConfigProperty() @Test public void testDefaultUserAgent() throws Exception { conf = new Configuration(); + skipIfCrossRegionClient(conf); fs = S3ATestUtils.createTestFileSystem(conf); assertNotNull(fs); - AmazonS3 s3 = fs.getAmazonS3ClientForTesting("User Agent"); - assertNotNull(s3); - ClientConfiguration awsConf = getField(s3, ClientConfiguration.class, + S3Client s3 = getS3Client("User Agent"); + SdkClientConfiguration clientConfiguration = getField(s3, SdkClientConfiguration.class, "clientConfiguration"); - assertEquals("Hadoop " + VersionInfo.getVersion(), - awsConf.getUserAgentPrefix()); + Assertions.assertThat(clientConfiguration.option(SdkClientOption.CLIENT_USER_AGENT)) + .describedAs("User Agent prefix") + .startsWith("Hadoop " + VersionInfo.getVersion()); } @Test public void testCustomUserAgent() throws Exception { conf = new Configuration(); + skipIfCrossRegionClient(conf); conf.set(Constants.USER_AGENT_PREFIX, "MyApp"); fs = S3ATestUtils.createTestFileSystem(conf); assertNotNull(fs); - AmazonS3 s3 = fs.getAmazonS3ClientForTesting("User agent"); - assertNotNull(s3); - ClientConfiguration awsConf = getField(s3, ClientConfiguration.class, + S3Client s3 = getS3Client("User agent"); + SdkClientConfiguration clientConfiguration = getField(s3, SdkClientConfiguration.class, "clientConfiguration"); - assertEquals("MyApp, Hadoop " + VersionInfo.getVersion(), - awsConf.getUserAgentPrefix()); + Assertions.assertThat(clientConfiguration.option(SdkClientOption.CLIENT_USER_AGENT)) + .describedAs("User Agent prefix") + .startsWith("MyApp, Hadoop " + VersionInfo.getVersion()); } @Test public void testRequestTimeout() throws Exception { conf = new Configuration(); + skipIfCrossRegionClient(conf); conf.set(REQUEST_TIMEOUT, "120"); fs = S3ATestUtils.createTestFileSystem(conf); - AmazonS3 s3 = fs.getAmazonS3ClientForTesting("Request timeout (ms)"); - ClientConfiguration awsConf = getField(s3, ClientConfiguration.class, + S3Client s3 = getS3Client("Request timeout (ms)"); + SdkClientConfiguration clientConfiguration = getField(s3, SdkClientConfiguration.class, "clientConfiguration"); assertEquals("Configured " + REQUEST_TIMEOUT + " is different than what AWS sdk configuration uses internally", - 120000, awsConf.getRequestTimeout()); + 120000, + clientConfiguration.option(SdkClientOption.API_CALL_ATTEMPT_TIMEOUT).toMillis()); } @Test - @SuppressWarnings("deprecation") public void testCloseIdempotent() throws Throwable { conf = new Configuration(); fs = S3ATestUtils.createTestFileSystem(conf); AWSCredentialProviderList credentials = - fs.shareCredentials("testCloseIdempotent"); + getS3AInternals().shareCredentials("testCloseIdempotent"); credentials.close(); fs.close(); assertTrue("Closing FS didn't close credentials " + credentials, @@ -514,64 +545,86 @@ public void testConfOptionPropagationToFS() throws Exception { } @Test(timeout = 10_000L) - public void testConnectTtlPropagation() throws Exception { - Configuration config = new Configuration(false); - ClientConfiguration awsConf = new ClientConfiguration(); - initConnectionSettings(config, awsConf); - Assertions.assertThat(awsConf.getConnectionTTL()) - .describedAs("connection ttl should be set to default value as" + - " %s is not set", CONNECTION_TTL) - .isEqualTo(DEFAULT_CONNECTION_TTL); - long connectionTtlTestVal = 1000; - config.setLong(CONNECTION_TTL, connectionTtlTestVal); - initConnectionSettings(config, awsConf); - Assertions.assertThat(awsConf.getConnectionTTL()) - .describedAs("%s not propagated to aws conf", CONNECTION_TTL) - .isEqualTo(connectionTtlTestVal); - - long connectionTtlTestVal1 = -1; - config.setLong(CONNECTION_TTL, connectionTtlTestVal1); - initConnectionSettings(config, awsConf); - Assertions.assertThat(awsConf.getConnectionTTL()) - .describedAs("%s not propagated to aws conf", CONNECTION_TTL) - .isEqualTo(connectionTtlTestVal1); - - long connectionTtlTestVal2 = -100; - config.setLong(CONNECTION_TTL, connectionTtlTestVal2); - intercept(IllegalArgumentException.class, () -> initConnectionSettings(config, awsConf)); + public void testS3SpecificSignerOverride() throws Exception { + Configuration config = new Configuration(); + removeBaseAndBucketOverrides(config, + CUSTOM_SIGNERS, SIGNING_ALGORITHM_S3, SIGNING_ALGORITHM_STS, AWS_REGION); + + config.set(CUSTOM_SIGNERS, + "CustomS3Signer:" + CustomS3Signer.class.getName() + + ",CustomSTSSigner:" + CustomSTSSigner.class.getName()); + + config.set(SIGNING_ALGORITHM_S3, "CustomS3Signer"); + config.set(SIGNING_ALGORITHM_STS, "CustomSTSSigner"); + + config.set(AWS_REGION, EU_WEST_1); + disableFilesystemCaching(config); + fs = S3ATestUtils.createTestFileSystem(config); + + S3Client s3Client = getS3Client("testS3SpecificSignerOverride"); + + final String bucket = fs.getBucket(); + StsClient stsClient = + STSClientFactory.builder(config, bucket, new AnonymousAWSCredentialsProvider(), "", + "").build(); + + intercept(StsException.class, "", () -> + stsClient.getSessionToken()); + + final IOException ioe = intercept(IOException.class, "", () -> + Invoker.once("head", bucket, () -> + s3Client.headBucket(HeadBucketRequest.builder().bucket(bucket).build()))); + + Assertions.assertThat(CustomS3Signer.isS3SignerCalled()) + .describedAs("Custom S3 signer not called").isTrue(); + + Assertions.assertThat(CustomSTSSigner.isSTSSignerCalled()) + .describedAs("Custom STS signer not called").isTrue(); } - @Test(timeout = 10_000L) - public void testS3SpecificSignerOverride() throws IOException { - ClientConfiguration clientConfiguration = null; - Configuration config; - - String signerOverride = "testSigner"; - String s3SignerOverride = "testS3Signer"; - - // Default SIGNING_ALGORITHM, overridden for S3 only - config = new Configuration(); - config.set(SIGNING_ALGORITHM_S3, s3SignerOverride); - clientConfiguration = S3AUtils - .createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_S3); - Assert.assertEquals(s3SignerOverride, - clientConfiguration.getSignerOverride()); - clientConfiguration = S3AUtils - .createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_STS); - Assert.assertNull(clientConfiguration.getSignerOverride()); - - // Configured base SIGNING_ALGORITHM, overridden for S3 only - config = new Configuration(); - config.set(SIGNING_ALGORITHM, signerOverride); - config.set(SIGNING_ALGORITHM_S3, s3SignerOverride); - clientConfiguration = S3AUtils - .createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_S3); - Assert.assertEquals(s3SignerOverride, - clientConfiguration.getSignerOverride()); - clientConfiguration = S3AUtils - .createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_STS); - Assert - .assertEquals(signerOverride, clientConfiguration.getSignerOverride()); + public static final class CustomS3Signer implements Signer { + + private static boolean s3SignerCalled = false; + + @Override + public SdkHttpFullRequest sign(SdkHttpFullRequest request, + ExecutionAttributes executionAttributes) { + LOG.debug("Custom S3 signer called"); + s3SignerCalled = true; + return request; + } + + public static boolean isS3SignerCalled() { + return s3SignerCalled; + } + } + + public static final class CustomSTSSigner implements Signer { + + private static boolean stsSignerCalled = false; + + @Override + public SdkHttpFullRequest sign(SdkHttpFullRequest request, + ExecutionAttributes executionAttributes) { + LOG.debug("Custom STS signer called"); + stsSignerCalled = true; + return request; + } + + public static boolean isSTSSignerCalled() { + return stsSignerCalled; + } } + /** + * Skip a test if client created is cross region. + * @param configuration configuration to probe + */ + private static void skipIfCrossRegionClient( + Configuration configuration) { + if (configuration.get(ENDPOINT, null) == null + && configuration.get(AWS_REGION, null) == null) { + skip("Skipping test as cross region client is in use "); + } + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContentEncoding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContentEncoding.java index 4a96bf5da91d7..ed86143100ac3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContentEncoding.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContentEncoding.java @@ -29,12 +29,16 @@ import org.apache.hadoop.fs.contract.ContractTestUtils; import static org.apache.hadoop.fs.s3a.Constants.CONTENT_ENCODING; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.raiseAsAssumption; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfNotEnabled; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_CONTENT_ENCODING; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.decodeBytes; /** * Tests of content encoding object meta data. + * Some stores don't support gzip; rejection of the content encoding + * is downgraded to a skipped test. */ public class ITestS3AContentEncoding extends AbstractS3ATestBase { @@ -43,6 +47,8 @@ public class ITestS3AContentEncoding extends AbstractS3ATestBase { @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); + skipIfNotEnabled(conf, KEY_CONTENT_ENCODING_ENABLED, + "Skipping storage class ACL tests"); removeBaseAndBucketOverrides(conf, CONTENT_ENCODING); conf.set(CONTENT_ENCODING, GZIP); @@ -51,20 +57,25 @@ protected Configuration createConfiguration() { @Test public void testCreatedObjectsHaveEncoding() throws Throwable { - S3AFileSystem fs = getFileSystem(); - Path dir = methodPath(); - fs.mkdirs(dir); - // even with content encoding enabled, directories do not have - // encoding. - Assertions.assertThat(getEncoding(dir)) - .describedAs("Encoding of object %s", dir) - .isNull(); - Path path = new Path(dir, "1"); - ContractTestUtils.touch(fs, path); - assertObjectHasEncoding(path); - Path path2 = new Path(dir, "2"); - fs.rename(path, path2); - assertObjectHasEncoding(path2); + try { + S3AFileSystem fs = getFileSystem(); + Path dir = methodPath(); + fs.mkdirs(dir); + // even with content encoding enabled, directories do not have + // encoding. + Assertions.assertThat(getEncoding(dir)) + .describedAs("Encoding of object %s", dir) + .isNull(); + Path path = new Path(dir, "1"); + ContractTestUtils.touch(fs, path); + assertObjectHasEncoding(path); + Path path2 = new Path(dir, "2"); + fs.rename(path, path2); + assertObjectHasEncoding(path2); + } catch (AWSUnsupportedFeatureException e) { + LOG.warn("Object store does not support {} content encoding", GZIP, e); + raiseAsAssumption(e); + } } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContractGetFileStatusV1List.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContractGetFileStatusV1List.java index 6d950d9bfaaa4..e07324300197b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContractGetFileStatusV1List.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AContractGetFileStatusV1List.java @@ -24,7 +24,9 @@ import org.apache.hadoop.fs.contract.s3a.S3AContract; import static org.apache.hadoop.fs.s3a.Constants.LIST_VERSION; +import static org.apache.hadoop.fs.s3a.S3ATestConstants.KEY_LIST_V1_ENABLED; import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfNotEnabled; /** * S3A contract tests for getFileStatus, using the v1 List Objects API. @@ -48,6 +50,8 @@ public void teardown() throws Exception { protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); disableFilesystemCaching(conf); + skipIfNotEnabled(conf, KEY_LIST_V1_ENABLED, + "Skipping V1 listing tests"); conf.setInt(Constants.MAX_PAGING_KEYS, 2); // Use v1 List Objects API diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADSSEEncryptionWithDefaultS3Settings.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADSSEEncryptionWithDefaultS3Settings.java new file mode 100644 index 0000000000000..a39490174424c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ADSSEEncryptionWithDefaultS3Settings.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.IOException; + +import org.junit.Ignore; +import org.junit.Test; + +import org.apache.commons.lang3.StringUtils; +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.auth.delegation.EncryptionSecrets; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; +import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; +import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_ALGORITHM; +import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM; +import static org.apache.hadoop.fs.s3a.EncryptionTestUtils.AWS_KMS_DSSE_ALGORITHM; +import static org.apache.hadoop.fs.s3a.S3AEncryptionMethods.DSSE_KMS; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfEncryptionNotSet; +import static org.apache.hadoop.fs.s3a.S3AUtils.getS3EncryptionKey; + +/** + * Concrete class that extends {@link AbstractTestS3AEncryption} + * and tests already configured bucket level DSSE encryption using s3 console. + */ +public class ITestS3ADSSEEncryptionWithDefaultS3Settings extends + AbstractTestS3AEncryption { + + @Override + public void setup() throws Exception { + super.setup(); + // get the KMS key for this test. + S3AFileSystem fs = getFileSystem(); + Configuration c = fs.getConf(); + skipIfEncryptionNotSet(c, getSSEAlgorithm()); + } + + @SuppressWarnings("deprecation") + @Override + protected void patchConfigurationEncryptionSettings( + final Configuration conf) { + removeBaseAndBucketOverrides(conf, + S3_ENCRYPTION_ALGORITHM, + SERVER_SIDE_ENCRYPTION_ALGORITHM); + conf.set(S3_ENCRYPTION_ALGORITHM, + getSSEAlgorithm().getMethod()); + } + + /** + * Setting this to NONE as we don't want to overwrite + * already configured encryption settings. + * @return the algorithm + */ + @Override + protected S3AEncryptionMethods getSSEAlgorithm() { + return S3AEncryptionMethods.NONE; + } + + /** + * The check here is that the object is encrypted + * and that the encryption key is the KMS key + * provided, not any default key. + * @param path path + */ + @Override + protected void assertEncrypted(Path path) throws IOException { + S3AFileSystem fs = getFileSystem(); + Configuration c = fs.getConf(); + String kmsKey = getS3EncryptionKey(getTestBucketName(c), c); + EncryptionTestUtils.assertEncrypted(fs, path, DSSE_KMS, kmsKey); + } + + @Override + @Ignore + @Test + public void testEncryptionSettingPropagation() throws Throwable { + } + + @Override + @Ignore + @Test + public void testEncryption() throws Throwable { + } + + /** + * Skipping if the test bucket is not configured with + * aws:kms encryption algorithm. + */ + @Override + public void testEncryptionOverRename() throws Throwable { + skipIfBucketNotKmsEncrypted(); + super.testEncryptionOverRename(); + } + + /** + * If the test bucket is not configured with aws:kms encryption algorithm, + * skip the test. + * + * @throws IOException If the object creation/deletion/access fails. + */ + private void skipIfBucketNotKmsEncrypted() throws IOException { + S3AFileSystem fs = getFileSystem(); + Path path = methodPath(); + ContractTestUtils.touch(fs, path); + try { + String sseAlgorithm = + getS3AInternals().getObjectMetadata(path).serverSideEncryptionAsString(); + if (StringUtils.isBlank(sseAlgorithm) || !sseAlgorithm.equals(AWS_KMS_DSSE_ALGORITHM)) { + skip("Test bucket is not configured with " + AWS_KMS_DSSE_ALGORITHM); + } + } finally { + ContractTestUtils.assertDeleted(fs, path, false); + } + } + + @Test + public void testEncryptionOverRename2() throws Throwable { + skipIfBucketNotKmsEncrypted(); + S3AFileSystem fs = getFileSystem(); + + // write the file with the unencrypted FS. + // this will pick up whatever defaults we have. + Path src = path(createFilename(1024)); + byte[] data = dataset(1024, 'a', 'z'); + EncryptionSecrets secrets = fs.getEncryptionSecrets(); + validateEncryptionSecrets(secrets); + writeDataset(fs, src, data, data.length, 1024 * 1024, true); + ContractTestUtils.verifyFileContents(fs, src, data); + + Configuration fs2Conf = new Configuration(fs.getConf()); + fs2Conf.set(S3_ENCRYPTION_ALGORITHM, + DSSE_KMS.getMethod()); + try (FileSystem kmsFS = FileSystem.newInstance(fs.getUri(), fs2Conf)) { + Path targetDir = path("target"); + kmsFS.mkdirs(targetDir); + ContractTestUtils.rename(kmsFS, src, targetDir); + Path renamedFile = new Path(targetDir, src.getName()); + ContractTestUtils.verifyFileContents(fs, renamedFile, data); + String kmsKey = getS3EncryptionKey(getTestBucketName(fs2Conf), fs2Conf); + // we assert that the renamed file has picked up the KMS key of our FS + EncryptionTestUtils.assertEncrypted(fs, renamedFile, DSSE_KMS, kmsKey); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionDSSEKMSUserDefinedKey.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionDSSEKMSUserDefinedKey.java new file mode 100644 index 0000000000000..028ba7f6e1fe0 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionDSSEKMSUserDefinedKey.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.IOException; +import java.io.UncheckedIOException; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.conf.Configuration; + +import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_KEY; +import static org.apache.hadoop.fs.s3a.S3AEncryptionMethods.DSSE_KMS; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfEncryptionNotSet; + +/** + * Concrete class that extends {@link AbstractTestS3AEncryption} + * and tests DSSE-KMS encryption. + */ +public class ITestS3AEncryptionDSSEKMSUserDefinedKey + extends AbstractTestS3AEncryption { + + @Override + protected Configuration createConfiguration() { + // get the KMS key for this test. + Configuration c = new Configuration(); + String kmsKey = S3AUtils.getS3EncryptionKey(getTestBucketName(c), c); + // skip the test if DSSE-KMS or KMS key not set. + try { + skipIfEncryptionNotSet(c, DSSE_KMS); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + assume("KMS key is expected to be present", StringUtils.isNotBlank(kmsKey)); + Configuration conf = super.createConfiguration(); + conf.set(S3_ENCRYPTION_KEY, kmsKey); + return conf; + } + + @Override + protected S3AEncryptionMethods getSSEAlgorithm() { + return DSSE_KMS; + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java index 64e37bf832b87..45b0c6c206f2d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java @@ -63,7 +63,7 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption { private static final String SERVICE_AMAZON_S3_STATUS_CODE_403 - = "Service: Amazon S3; Status Code: 403;"; + = "Service: S3, Status Code: 403"; private static final String KEY_1 = "4niV/jPK5VFRHY+KNb6wtqYd4xXyMgdJ9XQJpcQUVbs="; private static final String KEY_2 diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSDefaultKey.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSDefaultKey.java index 68ab5bd9e8c19..7e399f347100f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSDefaultKey.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSDefaultKey.java @@ -20,7 +20,7 @@ import java.io.IOException; -import com.amazonaws.services.s3.model.ObjectMetadata; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -51,9 +51,9 @@ protected S3AEncryptionMethods getSSEAlgorithm() { @Override protected void assertEncrypted(Path path) throws IOException { - ObjectMetadata md = getFileSystem().getObjectMetadata(path); + HeadObjectResponse md = getS3AInternals().getObjectMetadata(path); assertEquals("SSE Algorithm", EncryptionTestUtils.AWS_KMS_SSE_ALGORITHM, - md.getSSEAlgorithm()); - assertThat(md.getSSEAwsKmsKeyId(), containsString("arn:aws:kms:")); + md.serverSideEncryptionAsString()); + assertThat(md.ssekmsKeyId(), containsString("arn:aws:kms:")); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionWithDefaultS3Settings.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionWithDefaultS3Settings.java index a0fb762d89914..a6e373fcbdcce 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionWithDefaultS3Settings.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionWithDefaultS3Settings.java @@ -115,19 +115,34 @@ public void testEncryption() throws Throwable { */ @Override public void testEncryptionOverRename() throws Throwable { + skipIfBucketNotKmsEncrypted(); + super.testEncryptionOverRename(); + } + + /** + * If the test bucket is not configured with aws:kms encryption algorithm, + * skip the test. + * + * @throws IOException If the object creation/deletion/access fails. + */ + private void skipIfBucketNotKmsEncrypted() throws IOException { S3AFileSystem fs = getFileSystem(); Path path = path(getMethodName() + "find-encryption-algo"); ContractTestUtils.touch(fs, path); - String sseAlgorithm = fs.getObjectMetadata(path).getSSEAlgorithm(); - if(StringUtils.isBlank(sseAlgorithm) || - !sseAlgorithm.equals(AWS_KMS_SSE_ALGORITHM)) { - skip("Test bucket is not configured with " + AWS_KMS_SSE_ALGORITHM); + try { + String sseAlgorithm = + getS3AInternals().getObjectMetadata(path).serverSideEncryptionAsString(); + if (StringUtils.isBlank(sseAlgorithm) || !sseAlgorithm.equals(AWS_KMS_SSE_ALGORITHM)) { + skip("Test bucket is not configured with " + AWS_KMS_SSE_ALGORITHM); + } + } finally { + ContractTestUtils.assertDeleted(fs, path, false); } - super.testEncryptionOverRename(); } @Test public void testEncryptionOverRename2() throws Throwable { + skipIfBucketNotKmsEncrypted(); S3AFileSystem fs = getFileSystem(); // write the file with the unencrypted FS. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java index add6502d7da71..5d10590dfe30f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java @@ -21,21 +21,31 @@ import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; +import java.net.UnknownHostException; +import java.nio.file.AccessDeniedException; +import java.util.ArrayList; +import java.util.List; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.client.builder.AwsClientBuilder; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.util.AwsHostNameUtils; import org.assertj.core.api.Assertions; +import org.junit.Ignore; import org.junit.Test; +import software.amazon.awssdk.awscore.AwsExecutionAttribute; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.interceptor.Context; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.HeadBucketRequest; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION; -import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_CENTRAL_REGION; -import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ENDPOINT; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AWS_REGION_SYSPROP; +import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.io.IOUtils.closeStream; +import static org.apache.hadoop.fs.s3a.Constants.CENTRAL_ENDPOINT; + import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** @@ -44,114 +54,204 @@ */ public class ITestS3AEndpointRegion extends AbstractS3ATestBase { - private static final String AWS_REGION_TEST = "test-region"; private static final String AWS_ENDPOINT_TEST = "test-endpoint"; - private static final String AWS_ENDPOINT_TEST_WITH_REGION = - "test-endpoint.some-region.amazonaws.com"; - public static final String MARS_NORTH_2 = "mars-north-2"; + + private static final String US_EAST_1 = "us-east-1"; + + private static final String US_EAST_2 = "us-east-2"; + + private static final String US_WEST_2 = "us-west-2"; + + private static final String EU_WEST_2 = "eu-west-2"; + + private static final String CN_NORTHWEST_1 = "cn-northwest-1"; + + private static final String US_GOV_EAST_1 = "us-gov-east-1"; /** - * Test to verify that setting a region with the config would bypass the - * construction of region from endpoint. + * If anyone were ever to create a bucket with this UUID pair it would break the tests. */ - @Test - public void testWithRegionConfig() { - getFileSystem().getConf().set(AWS_REGION, AWS_REGION_TEST); - - //Creating an endpoint config with a custom endpoint. - AwsClientBuilder.EndpointConfiguration epr = createEpr(AWS_ENDPOINT_TEST, - getFileSystem().getConf().getTrimmed(AWS_REGION)); - //Checking if setting region config bypasses the endpoint region. - Assertions.assertThat(epr.getSigningRegion()) - .describedAs("There is a region mismatch") - .isEqualTo(getFileSystem().getConf().get(AWS_REGION)); - } + public static final String UNKNOWN_BUCKET = "23FA76D4-5F17-48B8-9D7D-9050269D0E40" + + "-8281BAF2-DBCF-47AA-8A27-F2FA3589656A"; + + private static final String EU_WEST_2_ENDPOINT = "s3.eu-west-2.amazonaws.com"; + + private static final String CN_ENDPOINT = "s3.cn-northwest-1.amazonaws.com.cn"; + + private static final String GOV_ENDPOINT = "s3-fips.us-gov-east-1.amazonaws.com"; + + private static final String VPC_ENDPOINT = "vpce-1a2b3c4d-5e6f.s3.us-west-2.vpce.amazonaws.com"; /** - * Test to verify that not setting the region config, would lead to using - * endpoint to construct the region. + * New FS instance which will be closed in teardown. */ + private S3AFileSystem newFS; + + @Override + public void teardown() throws Exception { + closeStream(newFS); + super.teardown(); + } + @Test - public void testWithoutRegionConfig() { - getFileSystem().getConf().unset(AWS_REGION); - - //Creating an endpoint config with a custom endpoint containing a region. - AwsClientBuilder.EndpointConfiguration eprRandom = - createEpr(AWS_ENDPOINT_TEST_WITH_REGION, - getFileSystem().getConf().getTrimmed(AWS_REGION)); - String regionFromEndpoint = - AwsHostNameUtils - .parseRegionFromAwsPartitionPattern(AWS_ENDPOINT_TEST_WITH_REGION); - //Checking if not setting region config leads to constructing the region - // from endpoint. - Assertions.assertThat(eprRandom.getSigningRegion()) - .describedAs("There is a region mismatch") - .isNotEqualTo(getFileSystem().getConf().get(AWS_REGION)) - .isEqualTo(regionFromEndpoint); + public void testWithoutRegionConfig() throws IOException { + describe("Verify that region lookup takes place"); + + Configuration conf = getConfiguration(); + removeBaseAndBucketOverrides(conf, AWS_REGION, PATH_STYLE_ACCESS); + conf.setBoolean(PATH_STYLE_ACCESS, false); + + newFS = new S3AFileSystem(); + + try { + newFS.initialize(getFileSystem().getUri(), conf); + newFS.getS3AInternals().getBucketMetadata(); + } catch (UnknownHostException | UnknownStoreException | AccessDeniedException allowed) { + // these are all valid failure modes from different test environments. + } } - /** - * Method to create EndpointConfiguration using an endpoint. - * - * @param endpoint the endpoint to be used for EndpointConfiguration creation. - * @return an instance of EndpointConfiguration. - */ - private AwsClientBuilder.EndpointConfiguration createEpr(String endpoint, - String awsRegion) { - return DefaultS3ClientFactory.createEndpointConfiguration(endpoint, - new ClientConfiguration(), awsRegion); + @Test + public void testUnknownBucket() throws Exception { + describe("Verify unknown bucket probe failure"); + Configuration conf = getConfiguration(); + removeBaseAndBucketOverrides(UNKNOWN_BUCKET, conf, AWS_REGION, PATH_STYLE_ACCESS); + + newFS = new S3AFileSystem(); + + try { + newFS.initialize(new URI("s3a://" + UNKNOWN_BUCKET), conf); + newFS.getS3AInternals().getBucketMetadata(); + // expect a failure by here + fail("Expected failure, got " + newFS); + } catch (UnknownHostException | UnknownStoreException expected) { + // this is good. + } } + @Test + public void testEndpointOverride() throws Throwable { + describe("Create a client with a configured endpoint"); + Configuration conf = getConfiguration(); + + S3Client client = createS3Client(conf, AWS_ENDPOINT_TEST, null, US_EAST_2); + + intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket( + HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build())); + } @Test - public void testInvalidRegionDefaultEndpoint() throws Throwable { - describe("Create a client with an invalid region and the default endpoint"); + public void testCentralEndpoint() throws Throwable { + describe("Create a client with the central endpoint"); Configuration conf = getConfiguration(); - // we are making a big assumption about the timetable for AWS - // region rollout. - // if this test ever fails because this region now exists - // -congratulations! - conf.set(AWS_REGION, MARS_NORTH_2); - createMarsNorth2Client(conf); + + S3Client client = createS3Client(conf, CENTRAL_ENDPOINT, null, US_EAST_1); + + intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket( + HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build())); } @Test - public void testUnsetRegionDefaultEndpoint() throws Throwable { - describe("Create a client with no region and the default endpoint"); + public void testWithRegionConfig() throws Throwable { + describe("Create a client with a configured region"); Configuration conf = getConfiguration(); - conf.unset(AWS_REGION); - createS3Client(conf, DEFAULT_ENDPOINT, AWS_S3_CENTRAL_REGION); + + S3Client client = createS3Client(conf, null, EU_WEST_2, EU_WEST_2); + + intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket( + HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build())); + } + + + public void testEUWest2Endpoint() throws Throwable { + describe("Create a client with the eu west 2 endpoint"); + Configuration conf = getConfiguration(); + + S3Client client = createS3Client(conf, EU_WEST_2_ENDPOINT, null, EU_WEST_2); + + intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket( + HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build())); } - /** - * By setting the system property {@code "aws.region"} we can - * guarantee that the SDK region resolution chain will always succeed - * (and fast). - * Clearly there is no validation of the region during the build process. - */ @Test - public void testBlankRegionTriggersSDKResolution() throws Throwable { - describe("Create a client with a blank region and the default endpoint." - + " This will trigger the SDK Resolution chain"); + public void testWithRegionAndEndpointConfig() throws Throwable { + describe("Test that when both region and endpoint are configured, region takes precedence"); Configuration conf = getConfiguration(); - conf.set(AWS_REGION, ""); - System.setProperty(AWS_REGION_SYSPROP, MARS_NORTH_2); - try { - createMarsNorth2Client(conf); - } finally { - System.clearProperty(AWS_REGION_SYSPROP); - } + + S3Client client = createS3Client(conf, EU_WEST_2_ENDPOINT, US_WEST_2, US_WEST_2); + + intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket( + HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build())); } - /** - * Create an S3 client bonded to an invalid region; - * verify that calling {@code getRegion()} triggers - * a failure. - * @param conf configuration to use in the building. - */ - private void createMarsNorth2Client(Configuration conf) throws Exception { - AmazonS3 client = createS3Client(conf, DEFAULT_ENDPOINT, MARS_NORTH_2); - intercept(IllegalArgumentException.class, MARS_NORTH_2, client::getRegion); + @Test + public void testWithChinaEndpoint() throws Throwable { + describe("Test with a china endpoint"); + Configuration conf = getConfiguration(); + + S3Client client = createS3Client(conf, CN_ENDPOINT, null, CN_NORTHWEST_1); + + intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket( + HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build())); + } + + @Test + public void testWithGovCloudEndpoint() throws Throwable { + describe("Test with a gov cloud endpoint"); + Configuration conf = getConfiguration(); + + S3Client client = createS3Client(conf, GOV_ENDPOINT, null, US_GOV_EAST_1); + + intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket( + HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build())); + } + + @Test + @Ignore("Pending HADOOP-18938. S3A region logic to handle vpce and non standard endpoints") + public void testWithVPCE() throws Throwable { + describe("Test with vpc endpoint"); + Configuration conf = getConfiguration(); + + S3Client client = createS3Client(conf, VPC_ENDPOINT, null, US_WEST_2); + + intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket( + HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build())); + } + + class RegionInterceptor implements ExecutionInterceptor { + private String endpoint; + private String region; + + RegionInterceptor(String endpoint, String region) { + this.endpoint = endpoint; + this.region = region; + } + + @Override + public void beforeExecution(Context.BeforeExecution context, + ExecutionAttributes executionAttributes) { + + if (endpoint != null) { + Assertions.assertThat( + executionAttributes.getAttribute(AwsExecutionAttribute.ENDPOINT_OVERRIDDEN)) + .describedAs("Endpoint not overridden").isTrue(); + + Assertions.assertThat( + executionAttributes.getAttribute(AwsExecutionAttribute.CLIENT_ENDPOINT).toString()) + .describedAs("There is an endpoint mismatch").isEqualTo("https://" + endpoint); + } else { + Assertions.assertThat( + executionAttributes.getAttribute(AwsExecutionAttribute.ENDPOINT_OVERRIDDEN)) + .describedAs("Endpoint is overridden").isEqualTo(null); + } + + Assertions.assertThat( + executionAttributes.getAttribute(AwsExecutionAttribute.AWS_REGION).toString()) + .describedAs("Incorrect region set").isEqualTo(region); + + // We don't actually want to make a request, so exit early. + throw AwsServiceException.builder().message("Exception thrown by interceptor").build(); + } } /** @@ -160,16 +260,18 @@ private void createMarsNorth2Client(Configuration conf) throws Exception { * value. * @param conf configuration to use. * @param endpoint endpoint. - * @param expectedRegion expected region + * @param expectedRegion the region that should be set in the client. * @return the client. * @throws URISyntaxException parse problems. * @throws IOException IO problems */ @SuppressWarnings("deprecation") - private AmazonS3 createS3Client(Configuration conf, - String endpoint, - String expectedRegion) - throws URISyntaxException, IOException { + private S3Client createS3Client(Configuration conf, + String endpoint, String configuredRegion, String expectedRegion) + throws IOException { + + List interceptors = new ArrayList<>(); + interceptors.add(new RegionInterceptor(endpoint, expectedRegion)); DefaultS3ClientFactory factory = new DefaultS3ClientFactory(); @@ -177,16 +279,16 @@ private AmazonS3 createS3Client(Configuration conf, S3ClientFactory.S3ClientCreationParameters parameters = new S3ClientFactory.S3ClientCreationParameters() .withCredentialSet(new AnonymousAWSCredentialsProvider()) - .withPathUri(new URI("s3a://localhost/")) .withEndpoint(endpoint) .withMetrics(new EmptyS3AStatisticsContext() - .newStatisticsFromAwsSdk()); - AmazonS3 client = factory.createS3Client( - new URI("s3a://localhost/"), + .newStatisticsFromAwsSdk()) + .withExecutionInterceptors(interceptors) + .withRegion(configuredRegion); + + + S3Client client = factory.createS3Client( + getFileSystem().getUri(), parameters); - Assertions.assertThat(client.getRegionName()) - .describedAs("Client region name") - .isEqualTo(expectedRegion); return client; } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java index c0f6a4b23226b..7c53f9c32a66a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java @@ -18,7 +18,8 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; +import software.amazon.awssdk.services.s3.model.S3Error; import org.assertj.core.api.Assertions; import org.junit.Assume; @@ -27,6 +28,7 @@ import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteException; import org.apache.hadoop.fs.statistics.StoreStatisticNames; import org.apache.hadoop.fs.store.audit.AuditSpan; @@ -34,13 +36,16 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.nio.file.AccessDeniedException; +import java.util.stream.Collectors; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.createFiles; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.isBulkDeleteEnabled; import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.failIf; import static org.apache.hadoop.test.LambdaTestUtils.*; import static org.apache.hadoop.util.functional.RemoteIterators.mappingRemoteIterator; @@ -85,7 +90,22 @@ public void testMultiObjectDeleteLargeNumKeys() throws Exception { S3AFileSystem fs = getFileSystem(); Path path = path("largeDir"); mkdirs(path); - createFiles(fs, path, 1, 1005, 0); + final boolean bulkDeleteEnabled = isBulkDeleteEnabled(getFileSystem()); + + // with single object delete, only create a few files for a faster + // test run. + int filesToCreate; + int pages; + if (bulkDeleteEnabled) { + filesToCreate = 1005; + pages = 5; + } else { + filesToCreate = 250; + pages = 0; + } + + + createFiles(fs, path, 1, filesToCreate, 0); RemoteIterator locatedFileStatusRemoteIterator = fs.listFiles(path, false); List keys = toList(mappingRemoteIterator(locatedFileStatusRemoteIterator, @@ -98,9 +118,10 @@ public void testMultiObjectDeleteLargeNumKeys() throws Exception { } Long bulkDeleteReqAfter = getNumberOfBulkDeleteRequestsMadeTillNow(fs); // number of delete requests is 5 as we have default page size of 250. + Assertions.assertThat(bulkDeleteReqAfter - bulkDeleteReqBefore) .describedAs("Number of batched bulk delete requests") - .isEqualTo(5); + .isEqualTo(pages); } private Long getNumberOfBulkDeleteRequestsMadeTillNow(S3AFileSystem fs) { @@ -115,12 +136,12 @@ private void removeKeys(S3AFileSystem fileSystem, String... keys) } } - private List buildDeleteRequest( + private List buildDeleteRequest( final String[] keys) { - List request = new ArrayList<>( + List request = new ArrayList<>( keys.length); for (String key : keys) { - request.add(new DeleteObjectsRequest.KeyVersion(key)); + request.add(ObjectIdentifier.builder().key(key).build()); } return request; } @@ -156,12 +177,26 @@ public void testMultiObjectDeleteNoPermissions() throws Throwable { // create a span, expect it to be activated. fs.getAuditSpanSource().createSpan(StoreStatisticNames.OP_DELETE, csvPath.toString(), null); - List keys + List keys = buildDeleteRequest( new String[]{ fs.pathToKey(csvPath), "missing-key.csv" }); + MultiObjectDeleteException ex = intercept( + MultiObjectDeleteException.class, + () -> fs.removeKeys(keys, false)); + final List undeleted = ex.errors().stream() + .map(S3Error::key) + .map(fs::keyToQualifiedPath) + .collect(Collectors.toList()); + final String undeletedFiles = undeleted.stream() + .map(Path::toString) + .collect(Collectors.joining(", ")); + failIf(undeleted.size() != 2, + "undeleted list size wrong: " + undeletedFiles, + ex); + assertTrue("no CSV in " +undeletedFiles, undeleted.contains(csvPath)); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java index 1a944ec29968a..66a211181bdde 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java @@ -25,19 +25,22 @@ import java.nio.charset.StandardCharsets; import java.nio.file.AccessDeniedException; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.GetBucketEncryptionResult; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.GetBucketEncryptionRequest; +import software.amazon.awssdk.services.s3.model.GetBucketEncryptionResponse; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.Test; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; +import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.store.EtagChecksum; import org.apache.hadoop.test.LambdaTestUtils; @@ -106,15 +109,15 @@ public void testCreateNonRecursiveSuccess() throws IOException { public void testPutObjectDirect() throws Throwable { final S3AFileSystem fs = getFileSystem(); try (AuditSpan span = span()) { - ObjectMetadata metadata = fs.newObjectMetadata(-1); - metadata.setContentLength(-1); + RequestFactory factory = RequestFactoryImpl.builder().withBucket(fs.getBucket()).build(); Path path = path("putDirect"); - final PutObjectRequest put = new PutObjectRequest(fs.getBucket(), - path.toUri().getPath(), - new ByteArrayInputStream("PUT".getBytes()), - metadata); + PutObjectRequest.Builder putObjectRequestBuilder = + factory.newPutObjectRequestBuilder(path.toUri().getPath(), null, -1, false); + putObjectRequestBuilder.contentLength(-1L); LambdaTestUtils.intercept(IllegalStateException.class, - () -> fs.putObjectDirect(put, PutObjectOptions.keepingDirs(), null)); + () -> fs.putObjectDirect(putObjectRequestBuilder.build(), PutObjectOptions.keepingDirs(), + new S3ADataBlocks.BlockUploadData(new ByteArrayInputStream("PUT".getBytes())), + false, null)); assertPathDoesNotExist("put object was created", path); } } @@ -179,14 +182,8 @@ public void testEmptyFileChecksums() throws Throwable { * non-null. */ private void assumeNoDefaultEncryption() throws IOException { - try { - skipIfClientSideEncryption(); - Assume.assumeThat(getDefaultEncryption(), nullValue()); - } catch (AccessDeniedException e) { - // if the user can't check the default encryption, assume that it is - // null and keep going - LOG.warn("User does not have permission to call getBucketEncryption()"); - } + skipIfClientSideEncryption(); + Assume.assumeThat(getDefaultEncryption(), nullValue()); } /** @@ -406,14 +403,16 @@ private static T verifyNoTrailingSlash(String role, T o) { * Gets default encryption settings for the bucket or returns null if default * encryption is disabled. */ - private GetBucketEncryptionResult getDefaultEncryption() throws IOException { + private GetBucketEncryptionResponse getDefaultEncryption() throws IOException { S3AFileSystem fs = getFileSystem(); - AmazonS3 s3 = fs.getAmazonS3ClientForTesting("check default encryption"); - try { + S3Client s3 = getS3AInternals().getAmazonS3Client("check default encryption"); + try (AuditSpan s = span()){ return Invoker.once("getBucketEncryption()", fs.getBucket(), - () -> s3.getBucketEncryption(fs.getBucket())); - } catch (FileNotFoundException e) { + () -> s3.getBucketEncryption(GetBucketEncryptionRequest.builder() + .bucket(fs.getBucket()) + .build())); + } catch (FileNotFoundException | AccessDeniedException | AWSBadRequestException e) { return null; } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java index 818d2fc889c17..e0559b7c49edc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java @@ -18,17 +18,20 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.services.s3.model.MultipartUpload; +import org.assertj.core.api.Assertions; +import org.junit.Test; +import software.amazon.awssdk.services.s3.model.MultipartUpload; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.store.audit.AuditSpan; -import org.junit.Test; - import java.io.IOException; import java.util.HashSet; import java.util.Set; +import static org.apache.hadoop.util.functional.RemoteIterators.foreach; /** * Tests for {@link MultipartUtils}. @@ -74,7 +77,7 @@ public void testListMultipartUploads() throws Exception { // 2. Verify all uploads are found listing by prefix describe("Verifying upload list by prefix"); - MultipartUtils.UploadIterator uploads = fs.listUploads(getPartPrefix(fs)); + RemoteIterator uploads = fs.listUploads(getPartPrefix(fs)); assertUploadsPresent(uploads, keySet); // 3. Verify all uploads are found listing without prefix @@ -95,26 +98,27 @@ public void testListMultipartUploads() throws Exception { * @param ourUploads set up uploads that should be present * @throws IOException on I/O error */ - private void assertUploadsPresent(MultipartUtils.UploadIterator list, + private void assertUploadsPresent(RemoteIterator list, Set ourUploads) throws IOException { // Don't modify passed-in set, use copy. Set uploads = new HashSet<>(ourUploads); - while (list.hasNext()) { - MultipartTestUtils.IdKey listing = toIdKey(list.next()); - if (uploads.contains(listing)) { + foreach(list, (upload) -> { + MultipartTestUtils.IdKey listing = toIdKey(upload); + if (uploads.remove(listing)) { LOG.debug("Matched: {},{}", listing.getKey(), listing.getUploadId()); - uploads.remove(listing); } else { LOG.debug("Not our upload {},{}", listing.getKey(), listing.getUploadId()); } - } - assertTrue("Not all our uploads were listed", uploads.isEmpty()); + }); + Assertions.assertThat(uploads) + .describedAs("Uploads which we expected to be listed.") + .isEmpty(); } private MultipartTestUtils.IdKey toIdKey(MultipartUpload mu) { - return new MultipartTestUtils.IdKey(mu.getKey(), mu.getUploadId()); + return new MultipartTestUtils.IdKey(mu.key(), mu.uploadId()); } private Path getPartFilename(int index) throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java index 6ad8ef58a7f68..e678df700b871 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java @@ -105,6 +105,7 @@ public synchronized void teardown() throws Exception { @Test public void testCacheFileExistence() throws Throwable { describe("Verify that FS cache files exist on local FS"); + skipIfClientSideEncryption(); try (FSDataInputStream in = fs.open(testFile)) { byte[] buffer = new byte[prefetchBlockSize]; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java index a7b59bb5d4624..4998cbc946e12 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java @@ -106,6 +106,7 @@ private static int calculateNumBlocks(long largeFileSize, int blockSize) { @Test public void testReadLargeFileFully() throws Throwable { describe("read a large file fully, uses S3ACachingInputStream"); + skipIfClientSideEncryption(); IOStatistics ioStats; createLargeFile(); @@ -139,6 +140,7 @@ public void testReadLargeFileFully() throws Throwable { public void testReadLargeFileFullyLazySeek() throws Throwable { describe("read a large file using readFully(position,buffer,offset,length)," + " uses S3ACachingInputStream"); + skipIfClientSideEncryption(); IOStatistics ioStats; createLargeFile(); @@ -170,6 +172,7 @@ public void testReadLargeFileFullyLazySeek() throws Throwable { @Test public void testRandomReadLargeFile() throws Throwable { describe("random read on a large file, uses S3ACachingInputStream"); + skipIfClientSideEncryption(); IOStatistics ioStats; createLargeFile(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index d3925d35a99d3..b88d0b4aab617 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -59,7 +59,7 @@ protected Configuration createConfiguration() { @Test public void testRequesterPaysOptionSuccess() throws Throwable { describe("Test requester pays enabled case by reading last then first byte"); - + skipIfClientSideEncryption(); Configuration conf = this.createConfiguration(); conf.setBoolean(ALLOW_REQUESTER_PAYS, true); // Enable bucket exists check, the first failure point people may encounter @@ -107,7 +107,7 @@ public void testRequesterPaysDisabledFails() throws Throwable { try (FileSystem fs = requesterPaysPath.getFileSystem(conf)) { intercept( AccessDeniedException.class, - "403 Forbidden", + "403", "Expected requester pays bucket to fail without header set", () -> fs.open(requesterPaysPath).close() ); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AStorageClass.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AStorageClass.java index 7c56f8d2ea050..56cb541e8233a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AStorageClass.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AStorageClass.java @@ -37,7 +37,6 @@ import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER; import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER_ARRAY; import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BUFFER_DISK; -import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD_BYTEBUFFER; import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS; import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_GLACIER; import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_REDUCED_REDUNDANCY; @@ -75,6 +74,7 @@ public ITestS3AStorageClass(String fastUploadBufferType) { @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); + skipIfStorageClassTestsDisabled(conf); disableFilesystemCaching(conf); removeBaseAndBucketOverrides(conf, STORAGE_CLASS, FAST_UPLOAD_BUFFER); conf.set(FAST_UPLOAD_BUFFER, fastUploadBufferType); @@ -82,12 +82,6 @@ protected Configuration createConfiguration() { return conf; } - @Override - public void setup() throws Exception { - super.setup(); - skipIfStorageClassTestsDisabled(getConfiguration()); - } - /* * This test ensures the default storage class configuration (no config or null) * works well with create and copy operations diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java index 0778662542d88..290a4d995c757 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ATemporaryCredentials.java @@ -25,10 +25,9 @@ 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.Credentials; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.StsClientBuilder; +import software.amazon.awssdk.services.sts.model.Credentials; import org.hamcrest.Matchers; import org.junit.Test; import org.slf4j.Logger; @@ -71,7 +70,6 @@ public class ITestS3ATemporaryCredentials extends AbstractS3ATestBase { private static final Logger LOG = LoggerFactory.getLogger(ITestS3ATemporaryCredentials.class); - @SuppressWarnings("deprecation") private static final String TEMPORARY_AWS_CREDENTIALS = TemporaryAWSCredentialsProvider.NAME; @@ -117,10 +115,10 @@ protected Configuration createConfiguration() { public void testSTS() throws IOException { Configuration conf = getContract().getConf(); S3AFileSystem testFS = getFileSystem(); - credentials = testFS.shareCredentials("testSTS"); + credentials = getS3AInternals().shareCredentials("testSTS"); String bucket = testFS.getBucket(); - AWSSecurityTokenServiceClientBuilder builder = STSClientFactory.builder( + StsClientBuilder builder = STSClientFactory.builder( conf, bucket, credentials, @@ -154,7 +152,7 @@ public void testSTS() throws IOException { // now create an invalid set of credentials by changing the session // token - conf2.set(SESSION_TOKEN, "invalid-" + sessionCreds.getSessionToken()); + conf2.set(SESSION_TOKEN, "invalid-" + sessionCreds.sessionToken()); try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf2)) { createAndVerifyFile(fs, path("testSTSInvalidToken"), TEST_FILE_SIZE); fail("Expected an access exception, but file access to " @@ -176,14 +174,13 @@ protected String getStsRegion(final Configuration conf) { } @Test - @SuppressWarnings("deprecation") public void testTemporaryCredentialValidation() throws Throwable { Configuration conf = new Configuration(); conf.set(ACCESS_KEY, "accesskey"); conf.set(SECRET_KEY, "secretkey"); conf.set(SESSION_TOKEN, ""); LambdaTestUtils.intercept(CredentialInitializationException.class, - () -> new TemporaryAWSCredentialsProvider(conf).getCredentials()); + () -> new TemporaryAWSCredentialsProvider(conf).resolveCredentials()); } /** @@ -360,25 +357,24 @@ public void testSessionCredentialsEndpointNoRegion() throws Throwable { * @return the caught exception. * @throws Exception any unexpected exception. */ - @SuppressWarnings("deprecation") public E expectedSessionRequestFailure( final Class clazz, final String endpoint, final String region, final String exceptionText) throws Exception { try(AWSCredentialProviderList parentCreds = - getFileSystem().shareCredentials("test"); + getS3AInternals().shareCredentials("test"); DurationInfo ignored = new DurationInfo(LOG, "requesting credentials")) { Configuration conf = new Configuration(getContract().getConf()); - ClientConfiguration awsConf = - S3AUtils.createAwsConf(conf, null, AWS_SERVICE_IDENTIFIER_STS); + return intercept(clazz, exceptionText, () -> { - AWSSecurityTokenService tokenService = + StsClient tokenService = STSClientFactory.builder(parentCreds, - awsConf, + conf, endpoint, - region) + region, + getFileSystem().getBucket()) .build(); Invoker invoker = new Invoker(new S3ARetryPolicy(conf), LOG_AT_ERROR); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java index 658ceb49173f3..adc4ef4c7c868 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java @@ -21,9 +21,9 @@ import java.io.IOException; import java.net.URI; -import com.amazonaws.AmazonClientException; -import com.amazonaws.AmazonWebServiceRequest; -import com.amazonaws.services.s3.AmazonS3; +import software.amazon.awssdk.core.SdkRequest; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.s3.S3Client; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,6 +51,7 @@ import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.util.Progressable; + import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.noopAuditor; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTrackerFactory; import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.checkNotNull; @@ -116,9 +117,7 @@ public MockS3AFileSystem(S3AFileSystem mock, root = new Path(FS_URI.toString()); } - private static T prepareRequest(T t) { - return t; - } + private static void prepareRequest(SdkRequest.Builder t) {} @Override public RequestFactory getRequestFactory() { @@ -210,7 +209,7 @@ public boolean isMultipartUploadEnabled() { * @param client client. */ @Override - public void setAmazonS3Client(AmazonS3 client) { + public void setAmazonS3Client(S3Client client) { LOG.debug("Setting S3 client to {}", client); super.setAmazonS3Client(client); } @@ -353,13 +352,13 @@ public long getDefaultBlockSize() { void deleteObjectAtPath(Path f, String key, boolean isFile) - throws AmazonClientException, IOException { + throws SdkException, IOException { deleteObject(key); } @Override protected void maybeCreateFakeParentDirectory(Path path) - throws IOException, AmazonClientException { + throws IOException, SdkException { // no-op } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java index 3240309aef971..0c61caacd055b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java @@ -23,32 +23,47 @@ import java.net.URI; import java.util.ArrayList; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.MultipartUploadListing; -import com.amazonaws.services.s3.model.Region; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest; +import software.amazon.awssdk.services.s3.model.GetBucketLocationResponse; +import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest; +import software.amazon.awssdk.services.s3.model.ListMultipartUploadsResponse; +import software.amazon.awssdk.transfer.s3.S3TransferManager; /** - * An {@link S3ClientFactory} that returns Mockito mocks of the {@link AmazonS3} + * An {@link S3ClientFactory} that returns Mockito mocks of the {@link S3Client} * interface suitable for unit testing. */ -@SuppressWarnings("deprecation") public class MockS3ClientFactory implements S3ClientFactory { + @Override - public AmazonS3 createS3Client(URI uri, - final S3ClientCreationParameters parameters) { - AmazonS3 s3 = mock(AmazonS3.class); - String bucket = uri.getHost(); - when(s3.doesBucketExist(bucket)).thenReturn(true); - when(s3.doesBucketExistV2(bucket)).thenReturn(true); + public S3Client createS3Client(URI uri, final S3ClientCreationParameters parameters) { + S3Client s3 = mock(S3Client.class); // this listing is used in startup if purging is enabled, so // return a stub value - MultipartUploadListing noUploads = new MultipartUploadListing(); - noUploads.setMultipartUploads(new ArrayList<>(0)); - when(s3.listMultipartUploads(any())) - .thenReturn(noUploads); - when(s3.getBucketLocation(anyString())) - .thenReturn(Region.US_West.toString()); + ListMultipartUploadsResponse noUploads = ListMultipartUploadsResponse.builder() + .uploads(new ArrayList<>(0)) + .isTruncated(false) + .build(); + when(s3.listMultipartUploads((ListMultipartUploadsRequest) any())).thenReturn(noUploads); + when(s3.getBucketLocation((GetBucketLocationRequest) any())).thenReturn( + GetBucketLocationResponse.builder().locationConstraint(Region.US_WEST_2.toString()) + .build()); + return s3; + } + + @Override + public S3AsyncClient createS3AsyncClient(URI uri, final S3ClientCreationParameters parameters) { + S3AsyncClient s3 = mock(S3AsyncClient.class); return s3; } + + @Override + public S3TransferManager createS3TransferManager(S3AsyncClient s3AsyncClient) { + S3TransferManager tm = mock(S3TransferManager.class); + return tm; + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java index 1ddff3c4cd5e9..3f6870be46b2a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java @@ -18,10 +18,12 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.services.s3.model.MultipartUpload; -import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.services.s3.model.MultipartUpload; +import software.amazon.awssdk.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.services.s3.model.UploadPartResponse; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.io.IOUtils; @@ -33,6 +35,7 @@ import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; +import java.util.Date; import java.util.List; import java.util.Objects; import java.util.Set; @@ -80,10 +83,11 @@ public static IdKey createPartUpload(S3AFileSystem fs, String key, int len, byte[] data = dataset(len, 'a', 'z'); InputStream in = new ByteArrayInputStream(data); String uploadId = writeHelper.initiateMultiPartUpload(key, PutObjectOptions.keepingDirs()); - UploadPartRequest req = writeHelper.newUploadPartRequest(key, uploadId, - partNo, len, in, null, 0L); - PartETag partEtag = writeHelper.uploadPart(req, null).getPartETag(); - LOG.debug("uploaded part etag {}, upid {}", partEtag.getETag(), uploadId); + UploadPartRequest req = writeHelper.newUploadPartRequestBuilder(key, uploadId, + partNo, len).build(); + RequestBody body = RequestBody.fromInputStream(in, len); + UploadPartResponse response = writeHelper.uploadPart(req, body, null); + LOG.debug("uploaded part etag {}, upid {}", response.eTag(), uploadId); return new IdKey(key, uploadId); } } @@ -93,16 +97,16 @@ public static void clearAnyUploads(S3AFileSystem fs, Path path) { String key = fs.pathToKey(path); AuditSpan span = null; try { - MultipartUtils.UploadIterator uploads = fs.listUploads(key); + RemoteIterator uploads = fs.listUploads(key); span = fs.createSpan("multipart", path.toString(), null); final WriteOperationHelper helper = fs.getWriteOperationHelper(); while (uploads.hasNext()) { MultipartUpload upload = uploads.next(); - LOG.debug("Cleaning up upload: {} {}", upload.getKey(), - truncatedUploadId(upload.getUploadId())); - helper.abortMultipartUpload(upload.getKey(), - upload.getUploadId(), true, LOG_EVENT); + LOG.debug("Cleaning up upload: {} {}", upload.key(), + truncatedUploadId(upload.uploadId())); + helper.abortMultipartUpload(upload.key(), + upload.uploadId(), true, LOG_EVENT); } } catch (IOException ioe) { LOG.info("Ignoring exception: ", ioe); @@ -115,11 +119,11 @@ public static void clearAnyUploads(S3AFileSystem fs, Path path) { public static void assertNoUploadsAt(S3AFileSystem fs, Path path) throws Exception { String key = fs.pathToKey(path); - MultipartUtils.UploadIterator uploads = fs.listUploads(key); + RemoteIterator uploads = fs.listUploads(key); while (uploads.hasNext()) { MultipartUpload upload = uploads.next(); - Assert.fail("Found unexpected upload " + upload.getKey() + " " + - truncatedUploadId(upload.getUploadId())); + Assert.fail("Found unexpected upload " + upload.key() + " " + + truncatedUploadId(upload.uploadId())); } } @@ -127,7 +131,7 @@ public static void assertNoUploadsAt(S3AFileSystem fs, Path path) throws public static int countUploadsAt(S3AFileSystem fs, Path path) throws IOException { String key = fs.pathToKey(path); - MultipartUtils.UploadIterator uploads = fs.listUploads(key); + RemoteIterator uploads = fs.listUploads(key); int count = 0; while (uploads.hasNext()) { MultipartUpload upload = uploads.next(); @@ -149,9 +153,9 @@ public static List listMultipartUploads(S3AFileSystem fs, return fs .listMultipartUploads(prefix).stream() .map(upload -> String.format("Upload to %s with ID %s; initiated %s", - upload.getKey(), - upload.getUploadId(), - S3ATestUtils.LISTING_FORMAT.format(upload.getInitiated()))) + upload.key(), + upload.uploadId(), + S3ATestUtils.LISTING_FORMAT.format(Date.from(upload.initiated())))) .collect(Collectors.toList()); } } @@ -196,5 +200,13 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(key, uploadId); } + + @Override + public String toString() { + return "IdKey{" + + "key='" + key + '\'' + + ", uploadId='" + uploadId + '\'' + + '}'; + } } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java index a6269c437665a..f2e0223adb1d9 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java @@ -58,6 +58,21 @@ public interface S3ATestConstants { */ String KEY_STORAGE_CLASS_TESTS_ENABLED = TEST_FS_S3A + "create.storage.class.enabled"; + /** + * A property set to true if ACL tests are enabled: {@value}. + */ + String KEY_ACL_TESTS_ENABLED = TEST_FS_S3A + "create.acl.enabled"; + + /** + * A property set to true if V1 tests are enabled: {@value}. + */ + String KEY_LIST_V1_ENABLED = TEST_FS_S3A + "list.v1.enabled"; + + /** + * A property set to true if content encoding tests are enabled: {@value}. + */ + String KEY_CONTENT_ENCODING_ENABLED = TEST_FS_S3A + "content.encoding.enabled"; + /** * Tell tests that they are being executed in parallel: {@value}. */ @@ -251,4 +266,9 @@ public interface S3ATestConstants { * Value: {@value}. */ String PROJECT_BUILD_DIRECTORY_PROPERTY = "project.build.directory"; + + /** + * AWS ireland region. + */ + String EU_WEST_1 = "eu-west-1"; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index 469562f9b33b9..b12e2d4b0af35 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -61,12 +61,13 @@ import org.apache.hadoop.util.functional.CallableRaisingIOE; import org.apache.hadoop.util.functional.FutureIO; -import com.amazonaws.auth.AWSCredentialsProvider; import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; +import org.junit.AssumptionViolatedException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import java.io.Closeable; import java.io.File; @@ -77,6 +78,7 @@ import java.text.DateFormat; import java.text.SimpleDateFormat; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Set; import java.util.TreeSet; @@ -435,17 +437,22 @@ public static Path getLandsatCSVPath(Configuration conf) { * @param clazz the expected exception class * @param ex the exception caught * @return the exception, if it is of the expected class - * @throws Exception the exception passed in. + * @throws AssertionError if the exception is {@code null}. + * @throws Exception the exception passed in if it is of a different type */ public static E verifyExceptionClass(Class clazz, Exception ex) throws Exception { + Assertions.assertThat(ex) + .describedAs("Exception expected of class %s", clazz) + .isNotNull(); if (!(ex.getClass().equals(clazz))) { throw ex; } return (E)ex; } + /** * Turn off FS Caching: use if a filesystem with different options from * the default is required. @@ -461,8 +468,20 @@ public static void disableFilesystemCaching(Configuration conf) { */ public static void skipIfEncryptionTestsDisabled( Configuration configuration) { - if (!configuration.getBoolean(KEY_ENCRYPTION_TESTS, true)) { - skip("Skipping encryption tests"); + skipIfNotEnabled(configuration, KEY_ENCRYPTION_TESTS, "Skipping encryption tests"); + } + + /** + * Skip a test suite/casee if a configuration has been explicitly disabled. + * @param configuration configuration to probe + * @param key key to resolve + * @param message assertion text + */ + public static void skipIfNotEnabled(final Configuration configuration, + final String key, + final String message) { + if (!configuration.getBoolean(key, true)) { + skip(message); } } @@ -472,9 +491,18 @@ public static void skipIfEncryptionTestsDisabled( */ public static void skipIfStorageClassTestsDisabled( Configuration configuration) { - if (!configuration.getBoolean(KEY_STORAGE_CLASS_TESTS_ENABLED, true)) { - skip("Skipping storage class tests"); - } + skipIfNotEnabled(configuration, KEY_STORAGE_CLASS_TESTS_ENABLED, + "Skipping storage class tests"); + } + + /** + * Skip a test if ACL class tests are disabled. + * @param configuration configuration to probe + */ + public static void skipIfACLTestsDisabled( + Configuration configuration) { + skipIfNotEnabled(configuration, KEY_ACL_TESTS_ENABLED, + "Skipping storage class ACL tests"); } /** @@ -612,8 +640,7 @@ public static void unsetHadoopCredentialProviders(final Configuration conf) { * @return a set of credentials * @throws IOException on a failure */ - @SuppressWarnings("deprecation") - public static AWSCredentialsProvider buildAwsCredentialsProvider( + public static AwsCredentialsProvider buildAwsCredentialsProvider( final Configuration conf) throws IOException { assumeSessionTestsEnabled(conf); @@ -631,9 +658,7 @@ public static AWSCredentialsProvider buildAwsCredentialsProvider( * @param conf configuration to examine */ public static void assumeSessionTestsEnabled(final Configuration conf) { - if (!conf.getBoolean(TEST_STS_ENABLED, true)) { - skip("STS functional tests disabled"); - } + skipIfNotEnabled(conf, TEST_STS_ENABLED, "STS functional tests disabled"); } /** @@ -668,13 +693,14 @@ public static MarshalledCredentials requestSessionCredentials( MarshalledCredentials sc = MarshalledCredentialBinding .requestSessionCredentials( buildAwsCredentialsProvider(conf), - S3AUtils.createAwsConf(conf, bucket, AWS_SERVICE_IDENTIFIER_STS), + conf, 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)); + new Invoker(new S3ARetryPolicy(conf), Invoker.LOG_EVENT), + bucket); sc.validate("requested session credentials: ", MarshalledCredentials.CredentialTypeRequired.SessionOnly); return sc; @@ -1240,6 +1266,14 @@ public static void assume(String message, boolean condition) { Assume.assumeTrue(message, condition); } + /** + * Convert a throwable to an assumption failure. + * @param t thrown exception. + */ + public static void raiseAsAssumption(Throwable t) { + throw new AssumptionViolatedException(t.toString(), t); + } + /** * Get the statistics from a wrapped block output stream. * @param out output stream @@ -1450,19 +1484,25 @@ public static S3AFileStatus innerGetFileStatus( * Skip a test if encryption algorithm or encryption key is not set. * * @param configuration configuration to probe. + * @param s3AEncryptionMethods list of encryption algorithms to probe. + * @throws IOException if the secret lookup fails. */ public static void skipIfEncryptionNotSet(Configuration configuration, - S3AEncryptionMethods s3AEncryptionMethod) throws IOException { + S3AEncryptionMethods... s3AEncryptionMethods) throws IOException { + if (s3AEncryptionMethods == null || s3AEncryptionMethods.length == 0) { + throw new IllegalArgumentException("Specify at least one encryption method"); + } // if S3 encryption algorithm is not set to desired method or AWS encryption // key is not set, then skip. String bucket = getTestBucketName(configuration); final EncryptionSecrets secrets = buildEncryptionSecrets(bucket, configuration); - if (!s3AEncryptionMethod.getMethod().equals(secrets.getEncryptionMethod().getMethod()) - || StringUtils.isBlank(secrets.getEncryptionKey())) { - skip(S3_ENCRYPTION_KEY + " is not set for " + s3AEncryptionMethod - .getMethod() + " or " + S3_ENCRYPTION_ALGORITHM + " is not set to " - + s3AEncryptionMethod.getMethod() - + " in " + secrets); + boolean encryptionMethodMatching = Arrays.stream(s3AEncryptionMethods).anyMatch( + s3AEncryptionMethod -> s3AEncryptionMethod.getMethod() + .equals(secrets.getEncryptionMethod().getMethod())); + if (!encryptionMethodMatching || StringUtils.isBlank(secrets.getEncryptionKey())) { + skip(S3_ENCRYPTION_KEY + " is not set or " + S3_ENCRYPTION_ALGORITHM + " is not set to " + + Arrays.stream(s3AEncryptionMethods).map(S3AEncryptionMethods::getMethod) + .collect(Collectors.toList()) + " in " + secrets); } } @@ -1510,4 +1550,15 @@ public static S3AInputStream getS3AInputStream( public static void disablePrefetching(Configuration conf) { removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY); } + + /** + * Does this FS support multi object delete? + * @param fs filesystem + * @return true if multi-delete is enabled. + */ + + public static boolean isBulkDeleteEnabled(FileSystem fs) { + return fs.getConf().getBoolean(Constants.ENABLE_MULTI_DELETE, + true); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestArnResource.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestArnResource.java index c881aac35d94d..28d0ca56f0eae 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestArnResource.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestArnResource.java @@ -18,12 +18,13 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.regions.Regions; +import software.amazon.awssdk.regions.Region; import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.test.HadoopTestBase; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -47,9 +48,9 @@ public void parseAccessPointFromArn() throws IllegalArgumentException { String accessPoint = "testAp"; String[][] regionPartitionEndpoints = new String[][] { - {Regions.EU_WEST_1.getName(), "aws"}, - {Regions.US_GOV_EAST_1.getName(), "aws-us-gov"}, - {Regions.CN_NORTH_1.getName(), "aws-cn"}, + {Region.EU_WEST_1.id(), "aws"}, + {Region.US_GOV_EAST_1.id(), "aws-us-gov"}, + {Region.CN_NORTH_1.id(), "aws-cn"}, }; for (String[] testPair : regionPartitionEndpoints) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInvoker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInvoker.java index 35199f4092790..f6e8f0c376d9b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInvoker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestInvoker.java @@ -22,14 +22,14 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.net.SocketTimeoutException; +import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; -import com.amazonaws.AmazonClientException; -import com.amazonaws.AmazonServiceException; -import com.amazonaws.SdkBaseException; -import com.amazonaws.SdkClientException; -import com.amazonaws.services.s3.model.AmazonS3Exception; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.s3.model.S3Exception; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -38,10 +38,12 @@ import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.net.ConnectTimeoutException; + import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.Invoker.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.verifyExceptionClass; import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_400_BAD_REQUEST; import static org.apache.hadoop.test.LambdaTestUtils.*; /** @@ -98,10 +100,12 @@ public class TestInvoker extends Assert { private int retryCount; private Invoker invoker = new Invoker(RETRY_POLICY, (text, e, retries, idempotent) -> retryCount++); - private static final AmazonClientException CLIENT_TIMEOUT_EXCEPTION = - new AmazonClientException(new Local.ConnectTimeoutException("timeout")); - private static final AmazonServiceException BAD_REQUEST = serviceException( - AWSBadRequestException.STATUS_CODE, + private static final SdkException CLIENT_TIMEOUT_EXCEPTION = + SdkException.builder() + .cause(new Local.ConnectTimeoutException("timeout")) + .build(); + private static final AwsServiceException BAD_REQUEST = serviceException( + SC_400_BAD_REQUEST, "bad request"); @Before @@ -109,24 +113,26 @@ public void setup() { resetCounters(); } - private static AmazonServiceException serviceException(int code, + private static AwsServiceException serviceException(int code, String text) { - AmazonServiceException ex = new AmazonServiceException(text); - ex.setStatusCode(code); - return ex; + return AwsServiceException.builder() + .message(text) + .statusCode(code) + .build(); } - private static AmazonS3Exception createS3Exception(int code) { + private static S3Exception createS3Exception(int code) { return createS3Exception(code, "", null); } - private static AmazonS3Exception createS3Exception(int code, + private static S3Exception createS3Exception(int code, String message, Throwable inner) { - AmazonS3Exception ex = new AmazonS3Exception(message); - ex.setStatusCode(code); - ex.initCause(inner); - return ex; + return (S3Exception) S3Exception.builder() + .message(message) + .statusCode(code) + .cause(inner) + .build(); } protected void verifyTranslated( @@ -136,7 +142,7 @@ protected void verifyTranslated( } private static E verifyTranslated(Class clazz, - SdkBaseException exception) throws Exception { + SdkException exception) throws Exception { return verifyExceptionClass(clazz, translateException("test", "/", exception)); } @@ -157,16 +163,22 @@ public void testS3500isStatus500Exception() throws Exception { @Test public void test500isStatus500Exception() throws Exception { - AmazonServiceException ex = new AmazonServiceException(""); - ex.setStatusCode(500); + AwsServiceException ex = AwsServiceException.builder() + .message("") + .statusCode(500) + .build(); verifyTranslated(AWSStatus500Exception.class, ex); } @Test public void testExceptionsWithTranslatableMessage() throws Exception { - SdkBaseException xmlParsing = new SdkBaseException(EOF_MESSAGE_IN_XML_PARSER); - SdkBaseException differentLength = new SdkBaseException(EOF_READ_DIFFERENT_LENGTH); + SdkException xmlParsing = SdkException.builder() + .message(EOF_MESSAGE_IN_XML_PARSER) + .build(); + SdkException differentLength = SdkException.builder() + .message(EOF_READ_DIFFERENT_LENGTH) + .build(); verifyTranslated(EOFException.class, xmlParsing); verifyTranslated(EOFException.class, differentLength); @@ -178,7 +190,9 @@ public void testSdkDifferentLengthExceptionIsTranslatable() throws Throwable { final AtomicInteger counter = new AtomicInteger(0); invoker.retry("test", null, false, () -> { if (counter.incrementAndGet() < ACTIVE_RETRY_LIMIT) { - throw new SdkClientException(EOF_READ_DIFFERENT_LENGTH); + throw SdkClientException.builder() + .message(EOF_READ_DIFFERENT_LENGTH) + .build(); } }); @@ -190,7 +204,9 @@ public void testSdkXmlParsingExceptionIsTranslatable() throws Throwable { final AtomicInteger counter = new AtomicInteger(0); invoker.retry("test", null, false, () -> { if (counter.incrementAndGet() < ACTIVE_RETRY_LIMIT) { - throw new SdkClientException(EOF_MESSAGE_IN_XML_PARSER); + throw SdkClientException.builder() + .message(EOF_MESSAGE_IN_XML_PARSER) + .build(); } }); @@ -201,14 +217,36 @@ public void testSdkXmlParsingExceptionIsTranslatable() throws Throwable { public void testExtractConnectTimeoutException() throws Throwable { throw extractException("", "", new ExecutionException( - new AmazonClientException(LOCAL_CONNECTION_TIMEOUT_EX))); + SdkException.builder() + .cause(LOCAL_CONNECTION_TIMEOUT_EX) + .build())); } @Test(expected = SocketTimeoutException.class) public void testExtractSocketTimeoutException() throws Throwable { throw extractException("", "", new ExecutionException( - new AmazonClientException(SOCKET_TIMEOUT_EX))); + SdkException.builder() + .cause(SOCKET_TIMEOUT_EX) + .build())); + } + + @Test(expected = org.apache.hadoop.net.ConnectTimeoutException.class) + public void testExtractConnectTimeoutExceptionFromCompletionException() throws Throwable { + throw extractException("", "", + new CompletionException( + SdkException.builder() + .cause(LOCAL_CONNECTION_TIMEOUT_EX) + .build())); + } + + @Test(expected = SocketTimeoutException.class) + public void testExtractSocketTimeoutExceptionFromCompletionException() throws Throwable { + throw extractException("", "", + new CompletionException( + SdkException.builder() + .cause(SOCKET_TIMEOUT_EX) + .build())); } /** @@ -259,7 +297,7 @@ public void testRetryThrottled() throws Throwable { ex, retries, false); } - protected AmazonServiceException newThrottledException() { + protected AwsServiceException newThrottledException() { return serviceException( AWSServiceThrottledException.STATUS_CODE, "throttled"); } @@ -354,7 +392,9 @@ public void testUnshadedConnectionTimeoutExceptionMatching() // connection timeout exceptions are special, but as AWS shades // theirs, we need to string match them verifyTranslated(ConnectTimeoutException.class, - new AmazonClientException(HTTP_CONNECTION_TIMEOUT_EX)); + SdkException.builder() + .cause(HTTP_CONNECTION_TIMEOUT_EX) + .build()); } @Test @@ -362,14 +402,18 @@ public void testShadedConnectionTimeoutExceptionMatching() throws Throwable { // connection timeout exceptions are special, but as AWS shades // theirs, we need to string match them verifyTranslated(ConnectTimeoutException.class, - new AmazonClientException(LOCAL_CONNECTION_TIMEOUT_EX)); + SdkException.builder() + .cause(LOCAL_CONNECTION_TIMEOUT_EX) + .build()); } @Test public void testShadedConnectionTimeoutExceptionNotMatching() throws Throwable { InterruptedIOException ex = verifyTranslated(InterruptedIOException.class, - new AmazonClientException(new Local.NotAConnectTimeoutException())); + SdkException.builder() + .cause(new Local.NotAConnectTimeoutException()) + .build()); if (ex instanceof ConnectTimeoutException) { throw ex; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java index 730bae0aeb101..f48fe5cd751f5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -30,31 +30,45 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import javax.annotation.Nullable; -import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.EnvironmentVariableCredentialsProvider; -import com.amazonaws.auth.InstanceProfileCredentialsProvider; -import org.apache.hadoop.thirdparty.com.google.common.collect.Sets; -import org.junit.Rule; +import org.assertj.core.api.Assertions; import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider; +import software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.auth.AbstractSessionCredentialsProvider; import org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider; +import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider; import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException; +import org.apache.hadoop.fs.s3a.impl.InstantiationIOException; import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.thirdparty.com.google.common.collect.Sets; -import static org.apache.hadoop.fs.s3a.Constants.*; -import static org.apache.hadoop.fs.s3a.S3ATestConstants.*; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; -import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.Constants.ASSUMED_ROLE_CREDENTIALS_PROVIDER; +import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER; +import static org.apache.hadoop.fs.s3a.S3ATestConstants.DEFAULT_CSVTEST_FILE; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.authenticationContains; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.buildClassListString; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getCSVTestPath; +import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.STANDARD_AWS_PROVIDERS; +import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.buildAWSProviderList; +import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSCredentialProviderList; +import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.DOES_NOT_IMPLEMENT; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; /** * Unit tests for {@link Constants#AWS_CREDENTIALS_PROVIDER} logic. @@ -67,19 +81,18 @@ public class TestS3AAWSCredentialsProvider { private static final URI TESTFILE_URI = new Path( DEFAULT_CSVTEST_FILE).toUri(); - @Rule - public ExpectedException exception = ExpectedException.none(); + private static final Logger LOG = LoggerFactory.getLogger(TestS3AAWSCredentialsProvider.class); @Test public void testProviderWrongClass() throws Exception { expectProviderInstantiationFailure(this.getClass(), - NOT_AWS_PROVIDER); + DOES_NOT_IMPLEMENT + " software.amazon.awssdk.auth.credentials.AwsCredentialsProvider"); } @Test public void testProviderAbstractClass() throws Exception { expectProviderInstantiationFailure(AbstractProvider.class, - ABSTRACT_PROVIDER); + InstantiationIOException.ABSTRACT_PROVIDER); } @Test @@ -92,18 +105,17 @@ public void testProviderNotAClass() throws Exception { public void testProviderConstructorError() throws Exception { expectProviderInstantiationFailure( ConstructorSignatureErrorProvider.class, - CONSTRUCTOR_EXCEPTION); + InstantiationIOException.CONSTRUCTOR_EXCEPTION); } @Test public void testProviderFailureError() throws Exception { expectProviderInstantiationFailure( ConstructorFailureProvider.class, - INSTANTIATION_EXCEPTION); + InstantiationIOException.INSTANTIATION_EXCEPTION); } @Test - @SuppressWarnings("deprecation") public void testInstantiationChain() throws Throwable { Configuration conf = new Configuration(false); conf.set(AWS_CREDENTIALS_PROVIDER, @@ -112,7 +124,7 @@ public void testInstantiationChain() throws Throwable { + " ,\n " + AnonymousAWSCredentialsProvider.NAME); Path testFile = getCSVTestPath(conf); - AWSCredentialProviderList list = createAWSCredentialProviderSet( + AWSCredentialProviderList list = createAWSCredentialProviderList( testFile.toUri(), conf); List> expectedClasses = Arrays.asList( @@ -123,15 +135,14 @@ public void testInstantiationChain() throws Throwable { } @Test - @SuppressWarnings("deprecation") public void testDefaultChain() throws Exception { URI uri1 = new URI("s3a://bucket1"), uri2 = new URI("s3a://bucket2"); Configuration conf = new Configuration(false); // use the default credential provider chain conf.unset(AWS_CREDENTIALS_PROVIDER); - AWSCredentialProviderList list1 = createAWSCredentialProviderSet( + AWSCredentialProviderList list1 = createAWSCredentialProviderList( uri1, conf); - AWSCredentialProviderList list2 = createAWSCredentialProviderSet( + AWSCredentialProviderList list2 = createAWSCredentialProviderList( uri2, conf); List> expectedClasses = STANDARD_AWS_PROVIDERS; assertCredentialProviders(expectedClasses, list1); @@ -144,30 +155,29 @@ public void testDefaultChainNoURI() throws Exception { // use the default credential provider chain conf.unset(AWS_CREDENTIALS_PROVIDER); assertCredentialProviders(STANDARD_AWS_PROVIDERS, - createAWSCredentialProviderSet(null, conf)); + createAWSCredentialProviderList(null, conf)); } @Test - @SuppressWarnings("deprecation") public void testConfiguredChain() throws Exception { URI uri1 = new URI("s3a://bucket1"), uri2 = new URI("s3a://bucket2"); List> expectedClasses = Arrays.asList( - EnvironmentVariableCredentialsProvider.class, - InstanceProfileCredentialsProvider.class, - AnonymousAWSCredentialsProvider.class); + IAMInstanceCredentialsProvider.class, + AnonymousAWSCredentialsProvider.class, + EnvironmentVariableCredentialsProvider.class + ); Configuration conf = createProviderConfiguration(buildClassListString(expectedClasses)); - AWSCredentialProviderList list1 = createAWSCredentialProviderSet( + AWSCredentialProviderList list1 = createAWSCredentialProviderList( uri1, conf); - AWSCredentialProviderList list2 = createAWSCredentialProviderSet( + AWSCredentialProviderList list2 = createAWSCredentialProviderList( uri2, conf); assertCredentialProviders(expectedClasses, list1); assertCredentialProviders(expectedClasses, list2); } @Test - @SuppressWarnings("deprecation") public void testConfiguredChainUsesSharedInstanceProfile() throws Exception { URI uri1 = new URI("s3a://bucket1"), uri2 = new URI("s3a://bucket2"); Configuration conf = new Configuration(false); @@ -175,9 +185,9 @@ public void testConfiguredChainUsesSharedInstanceProfile() throws Exception { Arrays.asList( InstanceProfileCredentialsProvider.class); conf.set(AWS_CREDENTIALS_PROVIDER, buildClassListString(expectedClasses)); - AWSCredentialProviderList list1 = createAWSCredentialProviderSet( + AWSCredentialProviderList list1 = createAWSCredentialProviderList( uri1, conf); - AWSCredentialProviderList list2 = createAWSCredentialProviderSet( + AWSCredentialProviderList list2 = createAWSCredentialProviderList( uri2, conf); assertCredentialProviders(expectedClasses, list1); assertCredentialProviders(expectedClasses, list2); @@ -194,54 +204,75 @@ public void testFallbackToDefaults() throws Throwable { EnvironmentVariableCredentialsProvider.class), Sets.newHashSet()); assertTrue("empty credentials", credentials.size() > 0); + } + + @Test + public void testProviderConstructor() throws Throwable { + final AWSCredentialProviderList list = new AWSCredentialProviderList("name", + new AnonymousAWSCredentialsProvider(), + new ErrorProvider(TESTFILE_URI, new Configuration())); + Assertions.assertThat(list.getProviders()) + .describedAs("provider list in %s", list) + .hasSize(2); + final AwsCredentials credentials = list.resolveCredentials(); + Assertions.assertThat(credentials) + .isInstanceOf(AwsBasicCredentials.class); + assertCredentialResolution(credentials, null, null); + } + + public static void assertCredentialResolution(AwsCredentials creds, String key, String secret) { + Assertions.assertThat(creds.accessKeyId()) + .describedAs("access key of %s", creds) + .isEqualTo(key); + Assertions.assertThat(creds.secretAccessKey()) + .describedAs("secret key of %s", creds) + .isEqualTo(secret); + } + + private String buildClassList(Class... classes) { + return Arrays.stream(classes) + .map(Class::getCanonicalName) + .collect(Collectors.joining(",")); + } + private String buildClassList(String... classes) { + return Arrays.stream(classes) + .collect(Collectors.joining(",")); } /** * A credential provider declared as abstract, so it cannot be instantiated. */ - static abstract class AbstractProvider implements AWSCredentialsProvider { + static abstract class AbstractProvider implements AwsCredentialsProvider { + + @Override + public AwsCredentials resolveCredentials() { + return null; + } } /** * A credential provider whose constructor signature doesn't match. */ protected static class ConstructorSignatureErrorProvider - implements AWSCredentialsProvider { + extends AbstractProvider { @SuppressWarnings("unused") public ConstructorSignatureErrorProvider(String str) { } - - @Override - public AWSCredentials getCredentials() { - return null; - } - - @Override - public void refresh() { - } } /** * A credential provider whose constructor raises an NPE. */ protected static class ConstructorFailureProvider - implements AWSCredentialsProvider { + extends AbstractProvider { @SuppressWarnings("unused") public ConstructorFailureProvider() { throw new NullPointerException("oops"); } - @Override - public AWSCredentials getCredentials() { - return null; - } - - @Override - public void refresh() { - } } @Test @@ -254,33 +285,23 @@ public void testAWSExceptionTranslation() throws Throwable { } } - protected static class AWSExceptionRaisingFactory implements AWSCredentialsProvider { + protected static class AWSExceptionRaisingFactory extends AbstractProvider { public static final String NO_AUTH = "No auth"; - public static AWSCredentialsProvider getInstance() { + public static AwsCredentialsProvider create() { 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); + InstantiationIOException.CONSTRUCTOR_EXCEPTION); } - static class FactoryOfWrongType implements AWSCredentialsProvider { + static class FactoryOfWrongType extends AbstractProvider { public static final String NO_AUTH = "No auth"; @@ -289,14 +310,10 @@ public static String getInstance() { } @Override - public AWSCredentials getCredentials() { + public AwsCredentials resolveCredentials() { return null; } - @Override - public void refresh() { - - } } /** @@ -309,7 +326,7 @@ public void refresh() { private IOException expectProviderInstantiationFailure(String option, String expectedErrorText) throws Exception { return intercept(IOException.class, expectedErrorText, - () -> createAWSCredentialProviderSet( + () -> createAWSCredentialProviderList( TESTFILE_URI, createProviderConfiguration(option))); } @@ -359,12 +376,14 @@ private static void assertCredentialProviders( List> expectedClasses, AWSCredentialProviderList list) { assertNotNull(list); - List providers = list.getProviders(); - assertEquals(expectedClasses.size(), providers.size()); + List providers = list.getProviders(); + Assertions.assertThat(providers) + .describedAs("providers") + .hasSize(expectedClasses.size()); for (int i = 0; i < expectedClasses.size(); ++i) { Class expectedClass = expectedClasses.get(i); - AWSCredentialsProvider provider = providers.get(i); + AwsCredentialsProvider provider = providers.get(i); assertNotNull( String.format("At position %d, expected class is %s, but found null.", i, expectedClass), provider); @@ -380,7 +399,6 @@ private static void assertCredentialProviders( * @see S3ATestUtils#authenticationContains(Configuration, String). */ @Test - @SuppressWarnings("deprecation") public void testAuthenticationContainsProbes() { Configuration conf = new Configuration(false); assertFalse("found AssumedRoleCredentialProvider", @@ -398,7 +416,7 @@ public void testExceptionLogic() throws Throwable { // verify you can't get credentials from it NoAuthWithAWSException noAuth = intercept(NoAuthWithAWSException.class, AWSCredentialProviderList.NO_AWS_CREDENTIAL_PROVIDERS, - () -> providers.getCredentials()); + () -> providers.resolveCredentials()); // but that it closes safely providers.close(); @@ -447,11 +465,10 @@ public void testRefCounting() throws Throwable { providers.close(); assertEquals("Ref count after close() for " + providers, 0, providers.getRefCount()); - providers.refresh(); intercept(NoAuthWithAWSException.class, AWSCredentialProviderList.CREDENTIALS_REQUESTED_WHEN_CLOSED, - () -> providers.getCredentials()); + () -> providers.resolveCredentials()); } /** @@ -470,35 +487,17 @@ public void testIOEInConstructorPropagation() throws Throwable { /** * Credential provider which raises an IOE when constructed. */ - protected static class IOERaisingProvider implements AWSCredentialsProvider { + protected static class IOERaisingProvider extends AbstractProvider { public IOERaisingProvider(URI uri, Configuration conf) throws IOException { throw new InterruptedIOException("expected"); } - @Override - public AWSCredentials getCredentials() { - return null; - } - - @Override - public void refresh() { - - } } - private static final AWSCredentials EXPECTED_CREDENTIALS = new AWSCredentials() { - @Override - public String getAWSAccessKeyId() { - return "expectedAccessKey"; - } - - @Override - public String getAWSSecretKey() { - return "expectedSecret"; - } - }; + private static final AwsCredentials EXPECTED_CREDENTIALS = + AwsBasicCredentials.create("expectedAccessKey", "expectedSecret"); /** * Credential provider that takes a long time. @@ -510,7 +509,7 @@ public SlowProvider(@Nullable URI uri, Configuration conf) { } @Override - protected AWSCredentials createCredentials(Configuration config) throws IOException { + protected AwsCredentials createCredentials(Configuration config) throws IOException { // yield to other callers to induce race condition Thread.yield(); return EXPECTED_CREDENTIALS; @@ -524,13 +523,13 @@ public void testConcurrentAuthentication() throws Throwable { Configuration conf = createProviderConfiguration(SlowProvider.class.getName()); Path testFile = getCSVTestPath(conf); - AWSCredentialProviderList list = createAWSCredentialProviderSet(testFile.toUri(), conf); + AWSCredentialProviderList list = createAWSCredentialProviderList(testFile.toUri(), conf); SlowProvider provider = (SlowProvider) list.getProviders().get(0); ExecutorService pool = Executors.newFixedThreadPool(CONCURRENT_THREADS); - List> results = new ArrayList<>(); + List> results = new ArrayList<>(); try { assertFalse( @@ -546,15 +545,15 @@ public void testConcurrentAuthentication() throws Throwable { } for (int i = 0; i < CONCURRENT_THREADS; i++) { - results.add(pool.submit(() -> list.getCredentials())); + results.add(pool.submit(() -> list.resolveCredentials())); } - for (Future result : results) { - AWSCredentials credentials = result.get(); + for (Future result : results) { + AwsCredentials credentials = result.get(); assertEquals("Access key from credential provider", - "expectedAccessKey", credentials.getAWSAccessKeyId()); + "expectedAccessKey", credentials.accessKeyId()); assertEquals("Secret key from credential provider", - "expectedSecret", credentials.getAWSSecretKey()); + "expectedSecret", credentials.secretAccessKey()); } } finally { pool.awaitTermination(10, TimeUnit.SECONDS); @@ -563,7 +562,7 @@ public void testConcurrentAuthentication() throws Throwable { assertTrue( "Provider initialized without errors. isInitialized should be true", - provider.isInitialized()); + provider.isInitialized()); assertTrue( "Provider initialized without errors. hasCredentials should be true", provider.hasCredentials()); @@ -584,7 +583,7 @@ public ErrorProvider(@Nullable URI uri, Configuration conf) { } @Override - protected AWSCredentials createCredentials(Configuration config) throws IOException { + protected AwsCredentials createCredentials(Configuration config) throws IOException { throw new IOException("expected error"); } } @@ -594,12 +593,12 @@ public void testConcurrentAuthenticationError() throws Throwable { Configuration conf = createProviderConfiguration(ErrorProvider.class.getName()); Path testFile = getCSVTestPath(conf); - AWSCredentialProviderList list = createAWSCredentialProviderSet(testFile.toUri(), conf); + AWSCredentialProviderList list = createAWSCredentialProviderList(testFile.toUri(), conf); ErrorProvider provider = (ErrorProvider) list.getProviders().get(0); ExecutorService pool = Executors.newFixedThreadPool(CONCURRENT_THREADS); - List> results = new ArrayList<>(); + List> results = new ArrayList<>(); try { assertFalse("Provider not initialized. isInitialized should be false", @@ -613,10 +612,10 @@ public void testConcurrentAuthenticationError() throws Throwable { } for (int i = 0; i < CONCURRENT_THREADS; i++) { - results.add(pool.submit(() -> list.getCredentials())); + results.add(pool.submit(() -> list.resolveCredentials())); } - for (Future result : results) { + for (Future result : results) { interceptFuture(CredentialInitializationException.class, "expected error", result @@ -637,4 +636,39 @@ public void testConcurrentAuthenticationError() throws Throwable { "Provider initialization failed. getInitializationException should contain the error", provider.getInitializationException().getMessage().contains("expected error")); } + + + /** + * V2 Credentials whose factory method raises ClassNotFoundException. + * This will fall back to an attempted v1 load which will fail because it + * is the wrong type. + * The exception raised will be from the v2 instantiation attempt, + * not the v1 attempt. + */ + @Test + public void testV2ClassNotFound() throws Throwable { + InstantiationIOException expected = intercept(InstantiationIOException.class, + "simulated v2 CNFE", + () -> createAWSCredentialProviderList( + TESTFILE_URI, + createProviderConfiguration(V2CredentialProviderDoesNotInstantiate.class.getName()))); + // print for the curious + LOG.info("{}", expected.toString()); + } + + /** + * V2 credentials which raises an instantiation exception in + * the factory method. + */ + public static final class V2CredentialProviderDoesNotInstantiate + extends AbstractProvider { + + private V2CredentialProviderDoesNotInstantiate() { + } + + public static AwsCredentialsProvider create() throws ClassNotFoundException { + throw new ClassNotFoundException("simulated v2 CNFE"); + } + } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java index ffa2c81e58adf..4f329afe7ad51 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java @@ -30,7 +30,6 @@ import org.junit.Before; import org.junit.Test; -import java.io.ByteArrayInputStream; import java.io.IOException; import java.util.concurrent.ExecutorService; @@ -105,16 +104,14 @@ public void testWriteOperationHelperPartLimits() throws Throwable { noopAuditor(conf), AuditTestSupport.NOOP_SPAN, new MinimalWriteOperationHelperCallbacks()); - ByteArrayInputStream inputStream = new ByteArrayInputStream( - "a".getBytes()); // first one works String key = "destKey"; - woh.newUploadPartRequest(key, - "uploadId", 1, 1024, inputStream, null, 0L); + woh.newUploadPartRequestBuilder(key, + "uploadId", 1, 1024); // but ask past the limit and a PathIOE is raised intercept(PathIOException.class, key, - () -> woh.newUploadPartRequest(key, - "uploadId", 50000, 1024, inputStream, null, 0L)); + () -> woh.newUploadPartRequestBuilder(key, + "uploadId", 50000, 1024)); } static class StreamClosedException extends IOException {} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java index 62a99d7209263..a4162f212179b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java @@ -27,9 +27,9 @@ import java.net.URI; import java.util.Date; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; -import com.amazonaws.services.s3.model.ObjectMetadata; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -37,6 +37,7 @@ import org.junit.Test; import org.mockito.ArgumentMatcher; + /** * deleteOnExit test for S3A. */ @@ -74,25 +75,25 @@ public void testDeleteOnExit() throws Exception { // unset S3CSE property from config to avoid pathIOE. conf.unset(Constants.S3_ENCRYPTION_ALGORITHM); testFs.initialize(uri, conf); - AmazonS3 testS3 = testFs.getAmazonS3ClientForTesting("mocking"); + S3Client testS3 = testFs.getS3AInternals().getAmazonS3Client("mocking"); Path path = new Path("/file"); String key = path.toUri().getPath().substring(1); - ObjectMetadata meta = new ObjectMetadata(); - meta.setContentLength(1L); - meta.setLastModified(new Date(2L)); - when(testS3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) - .thenReturn(meta); + HeadObjectResponse objectMetadata = + HeadObjectResponse.builder().contentLength(1L).lastModified(new Date(2L).toInstant()) + .build(); + when(testS3.headObject(argThat(correctGetMetadataRequest(BUCKET, key)))) + .thenReturn(objectMetadata); testFs.deleteOnExit(path); testFs.close(); assertEquals(0, testFs.getDeleteOnDnExitCount()); } - private ArgumentMatcher correctGetMetadataRequest( + private ArgumentMatcher correctGetMetadataRequest( String bucket, String key) { return request -> request != null - && request.getBucketName().equals(bucket) - && request.getKey().equals(key); + && request.bucket().equals(bucket) + && request.key().equals(key); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java index fd649c436bf59..c4d9ca7bcc04b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java @@ -18,30 +18,38 @@ package org.apache.hadoop.fs.s3a; +import static org.apache.hadoop.fs.s3a.AWSCredentialProviderList.maybeTranslateCredentialException; import static org.apache.hadoop.fs.s3a.Constants.*; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.verifyExceptionClass; import static org.apache.hadoop.fs.s3a.S3AUtils.*; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404; +import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.maybeTranslateAuditException; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.*; import static org.junit.Assert.*; import java.io.EOFException; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InterruptedIOException; -import java.net.SocketTimeoutException; import java.nio.file.AccessDeniedException; -import java.util.Collections; -import java.util.Map; import java.util.concurrent.ExecutionException; +import java.util.function.Consumer; -import com.amazonaws.AmazonClientException; -import com.amazonaws.AmazonServiceException; -import com.amazonaws.services.s3.model.AmazonS3Exception; +import org.assertj.core.api.Assertions; +import software.amazon.awssdk.awscore.exception.AwsErrorDetails; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.http.SdkHttpResponse; +import software.amazon.awssdk.services.s3.model.S3Exception; import org.junit.Test; +import org.apache.hadoop.fs.s3a.api.UnsupportedRequestException; +import org.apache.hadoop.fs.s3a.audit.AuditFailureException; +import org.apache.hadoop.fs.s3a.audit.AuditOperationRejectedException; import org.apache.hadoop.fs.s3a.impl.ErrorTranslation; + import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; /** @@ -51,28 +59,27 @@ @SuppressWarnings("ThrowableNotThrown") public class TestS3AExceptionTranslation { - private static final org.apache.http.conn.ConnectTimeoutException - HTTP_CONNECTION_TIMEOUT_EX - = new org.apache.http.conn.ConnectTimeoutException("apache"); - private static final SocketTimeoutException SOCKET_TIMEOUT_EX - = new SocketTimeoutException("socket"); - @Test - public void test301ContainsEndpoint() throws Exception { - String bucket = "bucket.s3-us-west-2.amazonaws.com"; - int sc301 = 301; - AmazonS3Exception s3Exception = createS3Exception("wrong endpoint", sc301, - Collections.singletonMap(S3AUtils.ENDPOINT_KEY, - bucket)); + public void test301ContainsRegion() throws Exception { + String region = "us-west-1"; + + AwsErrorDetails redirectError = AwsErrorDetails.builder() + .sdkHttpResponse( + SdkHttpResponse.builder().putHeader(BUCKET_REGION_HEADER, region).build()) + .build(); + + S3Exception s3Exception = createS3Exception("wrong region", + SC_301_MOVED_PERMANENTLY, + redirectError); AWSRedirectException ex = verifyTranslated( AWSRedirectException.class, s3Exception); - assertStatusCode(sc301, ex); + assertStatusCode(SC_301_MOVED_PERMANENTLY, ex); assertNotNull(ex.getMessage()); - assertContained(ex.getMessage(), bucket); - assertContained(ex.getMessage(), ENDPOINT); - assertExceptionContains(ENDPOINT, ex, "endpoint"); - assertExceptionContains(bucket, ex, "bucket name"); + assertContained(ex.getMessage(), region); + assertContained(ex.getMessage(), AWS_REGION); + assertExceptionContains(AWS_REGION, ex, "region"); + assertExceptionContains(region, ex, "region name"); } protected void assertContained(String text, String contained) { @@ -88,17 +95,17 @@ protected void verifyTranslated( @Test public void test400isBad() throws Exception { - verifyTranslated(400, AWSBadRequestException.class); + verifyTranslated(SC_400_BAD_REQUEST, AWSBadRequestException.class); } @Test public void test401isNotPermittedFound() throws Exception { - verifyTranslated(401, AccessDeniedException.class); + verifyTranslated(SC_401_UNAUTHORIZED, AccessDeniedException.class); } @Test public void test403isNotPermittedFound() throws Exception { - verifyTranslated(403, AccessDeniedException.class); + verifyTranslated(SC_403_FORBIDDEN, AccessDeniedException.class); } /** @@ -106,7 +113,7 @@ public void test403isNotPermittedFound() throws Exception { */ @Test public void test404isNotFound() throws Exception { - verifyTranslated(SC_404, FileNotFoundException.class); + verifyTranslated(SC_404_NOT_FOUND, FileNotFoundException.class); } /** @@ -114,8 +121,11 @@ public void test404isNotFound() throws Exception { */ @Test public void testUnknownBucketException() throws Exception { - AmazonS3Exception ex404 = createS3Exception(SC_404); - ex404.setErrorCode(ErrorTranslation.AwsErrorCodes.E_NO_SUCH_BUCKET); + S3Exception ex404 = createS3Exception(b -> b + .statusCode(SC_404_NOT_FOUND) + .awsErrorDetails(AwsErrorDetails.builder() + .errorCode(ErrorTranslation.AwsErrorCodes.E_NO_SUCH_BUCKET) + .build())); verifyTranslated( UnknownStoreException.class, ex404); @@ -123,12 +133,12 @@ public void testUnknownBucketException() throws Exception { @Test public void test410isNotFound() throws Exception { - verifyTranslated(410, FileNotFoundException.class); + verifyTranslated(SC_410_GONE, FileNotFoundException.class); } @Test public void test416isEOF() throws Exception { - verifyTranslated(416, EOFException.class); + verifyTranslated(SC_416_RANGE_NOT_SATISFIABLE, EOFException.class); } @Test @@ -143,19 +153,21 @@ public void testGenericS3Exception() throws Exception { @Test public void testGenericServiceS3Exception() throws Exception { // service exception of no known type - AmazonServiceException ase = new AmazonServiceException("unwind"); - ase.setStatusCode(500); + AwsServiceException ase = AwsServiceException.builder() + .message("unwind") + .statusCode(SC_500_INTERNAL_SERVER_ERROR) + .build(); AWSServiceIOException ex = verifyTranslated( AWSStatus500Exception.class, ase); - assertStatusCode(500, ex); + assertStatusCode(SC_500_INTERNAL_SERVER_ERROR, ex); } protected void assertStatusCode(int expected, AWSServiceIOException ex) { assertNotNull("Null exception", ex); - if (expected != ex.getStatusCode()) { + if (expected != ex.statusCode()) { throw new AssertionError("Expected status code " + expected - + "but got " + ex.getStatusCode(), + + "but got " + ex.statusCode(), ex); } } @@ -164,23 +176,35 @@ protected void assertStatusCode(int expected, AWSServiceIOException ex) { public void testGenericClientException() throws Exception { // Generic Amazon exception verifyTranslated(AWSClientIOException.class, - new AmazonClientException("")); + SdkException.builder().message("").build()); } - private static AmazonS3Exception createS3Exception(int code) { - return createS3Exception("", code, null); + private static S3Exception createS3Exception( + Consumer consumer) { + S3Exception.Builder builder = S3Exception.builder() + .awsErrorDetails(AwsErrorDetails.builder() + .build()); + consumer.accept(builder); + return (S3Exception) builder.build(); } - private static AmazonS3Exception createS3Exception(String message, int code, - Map additionalDetails) { - AmazonS3Exception source = new AmazonS3Exception(message); - source.setStatusCode(code); - source.setAdditionalDetails(additionalDetails); + private static S3Exception createS3Exception(int code) { + return createS3Exception(b -> b.message("").statusCode(code)); + } + + private static S3Exception createS3Exception(String message, int code, + AwsErrorDetails additionalDetails) { + + S3Exception source = (S3Exception) S3Exception.builder() + .message(message) + .statusCode(code) + .awsErrorDetails(additionalDetails) + .build(); return source; } private static E verifyTranslated(Class clazz, - AmazonClientException exception) throws Exception { + SdkException exception) throws Exception { // Verifying that the translated exception have the correct error message. IOException ioe = translateException("test", "/", exception); assertExceptionContains(exception.getMessage(), ioe, @@ -212,16 +236,98 @@ public void testInterruptExceptionDetecting() throws Throwable { public void testExtractInterrupted() throws Throwable { throw extractException("", "", new ExecutionException( - new AmazonClientException( - new InterruptedException("")))); + SdkException.builder() + .cause(new InterruptedException("")) + .build())); } @Test(expected = InterruptedIOException.class) public void testExtractInterruptedIO() throws Throwable { throw extractException("", "", new ExecutionException( - new AmazonClientException( - new InterruptedIOException("")))); + SdkException.builder() + .cause(new InterruptedIOException("")) + .build())); + } + + private SdkClientException sdkClientException(String message, Throwable cause) { + return SdkClientException.builder() + .message(message) + .cause(cause) + .build(); + } + @Test + public void testTranslateCredentialException() throws Throwable { + verifyExceptionClass(AccessDeniedException.class, + maybeTranslateCredentialException("/", + new CredentialInitializationException("Credential initialization failed"))); + } + + @Test + public void testTranslateNestedCredentialException() throws Throwable { + final AccessDeniedException ex = + verifyExceptionClass(AccessDeniedException.class, + maybeTranslateCredentialException("/", + sdkClientException("", + new CredentialInitializationException("Credential initialization failed")))); + // unwrap and verify that the initial client exception has been stripped + final Throwable cause = ex.getCause(); + Assertions.assertThat(cause) + .isInstanceOf(CredentialInitializationException.class); + CredentialInitializationException cie = (CredentialInitializationException) cause; + Assertions.assertThat(cie.retryable()) + .describedAs("Retryable flag") + .isFalse(); + } + + + @Test + public void testTranslateNonCredentialException() throws Throwable { + Assertions.assertThat( + maybeTranslateCredentialException("/", + sdkClientException("not a credential exception", null))) + .isNull(); + Assertions.assertThat( + maybeTranslateCredentialException("/", + sdkClientException("", sdkClientException("not a credential exception", null)))) + .isNull(); + } + + @Test + public void testTranslateAuditException() throws Throwable { + verifyExceptionClass(AccessDeniedException.class, + maybeTranslateAuditException("/", + new AuditFailureException("failed"))); + } + + @Test + public void testTranslateNestedAuditException() throws Throwable { + verifyExceptionClass(AccessDeniedException.class, + maybeTranslateAuditException("/", + sdkClientException("", new AuditFailureException("failed")))); + } + + + @Test + public void testTranslateNestedAuditRejectedException() throws Throwable { + final UnsupportedRequestException ex = + verifyExceptionClass(UnsupportedRequestException.class, + maybeTranslateAuditException("/", + sdkClientException("", new AuditOperationRejectedException("rejected")))); + Assertions.assertThat(ex.getCause()) + .isInstanceOf(AuditOperationRejectedException.class); + } + + @Test + public void testTranslateNonAuditException() throws Throwable { + Assertions.assertThat( + maybeTranslateAuditException("/", + sdkClientException("not an audit exception", null))) + .isNull(); + Assertions.assertThat( + maybeTranslateAuditException("/", + sdkClientException("", sdkClientException("not an audit exception", null)))) + .isNull(); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java index 34a275b580f25..1a2a21a6e5111 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java @@ -21,21 +21,22 @@ import static org.junit.Assert.*; import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import java.io.FileNotFoundException; +import java.util.ArrayList; import java.util.Collections; import java.util.Date; import java.util.List; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsV2Request; -import com.amazonaws.services.s3.model.ListObjectsV2Result; -import com.amazonaws.services.s3.model.ObjectListing; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.S3ObjectSummary; +import software.amazon.awssdk.services.s3.model.CommonPrefix; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsResponse; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; +import software.amazon.awssdk.services.s3.model.S3Object; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -44,6 +45,7 @@ import org.junit.Test; import org.mockito.ArgumentMatcher; + /** * S3A tests for getFileStatus using mock S3 client. */ @@ -53,17 +55,16 @@ public class TestS3AGetFileStatus extends AbstractS3AMockTest { public void testFile() throws Exception { Path path = new Path("/file"); String key = path.toUri().getPath().substring(1); - ObjectMetadata meta = new ObjectMetadata(); - meta.setContentLength(1L); - meta.setLastModified(new Date(2L)); - when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) - .thenReturn(meta); + HeadObjectResponse objectMetadata = + HeadObjectResponse.builder().contentLength(1L).lastModified(new Date(2L).toInstant()) + .build(); + when(s3.headObject(argThat(correctGetMetadataRequest(BUCKET, key)))).thenReturn(objectMetadata); FileStatus stat = fs.getFileStatus(path); assertNotNull(stat); assertEquals(fs.makeQualified(path), stat.getPath()); assertTrue(stat.isFile()); - assertEquals(meta.getContentLength(), stat.getLen()); - assertEquals(meta.getLastModified().getTime(), stat.getModificationTime()); + assertEquals(objectMetadata.contentLength().longValue(), stat.getLen()); + assertEquals(Date.from(objectMetadata.lastModified()).getTime(), stat.getModificationTime()); ContractTestUtils.assertNotErasureCoded(fs, path); assertTrue(path + " should have erasure coding unset in " + "FileStatus#toString(): " + stat, @@ -74,17 +75,16 @@ public void testFile() throws Exception { public void testFakeDirectory() throws Exception { Path path = new Path("/dir"); String key = path.toUri().getPath().substring(1); - when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) + when(s3.headObject(argThat(correctGetMetadataRequest(BUCKET, key)))) .thenThrow(NOT_FOUND); String keyDir = key + "/"; - ListObjectsV2Result listResult = new ListObjectsV2Result(); - S3ObjectSummary objectSummary = new S3ObjectSummary(); - objectSummary.setKey(keyDir); - objectSummary.setSize(0L); - listResult.getObjectSummaries().add(objectSummary); + List s3Objects = new ArrayList<>(1); + s3Objects.add(S3Object.builder().key(keyDir).size(0L).build()); + ListObjectsV2Response listObjectsV2Response = + ListObjectsV2Response.builder().contents(s3Objects).build(); when(s3.listObjectsV2(argThat( matchListV2Request(BUCKET, keyDir)) - )).thenReturn(listResult); + )).thenReturn(listObjectsV2Response); FileStatus stat = fs.getFileStatus(path); assertNotNull(stat); assertEquals(fs.makeQualified(path), stat.getPath()); @@ -95,12 +95,13 @@ public void testFakeDirectory() throws Exception { public void testImplicitDirectory() throws Exception { Path path = new Path("/dir"); String key = path.toUri().getPath().substring(1); - when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) + when(s3.headObject(argThat(correctGetMetadataRequest(BUCKET, key)))) .thenThrow(NOT_FOUND); - when(s3.getObjectMetadata(argThat( + when(s3.headObject(argThat( correctGetMetadataRequest(BUCKET, key + "/")) )).thenThrow(NOT_FOUND); - setupListMocks(Collections.singletonList("dir/"), Collections.emptyList()); + setupListMocks(Collections.singletonList(CommonPrefix.builder().prefix("dir/").build()), + Collections.emptyList()); FileStatus stat = fs.getFileStatus(path); assertNotNull(stat); assertEquals(fs.makeQualified(path), stat.getPath()); @@ -115,9 +116,9 @@ public void testImplicitDirectory() throws Exception { public void testRoot() throws Exception { Path path = new Path("/"); String key = path.toUri().getPath().substring(1); - when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) + when(s3.headObject(argThat(correctGetMetadataRequest(BUCKET, key)))) .thenThrow(NOT_FOUND); - when(s3.getObjectMetadata(argThat( + when(s3.headObject(argThat( correctGetMetadataRequest(BUCKET, key + "/") ))).thenThrow(NOT_FOUND); setupListMocks(Collections.emptyList(), Collections.emptyList()); @@ -132,9 +133,9 @@ public void testRoot() throws Exception { public void testNotFound() throws Exception { Path path = new Path("/dir"); String key = path.toUri().getPath().substring(1); - when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) + when(s3.headObject(argThat(correctGetMetadataRequest(BUCKET, key)))) .thenThrow(NOT_FOUND); - when(s3.getObjectMetadata(argThat( + when(s3.headObject(argThat( correctGetMetadataRequest(BUCKET, key + "/") ))).thenThrow(NOT_FOUND); setupListMocks(Collections.emptyList(), Collections.emptyList()); @@ -142,36 +143,38 @@ public void testNotFound() throws Exception { fs.getFileStatus(path); } - private void setupListMocks(List prefixes, - List summaries) { - + private void setupListMocks(List prefixes, + List s3Objects) { // V1 list API mock - ObjectListing objects = mock(ObjectListing.class); - when(objects.getCommonPrefixes()).thenReturn(prefixes); - when(objects.getObjectSummaries()).thenReturn(summaries); - when(s3.listObjects(any(ListObjectsRequest.class))).thenReturn(objects); + ListObjectsResponse v1Response = ListObjectsResponse.builder() + .commonPrefixes(prefixes) + .contents(s3Objects) + .build(); + when(s3.listObjects(any(ListObjectsRequest.class))).thenReturn(v1Response); // V2 list API mock - ListObjectsV2Result v2Result = mock(ListObjectsV2Result.class); - when(v2Result.getCommonPrefixes()).thenReturn(prefixes); - when(v2Result.getObjectSummaries()).thenReturn(summaries); - when(s3.listObjectsV2(any(ListObjectsV2Request.class))) - .thenReturn(v2Result); + ListObjectsV2Response v2Result = ListObjectsV2Response.builder() + .commonPrefixes(prefixes) + .contents(s3Objects) + .build(); + when(s3.listObjectsV2( + any(software.amazon.awssdk.services.s3.model.ListObjectsV2Request.class))).thenReturn( + v2Result); } - private ArgumentMatcher correctGetMetadataRequest( + private ArgumentMatcher correctGetMetadataRequest( String bucket, String key) { return request -> request != null - && request.getBucketName().equals(bucket) - && request.getKey().equals(key); + && request.bucket().equals(bucket) + && request.key().equals(key); } private ArgumentMatcher matchListV2Request( String bucket, String key) { return (ListObjectsV2Request request) -> { return request != null - && request.getBucketName().equals(bucket) - && request.getPrefix().equals(key); + && request.bucket().equals(bucket) + && request.prefix().equals(key); }; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java index c62bf5daca3a4..da1284343da9f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java @@ -19,16 +19,18 @@ package org.apache.hadoop.fs.s3a; import javax.net.ssl.SSLException; +import java.io.FilterInputStream; import java.io.IOException; import java.net.SocketException; import java.nio.charset.StandardCharsets; import java.util.concurrent.CompletableFuture; -import com.amazonaws.SdkClientException; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.S3ObjectInputStream; +import software.amazon.awssdk.awscore.exception.AwsErrorDetails; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.http.AbortableInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; import org.junit.Test; import org.apache.commons.io.IOUtils; @@ -37,6 +39,7 @@ import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.util.functional.CallableRaisingIOE; + import static java.lang.Math.min; import static org.apache.hadoop.util.functional.FutureIO.eval; import static org.junit.Assert.assertArrayEquals; @@ -121,13 +124,22 @@ private S3AInputStream getMockedS3AInputStream() { * @return mocked object. */ private S3AInputStream.InputStreamCallbacks getMockedInputStreamCallback() { - return new S3AInputStream.InputStreamCallbacks() { + GetObjectResponse objectResponse = GetObjectResponse.builder() + .eTag("test-etag") + .build(); - private final S3Object mockedS3Object = getMockedS3Object(); + ResponseInputStream[] responseInputStreams = + new ResponseInputStream[] { + getMockedInputStream(objectResponse, true), + getMockedInputStream(objectResponse, true), + getMockedInputStream(objectResponse, false) + }; + + return new S3AInputStream.InputStreamCallbacks() { private Integer mockedS3ObjectIndex = 0; @Override - public S3Object getObject(GetObjectRequest request) { + public ResponseInputStream getObject(GetObjectRequest request) { // Set s3 client to return mocked s3object with defined read behavior. mockedS3ObjectIndex++; // open() -> lazySeek() -> reopen() @@ -144,14 +156,17 @@ public S3Object getObject(GetObjectRequest request) { // -> getObjectContent(objectInputStreamGood)-> objectInputStreamGood // -> wrappedStream.read if (mockedS3ObjectIndex == 3) { - throw new SdkClientException("Failed to get S3Object"); + throw AwsServiceException.builder() + .message("Failed to get S3Object") + .awsErrorDetails(AwsErrorDetails.builder().errorCode("test-code").build()) + .build(); } - return mockedS3Object; + return responseInputStreams[min(mockedS3ObjectIndex, responseInputStreams.length) - 1]; } @Override - public GetObjectRequest newGetRequest(String key) { - return new GetObjectRequest(fs.getBucket(), key); + public GetObjectRequest.Builder newGetRequestBuilder(String key) { + return GetObjectRequest.builder().bucket(fs.getBucket()).key(key); } @Override @@ -166,70 +181,41 @@ public void close() { } /** - * Get mocked S3Object that returns bad input stream on the initial of - * getObjectContent calls. - * - * @return mocked object. - */ - private S3Object getMockedS3Object() { - S3ObjectInputStream objectInputStreamBad1 = getMockedInputStream(true); - S3ObjectInputStream objectInputStreamBad2 = getMockedInputStream(true); - S3ObjectInputStream objectInputStreamGood = getMockedInputStream(false); - - return new S3Object() { - private final S3ObjectInputStream[] inputStreams = - {objectInputStreamBad1, objectInputStreamBad2, objectInputStreamGood}; - - private Integer inputStreamIndex = 0; - - @Override - public S3ObjectInputStream getObjectContent() { - // Set getObjectContent behavior: - // Returns bad stream twice, and good stream afterwards. - inputStreamIndex++; - return inputStreams[min(inputStreamIndex, inputStreams.length) - 1]; - } - - @Override - public ObjectMetadata getObjectMetadata() { - // Set getObjectMetadata behavior: returns dummy metadata - ObjectMetadata metadata = new ObjectMetadata(); - metadata.setHeader("ETag", "test-etag"); - return metadata; - } - }; - } - - /** - * Get mocked S3ObjectInputStream where we can trigger IOException to + * Get mocked ResponseInputStream where we can trigger IOException to * simulate the read failure. * * @param triggerFailure true when a failure injection is enabled. * @return mocked object. */ - private S3ObjectInputStream getMockedInputStream(boolean triggerFailure) { - return new S3ObjectInputStream(IOUtils.toInputStream(INPUT, StandardCharsets.UTF_8), null) { - - private final IOException exception = - new SSLException(new SocketException("Connection reset")); - - @Override - public int read() throws IOException { - int result = super.read(); - if (triggerFailure) { - throw exception; - } - return result; - } - - @Override - public int read(byte[] b, int off, int len) throws IOException { - int result = super.read(b, off, len); - if (triggerFailure) { - throw exception; - } - return result; - } - }; + private ResponseInputStream getMockedInputStream( + GetObjectResponse objectResponse, boolean triggerFailure) { + + FilterInputStream inputStream = + new FilterInputStream(IOUtils.toInputStream(INPUT, StandardCharsets.UTF_8)) { + + private final IOException exception = + new SSLException(new SocketException("Connection reset")); + + @Override + public int read() throws IOException { + int result = super.read(); + if (triggerFailure) { + throw exception; + } + return result; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + int result = super.read(b, off, len); + if (triggerFailure) { + throw exception; + } + return result; + } + }; + + return new ResponseInputStream(objectResponse, + AbortableInputStream.create(inputStream, () -> {})); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AProxy.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AProxy.java index e05ee25adfa74..0982c8cbd4761 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AProxy.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AProxy.java @@ -20,18 +20,17 @@ import java.io.IOException; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.Protocol; import org.assertj.core.api.Assertions; import org.junit.Test; +import software.amazon.awssdk.http.apache.ProxyConfiguration; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.impl.AWSClientConfig; import org.apache.hadoop.test.AbstractHadoopTestBase; import static org.apache.hadoop.fs.s3a.Constants.PROXY_HOST; import static org.apache.hadoop.fs.s3a.Constants.PROXY_PORT; import static org.apache.hadoop.fs.s3a.Constants.PROXY_SECURED; -import static org.apache.hadoop.fs.s3a.S3AUtils.initProxySupport; /** * Tests to verify {@link S3AUtils} translates the proxy configurations @@ -79,11 +78,16 @@ public void testProxyDefault() throws IOException { private void verifyProxy(Configuration proxyConfig, boolean isExpectedSecured) throws IOException { - ClientConfiguration awsConf = new ClientConfiguration(); - initProxySupport(proxyConfig, "test-bucket", awsConf); - Assertions.assertThat(awsConf.getProxyProtocol()) + ProxyConfiguration config = + AWSClientConfig.createProxyConfiguration(proxyConfig, "testBucket"); + ProxyConfiguration asyncConfig = + AWSClientConfig.createProxyConfiguration(proxyConfig, "testBucket"); + Assertions.assertThat(config.scheme()) .describedAs("Proxy protocol not as expected") - .isEqualTo(isExpectedSecured ? Protocol.HTTPS : Protocol.HTTP); + .isEqualTo(isExpectedSecured ? "https" : "http"); + Assertions.assertThat(asyncConfig.scheme()) + .describedAs("Proxy protocol not as expected") + .isEqualTo(isExpectedSecured ? "https" : "http"); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java index 0e105c25c3a45..643db02087b46 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java @@ -18,17 +18,22 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.S3ObjectInputStream; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.http.AbortableInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.junit.Test; + import java.io.IOException; -import java.util.Date; +import java.io.InputStream; +import java.time.Instant; import static org.junit.Assert.assertEquals; @@ -40,10 +45,10 @@ import static org.mockito.Mockito.when; /** - * Uses mocks to check that the {@link S3ObjectInputStream} is closed when - * {@link org.apache.hadoop.fs.CanUnbuffer#unbuffer} is called. Unlike the - * other unbuffer tests, this specifically tests that the underlying S3 object - * stream is closed. + * Uses mocks to check that the {@link ResponseInputStream} is + * closed when {@link org.apache.hadoop.fs.CanUnbuffer#unbuffer} is called. + * Unlike the other unbuffer tests, this specifically tests that the underlying + * S3 object stream is closed. */ public class TestS3AUnbuffer extends AbstractS3AMockTest { @@ -51,22 +56,27 @@ public class TestS3AUnbuffer extends AbstractS3AMockTest { public void testUnbuffer() throws IOException { // Create mock ObjectMetadata for getFileStatus() Path path = new Path("/file"); - ObjectMetadata meta = mock(ObjectMetadata.class); - when(meta.getContentLength()).thenReturn(1L); - when(meta.getLastModified()).thenReturn(new Date(2L)); - when(meta.getETag()).thenReturn("mock-etag"); - when(s3.getObjectMetadata(any())).thenReturn(meta); + HeadObjectResponse objectMetadata = HeadObjectResponse.builder() + .contentLength(1L) + .lastModified(Instant.ofEpochMilli(2L)) + .eTag("mock-etag") + .build(); + when(s3.headObject((HeadObjectRequest) any())).thenReturn(objectMetadata); - // Create mock S3ObjectInputStream and S3Object for open() - S3ObjectInputStream objectStream = mock(S3ObjectInputStream.class); + // Create mock ResponseInputStream and GetObjectResponse for open() + GetObjectResponse objectResponse = GetObjectResponse.builder() + .contentLength(1L) + .lastModified(Instant.ofEpochMilli(2L)) + .eTag("mock-etag") + .build(); + InputStream objectStream = mock(InputStream.class); when(objectStream.read()).thenReturn(-1); when(objectStream.read(any(byte[].class))).thenReturn(-1); when(objectStream.read(any(byte[].class), anyInt(), anyInt())).thenReturn(-1); - - S3Object s3Object = mock(S3Object.class); - when(s3Object.getObjectContent()).thenReturn(objectStream); - when(s3Object.getObjectMetadata()).thenReturn(meta); - when(s3.getObject(any())).thenReturn(s3Object); + ResponseInputStream getObjectResponseInputStream = + new ResponseInputStream(objectResponse, + AbortableInputStream.create(objectStream, () -> {})); + when(s3.getObject((GetObjectRequest) any())).thenReturn(getObjectResponseInputStream); // Call read and then unbuffer FSDataInputStream stream = fs.open(path); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java index 42de7cdffc80e..66d9032e858eb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java @@ -18,18 +18,18 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.AmazonServiceException; -import com.amazonaws.SdkBaseException; -import com.amazonaws.services.s3.Headers; -import com.amazonaws.services.s3.model.CopyObjectRequest; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.transfer.model.CopyResult; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.CopyObjectResponse; +import software.amazon.awssdk.services.s3.model.CopyObjectResult; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; @@ -40,6 +40,7 @@ import static org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.CHANGE_DETECTED; import static org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.createPolicy; import static org.apache.hadoop.fs.s3a.impl.ChangeTracker.CHANGE_REPORTED_BY_S3; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_412_PRECONDITION_FAILED; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** @@ -68,7 +69,7 @@ public void testVersionCheckingHandlingNoVersions() throws Throwable { ChangeDetectionPolicy.Source.VersionId, false); assertFalse("Tracker should not have applied contraints " + tracker, - tracker.maybeApplyConstraint(newGetObjectRequest())); + tracker.maybeApplyConstraint(newGetObjectRequestBuilder())); tracker.processResponse( newResponse(null, null), "", 0); @@ -96,7 +97,7 @@ public void testEtagCheckingWarn() throws Throwable { ChangeDetectionPolicy.Source.ETag, false); assertFalse("Tracker should not have applied constraints " + tracker, - tracker.maybeApplyConstraint(newGetObjectRequest())); + tracker.maybeApplyConstraint(newGetObjectRequestBuilder())); tracker.processResponse( newResponse("e1", null), "", 0); @@ -122,13 +123,13 @@ public void testVersionCheckingOnClient() throws Throwable { ChangeDetectionPolicy.Source.VersionId, false); assertFalse("Tracker should not have applied constraints " + tracker, - tracker.maybeApplyConstraint(newGetObjectRequest())); + tracker.maybeApplyConstraint(newGetObjectRequestBuilder())); tracker.processResponse( newResponse(null, "rev1"), "", 0); assertTrackerMismatchCount(tracker, 0); assertRevisionId(tracker, "rev1"); - GetObjectRequest request = newGetObjectRequest(); + GetObjectRequest request = newGetObjectRequestBuilder().build(); expectChangeException(tracker, newResponse(null, "rev2"), "change detected"); // mismatch was noted (so gets to FS stats) @@ -149,14 +150,14 @@ public void testVersionCheckingOnServer() throws Throwable { ChangeDetectionPolicy.Source.VersionId, false); assertFalse("Tracker should not have applied contraints " + tracker, - tracker.maybeApplyConstraint(newGetObjectRequest())); + tracker.maybeApplyConstraint(newGetObjectRequestBuilder())); tracker.processResponse( newResponse(null, "rev1"), "", 0); assertTrackerMismatchCount(tracker, 0); assertRevisionId(tracker, "rev1"); - GetObjectRequest request = newGetObjectRequest(); - assertConstraintApplied(tracker, request); + GetObjectRequest.Builder builder = newGetObjectRequestBuilder(); + assertConstraintApplied(tracker, builder); // now, the tracker expects a null response expectChangeException(tracker, null, CHANGE_REPORTED_BY_S3); assertTrackerMismatchCount(tracker, 1); @@ -249,31 +250,33 @@ public void testCopyVersionMismatch() throws Throwable { // 412 is translated to RemoteFileChangedException // note: this scenario is never currently hit due to // https://github.com/aws/aws-sdk-java/issues/1644 - AmazonServiceException awsException = - new AmazonServiceException("aws exception"); - awsException.setStatusCode(ChangeTracker.SC_PRECONDITION_FAILED); + AwsServiceException awsException = + AwsServiceException.builder() + .message("aws exception") + .statusCode(SC_412_PRECONDITION_FAILED) + .build(); expectChangeException(tracker, awsException, "copy", RemoteFileChangedException.PRECONDITIONS_FAILED); // processing another type of exception does nothing - tracker.processException(new SdkBaseException("foo"), "copy"); + tracker.processException(SdkException.builder().message("foo").build(), "copy"); } protected void assertConstraintApplied(final ChangeTracker tracker, - final GetObjectRequest request) { + final GetObjectRequest.Builder builder) { assertTrue("Tracker should have applied contraints " + tracker, - tracker.maybeApplyConstraint(request)); + tracker.maybeApplyConstraint(builder)); } protected void assertConstraintApplied(final ChangeTracker tracker, - final CopyObjectRequest request) throws PathIOException { + final CopyObjectRequest.Builder requestBuilder) throws PathIOException { assertTrue("Tracker should have applied contraints " + tracker, - tracker.maybeApplyConstraint(request)); + tracker.maybeApplyConstraint(requestBuilder)); } protected RemoteFileChangedException expectChangeException( final ChangeTracker tracker, - final S3Object response, + final GetObjectResponse response, final String message) throws Exception { return expectException(tracker, response, message, RemoteFileChangedException.class); @@ -281,7 +284,7 @@ protected RemoteFileChangedException expectChangeException( protected RemoteFileChangedException expectChangeException( final ChangeTracker tracker, - final SdkBaseException exception, + final SdkException exception, final String operation, final String message) throws Exception { return expectException(tracker, exception, operation, message, @@ -290,7 +293,7 @@ protected RemoteFileChangedException expectChangeException( protected PathIOException expectNoVersionAttributeException( final ChangeTracker tracker, - final S3Object response, + final GetObjectResponse response, final String message) throws Exception { return expectException(tracker, response, message, NoVersionAttributeException.class); @@ -298,7 +301,7 @@ protected PathIOException expectNoVersionAttributeException( protected PathIOException expectNoVersionAttributeException( final ChangeTracker tracker, - final CopyResult response, + final CopyObjectResponse response, final String message) throws Exception { return expectException(tracker, response, message, NoVersionAttributeException.class); @@ -306,7 +309,7 @@ protected PathIOException expectNoVersionAttributeException( protected T expectException( final ChangeTracker tracker, - final S3Object response, + final GetObjectResponse response, final String message, final Class clazz) throws Exception { return intercept( @@ -320,7 +323,7 @@ protected T expectException( protected T expectException( final ChangeTracker tracker, - final CopyResult response, + final CopyObjectResponse response, final String message, final Class clazz) throws Exception { return intercept( @@ -334,7 +337,7 @@ protected T expectException( protected T expectException( final ChangeTracker tracker, - final SdkBaseException exception, + final SdkException exception, final String operation, final String message, final Class clazz) throws Exception { @@ -389,48 +392,36 @@ protected ChangeTracker newTracker(final ChangeDetectionPolicy.Mode mode, if (objectAttributes.getVersionId() == null && objectAttributes.getETag() == null) { assertFalse("Tracker should not have applied constraints " + tracker, - tracker.maybeApplyConstraint(newGetObjectRequest())); + tracker.maybeApplyConstraint(newGetObjectRequestBuilder())); } return tracker; } - private GetObjectRequest newGetObjectRequest() { - return new GetObjectRequest(BUCKET, OBJECT); + private GetObjectRequest.Builder newGetObjectRequestBuilder() { + return GetObjectRequest.builder().bucket(BUCKET).key(OBJECT); } - private CopyObjectRequest newCopyObjectRequest() { - return new CopyObjectRequest(BUCKET, OBJECT, BUCKET, DEST_OBJECT); + private CopyObjectRequest.Builder newCopyObjectRequest() { + return CopyObjectRequest.builder().sourceBucket(BUCKET).sourceKey(OBJECT) + .destinationBucket(BUCKET).destinationKey(DEST_OBJECT); } - private CopyResult newCopyResult(String eTag, String versionId) { - CopyResult copyResult = new CopyResult(); - copyResult.setSourceBucketName(BUCKET); - copyResult.setSourceKey(OBJECT); - copyResult.setDestinationBucketName(BUCKET); - copyResult.setDestinationKey(DEST_OBJECT); - copyResult.setETag(eTag); - copyResult.setVersionId(versionId); - return copyResult; + private CopyObjectResponse newCopyResult(String eTag, String versionId) { + CopyObjectResponse.Builder copyObjectResponseBuilder = CopyObjectResponse.builder(); + + return copyObjectResponseBuilder.versionId(versionId) + .copyObjectResult(CopyObjectResult.builder().eTag(eTag).build()).build(); } - private S3Object newResponse(String etag, String versionId) { - ObjectMetadata md = new ObjectMetadata(); + private GetObjectResponse newResponse(String etag, String versionId) { + GetObjectResponse.Builder builder = GetObjectResponse.builder(); if (etag != null) { - md.setHeader(Headers.ETAG, etag); + builder.eTag(etag); } if (versionId != null) { - md.setHeader(Headers.S3_VERSION_ID, versionId); + builder.versionId(versionId); } - S3Object response = emptyResponse(); - response.setObjectMetadata(md); - return response; - } - - private S3Object emptyResponse() { - S3Object response = new S3Object(); - response.setBucketName(BUCKET); - response.setKey(OBJECT); - return response; + return builder.build(); } private S3ObjectAttributes objectAttributes( diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestWildflyAndOpenSSLBinding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestWildflyAndOpenSSLBinding.java index a2b013f468a79..9e903fd85ff49 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestWildflyAndOpenSSLBinding.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestWildflyAndOpenSSLBinding.java @@ -20,10 +20,9 @@ import java.io.IOException; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.Protocol; import org.junit.Before; import org.junit.Test; +import software.amazon.awssdk.http.apache.ApacheHttpClient; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; @@ -74,7 +73,7 @@ public void testUnknownMode() throws Throwable { Configuration conf = new Configuration(false); conf.set(SSL_CHANNEL_MODE, "no-such-mode "); intercept(IllegalArgumentException.class, () -> - bindSSLChannelMode(conf, new ClientConfiguration())); + bindSSLChannelMode(conf, ApacheHttpClient.builder())); } @Test @@ -143,9 +142,7 @@ private DelegatingSSLSocketFactory.SSLChannelMode bindSocketFactory( DelegatingSSLSocketFactory.resetDefaultFactory(); Configuration conf = new Configuration(false); conf.set(SSL_CHANNEL_MODE, channelMode.name()); - ClientConfiguration awsConf = new ClientConfiguration(); - awsConf.setProtocol(Protocol.HTTPS); - bindSSLChannelMode(conf, awsConf); + bindSSLChannelMode(conf, ApacheHttpClient.builder()); return DelegatingSSLSocketFactory.getDefaultFactory().getChannelMode(); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/adapter/TestV1CredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/adapter/TestV1CredentialsProvider.java new file mode 100644 index 0000000000000..b0e1b57d75471 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/adapter/TestV1CredentialsProvider.java @@ -0,0 +1,222 @@ +/** + * 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.adapter; + +import java.io.IOException; +import java.net.URI; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.AWSCredentialsProvider; +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.AWSCredentialProviderList; +import org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider; +import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider; +import org.apache.hadoop.fs.s3a.impl.InstantiationIOException; + +import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER; +import static org.apache.hadoop.fs.s3a.S3ATestConstants.DEFAULT_CSVTEST_FILE; +import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.ANONYMOUS_CREDENTIALS_V1; +import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.EC2_CONTAINER_CREDENTIALS_V1; +import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.ENVIRONMENT_CREDENTIALS_V1; +import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSCredentialProviderList; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Unit tests for v1 to v2 credential provider logic. + */ +public class TestV1CredentialsProvider { + + /** + * URI of the landsat images. + */ + private static final URI TESTFILE_URI = new Path( + DEFAULT_CSVTEST_FILE).toUri(); + + private static final Logger LOG = LoggerFactory.getLogger(TestV1CredentialsProvider.class); + + + @Test + public void testV1V2Mapping() throws Exception { + URI uri1 = new URI("s3a://bucket1"); + + List> expectedClasses = + Arrays.asList( + IAMInstanceCredentialsProvider.class, + AnonymousAWSCredentialsProvider.class, + EnvironmentVariableCredentialsProvider.class); + Configuration conf = + createProviderConfiguration(buildClassList( + EC2_CONTAINER_CREDENTIALS_V1, + ANONYMOUS_CREDENTIALS_V1, + ENVIRONMENT_CREDENTIALS_V1)); + AWSCredentialProviderList list1 = createAWSCredentialProviderList( + uri1, conf); + assertCredentialProviders(expectedClasses, list1); + } + + @Test + public void testV1Wrapping() throws Exception { + URI uri1 = new URI("s3a://bucket1"); + + List> expectedClasses = + Arrays.asList( + V1ToV2AwsCredentialProviderAdapter.class, + V1ToV2AwsCredentialProviderAdapter.class); + Configuration conf = + createProviderConfiguration(buildClassList( + LegacyV1CredentialProvider.class.getName(), + LegacyV1CredentialProviderWithConf.class.getName())); + AWSCredentialProviderList list1 = createAWSCredentialProviderList( + uri1, conf); + assertCredentialProviders(expectedClasses, list1); + } + + private String buildClassList(String... classes) { + return Arrays.stream(classes) + .collect(Collectors.joining(",")); + } + + + /** + * 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 { + return intercept(IOException.class, expectedErrorText, + () -> createAWSCredentialProviderList( + TESTFILE_URI, + createProviderConfiguration(option))); + } + + /** + * 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; + } + + /** + * Asserts expected provider classes in list. + * @param expectedClasses expected provider classes + * @param list providers to check + */ + private static void assertCredentialProviders( + List> expectedClasses, + AWSCredentialProviderList list) { + assertNotNull(list); + List providers = list.getProviders(); + Assertions.assertThat(providers) + .describedAs("providers") + .hasSize(expectedClasses.size()); + for (int i = 0; i < expectedClasses.size(); ++i) { + Class expectedClass = + expectedClasses.get(i); + AwsCredentialsProvider provider = providers.get(i); + assertNotNull( + String.format("At position %d, expected class is %s, but found null.", + i, expectedClass), provider); + assertTrue( + String.format("At position %d, expected class is %s, but found %s.", + i, expectedClass, provider.getClass()), + expectedClass.isAssignableFrom(provider.getClass())); + } + } + + + public static class LegacyV1CredentialProvider implements AWSCredentialsProvider { + + public LegacyV1CredentialProvider() { + } + + @Override + public AWSCredentials getCredentials() { + return null; + } + + @Override + public void refresh() { + + } + } + + /** + * V1 credentials with a configuration constructor. + */ + public static final class LegacyV1CredentialProviderWithConf + extends LegacyV1CredentialProvider { + + public LegacyV1CredentialProviderWithConf(Configuration conf) { + } + } + + /** + * V1 Credentials whose factory method raises ClassNotFoundException. + * Expect this to fail rather than trigger recursive recovery; + * exception will be wrapped with something intended to be informative. + */ + @Test + public void testV1InstantiationFailurePropagation() throws Throwable { + InstantiationIOException expected = intercept(InstantiationIOException.class, + "simulated CNFE", + () -> createAWSCredentialProviderList( + TESTFILE_URI, + createProviderConfiguration(V1CredentialProviderDoesNotInstantiate.class.getName()))); + // print for the curious + LOG.info("{}", expected.toString()); + } + + + /** + * V1 credentials which raises an instantiation exception. + */ + public static final class V1CredentialProviderDoesNotInstantiate + extends LegacyV1CredentialProvider { + + private V1CredentialProviderDoesNotInstantiate() { + } + + public static AWSCredentialsProvider getInstance() throws ClassNotFoundException { + throw new ClassNotFoundException("simulated CNFE"); + } + } + + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java index 298c1444bb9b3..e2297e37e50c4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java @@ -21,13 +21,23 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.net.URI; +import java.util.ArrayList; +import java.util.HashMap; import java.util.Map; -import java.util.function.Consumer; import java.util.stream.Collectors; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; -import com.amazonaws.services.s3.model.GetObjectRequest; + +import software.amazon.awssdk.awscore.AwsExecutionAttribute; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.interceptor.InterceptorContext; +import software.amazon.awssdk.http.SdkHttpMethod; +import software.amazon.awssdk.http.SdkHttpRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; + import org.junit.After; import org.junit.Before; import org.slf4j.Logger; @@ -137,22 +147,58 @@ protected AuditSpanS3A activeSpan() { /** * Create a head request and pass it through the manager's beforeExecution() * callback. + * * @return a processed request. */ - protected GetObjectMetadataRequest head() { - return manager.beforeExecution( - requestFactory.newGetObjectMetadataRequest("/")); + protected SdkHttpRequest head() { + HeadObjectRequest.Builder headObjectRequestBuilder = + requestFactory.newHeadObjectRequestBuilder("/"); + manager.requestCreated(headObjectRequestBuilder); + HeadObjectRequest headObjectRequest = headObjectRequestBuilder.build(); + ExecutionAttributes executionAttributes = ExecutionAttributes.builder().build(); + InterceptorContext context = InterceptorContext.builder() + .request(headObjectRequest) + .httpRequest(SdkHttpRequest.builder() + .uri(URI.create("https://test")) + .method(SdkHttpMethod.HEAD) + .build()) + .build(); + manager.beforeExecution(context, executionAttributes); + return manager.modifyHttpRequest(context, executionAttributes); } /** - * Create a GetObject request and modify it before passing it through auditor. - * @param modifyRequest Consumer Interface for changing the request before passing to the auditor - * @return the request + * Create a get request and pass it through the manager's beforeExecution() + * callback. + * + * @return a processed request. */ - protected GetObjectRequest get(Consumer modifyRequest) { - GetObjectRequest req = requestFactory.newGetObjectRequest("/"); - modifyRequest.accept(req); - return manager.beforeExecution(req); + protected SdkHttpRequest get(String range) { + GetObjectRequest.Builder getObjectRequestBuilder = + requestFactory.newGetObjectRequestBuilder("/"); + + SdkHttpRequest.Builder httpRequestBuilder = + SdkHttpRequest.builder().uri(URI.create("https://test")).method(SdkHttpMethod.GET); + + if (!range.isEmpty()) { + getObjectRequestBuilder.range(range); + List rangeHeader = new ArrayList<>(); + rangeHeader.add(range); + Map> headers = new HashMap<>(); + headers.put("Range", rangeHeader); + httpRequestBuilder.headers(headers); + } + + manager.requestCreated(getObjectRequestBuilder); + GetObjectRequest getObjectRequest = getObjectRequestBuilder.build(); + ExecutionAttributes executionAttributes = ExecutionAttributes.builder().build().putAttribute( + AwsExecutionAttribute.OPERATION_NAME, "GetObject"); + InterceptorContext context = InterceptorContext.builder() + .request(getObjectRequest) + .httpRequest(httpRequestBuilder.build()) + .build(); + manager.beforeExecution(context, executionAttributes); + return manager.modifyHttpRequest(context, executionAttributes); } /** @@ -244,15 +290,31 @@ protected void assertMapNotContains(final Map params, final Stri * @param keys keys to be provided in the bulk delete request. * @return a processed request. */ - protected DeleteObjectsRequest headForBulkDelete(String... keys) { + protected SdkHttpRequest headForBulkDelete(String... keys) { if (keys == null || keys.length == 0) { return null; } - List keysToDelete = Arrays + + List keysToDelete = Arrays .stream(keys) - .map(DeleteObjectsRequest.KeyVersion::new) + .map(key -> ObjectIdentifier.builder().key(key).build()) .collect(Collectors.toList()); - return manager.beforeExecution(requestFactory.newBulkDeleteRequest(keysToDelete)); + + ExecutionAttributes executionAttributes = ExecutionAttributes.builder().build(); + + SdkHttpRequest.Builder httpRequestBuilder = + SdkHttpRequest.builder().uri(URI.create("https://test")).method(SdkHttpMethod.POST); + + DeleteObjectsRequest deleteObjectsRequest = + requestFactory.newBulkDeleteRequestBuilder(keysToDelete).build(); + + InterceptorContext context = InterceptorContext.builder() + .request(deleteObjectsRequest) + .httpRequest(httpRequestBuilder.build()) + .build(); + + manager.beforeExecution(context, executionAttributes); + return manager.modifyHttpRequest(context, executionAttributes); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java index ad72d75081b27..1520e588e544e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AuditTestSupport.java @@ -30,7 +30,7 @@ import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION; import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_SPAN_CREATION; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_ENABLED; -import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_EXECUTION_INTERCEPTORS; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_SERVICE_CLASSNAME; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.LOGGING_AUDIT_SERVICE; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.NOOP_AUDIT_SERVICE; @@ -119,7 +119,7 @@ public static Configuration resetAuditOptions(Configuration conf) { S3ATestUtils.removeBaseAndBucketOverrides(conf, REFERRER_HEADER_ENABLED, REJECT_OUT_OF_SPAN_OPERATIONS, - AUDIT_REQUEST_HANDLERS, + AUDIT_EXECUTION_INTERCEPTORS, AUDIT_SERVICE_CLASSNAME, AUDIT_ENABLED); return conf; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java index 9e6d82ce6ac75..ea7a1a34da735 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManager.java @@ -33,6 +33,7 @@ import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION; import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.enableLoggingAuditor; import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.resetAuditOptions; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_EXECUTION_INTERCEPTORS; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.UNAUDITED_OPERATION; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; @@ -57,8 +58,9 @@ public Configuration createConfiguration() { Configuration conf = super.createConfiguration(); resetAuditOptions(conf); enableLoggingAuditor(conf); - conf.set(AUDIT_REQUEST_HANDLERS, - SimpleAWSRequestHandler.CLASS); + conf.set(AUDIT_EXECUTION_INTERCEPTORS, + SimpleAWSExecutionInterceptor.CLASS); + conf.set(AUDIT_REQUEST_HANDLERS, "not-valid-class"); return conf; } @@ -114,22 +116,26 @@ public void testInvokeOutOfSpanRejected() throws Throwable { } @Test - public void testRequestHandlerBinding() throws Throwable { - describe("Verify that extra request handlers can be added and that they" + public void testExecutionInterceptorBinding() throws Throwable { + describe("Verify that extra ExecutionInterceptor can be added and that they" + " will be invoked during request execution"); - final long baseCount = SimpleAWSRequestHandler.getInvocationCount(); + final long baseCount = SimpleAWSExecutionInterceptor.getInvocationCount(); final S3AFileSystem fs = getFileSystem(); final long exec0 = lookupCounterStatistic(iostats(), AUDIT_REQUEST_EXECUTION.getSymbol()); // API call to a known path, `getBucketLocation()` does not always result in an API call. fs.listStatus(path("/")); // which MUST have ended up calling the extension request handler - Assertions.assertThat(SimpleAWSRequestHandler.getInvocationCount()) + Assertions.assertThat(SimpleAWSExecutionInterceptor.getInvocationCount()) .describedAs("Invocation count of plugged in request handler") .isGreaterThan(baseCount); assertThatStatisticCounter(iostats(), AUDIT_REQUEST_EXECUTION.getSymbol()) .isGreaterThan(exec0); assertThatStatisticCounter(iostats(), AUDIT_FAILURE.getSymbol()) .isZero(); + Assertions.assertThat(SimpleAWSExecutionInterceptor.getStaticConf()) + .describedAs("configuratin of SimpleAWSExecutionInterceptor") + .isNotNull() + .isSameAs(fs.getConf()); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/SimpleAWSRequestHandler.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/SimpleAWSExecutionInterceptor.java similarity index 56% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/SimpleAWSRequestHandler.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/SimpleAWSExecutionInterceptor.java index 6f5a0445a92f7..bf9b90bcdf31e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/SimpleAWSRequestHandler.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/SimpleAWSExecutionInterceptor.java @@ -20,28 +20,35 @@ import java.util.concurrent.atomic.AtomicLong; -import com.amazonaws.AmazonWebServiceRequest; -import com.amazonaws.handlers.RequestHandler2; +import software.amazon.awssdk.core.interceptor.Context; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; /** - * Simple AWS handler to verify dynamic loading of extra request - * handlers during auditing setup. + * Simple AWS interceptor to verify dynamic loading of extra + * execution interceptors during auditing setup. * The invocation counter tracks the count of calls to - * {@link #beforeExecution(AmazonWebServiceRequest)}. + * {@link #beforeExecution}. */ -public final class SimpleAWSRequestHandler extends RequestHandler2 { +public final class SimpleAWSExecutionInterceptor extends Configured + implements ExecutionInterceptor { public static final String CLASS - = "org.apache.hadoop.fs.s3a.audit.SimpleAWSRequestHandler"; + = "org.apache.hadoop.fs.s3a.audit.SimpleAWSExecutionInterceptor"; + + private static Configuration staticConf; /** Count of invocations. */ private static final AtomicLong INVOCATIONS = new AtomicLong(0); @Override - public AmazonWebServiceRequest beforeExecution( - final AmazonWebServiceRequest request) { + public void beforeExecution(Context.BeforeExecution context, + ExecutionAttributes executionAttributes) { INVOCATIONS.incrementAndGet(); - return request; + staticConf = getConf(); } /** @@ -51,4 +58,14 @@ public AmazonWebServiceRequest beforeExecution( public static long getInvocationCount() { return INVOCATIONS.get(); } + + /** + * get the static conf, which is set the config of the + * last executor invoked. + * @return the static configuration. + */ + + public static Configuration getStaticConf() { + return staticConf; + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java index 7cdab4c4b75e0..4f476604332b1 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java @@ -22,9 +22,12 @@ import java.nio.file.AccessDeniedException; import java.util.List; -import com.amazonaws.DefaultRequest; -import com.amazonaws.handlers.RequestHandler2; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; +import software.amazon.awssdk.core.interceptor.InterceptorContext; +import software.amazon.awssdk.http.SdkHttpMethod; +import software.amazon.awssdk.http.SdkHttpRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; import org.assertj.core.api.Assertions; import org.junit.Test; @@ -39,13 +42,15 @@ import org.apache.hadoop.service.Service; import org.apache.hadoop.test.AbstractHadoopTestBase; + import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.attachSpanToRequest; import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.retrieveAttachedSpan; import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.createIOStatisticsStoreForAuditing; import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.noopAuditConfig; -import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_EXECUTION_INTERCEPTORS; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_SERVICE_CLASSNAME; +import static org.apache.hadoop.fs.s3a.audit.impl.S3AInternalAuditConstants.AUDIT_SPAN_EXECUTION_ATTRIBUTE; import static org.apache.hadoop.service.ServiceAssert.assertServiceStateStarted; import static org.apache.hadoop.service.ServiceAssert.assertServiceStateStopped; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -159,30 +164,50 @@ public void testAuditManagerLifecycle() throws Throwable { } @Test - public void testSingleRequestHandler() throws Throwable { + public void testSingleExecutionInterceptor() throws Throwable { AuditManagerS3A manager = AuditIntegration.createAndStartAuditManager( noopAuditConfig(), ioStatistics); - List handlers - = manager.createRequestHandlers(); - assertThat(handlers) + List interceptors + = manager.createExecutionInterceptors(); + assertThat(interceptors) .hasSize(1); - RequestHandler2 handler = handlers.get(0); + ExecutionInterceptor interceptor = interceptors.get(0); + RequestFactory requestFactory = RequestFactoryImpl.builder() .withBucket("bucket") .build(); + HeadObjectRequest.Builder requestBuilder = + requestFactory.newHeadObjectRequestBuilder("/"); + + assertThat(interceptor instanceof AWSAuditEventCallbacks).isTrue(); + ((AWSAuditEventCallbacks)interceptor).requestCreated(requestBuilder); + + HeadObjectRequest request = requestBuilder.build(); + SdkHttpRequest httpRequest = SdkHttpRequest.builder() + .protocol("https") + .host("test") + .method(SdkHttpMethod.HEAD) + .build(); + + ExecutionAttributes attributes = ExecutionAttributes.builder().build(); + InterceptorContext context = InterceptorContext.builder() + .request(request) + .httpRequest(httpRequest) + .build(); + // test the basic pre-request sequence while avoiding // the complexity of recreating the full sequence // (and probably getting it wrong) - GetObjectMetadataRequest r - = requestFactory.newGetObjectMetadataRequest("/"); - DefaultRequest dr = new DefaultRequest(r, "S3"); - assertThat(handler.beforeMarshalling(r)) - .isNotNull(); - assertThat(handler.beforeExecution(r)) - .isNotNull(); - handler.beforeRequest(dr); - + interceptor.beforeExecution(context, attributes); + interceptor.modifyRequest(context, attributes); + interceptor.beforeMarshalling(context, attributes); + interceptor.afterMarshalling(context, attributes); + interceptor.modifyHttpRequest(context, attributes); + interceptor.beforeTransmission(context, attributes); + AuditSpanS3A span = attributes.getAttribute(AUDIT_SPAN_EXECUTION_ATTRIBUTE); + assertThat(span).isNotNull(); + assertThat(span.isValidSpan()).isFalse(); } /** @@ -192,14 +217,14 @@ public void testSingleRequestHandler() throws Throwable { public void testRequestHandlerLoading() throws Throwable { Configuration conf = noopAuditConfig(); conf.setClassLoader(this.getClass().getClassLoader()); - conf.set(AUDIT_REQUEST_HANDLERS, - SimpleAWSRequestHandler.CLASS); + conf.set(AUDIT_EXECUTION_INTERCEPTORS, + SimpleAWSExecutionInterceptor.CLASS); AuditManagerS3A manager = AuditIntegration.createAndStartAuditManager( conf, ioStatistics); - assertThat(manager.createRequestHandlers()) + assertThat(manager.createExecutionInterceptors()) .hasSize(2) - .hasAtLeastOneElementOfType(SimpleAWSRequestHandler.class); + .hasAtLeastOneElementOfType(SimpleAWSExecutionInterceptor.class); } @Test @@ -216,8 +241,8 @@ public void testLoggingAuditorBinding() throws Throwable { @Test public void testNoopAuditManager() throws Throwable { AuditManagerS3A manager = AuditIntegration.stubAuditManager(); - assertThat(manager.createStateChangeListener()) - .describedAs("transfer state change listener") + assertThat(manager.createTransferListener()) + .describedAs("transfer listener") .isNotNull(); } @@ -226,11 +251,10 @@ public void testSpanAttachAndRetrieve() throws Throwable { AuditManagerS3A manager = AuditIntegration.stubAuditManager(); AuditSpanS3A span = manager.createSpan("op", null, null); - GetObjectMetadataRequest request = - new GetObjectMetadataRequest("bucket", "key"); - attachSpanToRequest(request, span); - AWSAuditEventCallbacks callbacks = retrieveAttachedSpan(request); - assertThat(callbacks).isSameAs(span); + ExecutionAttributes attributes = ExecutionAttributes.builder().build(); + attachSpanToRequest(attributes, span); + AuditSpanS3A retrievedSpan = retrieveAttachedSpan(attributes); + assertThat(retrievedSpan).isSameAs(span); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditSpanLifecycle.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditSpanLifecycle.java index 608667d9dfed8..e5e4afc434c8e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditSpanLifecycle.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditSpanLifecycle.java @@ -20,13 +20,14 @@ import java.util.List; -import com.amazonaws.handlers.RequestHandler2; +import software.amazon.awssdk.core.interceptor.ExecutionInterceptor; import org.junit.Before; import org.junit.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.store.audit.AuditSpan; + import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.noopAuditConfig; import static org.assertj.core.api.Assertions.assertThat; @@ -56,10 +57,10 @@ public void testStop() throws Throwable { } @Test - public void testCreateRequestHandlers() throws Throwable { - List handlers - = getManager().createRequestHandlers(); - assertThat(handlers).isNotEmpty(); + public void testCreateExecutionInterceptors() throws Throwable { + List interceptors + = getManager().createExecutionInterceptors(); + assertThat(interceptors).isNotEmpty(); } @Test diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java index b772e6dfc06fc..7f8dd043261b2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestHttpReferrerAuditHeader.java @@ -20,12 +20,11 @@ import java.io.IOException; import java.net.URISyntaxException; +import java.util.List; import java.util.Map; import java.util.regex.Matcher; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; -import com.amazonaws.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.http.SdkHttpRequest; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; @@ -38,6 +37,7 @@ import org.apache.hadoop.fs.store.audit.HttpReferrerAuditHeader; import org.apache.hadoop.security.UserGroupInformation; + import static org.apache.hadoop.fs.audit.AuditConstants.DELETE_KEYS_SIZE; import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.loggingAuditConfig; import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.REFERRER_HEADER_FILTER; @@ -97,13 +97,16 @@ protected Configuration createConfig() { public void testHttpReferrerPatchesTheRequest() throws Throwable { AuditSpan span = span(); long ts = span.getTimestamp(); - GetObjectMetadataRequest request = head(); - Map headers - = request.getCustomRequestHeaders(); + SdkHttpRequest request = head(); + Map> headers = request.headers(); assertThat(headers) .describedAs("Custom headers") .containsKey(HEADER_REFERRER); - String header = headers.get(HEADER_REFERRER); + List headerValues = headers.get(HEADER_REFERRER); + assertThat(headerValues) + .describedAs("Multiple referrer headers") + .hasSize(1); + String header = headerValues.get(0); LOG.info("Header is {}", header); Map params = HttpReferrerAuditHeader.extractQueryParameters(header); @@ -305,13 +308,16 @@ public void testStripWrappedQuotes() throws Throwable { @Test public void testGetObjectRange() throws Throwable { AuditSpan span = span(); - GetObjectRequest request = get(getObjectRequest -> getObjectRequest.setRange(100, 200)); - Map headers - = request.getCustomRequestHeaders(); + SdkHttpRequest request = get("bytes=100-200"); + Map> headers = request.headers(); assertThat(headers) - .describedAs("Custom headers") - .containsKey(HEADER_REFERRER); - String header = headers.get(HEADER_REFERRER); + .describedAs("Custom headers") + .containsKey(HEADER_REFERRER); + List headerValues = headers.get(HEADER_REFERRER); + assertThat(headerValues) + .describedAs("Multiple referrer headers") + .hasSize(1); + String header = headerValues.get(0); LOG.info("Header is {}", header); Map params = HttpReferrerAuditHeader.extractQueryParameters(header); @@ -324,13 +330,16 @@ public void testGetObjectRange() throws Throwable { @Test public void testGetObjectWithoutRange() throws Throwable { AuditSpan span = span(); - GetObjectRequest request = get(getObjectRequest -> {}); - Map headers - = request.getCustomRequestHeaders(); + SdkHttpRequest request = get(""); + Map> headers = request.headers(); assertThat(headers) .describedAs("Custom headers") .containsKey(HEADER_REFERRER); - String header = headers.get(HEADER_REFERRER); + List headerValues = headers.get(HEADER_REFERRER); + assertThat(headerValues) + .describedAs("Multiple referrer headers") + .hasSize(1); + String header = headerValues.get(0); LOG.info("Header is {}", header); Map params = HttpReferrerAuditHeader.extractQueryParameters(header); @@ -341,16 +350,20 @@ public void testGetObjectWithoutRange() throws Throwable { public void testHttpReferrerForBulkDelete() throws Throwable { AuditSpan span = span(); long ts = span.getTimestamp(); - DeleteObjectsRequest request = headForBulkDelete( + SdkHttpRequest request = headForBulkDelete( "key_01", "key_02", "key_03"); - Map headers - = request.getCustomRequestHeaders(); + Map> headers + = request.headers(); assertThat(headers) .describedAs("Custom headers") .containsKey(HEADER_REFERRER); - String header = headers.get(HEADER_REFERRER); + List headerValues = headers.get(HEADER_REFERRER); + assertThat(headerValues) + .describedAs("Multiple referrer headers") + .hasSize(1); + String header = headerValues.get(0); LOG.info("Header is {}", header); Map params = HttpReferrerAuditHeader.extractQueryParameters(header); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java index 8d37b432acb56..0059e5b6c5392 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestLoggingAuditor.java @@ -18,9 +18,12 @@ package org.apache.hadoop.fs.s3a.audit; -import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; -import com.amazonaws.services.s3.model.CopyPartRequest; -import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.interceptor.InterceptorContext; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest; +import software.amazon.awssdk.services.s3.model.UploadPartCopyRequest; +import software.amazon.awssdk.transfer.s3.progress.TransferListener; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; @@ -30,6 +33,7 @@ import org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor; import org.apache.hadoop.fs.store.audit.AuditSpan; + import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.loggingAuditConfig; import static org.assertj.core.api.Assertions.assertThat; @@ -131,8 +135,23 @@ public void testLoggingSpan() throws Throwable { */ @Test public void testCopyOutsideSpanAllowed() throws Throwable { - getManager().beforeExecution(new CopyPartRequest()); - getManager().beforeExecution(new CompleteMultipartUploadRequest()); + getManager().beforeExecution( + InterceptorContext.builder() + .request(UploadPartCopyRequest.builder().build()) + .build(), + ExecutionAttributes.builder().build()); + getManager().beforeExecution( + InterceptorContext.builder() + .request(GetBucketLocationRequest.builder().build()) + .build(), + ExecutionAttributes.builder().build()); + getManager().beforeExecution( + InterceptorContext.builder() + .request(CompleteMultipartUploadRequest.builder() + .multipartUpload(u -> {}) + .build()) + .build(), + ExecutionAttributes.builder().build()); } /** @@ -141,9 +160,9 @@ public void testCopyOutsideSpanAllowed() throws Throwable { */ @Test public void testTransferStateListenerOutsideSpan() throws Throwable { - TransferStateChangeListener listener - = getManager().createStateChangeListener(); - listener.transferStateChanged(null, null); + TransferListener listener + = getManager().createTransferListener(); + listener.transferInitiated(null); assertHeadUnaudited(); } @@ -158,15 +177,15 @@ public void testTransferStateListenerInSpan() throws Throwable { AuditSpan span = span(); // create the listener in the span - TransferStateChangeListener listener - = getManager().createStateChangeListener(); + TransferListener listener + = getManager().createTransferListener(); span.deactivate(); // head calls fail assertHeadUnaudited(); // until the state change switches this thread back to the span - listener.transferStateChanged(null, null); + listener.transferInitiated(null); // which can be probed assertActiveSpan(span); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 9fb09b4cede52..12234301b50d8 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -26,23 +26,25 @@ import java.util.List; import java.util.stream.IntStream; -import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.services.s3.model.MultipartUpload; +import software.amazon.awssdk.services.sts.model.StsException; import com.fasterxml.jackson.core.JsonProcessingException; import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.AWSBadRequestException; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; -import org.apache.hadoop.fs.s3a.MultipartUtils; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestConstants; import org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider; @@ -51,13 +53,14 @@ import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; +import org.apache.hadoop.fs.s3a.impl.InstantiationIOException; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; -import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.E_FORBIDDEN_AWS_PROVIDER; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.*; import static org.apache.hadoop.fs.s3a.auth.RoleModel.*; import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*; @@ -140,7 +143,6 @@ private E expectFileSystemCreateFailure( } @Test - @SuppressWarnings("deprecation") public void testCreateCredentialProvider() throws IOException { describe("Create the credential provider"); @@ -148,13 +150,12 @@ public void testCreateCredentialProvider() throws IOException { try (AssumedRoleCredentialProvider provider = new AssumedRoleCredentialProvider(uri, conf)) { LOG.info("Provider is {}", provider); - AWSCredentials credentials = provider.getCredentials(); + AwsCredentials credentials = provider.resolveCredentials(); assertNotNull("Null credentials from " + provider, credentials); } } @Test - @SuppressWarnings("deprecation") public void testCreateCredentialProviderNoURI() throws IOException { describe("Create the credential provider"); @@ -162,7 +163,7 @@ public void testCreateCredentialProviderNoURI() throws IOException { try (AssumedRoleCredentialProvider provider = new AssumedRoleCredentialProvider(null, conf)) { LOG.info("Provider is {}", provider); - AWSCredentials credentials = provider.getCredentials(); + AwsCredentials credentials = provider.resolveCredentials(); assertNotNull("Null credentials from " + provider, credentials); } } @@ -172,7 +173,6 @@ public void testCreateCredentialProviderNoURI() throws IOException { * @return a configuration set to use to the role ARN. * @throws JsonProcessingException problems working with JSON policies. */ - @SuppressWarnings("deprecation") protected Configuration createValidRoleConf() throws JsonProcessingException { String roleARN = getAssumedRoleARN(); @@ -186,13 +186,17 @@ protected Configuration createValidRoleConf() throws JsonProcessingException { } @Test - @SuppressWarnings("deprecation") public void testAssumedInvalidRole() throws Throwable { Configuration conf = new Configuration(); conf.set(ASSUMED_ROLE_ARN, ROLE_ARN_EXAMPLE); - interceptClosing(AWSSecurityTokenServiceException.class, + interceptClosing(StsException.class, "", - () -> new AssumedRoleCredentialProvider(uri, conf)); + () -> { + AssumedRoleCredentialProvider p = + new AssumedRoleCredentialProvider(uri, conf); + p.resolveCredentials(); + return p; + }); } @Test @@ -204,7 +208,6 @@ public void testAssumeRoleFSBadARN() throws Exception { } @Test - @SuppressWarnings("deprecation") public void testAssumeRoleNoARN() throws Exception { describe("Attemnpt to create the FS with no ARN"); Configuration conf = createAssumedRoleConfig(); @@ -237,7 +240,6 @@ public void testAssumeRoleFSBadPolicy2() throws Exception { } @Test - @SuppressWarnings("deprecation") public void testAssumeRoleCannotAuthAssumedRole() throws Exception { describe("Assert that you can't use assumed roles to auth assumed roles"); @@ -246,12 +248,11 @@ public void testAssumeRoleCannotAuthAssumedRole() throws Exception { conf.set(ASSUMED_ROLE_CREDENTIALS_PROVIDER, AssumedRoleCredentialProvider.NAME); expectFileSystemCreateFailure(conf, - IOException.class, + InstantiationIOException.class, E_FORBIDDEN_AWS_PROVIDER); } @Test - @SuppressWarnings("deprecation") public void testAssumeRoleBadInnerAuth() throws Exception { describe("Try to authenticate with a keypair with spaces"); @@ -267,7 +268,6 @@ public void testAssumeRoleBadInnerAuth() throws Exception { } @Test - @SuppressWarnings("deprecation") public void testAssumeRoleBadInnerAuth2() throws Exception { describe("Try to authenticate with an invalid keypair"); @@ -351,7 +351,6 @@ private Configuration createAssumedRoleConfig(String roleARN) { } @Test - @SuppressWarnings("deprecation") public void testAssumeRoleUndefined() throws Throwable { describe("Verify that you cannot instantiate the" + " AssumedRoleCredentialProvider without a role ARN"); @@ -363,12 +362,11 @@ public void testAssumeRoleUndefined() throws Throwable { } @Test - @SuppressWarnings("deprecation") public void testAssumedIllegalDuration() throws Throwable { describe("Expect the constructor to fail if the session is to short"); Configuration conf = new Configuration(); conf.set(ASSUMED_ROLE_SESSION_DURATION, "30s"); - interceptClosing(AWSSecurityTokenServiceException.class, "", + interceptClosing(StsException.class, "", () -> new AssumedRoleCredentialProvider(uri, conf)); } @@ -426,8 +424,7 @@ public void testAssumeRolePoliciesOverrideRolePerms() throws Throwable { bindRolePolicy(conf, policy( statement(false, S3_ALL_BUCKETS, S3_GET_OBJECT_TORRENT), - ALLOW_S3_GET_BUCKET_LOCATION, - STATEMENT_ALLOW_SSE_KMS_RW)); + ALLOW_S3_GET_BUCKET_LOCATION, STATEMENT_ALLOW_KMS_RW)); Path path = path("testAssumeRoleStillIncludesRolePerms"); roleFS = (S3AFileSystem) path.getFileSystem(conf); assertTouchForbidden(roleFS, path); @@ -447,8 +444,7 @@ public void testReadOnlyOperations() throws Throwable { bindRolePolicy(conf, policy( statement(false, S3_ALL_BUCKETS, S3_PATH_WRITE_OPERATIONS), - STATEMENT_ALL_S3, - STATEMENT_ALLOW_SSE_KMS_READ)); + STATEMENT_ALL_S3, STATEMENT_ALLOW_KMS_RW)); Path path = methodPath(); roleFS = (S3AFileSystem) path.getFileSystem(conf); // list the root path, expect happy @@ -468,7 +464,7 @@ public void testReadOnlyOperations() throws Throwable { // list multipart uploads. // This is part of the read policy. int counter = 0; - MultipartUtils.UploadIterator iterator = roleFS.listUploads("/"); + RemoteIterator iterator = roleFS.listUploads("/"); while (iterator.hasNext()) { counter++; iterator.next(); @@ -495,8 +491,7 @@ public void testRestrictedWriteSubdir() throws Throwable { Configuration conf = createAssumedRoleConfig(); bindRolePolicyStatements(conf, - STATEMENT_ALL_BUCKET_READ_ACCESS, - STATEMENT_ALLOW_SSE_KMS_RW, + STATEMENT_ALL_BUCKET_READ_ACCESS, STATEMENT_ALLOW_KMS_RW, new Statement(Effects.Allow) .addActions(S3_ALL_OPERATIONS) .addResources(directory(restrictedDir))); @@ -537,7 +532,6 @@ public Path methodPath() throws IOException { * don't break. */ @Test - @SuppressWarnings("deprecation") public void testAssumedRoleRetryHandler() throws Throwable { try(AssumedRoleCredentialProvider provider = new AssumedRoleCredentialProvider(getFileSystem().getUri(), @@ -563,8 +557,7 @@ public void testRestrictedCommitActions() throws Throwable { fs.delete(basePath, true); fs.mkdirs(readOnlyDir); - bindRolePolicyStatements(conf, - STATEMENT_ALLOW_SSE_KMS_RW, + bindRolePolicyStatements(conf, STATEMENT_ALLOW_KMS_RW, STATEMENT_ALL_BUCKET_READ_ACCESS, new Statement(Effects.Allow) .addActions(S3_PATH_RW_OPERATIONS) @@ -714,8 +707,7 @@ public void executePartialDelete(final Configuration conf, S3AFileSystem fs = getFileSystem(); fs.delete(destDir, true); - bindRolePolicyStatements(conf, - STATEMENT_ALLOW_SSE_KMS_RW, + bindRolePolicyStatements(conf, STATEMENT_ALLOW_KMS_RW, statement(true, S3_ALL_BUCKETS, S3_ALL_OPERATIONS), new Statement(Effects.Deny) .addActions(S3_PATH_WRITE_OPERATIONS) @@ -746,8 +738,7 @@ public void testBucketLocationForbidden() throws Throwable { describe("Restrict role to read only"); Configuration conf = createAssumedRoleConfig(); - bindRolePolicyStatements(conf, - STATEMENT_ALLOW_SSE_KMS_RW, + bindRolePolicyStatements(conf, STATEMENT_ALLOW_KMS_RW, statement(true, S3_ALL_BUCKETS, S3_ALL_OPERATIONS), statement(false, S3_ALL_BUCKETS, S3_GET_BUCKET_LOCATION)); Path path = methodPath(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumedRoleCommitOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumedRoleCommitOperations.java index dabc0abc2af21..2dc8497d61804 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumedRoleCommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumedRoleCommitOperations.java @@ -61,8 +61,7 @@ public void setup() throws Exception { restrictedDir = super.path("restricted"); Configuration conf = newAssumedRoleConfig(getConfiguration(), getAssumedRoleARN()); - bindRolePolicyStatements(conf, - STATEMENT_ALLOW_SSE_KMS_RW, + bindRolePolicyStatements(conf, STATEMENT_ALLOW_KMS_RW, statement(true, S3_ALL_BUCKETS, S3_BUCKET_READ_OPERATIONS), new RoleModel.Statement(RoleModel.Effects.Allow) .addActions(S3_PATH_RW_OPERATIONS) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java index a829d470e7a66..fe48cd8911578 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java @@ -25,12 +25,12 @@ import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; -import com.amazonaws.SignableRequest; -import com.amazonaws.auth.AWS4Signer; -import com.amazonaws.arn.Arn; -import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.auth.Signer; -import com.amazonaws.services.s3.internal.AWSS3V4Signer; +import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.auth.signer.Aws4Signer; +import software.amazon.awssdk.auth.signer.AwsS3V4Signer; +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.signer.Signer; +import software.amazon.awssdk.http.SdkHttpFullRequest; import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; @@ -70,12 +70,15 @@ public class ITestCustomSigner extends AbstractS3ATestBase { public void setup() throws Exception { super.setup(); final S3AFileSystem fs = getFileSystem(); - regionName = determineRegion(fs.getBucket()); + final Configuration conf = fs.getConf(); + endpoint = conf.getTrimmed(Constants.ENDPOINT, Constants.CENTRAL_ENDPOINT); + LOG.info("Test endpoint is {}", endpoint); + regionName = conf.getTrimmed(Constants.AWS_REGION, ""); + if (regionName.isEmpty()) { + regionName = determineRegion(fs.getBucket()); + } LOG.info("Determined region name to be [{}] for bucket [{}]", regionName, fs.getBucket()); - endpoint = fs.getConf() - .get(Constants.ENDPOINT, Constants.CENTRAL_ENDPOINT); - LOG.info("Test endpoint is {}", endpoint); } @Test @@ -118,11 +121,14 @@ private FileSystem runMkDirAndVerify(UserGroupInformation ugi, .isGreaterThan(invocationCount); Assertions.assertThat(CustomSigner.lastStoreValue) - .as("Store value should not be null").isNotNull(); + .as("Store value should not be null in %s", CustomSigner.description()) + .isNotNull(); Assertions.assertThat(CustomSigner.lastStoreValue.conf) - .as("Configuration should not be null").isNotNull(); + .as("Configuration should not be null in %s", CustomSigner.description()) + .isNotNull(); Assertions.assertThat(CustomSigner.lastStoreValue.conf.get(TEST_ID_KEY)) - .as("Configuration TEST_KEY mismatch").isEqualTo(identifier); + .as("Configuration TEST_KEY mismatch in %s", CustomSigner.description()) + .isEqualTo(identifier); return fs; }); @@ -152,7 +158,7 @@ private Configuration createTestConfig(String identifier) { } private String determineRegion(String bucketName) throws IOException { - return getFileSystem().getBucketLocation(bucketName); + return getS3AInternals().getBucketLocation(bucketName); } @Private @@ -183,35 +189,29 @@ public CustomSigner() { * request because the signature calculated by the service doesn't match * what we sent. * @param request the request to sign. - * @param credentials credentials used to sign the request. + * @param executionAttributes request executionAttributes which contain the credentials. */ @Override - public void sign(SignableRequest request, AWSCredentials credentials) { + public SdkHttpFullRequest sign(SdkHttpFullRequest request, + ExecutionAttributes executionAttributes) { int c = INVOCATION_COUNT.incrementAndGet(); LOG.info("Signing request #{}", c); - String host = request.getEndpoint().getHost(); + String host = request.host(); String bucketName = parseBucketFromHost(host); try { lastStoreValue = CustomSignerInitializer .getStoreValue(bucketName, UserGroupInformation.getCurrentUser()); + LOG.info("Store value for bucket {} is {}", bucketName, lastStoreValue); } catch (IOException e) { - throw new RuntimeException("Failed to get current Ugi", e); + throw new RuntimeException("Failed to get current Ugi " + e, e); } if (bucketName.equals("kms")) { - AWS4Signer realKMSSigner = new AWS4Signer(); - realKMSSigner.setServiceName("kms"); - if (lastStoreValue != null) { - realKMSSigner.setRegionName(lastStoreValue.conf.get(TEST_REGION_KEY)); - } - realKMSSigner.sign(request, credentials); + Aws4Signer realKMSSigner = Aws4Signer.create(); + return realKMSSigner.sign(request, executionAttributes); } else { - AWSS3V4Signer realSigner = new AWSS3V4Signer(); - realSigner.setServiceName("s3"); - if (lastStoreValue != null) { - realSigner.setRegionName(lastStoreValue.conf.get(TEST_REGION_KEY)); - } - realSigner.sign(request, credentials); + AwsS3V4Signer realSigner = AwsS3V4Signer.create(); + return realSigner.sign(request, executionAttributes); } } @@ -235,11 +235,11 @@ private String parseBucketFromHost(String host) { String accessPointName = bucketName.substring(0, bucketName.length() - (accountId.length() + 1)); Arn arn = Arn.builder() - .withAccountId(accountId) - .withPartition("aws") - .withRegion(hostBits[2]) - .withResource("accesspoint" + "/" + accessPointName) - .withService("s3").build(); + .accountId(accountId) + .partition("aws") + .region(hostBits[2]) + .resource("accesspoint" + "/" + accessPointName) + .service("s3").build(); bucketName = arn.toString(); } @@ -254,6 +254,14 @@ public static int getInstantiationCount() { public static int getInvocationCount() { return INVOCATION_COUNT.get(); } + + public static String description() { + return "CustomSigner{" + + "invocations=" + INVOCATION_COUNT.get() + + ", instantiations=" + INSTANTIATION_COUNT.get() + + ", lastStoreValue=" + lastStoreValue + + "}"; + } } @Private @@ -267,6 +275,7 @@ public void registerStore(String bucketName, Configuration storeConf, DelegationTokenProvider dtProvider, UserGroupInformation storeUgi) { StoreKey storeKey = new StoreKey(bucketName, storeUgi); StoreValue storeValue = new StoreValue(storeConf, dtProvider); + LOG.info("Registering store {} with value {}", storeKey, storeValue); knownStores.put(storeKey, storeValue); } @@ -274,6 +283,7 @@ public void registerStore(String bucketName, Configuration storeConf, public void unregisterStore(String bucketName, Configuration storeConf, DelegationTokenProvider dtProvider, UserGroupInformation storeUgi) { StoreKey storeKey = new StoreKey(bucketName, storeUgi); + LOG.info("Unregistering store {}", storeKey); knownStores.remove(storeKey); } @@ -309,6 +319,14 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(bucketName, ugi); } + + @Override + public String toString() { + return "StoreKey{" + + "bucketName='" + bucketName + '\'' + + ", ugi=" + ugi + + '}'; + } } static class StoreValue { @@ -320,6 +338,14 @@ public StoreValue(Configuration conf, this.conf = conf; this.dtProvider = dtProvider; } + + @Override + public String toString() { + return "StoreValue{" + + "conf=" + conf + + ", dtProvider=" + dtProvider + + '}'; + } } } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java index a16e1b5e492fb..7151c38ad3e27 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java @@ -260,8 +260,7 @@ public void initNoReadAccess() throws Throwable { // it still has write access, which can be explored in the final // step to delete files and directories. roleConfig = createAssumedRoleConfig(); - bindRolePolicyStatements(roleConfig, - STATEMENT_ALLOW_SSE_KMS_RW, + bindRolePolicyStatements(roleConfig, STATEMENT_ALLOW_KMS_RW, statement(true, S3_ALL_BUCKETS, S3_ALL_OPERATIONS), new Statement(Effects.Deny) .addActions(S3_ALL_GET) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java index 37c2dce4e1d72..852f03ea618fd 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java @@ -146,19 +146,22 @@ public static void assertTouchForbidden(final FileSystem fs, final Path path) * @param roleARN ARN of role * @return the new configuration */ - @SuppressWarnings("deprecation") public static Configuration newAssumedRoleConfig( final Configuration srcConf, final String roleARN) { Configuration conf = new Configuration(srcConf); removeBaseAndBucketOverrides(conf, + S3A_BUCKET_PROBE, DELEGATION_TOKEN_BINDING, ASSUMED_ROLE_ARN, - AWS_CREDENTIALS_PROVIDER); + AWS_CREDENTIALS_PROVIDER, + ASSUMED_ROLE_SESSION_DURATION); conf.set(AWS_CREDENTIALS_PROVIDER, AssumedRoleCredentialProvider.NAME); conf.set(ASSUMED_ROLE_ARN, roleARN); conf.set(ASSUMED_ROLE_SESSION_NAME, "test"); conf.set(ASSUMED_ROLE_SESSION_DURATION, "15m"); + // force in bucket resolution during startup + conf.setInt(S3A_BUCKET_PROBE, 1); disableFilesystemCaching(conf); return conf; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestIAMInstanceCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestIAMInstanceCredentialsProvider.java new file mode 100644 index 0000000000000..c8986eab9b850 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestIAMInstanceCredentialsProvider.java @@ -0,0 +1,107 @@ +/** + * 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 org.assertj.core.api.Assertions; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentials; + +import org.apache.hadoop.test.AbstractHadoopTestBase; + +/** + * Unit tests for IAMInstanceCredentials provider. + * This is a bit tricky as we don't want to require running in EC2, + * but nor do we want a test which doesn't work in EC2. + */ +public class TestIAMInstanceCredentialsProvider extends AbstractHadoopTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestIAMInstanceCredentialsProvider.class); + + /** + * Error string from + * software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider, + * if IAM resolution has been disabled: {@value}. + */ + public static final String DISABLED = + "IMDS credentials have been disabled by environment variable or system property"; + + /** + * Test an immediate create/close. + */ + @Test + public void testIAMInstanceCredentialsProviderClose() throws Throwable { + new IAMInstanceCredentialsProvider().close(); + } + + /** + * Test instantiation. + * Multiple outcomes depending on host setup. + *

      + *
    1. In EC2: credentials resolved. + * Assert the credentials comes with a key.
    2. + *
    3. Not in EC2: NoAwsCredentialsException wraps network error trying + * to talk to the service. + * Assert wrapped exception is an IOE.
    4. + *
    5. IMDS resolution disabled by env var/sysprop. + * NoAwsCredentialsException raised doesn't contain an IOE. + * Require the message to contain the {@link #DISABLED} text.
    6. j + *
    + */ + @Test + public void testIAMInstanceCredentialsInstantiate() throws Throwable { + try (IAMInstanceCredentialsProvider provider = new IAMInstanceCredentialsProvider()) { + try { + final AwsCredentials credentials = provider.resolveCredentials(); + // if we get here this test suite is running in a container/EC2 + LOG.info("Credentials: retrieved from {}: key={}", + provider.isContainerCredentialsProvider() ? "container" : "EC2", + credentials.accessKeyId()); + Assertions.assertThat(credentials.accessKeyId()) + .describedAs("Access key from IMDS") + .isNotBlank(); + + // and if we get here, so does a second call + provider.resolveCredentials(); + } catch (NoAwsCredentialsException expected) { + // this is expected if the test is not running in a container/EC2 + LOG.info("Not running in a container/EC2"); + LOG.info("Exception raised", expected); + // and we expect to have fallen back to InstanceProfileCredentialsProvider + Assertions.assertThat(provider.isContainerCredentialsProvider()) + .describedAs("%s: shoud be using InstanceProfileCredentialsProvider") + .isFalse(); + final Throwable cause = expected.getCause(); + if (cause == null) { + throw expected; + } + if (!(cause instanceof IOException) + && !cause.toString().contains(DISABLED)) { + throw new AssertionError("Cause not a IOException", cause); + } + } + } + } + + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestMarshalledCredentials.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestMarshalledCredentials.java index c5ed9dbaac429..b9d547635f7f3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestMarshalledCredentials.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestMarshalledCredentials.java @@ -21,7 +21,7 @@ import java.net.URI; import java.net.URISyntaxException; -import com.amazonaws.auth.AWSCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; import org.junit.Before; import org.junit.Test; @@ -94,13 +94,13 @@ public void testMarshalledCredentialProviderSession() throws Throwable { new Configuration(false), credentials, MarshalledCredentials.CredentialTypeRequired.SessionOnly); - AWSCredentials aws = provider.getCredentials(); + AwsCredentials aws = provider.resolveCredentials(); assertEquals(credentials.toString(), credentials.getAccessKey(), - aws.getAWSAccessKeyId()); + aws.accessKeyId()); assertEquals(credentials.toString(), credentials.getSecretKey(), - aws.getAWSSecretKey()); + aws.secretAccessKey()); // because the credentials are set to full only, creation will fail } @@ -119,7 +119,7 @@ public void testCredentialTypeMismatch() throws Throwable { MarshalledCredentials.CredentialTypeRequired.FullOnly); // because the credentials are set to full only, creation will fail intercept(NoAuthWithAWSException.class, "test", - () -> provider.getCredentials()); + () -> provider.resolveCredentials()); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java index ca87b5c1b34a6..b56b8c20bfe77 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java @@ -19,43 +19,39 @@ import java.io.Closeable; import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; import java.nio.charset.StandardCharsets; import java.security.PrivilegedExceptionAction; import java.util.HashMap; import java.util.Map; import java.util.Objects; -import java.util.concurrent.TimeUnit; - -import com.amazonaws.AmazonWebServiceRequest; -import com.amazonaws.DefaultRequest; -import com.amazonaws.SignableRequest; -import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.auth.Signer; -import com.amazonaws.auth.SignerFactory; + +import software.amazon.awssdk.core.interceptor.ExecutionAttributes; +import software.amazon.awssdk.core.signer.Signer; +import software.amazon.awssdk.http.SdkHttpFullRequest; +import software.amazon.awssdk.http.SdkHttpMethod; import org.assertj.core.api.Assertions; import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.Timeout; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.s3a.auth.TestSignerManager.SignerInitializerForTest.StoreValue; import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider; +import org.apache.hadoop.fs.s3a.impl.InstantiationIOException; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.test.LambdaTestUtils; +import org.apache.hadoop.test.AbstractHadoopTestBase; import static org.apache.hadoop.fs.s3a.Constants.CUSTOM_SIGNERS; +import static org.apache.hadoop.fs.s3a.auth.SignerFactory.S3_V2_SIGNER; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Tests for the SignerManager. */ -public class TestSignerManager { +public class TestSignerManager extends AbstractHadoopTestBase { private static final Text TEST_TOKEN_KIND = new Text("TestTokenKind"); private static final Text TEST_TOKEN_SERVICE = new Text("TestTokenService"); @@ -65,9 +61,6 @@ public class TestSignerManager { private static final String TESTUSER1 = "testuser1"; private static final String TESTUSER2 = "testuser2"; - @Rule public Timeout testTimeout = new Timeout(10_000L, - TimeUnit.MILLISECONDS); - @Before public void beforeTest() { SignerForTest1.reset(); @@ -99,11 +92,8 @@ public void testCustomSignerFailureIfNotRegistered() throws Exception { // Make sure the config is respected. signerManager.initCustomSigners(); // Simulate a call from the AWS SDK to create the signer. - LambdaTestUtils.intercept(Exception.class, + intercept(InstantiationIOException.class, () -> SignerFactory.createSigner("testsignerUnregistered", null)); - // Expecting generic Exception.class to handle future implementation - // changes. - // For now, this is an NPE } @Test @@ -284,7 +274,7 @@ private void attemptSignAndVerify(String identifier, String bucket, throws IOException, InterruptedException { ugi.doAs((PrivilegedExceptionAction) () -> { Signer signer = new SignerForInitializerTest(); - SignableRequest signableRequest = constructSignableRequest(bucket); + SdkHttpFullRequest signableRequest = constructSignableRequest(bucket); signer.sign(signableRequest, null); verifyStoreValueInSigner(expectNullStoreInfo, bucket, identifier); return null; @@ -336,8 +326,10 @@ public static class SignerForTest1 implements Signer { private static boolean initialized = false; @Override - public void sign(SignableRequest request, AWSCredentials credentials) { + public SdkHttpFullRequest sign(SdkHttpFullRequest sdkHttpFullRequest, + ExecutionAttributes executionAttributes) { initialized = true; + return sdkHttpFullRequest; } public static void reset() { @@ -354,8 +346,10 @@ public static class SignerForTest2 implements Signer { private static boolean initialized = false; @Override - public void sign(SignableRequest request, AWSCredentials credentials) { + public SdkHttpFullRequest sign(SdkHttpFullRequest sdkHttpFullRequest, + ExecutionAttributes executionAttributes) { initialized = true; + return sdkHttpFullRequest; } public static void reset() { @@ -472,11 +466,15 @@ public static class SignerForInitializerTest implements Signer { private static StoreValue retrievedStoreValue; @Override - public void sign(SignableRequest request, AWSCredentials credentials) { - String bucketName = request.getEndpoint().getHost(); + public SdkHttpFullRequest sign(SdkHttpFullRequest sdkHttpFullRequest, + ExecutionAttributes executionAttributes) { + String bucket = sdkHttpFullRequest.host().split("//")[1]; + // remove trailing slash + String bucketName = bucket.substring(0, bucket.length() - 1); try { retrievedStoreValue = SignerInitializerForTest .getStoreInfo(bucketName, UserGroupInformation.getCurrentUser()); + return sdkHttpFullRequest; } catch (IOException e) { throw new RuntimeException("Failed to get current ugi", e); } @@ -579,12 +577,22 @@ private String createTokenIdentifierString(String identifier, return identifier + "_" + bucketName + "_" + user; } - private SignableRequest constructSignableRequest(String bucketName) - throws URISyntaxException { - DefaultRequest signableRequest = new DefaultRequest( - AmazonWebServiceRequest.NOOP, "fakeservice"); - URI uri = new URI("s3://" + bucketName + "/"); - signableRequest.setEndpoint(uri); - return signableRequest; + private SdkHttpFullRequest constructSignableRequest(String bucketName) { + String host = "s3://" + bucketName + "/"; + return SdkHttpFullRequest.builder().host(host).protocol("https").method(SdkHttpMethod.GET) + .build(); + } + + @Test + public void testV2SignerRejected() throws Throwable { + intercept(InstantiationIOException.class, "no longer supported", + () -> SignerFactory.createSigner(S3_V2_SIGNER, "key")); } + + @Test + public void testUnknownSignerRejected() throws Throwable { + intercept(InstantiationIOException.class, "unknownSigner", + () -> SignerFactory.createSigner("unknownSigner", "key")); + } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/CountInvocationsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/CountInvocationsProvider.java index 3a7d78d68f7d5..4c7cd5c667999 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/CountInvocationsProvider.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/CountInvocationsProvider.java @@ -20,8 +20,10 @@ import java.util.concurrent.atomic.AtomicLong; -import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.auth.AWSCredentialsProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import org.apache.hadoop.fs.s3a.CredentialInitializationException; @@ -29,21 +31,37 @@ * Simple AWS credential provider which counts how often it is invoked. */ public class CountInvocationsProvider - implements AWSCredentialsProvider { + implements AwsCredentialsProvider { + + private static final Logger LOG = LoggerFactory.getLogger( + CountInvocationsProvider.class); public static final String NAME = CountInvocationsProvider.class.getName(); public static final AtomicLong COUNTER = new AtomicLong(0); + private final AtomicLong instanceCounter = new AtomicLong(0); + @Override - public AWSCredentials getCredentials() { - COUNTER.incrementAndGet(); - throw new CredentialInitializationException("no credentials"); + public AwsCredentials resolveCredentials() { + final long global = COUNTER.incrementAndGet(); + final long local = instanceCounter.incrementAndGet(); + final String msg = + String.format("counter with global count %d and local count %d", global, local); + LOG.debug("resolving credentials from {}", msg); + throw new CredentialInitializationException("no credentials from " + msg); } - @Override - public void refresh() { + public long getInstanceCounter() { + return instanceCounter.get(); + } + @Override + public String toString() { + return "CountInvocationsProvider{" + + "instanceCounter=" + instanceCounter.get() + + "; global counter=" + COUNTER.get() + + '}'; } public static long getInvocationCount() { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFilesystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFilesystem.java index 295125169a00c..e85e33c844d52 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFilesystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFilesystem.java @@ -26,14 +26,15 @@ import java.net.URI; import java.nio.file.AccessDeniedException; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.ObjectMetadata; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.HeadBucketResponse; 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; @@ -254,7 +255,6 @@ public void testGetDTfromFileSystem() throws Throwable { } @Test - @SuppressWarnings("deprecation") public void testAddTokensFromFileSystem() throws Throwable { describe("verify FileSystem.addDelegationTokens() collects tokens"); S3AFileSystem fs = getFileSystem(); @@ -276,7 +276,7 @@ public void testAddTokensFromFileSystem() throws Throwable { AWSCredentialProviderList providerList = requireNonNull( delegationTokens.getCredentialProviders(), "providers"); - providerList.getCredentials(); + providerList.resolveCredentials(); } @Test @@ -323,14 +323,15 @@ protected Credentials createDelegationTokens() throws IOException { * 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. */ - @SuppressWarnings("deprecation") @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(); // force a probe of the remote FS to make sure its endpoint is valid - fs.getObjectMetadata(new Path("/")); + // TODO: Check what should happen here. Calling headObject() on the root path fails in V2, + // with the error that key cannot be empty. + // fs.getObjectMetadata(new Path("/")); readLandsatMetadata(fs); URI uri = fs.getUri(); @@ -577,29 +578,29 @@ public void testDelegationBindingMismatch2() throws Throwable { * @return result of the HEAD * @throws Exception failure */ - @SuppressWarnings("deprecation") - protected ObjectMetadata readLandsatMetadata(final S3AFileSystem delegatedFS) + protected HeadBucketResponse readLandsatMetadata(final S3AFileSystem delegatedFS) throws Exception { AWSCredentialProviderList testingCreds - = delegatedFS.shareCredentials("testing"); + = delegatedFS.getS3AInternals().shareCredentials("testing"); URI landsat = new URI(DEFAULT_CSVTEST_FILE); DefaultS3ClientFactory factory = new DefaultS3ClientFactory(); - factory.setConf(new Configuration(delegatedFS.getConf())); + Configuration conf = delegatedFS.getConf(); + factory.setConf(conf); String host = landsat.getHost(); S3ClientFactory.S3ClientCreationParameters parameters = null; parameters = new S3ClientFactory.S3ClientCreationParameters() .withCredentialSet(testingCreds) .withPathUri(new URI("s3a://localhost/")) - .withEndpoint(DEFAULT_ENDPOINT) .withMetrics(new EmptyS3AStatisticsContext() .newStatisticsFromAwsSdk()) .withUserAgentSuffix("ITestSessionDelegationInFilesystem"); - AmazonS3 s3 = factory.createS3Client(landsat, parameters); + + S3Client s3 = factory.createS3Client(landsat, parameters); return Invoker.once("HEAD", host, - () -> s3.getObjectMetadata(host, landsat.getPath().substring(1))); + () -> s3.headBucket(b -> b.bucket(host))); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationTokens.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationTokens.java index fab7ffdbb76f8..efc775966859d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationTokens.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationTokens.java @@ -22,8 +22,8 @@ import java.io.IOException; import java.net.URI; -import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.auth.AWSSessionCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; import org.hamcrest.Matchers; import org.junit.Test; import org.slf4j.Logger; @@ -186,11 +186,15 @@ public void testCreateAndUseDT() throws Throwable { final MarshalledCredentials creds; try(S3ADelegationTokens dt2 = instantiateDTSupport(getConfiguration())) { dt2.start(); + // first creds are good + dt2.getCredentialProviders().resolveCredentials(); + + // reset to the original dt dt2.resetTokenBindingToDT(originalDT); - final AWSSessionCredentials awsSessionCreds + final AwsSessionCredentials awsSessionCreds = verifySessionCredentials( - dt2.getCredentialProviders().getCredentials()); + dt2.getCredentialProviders().resolveCredentials()); final MarshalledCredentials origCreds = fromAWSCredentials( awsSessionCreds); @@ -249,7 +253,7 @@ public void testCreateWithRenewer() throws Throwable { * @return the retrieved DT. This is only for error reporting. * @throws IOException failure. */ - @SuppressWarnings({"OptionalGetWithoutIsPresent", "deprecation"}) + @SuppressWarnings({"OptionalGetWithoutIsPresent"}) protected AbstractS3ATokenIdentifier verifyCredentialPropagation( final S3AFileSystem fs, final MarshalledCredentials session, @@ -278,7 +282,7 @@ protected AbstractS3ATokenIdentifier verifyCredentialPropagation( LOG.info("Regenerated DT is {}", newDT); final MarshalledCredentials creds2 = fromAWSCredentials( verifySessionCredentials( - delegationTokens2.getCredentialProviders().getCredentials())); + delegationTokens2.getCredentialProviders().resolveCredentials())); assertEquals("Credentials", session, creds2); assertTrue("Origin in " + boundId, boundId.getOrigin() @@ -287,12 +291,12 @@ protected AbstractS3ATokenIdentifier verifyCredentialPropagation( } } - 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()); + private AwsSessionCredentials verifySessionCredentials( + final AwsCredentials creds) { + AwsSessionCredentials session = (AwsSessionCredentials) creds; + assertNotNull("access key", session.accessKeyId()); + assertNotNull("secret key", session.secretAccessKey()); + assertNotNull("session token", session.sessionToken()); return session; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/TestS3ADelegationTokenSupport.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/TestS3ADelegationTokenSupport.java index 88d9ebfcdfdc3..992643ff8ce98 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/TestS3ADelegationTokenSupport.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/TestS3ADelegationTokenSupport.java @@ -37,7 +37,6 @@ 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; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java index 98c27eb13fd8d..67c88039aad1b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java @@ -741,8 +741,8 @@ private void validateContent(Path dir, */ private void validateStorageClass(Path dir, String expectedStorageClass) throws Exception { Path expectedFile = getPart0000(dir); - S3AFileSystem fs = getFileSystem(); - String actualStorageClass = fs.getObjectMetadata(expectedFile).getStorageClass(); + String actualStorageClass = getS3AInternals().getObjectMetadata(expectedFile) + .storageClassAsString(); Assertions.assertThat(actualStorageClass) .describedAs("Storage class of object %s", expectedFile) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java index 613fab192874a..116d48e9de5fc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java @@ -23,6 +23,7 @@ import java.util.Map; import org.assertj.core.api.Assertions; +import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,8 +40,10 @@ import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import org.apache.hadoop.fs.s3a.scale.AbstractSTestS3AHugeFiles; +import org.apache.hadoop.fs.store.audit.AuditSpan; import static org.apache.hadoop.fs.s3a.MultipartTestUtils.listMultipartUploads; +import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.extractXAttrLongValue; @@ -118,6 +121,19 @@ protected Path getPathOfFileToCreate() { return magicOutputFile; } + @Test + public void test_000_CleanupPendingUploads() throws IOException { + describe("Cleanup any existing pending uploads"); + final S3AFileSystem fs = getFileSystem(); + final String key = fs.pathToKey(finalDirectory); + final AuditSpan span = fs.getAuditSpanSource().createSpan( + MULTIPART_UPLOAD_ABORT_UNDER_PATH_INVOKED.getSymbol(), + key, null); + final int count = fs.createWriteOperationHelper(span) + .abortMultipartUploadsUnderPath(key + "/"); + LOG.info("Aborted {} uploads under {}", count, key); + } + @Override public void test_030_postCreationAssertions() throws Throwable { describe("Committing file"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java index 62b76b7c395bc..a6d9b7fcaf9ea 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java @@ -29,21 +29,20 @@ import java.util.UUID; import java.util.stream.Collectors; -import com.amazonaws.AmazonClientException; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.AmazonS3Client; -import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; -import com.amazonaws.services.s3.model.AmazonS3Exception; -import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; -import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; -import com.amazonaws.services.s3.model.DeleteObjectRequest; -import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; -import com.amazonaws.services.s3.model.InitiateMultipartUploadResult; -import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; -import com.amazonaws.services.s3.model.MultipartUpload; -import com.amazonaws.services.s3.model.MultipartUploadListing; -import com.amazonaws.services.s3.model.UploadPartRequest; -import com.amazonaws.services.s3.model.UploadPartResult; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse; +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadResponse; +import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; +import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest; +import software.amazon.awssdk.services.s3.model.ListMultipartUploadsResponse; +import software.amazon.awssdk.services.s3.model.MultipartUpload; +import software.amazon.awssdk.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.services.s3.model.UploadPartResponse; import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; import org.junit.AfterClass; @@ -81,6 +80,7 @@ import org.apache.hadoop.service.ServiceOperations; import org.apache.hadoop.test.HadoopTestBase; + import static org.mockito.ArgumentMatchers.*; import static org.mockito.Mockito.*; @@ -339,7 +339,7 @@ public abstract static class JobCommitterTest // created in Before private StagingTestBase.ClientResults results = null; private StagingTestBase.ClientErrors errors = null; - private AmazonS3 mockClient = null; + private S3Client mockClient = null; @Before public void setupJob() throws Exception { @@ -448,7 +448,7 @@ protected File getTempDir() { public static class ClientResults implements Serializable { private static final long serialVersionUID = -3137637327090709905L; // For inspection of what the committer did - private final Map requests = + private final Map requests = Maps.newHashMap(); private final List uploads = Lists.newArrayList(); private final List parts = Lists.newArrayList(); @@ -461,7 +461,7 @@ public static class ClientResults implements Serializable { Maps.newHashMap(); private final List deletes = Lists.newArrayList(); - public Map getRequests() { + public Map getRequests() { return requests; } @@ -490,7 +490,7 @@ public List getDeletes() { } public List getDeletePaths() { - return deletes.stream().map(DeleteObjectRequest::getKey).collect( + return deletes.stream().map(DeleteObjectRequest::key).collect( Collectors.toList()); } @@ -619,197 +619,163 @@ private static T getArgumentAt(InvocationOnMock invocation, int index, * @param errors when (if any) to fail * @return the mock client to patch in to a committer/FS instance */ - public static AmazonS3 newMockS3Client(final ClientResults results, + public static S3Client newMockS3Client(final ClientResults results, final ClientErrors errors) { - AmazonS3Client mockClient = mock(AmazonS3Client.class); + S3Client mockClientV2 = mock(S3Client.class); final Object lock = new Object(); // initiateMultipartUpload - when(mockClient - .initiateMultipartUpload(any(InitiateMultipartUploadRequest.class))) + when(mockClientV2 + .createMultipartUpload(any(CreateMultipartUploadRequest.class))) .thenAnswer(invocation -> { - LOG.debug("initiateMultipartUpload for {}", mockClient); + LOG.debug("initiateMultipartUpload for {}", mockClientV2); synchronized (lock) { if (results.requests.size() == errors.failOnInit) { if (errors.recover) { errors.failOnInit(-1); } - throw new AmazonClientException( - "Mock Fail on init " + results.requests.size()); + throw AwsServiceException.builder() + .message("Mock Fail on init " + results.requests.size()) + .build(); } String uploadId = UUID.randomUUID().toString(); - InitiateMultipartUploadRequest req = getArgumentAt(invocation, - 0, InitiateMultipartUploadRequest.class); + CreateMultipartUploadRequest req = getArgumentAt(invocation, + 0, CreateMultipartUploadRequest.class); results.requests.put(uploadId, req); - results.activeUploads.put(uploadId, req.getKey()); + results.activeUploads.put(uploadId, req.key()); results.uploads.add(uploadId); - return newResult(results.requests.get(uploadId), uploadId); + return CreateMultipartUploadResponse.builder() + .uploadId(uploadId) + .build(); } }); // uploadPart - when(mockClient.uploadPart(any(UploadPartRequest.class))) + when(mockClientV2.uploadPart(any(UploadPartRequest.class), any(RequestBody.class))) .thenAnswer(invocation -> { - LOG.debug("uploadPart for {}", mockClient); + LOG.debug("uploadPart for {}", mockClientV2); synchronized (lock) { if (results.parts.size() == errors.failOnUpload) { if (errors.recover) { errors.failOnUpload(-1); } LOG.info("Triggering upload failure"); - throw new AmazonClientException( - "Mock Fail on upload " + results.parts.size()); + throw AwsServiceException.builder() + .message("Mock Fail on upload " + results.parts.size()) + .build(); } UploadPartRequest req = getArgumentAt(invocation, 0, UploadPartRequest.class); results.parts.add(req); String etag = UUID.randomUUID().toString(); - List etags = results.tagsByUpload.get(req.getUploadId()); + List etags = results.tagsByUpload.get(req.uploadId()); if (etags == null) { etags = Lists.newArrayList(); - results.tagsByUpload.put(req.getUploadId(), etags); + results.tagsByUpload.put(req.uploadId(), etags); } etags.add(etag); - return newResult(req, etag); + return UploadPartResponse.builder().eTag(etag).build(); } }); // completeMultipartUpload - when(mockClient + when(mockClientV2 .completeMultipartUpload(any(CompleteMultipartUploadRequest.class))) .thenAnswer(invocation -> { - LOG.debug("completeMultipartUpload for {}", mockClient); + LOG.debug("completeMultipartUpload for {}", mockClientV2); synchronized (lock) { if (results.commits.size() == errors.failOnCommit) { if (errors.recover) { errors.failOnCommit(-1); } - throw new AmazonClientException( - "Mock Fail on commit " + results.commits.size()); + throw AwsServiceException.builder() + .message("Mock Fail on commit " + results.commits.size()) + .build(); } CompleteMultipartUploadRequest req = getArgumentAt(invocation, 0, CompleteMultipartUploadRequest.class); - String uploadId = req.getUploadId(); + String uploadId = req.uploadId(); removeUpload(results, uploadId); results.commits.add(req); - - return newResult(req); + return CompleteMultipartUploadResponse.builder().build(); } }); // abortMultipartUpload mocking doAnswer(invocation -> { - LOG.debug("abortMultipartUpload for {}", mockClient); + LOG.debug("abortMultipartUpload for {}", mockClientV2); synchronized (lock) { if (results.aborts.size() == errors.failOnAbort) { if (errors.recover) { errors.failOnAbort(-1); } - throw new AmazonClientException( - "Mock Fail on abort " + results.aborts.size()); + throw AwsServiceException.builder() + .message("Mock Fail on abort " + results.aborts.size()) + .build(); } AbortMultipartUploadRequest req = getArgumentAt(invocation, 0, AbortMultipartUploadRequest.class); - String id = req.getUploadId(); + String id = req.uploadId(); removeUpload(results, id); results.aborts.add(req); return null; } }) - .when(mockClient) + .when(mockClientV2) .abortMultipartUpload(any(AbortMultipartUploadRequest.class)); // deleteObject mocking doAnswer(invocation -> { - LOG.debug("deleteObject for {}", mockClient); + LOG.debug("deleteObject for {}", mockClientV2); synchronized (lock) { results.deletes.add(getArgumentAt(invocation, 0, DeleteObjectRequest.class)); return null; } }) - .when(mockClient) + .when(mockClientV2) .deleteObject(any(DeleteObjectRequest.class)); - // deleteObject mocking - doAnswer(invocation -> { - LOG.debug("deleteObject for {}", mockClient); - synchronized (lock) { - results.deletes.add(new DeleteObjectRequest( - getArgumentAt(invocation, 0, String.class), - getArgumentAt(invocation, 1, String.class) - )); - return null; - } - }).when(mockClient) - .deleteObject(any(String.class), any(String.class)); - // to String returns the debug information - when(mockClient.toString()).thenAnswer( + when(mockClientV2.toString()).thenAnswer( invocation -> "Mock3AClient " + results + " " + errors); - when(mockClient + when(mockClientV2 .listMultipartUploads(any(ListMultipartUploadsRequest.class))) .thenAnswer(invocation -> { synchronized (lock) { - MultipartUploadListing l = new MultipartUploadListing(); - l.setMultipartUploads( - results.activeUploads.entrySet().stream() - .map(e -> newMPU(e.getKey(), e.getValue())) - .collect(Collectors.toList())); - return l; + return ListMultipartUploadsResponse.builder() + .uploads(results.activeUploads.entrySet().stream() + .map(e -> MultipartUpload.builder() + .uploadId(e.getKey()) + .key(e.getValue()) + .build()) + .collect(Collectors.toList())) + .build(); } }); - return mockClient; + return mockClientV2; } /** * Remove an upload from the upload map. * @param results result set * @param uploadId The upload ID to remove - * @throws AmazonS3Exception with error code 404 if the id is unknown. + * @throws AwsServiceException with error code 404 if the id is unknown. */ protected static void removeUpload(final ClientResults results, final String uploadId) { String removed = results.activeUploads.remove(uploadId); if (removed == null) { // upload doesn't exist - AmazonS3Exception ex = new AmazonS3Exception( - "not found " + uploadId); - ex.setStatusCode(404); - throw ex; + throw AwsServiceException.builder() + .message("not found " + uploadId) + .statusCode(404) + .build(); } } - private static CompleteMultipartUploadResult newResult( - CompleteMultipartUploadRequest req) { - return new CompleteMultipartUploadResult(); - } - - - private static MultipartUpload newMPU(String id, String path) { - MultipartUpload up = new MultipartUpload(); - up.setUploadId(id); - up.setKey(path); - return up; - } - - private static UploadPartResult newResult(UploadPartRequest request, - String etag) { - UploadPartResult result = new UploadPartResult(); - result.setPartNumber(request.getPartNumber()); - result.setETag(etag); - return result; - } - - private static InitiateMultipartUploadResult newResult( - InitiateMultipartUploadRequest request, String uploadId) { - InitiateMultipartUploadResult result = new InitiateMultipartUploadResult(); - result.setUploadId(uploadId); - return result; - } - /** * create files in the attempt path that should be found by * {@code getTaskOutput}. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java index 4d24c07dacfe2..f96cf97ebd7f4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java @@ -27,7 +27,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import com.amazonaws.services.s3.model.PartETag; +import software.amazon.awssdk.services.s3.model.CompletedPart; import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; import org.assertj.core.api.Assertions; @@ -39,6 +39,7 @@ 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.FileSystem; @@ -159,8 +160,8 @@ private void createTasks() throws IOException { // step1: a list of tags. // this is the md5sum of hadoop 3.2.1.tar String tag = "9062dcf18ffaee254821303bbd11c72b"; - List etags = IntStream.rangeClosed(1, BLOCKS_PER_TASK + 1) - .mapToObj(i -> new PartETag(i, tag)) + List etags = IntStream.rangeClosed(1, BLOCKS_PER_TASK + 1) + .mapToObj(i -> CompletedPart.builder().partNumber(i).eTag(tag).build()) .collect(Collectors.toList()); SinglePendingCommit base = new SinglePendingCommit(); base.setBucket(BUCKET); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java index b4cc21b61e210..5ac7e9ce756c6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java @@ -31,9 +31,10 @@ import java.util.UUID; import java.util.stream.Collectors; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; -import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; + import org.apache.hadoop.thirdparty.com.google.common.collect.Sets; import org.assertj.core.api.Assertions; import org.junit.After; @@ -50,7 +51,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.AWSClientIOException; import org.apache.hadoop.fs.s3a.MockS3AFileSystem; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; @@ -69,6 +69,7 @@ import org.apache.hadoop.mapreduce.task.JobContextImpl; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; + import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.*; @@ -111,7 +112,7 @@ public class TestStagingCommitter extends StagingTestBase.MiniDFSTest { // created in Before private StagingTestBase.ClientResults results = null; private StagingTestBase.ClientErrors errors = null; - private AmazonS3 mockClient = null; + private S3Client mockClient = null; private File tmpDir; /** @@ -472,7 +473,7 @@ public void testTaskInitializeFailure() throws Exception { writeOutputFile(tac.getTaskAttemptID(), attemptPath, UUID.randomUUID().toString(), 10); - intercept(AWSClientIOException.class, + intercept(IOException.class, "Fail on init 1", "Should fail during init", () -> committer.commitTask(tac)); @@ -500,7 +501,7 @@ public void testTaskSingleFileUploadFailure() throws Exception { writeOutputFile(tac.getTaskAttemptID(), attemptPath, UUID.randomUUID().toString(), 10); - intercept((Class) AWSClientIOException.class, + intercept(IOException.class, "Fail on upload 2", "Should fail during upload", () -> { @@ -512,7 +513,7 @@ public void testTaskSingleFileUploadFailure() throws Exception { 1, results.getUploads().size()); assertEquals("Should abort the upload", results.getUploads().get(0), - results.getAborts().get(0).getUploadId()); + results.getAborts().get(0).uploadId()); assertPathDoesNotExist(fs, "Should remove the attempt path", attemptPath); } @@ -531,7 +532,7 @@ public void testTaskMultiFileUploadFailure() throws Exception { writeOutputFile(tac.getTaskAttemptID(), attemptPath, UUID.randomUUID().toString(), 10); - intercept((Class) AWSClientIOException.class, + intercept(IOException.class, "Fail on upload 5", "Should fail during upload", () -> { @@ -563,7 +564,7 @@ public void testTaskUploadAndAbortFailure() throws Exception { writeOutputFile(tac.getTaskAttemptID(), attemptPath, UUID.randomUUID().toString(), 10); - intercept((Class) AWSClientIOException.class, + intercept(IOException.class, "Fail on upload 5", "Should suppress abort failure, propagate upload failure", ()-> { @@ -636,7 +637,7 @@ public void testJobCommitFailure() throws Exception { errors.failOnCommit(5); setMockLogLevel(MockS3AFileSystem.LOG_NAME); - intercept(AWSClientIOException.class, + intercept(IOException.class, "Fail on commit 5", "Should propagate the commit failure", () -> { @@ -644,17 +645,16 @@ public void testJobCommitFailure() throws Exception { return jobCommitter.toString(); }); - Set commits = results.getCommits() .stream() .map(commit -> - "s3a://" + commit.getBucketName() + "/" + commit.getKey()) + "s3a://" + commit.bucket() + "/" + commit.key()) .collect(Collectors.toSet()); Set deletes = results.getDeletes() .stream() .map(delete -> - "s3a://" + delete.getBucketName() + "/" + delete.getKey()) + "s3a://" + delete.bucket() + "/" + delete.key()) .collect(Collectors.toSet()); Assertions.assertThat(commits) @@ -728,14 +728,14 @@ private Set runTasks(JobContext jobContext, private static Set getAbortedIds( List aborts) { return aborts.stream() - .map(AbortMultipartUploadRequest::getUploadId) + .map(AbortMultipartUploadRequest::uploadId) .collect(Collectors.toSet()); } private static Set getCommittedIds( List commits) { return commits.stream() - .map(CompleteMultipartUploadRequest::getUploadId) + .map(CompleteMultipartUploadRequest::uploadId) .collect(Collectors.toSet()); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java index 01be1e063060c..8da7e9c2c5c51 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java @@ -24,7 +24,7 @@ import java.util.Set; import java.util.UUID; -import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest; import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; import org.apache.hadoop.thirdparty.com.google.common.collect.Sets; import org.assertj.core.api.Assertions; @@ -36,6 +36,7 @@ import org.apache.hadoop.fs.PathExistsException; import org.apache.hadoop.mapreduce.JobContext; + import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.Mockito.*; @@ -146,10 +147,10 @@ public void testAppend() throws Exception { protected void verifyFilesCreated( final PartitionedStagingCommitter committer) { Set files = Sets.newHashSet(); - for (InitiateMultipartUploadRequest request : + for (CreateMultipartUploadRequest request : getMockResults().getRequests().values()) { - assertEquals(BUCKET, request.getBucketName()); - files.add(request.getKey()); + assertEquals(BUCKET, request.bucket()); + files.add(request.key()); } Assertions.assertThat(files) .describedAs("Should have the right number of uploads") diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java index d28f4279f1ba0..fbad671e1fa66 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java @@ -16,7 +16,6 @@ import java.io.IOException; import java.net.URI; -import com.amazonaws.services.s3.model.CryptoStorageMode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,7 +76,7 @@ protected void verifyReadBytes(FileSystem.Statistics stats) { *
    * NOTE: if Client side encryption is enabled, expected bytes written * should increase by 16(padding of data) + bytes for the key ID set + 94(KMS - * key generation) in case of storage type{@link CryptoStorageMode} as + * key generation) in case of storage type CryptoStorageMode as * ObjectMetadata(Default). If Crypto Storage mode is instruction file then * add additional bytes as that file is stored separately and would account * for bytes written. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java index 378f4a70433d7..967fb2df73be3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java @@ -28,7 +28,6 @@ import java.util.TreeSet; import java.util.stream.Collectors; -import com.amazonaws.services.s3.model.MultiObjectDeleteException; import org.assertj.core.api.Assertions; import org.junit.Test; import org.junit.runner.RunWith; @@ -56,6 +55,7 @@ import static org.apache.hadoop.fs.s3a.auth.RoleModel.Effects; import static org.apache.hadoop.fs.s3a.auth.RoleModel.Statement; import static org.apache.hadoop.fs.s3a.auth.RoleModel.directory; +import static org.apache.hadoop.fs.s3a.auth.RoleModel.resource; import static org.apache.hadoop.fs.s3a.auth.RoleModel.statement; import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.bindRolePolicyStatements; @@ -144,6 +144,11 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { */ private Path writableDir; + /** + * Instruction file created when using CSE, required to be added to policies. + */ + private Path writableDirInstructionFile; + /** * A directory to which restricted roles have only read access. */ @@ -216,6 +221,7 @@ public void setup() throws Exception { basePath = uniquePath(); readOnlyDir = new Path(basePath, "readonlyDir"); writableDir = new Path(basePath, "writableDir"); + writableDirInstructionFile = new Path(basePath, "writableDir.instruction"); readOnlyChild = new Path(readOnlyDir, "child"); noReadDir = new Path(basePath, "noReadDir"); // the full FS @@ -225,8 +231,7 @@ public void setup() throws Exception { // create the baseline assumed role assumedRoleConfig = createAssumedRoleConfig(); - bindRolePolicyStatements(assumedRoleConfig, - STATEMENT_ALLOW_SSE_KMS_RW, + bindRolePolicyStatements(assumedRoleConfig, STATEMENT_ALLOW_KMS_RW, STATEMENT_ALL_BUCKET_READ_ACCESS, // root: r-x new Statement(Effects.Allow) // dest: rwx .addActions(S3_PATH_RW_OPERATIONS) @@ -365,13 +370,13 @@ public void testMultiDeleteOptionPropagated() throws Throwable { public void testRenameParentPathNotWriteable() throws Throwable { describe("rename with parent paths not writeable; multi=%s", multiDelete); final Configuration conf = createAssumedRoleConfig(); - bindRolePolicyStatements(conf, - STATEMENT_ALLOW_SSE_KMS_RW, + bindRolePolicyStatements(conf, STATEMENT_ALLOW_KMS_RW, STATEMENT_ALL_BUCKET_READ_ACCESS, new Statement(Effects.Allow) .addActions(S3_PATH_RW_OPERATIONS) .addResources(directory(readOnlyDir)) - .addResources(directory(writableDir))); + .addResources(directory(writableDir)) + .addResources(resource(writableDirInstructionFile, false, false))); roleFS = (S3AFileSystem) readOnlyDir.getFileSystem(conf); S3AFileSystem fs = getFileSystem(); @@ -733,8 +738,7 @@ public void testRenamePermissionRequirements() throws Throwable { // s3:DeleteObjectVersion permission, and attempt rename // and then delete. Configuration roleConfig = createAssumedRoleConfig(); - bindRolePolicyStatements(roleConfig, - STATEMENT_ALLOW_SSE_KMS_RW, + bindRolePolicyStatements(roleConfig, STATEMENT_ALLOW_KMS_RW, STATEMENT_ALL_BUCKET_READ_ACCESS, // root: r-x new Statement(Effects.Allow) // dest: rwx .addActions(S3_PATH_RW_OPERATIONS) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestRenameDeleteRace.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestRenameDeleteRace.java index 2610f54b44e9e..73db942973211 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestRenameDeleteRace.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestRenameDeleteRace.java @@ -23,13 +23,14 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; -import com.amazonaws.AmazonClientException; +import software.amazon.awssdk.core.exception.SdkException; import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; @@ -207,7 +208,7 @@ private BlockingFakeDirMarkerFS() { @Override protected void maybeCreateFakeParentDirectory(final Path path) - throws IOException, AmazonClientException { + throws IOException, SdkException { LOG.info("waking anything blocked on the signal semaphore"); // notify anything waiting signalCreatingFakeParentDirectory.release(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestUploadPurgeOnDirectoryOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestUploadPurgeOnDirectoryOperations.java new file mode 100644 index 0000000000000..9e07027375989 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestUploadPurgeOnDirectoryOperations.java @@ -0,0 +1,163 @@ +/* + * 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.impl; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import software.amazon.awssdk.services.s3.model.MultipartUpload; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; +import org.apache.hadoop.fs.store.audit.AuditSpan; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertFileHasLength; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertHasPathCapabilities; +import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_OPERATIONS_PURGE_UPLOADS; +import static org.apache.hadoop.fs.s3a.MultipartTestUtils.clearAnyUploads; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_LIST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_MULTIPART_UPLOAD_ABORTED; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_MULTIPART_UPLOAD_LIST; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_COMMITTER_ENABLED; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_PATH_PREFIX; +import static org.apache.hadoop.util.functional.RemoteIterators.toList; + +/** + * Test behavior of purging uploads in rename and delete. + */ +public class ITestUploadPurgeOnDirectoryOperations extends AbstractS3ACostTest { + + @Override + public Configuration createConfiguration() { + final Configuration conf = super.createConfiguration(); + removeBaseAndBucketOverrides(conf, + DIRECTORY_OPERATIONS_PURGE_UPLOADS, + MAGIC_COMMITTER_ENABLED); + conf.setBoolean(DIRECTORY_OPERATIONS_PURGE_UPLOADS, true); + conf.setBoolean(MAGIC_COMMITTER_ENABLED, true); + return conf; + } + + @Override + public void setup() throws Exception { + super.setup(); + final S3AFileSystem fs = getFileSystem(); + assertHasPathCapabilities(fs, new Path("/"), + DIRECTORY_OPERATIONS_PURGE_UPLOADS); + clearAnyUploads(fs, methodPath()); + } + + @Test + public void testDeleteWithPendingUpload() throws Throwable { + + final S3AFileSystem fs = getFileSystem(); + final Path dir = methodPath(); + + // create a magic file. + createMagicFile(fs, dir); + + // and there's a pending upload + assertUploadCount(dir, 1); + + // delete the dir, with a cost of 1 abort, 1 list. + verifyMetrics(() -> fs.delete(dir, true), + with(OBJECT_MULTIPART_UPLOAD_ABORTED, 1), // abort + with(OBJECT_MULTIPART_UPLOAD_LIST, 1), // HTTP request inside iterator + with(MULTIPART_UPLOAD_LIST, 0)); // api list call + + + // and the pending upload is gone + assertUploadCount(dir, 0); + } + + @Test + public void testRenameWithPendingUpload() throws Throwable { + + final S3AFileSystem fs = getFileSystem(); + final Path base = methodPath(); + final Path dir = new Path(base, "src"); + final Path dest = new Path(base, "dest"); + + // create a magic file. + createMagicFile(fs, dir); + + // and there's a pending upload + assertUploadCount(dir, 1); + + // rename the dir, with a cost of 1 abort, 1 list. + verifyMetrics(() -> fs.rename(dir, dest), + with(OBJECT_MULTIPART_UPLOAD_ABORTED, 1), // abort + with(OBJECT_MULTIPART_UPLOAD_LIST, 1), // HTTP request inside iterator + with(MULTIPART_UPLOAD_LIST, 0)); // api list call + + // and there isn't + assertUploadCount(dir, 0); + } + + /** + * Create a magic file of "real" length more than 0 bytes long. + * @param fs filesystem + * @param dir directory + * @return the path + * @throws IOException creation failure.p + */ + private static Path createMagicFile(final S3AFileSystem fs, final Path dir) throws IOException { + Path magicFile = new Path(dir, MAGIC_PATH_PREFIX + "001/file.txt"); + createFile(fs, magicFile, true, "123".getBytes(StandardCharsets.UTF_8)); + + // the file exists but is a 0 byte marker file. + assertFileHasLength(fs, magicFile, 0); + return magicFile; + } + + /** + * Assert the upload count under a dir is the expected value. + * Failure message will include the list of entries. + * @param dir dir + * @param expected expected count + * @throws IOException listing problem + */ + private void assertUploadCount(final Path dir, final int expected) throws IOException { + Assertions.assertThat(toList(listUploads(dir))) + .describedAs("uploads under %s", dir) + .hasSize(expected); + } + + /** + * List uploads; use the same APIs that the directory operations use, + * so implicitly validating them. + * @param dir directory to list + * @return full list of entries + * @throws IOException listing problem + */ + private RemoteIterator listUploads(Path dir) throws IOException { + final S3AFileSystem fs = getFileSystem(); + try (AuditSpan ignored = span()) { + final StoreContext sc = fs.createStoreContext(); + return fs.listUploadsUnderPrefix(sc, sc.pathToKey(dir)); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java index 3a390e34ecad2..f6fe68b8e6d33 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java @@ -37,7 +37,6 @@ import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_XATTR_GET_MAP; import static org.apache.hadoop.fs.s3a.Statistic.INVOCATION_XATTR_GET_NAMED; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_OCTET_STREAM; -import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_APPLICATION_XML; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.CONTENT_TYPE_X_DIRECTORY; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_CONTENT_LENGTH; import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_CONTENT_TYPE; @@ -74,15 +73,12 @@ public void testXAttrRoot() throws Throwable { fs.listXAttrs(root), with(INVOCATION_OP_XATTR_LIST, GET_METADATA_ON_OBJECT)); - // verify this contains all the standard markers, - // but not the magic marker header + // don't make any assertions on the headers entries + // as different S3 providers may have different headers + // and they may even change over time. Assertions.assertThat(headerList) .describedAs("Headers on root object") - .containsOnly( - XA_CONTENT_LENGTH, - XA_CONTENT_TYPE); - assertHeaderEntry(xAttrs, XA_CONTENT_TYPE) - .isEqualTo(CONTENT_TYPE_APPLICATION_XML); + .hasSize(xAttrs.size()); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestErrorTranslation.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestErrorTranslation.java new file mode 100644 index 0000000000000..0f0b2c0c34bb5 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestErrorTranslation.java @@ -0,0 +1,156 @@ +/* + * 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.impl; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.ConnectException; +import java.net.NoRouteToHostException; +import java.net.SocketTimeoutException; +import java.net.UnknownHostException; +import java.util.Collections; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import software.amazon.awssdk.awscore.retry.conditions.RetryOnErrorCodeCondition; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.core.retry.RetryPolicyContext; + +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.auth.NoAwsCredentialsException; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.maybeExtractIOException; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.junit.Assert.assertTrue; + +/** + * Unit tests related to the {@link ErrorTranslation} class. + */ +public class TestErrorTranslation extends AbstractHadoopTestBase { + + /** + * Create an sdk exception with the given cause. + * @param message error message + * @param cause cause + * @return a new exception + */ + private static SdkClientException sdkException( + String message, + Throwable cause) { + return SdkClientException.builder() + .message(message) + .cause(cause) + .build(); + } + + @Test + public void testUnknownHostExceptionExtraction() throws Throwable { + final SdkClientException thrown = sdkException("top", + sdkException("middle", + new UnknownHostException("bottom"))); + final IOException ioe = intercept(UnknownHostException.class, "top", + () -> { + throw maybeExtractIOException("", thrown); + }); + + // the wrapped exception is the top level one: no stack traces have + // been lost + if (ioe.getCause() != thrown) { + throw new AssertionError("Cause of " + ioe + " is not " + thrown, thrown); + } + + } + + @Test + public void testNoRouteToHostExceptionExtraction() throws Throwable { + intercept(NoRouteToHostException.class, "top", + () -> { + throw maybeExtractIOException("p2", + sdkException("top", + sdkException("middle", + new NoRouteToHostException("bottom")))); + }); + } + + @Test + public void testConnectExceptionExtraction() throws Throwable { + intercept(ConnectException.class, "top", + () -> { + throw maybeExtractIOException("p1", + sdkException("top", + sdkException("middle", + new ConnectException("bottom")))); + }); + } + + /** + * When there is an UncheckedIOException, its inner class is + * extracted. + */ + @Test + public void testUncheckedIOExceptionExtraction() throws Throwable { + intercept(SocketTimeoutException.class, "top", + () -> { + final SdkClientException thrown = sdkException("top", + sdkException("middle", + new UncheckedIOException( + new SocketTimeoutException("bottom")))); + throw maybeExtractIOException("p1", + new NoAwsCredentialsException("IamProvider", thrown.toString(), thrown)); + }); + } + + @Test + public void testNoConstructorExtraction() throws Throwable { + intercept(PathIOException.class, NoConstructorIOE.MESSAGE, + () -> { + throw maybeExtractIOException("p1", + sdkException("top", + sdkException("middle", + new NoConstructorIOE()))); + }); + } + + + public static final class NoConstructorIOE extends IOException { + + public static final String MESSAGE = "no-arg constructor"; + + public NoConstructorIOE() { + super(MESSAGE); + } + } + + + @Test + public void testMultiObjectExceptionFilledIn() throws Throwable { + + MultiObjectDeleteException ase = + new MultiObjectDeleteException(Collections.emptyList()); + RetryPolicyContext context = RetryPolicyContext.builder() + .exception(ase) + .build(); + RetryOnErrorCodeCondition retry = RetryOnErrorCodeCondition.create(""); + + Assertions.assertThat(retry.shouldRetry(context)) + .describedAs("retry policy of MultiObjectException") + .isFalse(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java index cd71a2949ac6c..f6be1f75cfa68 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java @@ -26,7 +26,9 @@ import java.util.List; import java.util.Map; -import com.amazonaws.services.s3.model.ObjectMetadata; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadBucketResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.assertj.core.api.Assertions; import org.assertj.core.util.Lists; import org.junit.Before; @@ -206,20 +208,20 @@ public void testMetadataCopySkipsMagicAttribute() throws Throwable { final String owner = "x-header-owner"; final String root = "root"; CONTEXT_ACCESSORS.userHeaders.put(owner, root); - final ObjectMetadata source = CONTEXT_ACCESSORS + final HeadObjectResponse source = CONTEXT_ACCESSORS .getObjectMetadata(MAGIC_KEY); - final Map sourceUserMD = source.getUserMetadata(); + final Map sourceUserMD = source.metadata(); Assertions.assertThat(sourceUserMD.get(owner)) .describedAs("owner header in copied MD") .isEqualTo(root); - ObjectMetadata dest = new ObjectMetadata(); - headerProcessing.cloneObjectMetadata(source, dest); + Map destUserMetadata = new HashMap<>(); + headerProcessing.cloneObjectMetadata(source, destUserMetadata, CopyObjectRequest.builder()); - Assertions.assertThat(dest.getUserMetadata().get(X_HEADER_MAGIC_MARKER)) + Assertions.assertThat(destUserMetadata.get(X_HEADER_MAGIC_MARKER)) .describedAs("Magic marker header in copied MD") .isNull(); - Assertions.assertThat(dest.getUserMetadata().get(owner)) + Assertions.assertThat(destUserMetadata.get(owner)) .describedAs("owner header in copied MD") .isEqualTo(root); } @@ -307,20 +309,24 @@ public RequestFactory getRequestFactory() { } @Override - public ObjectMetadata getObjectMetadata(final String key) + public HeadObjectResponse getObjectMetadata(final String key) throws IOException { if (MAGIC_KEY.equals(key)) { - ObjectMetadata omd = new ObjectMetadata(); - omd.setUserMetadata(userHeaders); - omd.setContentLength(len); - omd.setLastModified(date); - return omd; + return HeadObjectResponse.builder() + .metadata(userHeaders) + .contentLength(len) + .lastModified(date.toInstant()).build(); } else { throw new FileNotFoundException(key); } } + @Override + public HeadBucketResponse getBucketMetadata() throws IOException { + return HeadBucketResponse.builder().build(); + } + public void setHeader(String key, String val) { userHeaders.put(key, val); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java index 7f51d2b45362c..919a89b8c1dd0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java @@ -18,14 +18,10 @@ package org.apache.hadoop.fs.s3a.impl; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.client.builder.AwsClientBuilder; -import org.junit.Ignore; import org.junit.Test; import org.apache.hadoop.test.AbstractHadoopTestBase; -import static org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createEndpointConfiguration; import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion; import static org.assertj.core.api.Assertions.assertThat; @@ -63,43 +59,4 @@ private static void assertRegionFixup(String region, String expected) { .describedAs("Fixup of %s", region) .isEqualTo(expected); } - - @Test - public void testNull() throws Throwable { - expectEndpoint("", true, "unused"); - } - - @Test - @Ignore("disabled until endpoint logic works for S3 client builder API") - public void testUSEastEndpoint() throws Throwable { - expectEndpoint(US_EAST_1, false, US_EAST_1); - } - - @Test - @Ignore("disabled until endpoint logic works for S3 client builder API") - public void testUSWestEndpoint() throws Throwable { - expectEndpoint(US_WEST_2, false, US_WEST_2); - } - - public void expectEndpoint(final String src, - final boolean expectNull, - final String expectRegion) { - AwsClientBuilder.EndpointConfiguration epr = - createEndpointConfiguration(src, new ClientConfiguration(), src); - String eprStr = epr == null - ? "(empty)" - : ("(" + epr.getServiceEndpoint() + " " + epr.getSigningRegion()); - if (expectNull) { - assertThat(epr) - .describedAs("Endpoint configuration of %s =", - src, eprStr) - .isNull(); - } else { - assertThat(epr) - .describedAs("Endpoint configuration of %s =", - src, eprStr) - .hasFieldOrPropertyWithValue("serviceEndpoint", src) - .hasFieldOrPropertyWithValue("signingRegion", expectRegion); - } - } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java index 7c85142d4376d..1fb576a55514c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java @@ -18,21 +18,20 @@ package org.apache.hadoop.fs.s3a.impl; -import java.io.ByteArrayInputStream; -import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.concurrent.atomic.AtomicLong; -import com.amazonaws.AmazonWebServiceRequest; -import com.amazonaws.services.s3.model.CannedAccessControlList; -import com.amazonaws.services.s3.model.ObjectListing; -import com.amazonaws.services.s3.model.ObjectMetadata; +import software.amazon.awssdk.awscore.AwsRequest; +import software.amazon.awssdk.core.SdkRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.audit.AWSRequestAnalyzer; @@ -57,7 +56,7 @@ public class TestRequestFactory extends AbstractHadoopTestBase { private final AWSRequestAnalyzer analyzer = new AWSRequestAnalyzer(); /** - * Count of requests analyzed via the {@link #a(AmazonWebServiceRequest)} + * Count of requests analyzed via the {@link #a(AwsRequest.Builder)} * call. */ private int requestsAnalyzed; @@ -81,27 +80,31 @@ public void testRequestFactoryWithEncryption() throws Throwable { */ @Test public void testRequestFactoryWithCannedACL() throws Throwable { - CannedAccessControlList acl = CannedAccessControlList.BucketOwnerFullControl; + String acl = "bucket-owner-full-control"; RequestFactory factory = RequestFactoryImpl.builder() .withBucket("bucket") .withCannedACL(acl) .build(); String path = "path"; String path2 = "path2"; - ObjectMetadata md = factory.newObjectMetadata(128); - Assertions.assertThat( - factory.newPutObjectRequest(path, md, - null, new ByteArrayInputStream(new byte[0])) - .getCannedAcl()) + HeadObjectResponse md = HeadObjectResponse.builder().contentLength(128L).build(); + + Assertions.assertThat(factory.newPutObjectRequestBuilder(path, null, 128, false) + .build() + .acl() + .toString()) .describedAs("ACL of PUT") .isEqualTo(acl); - Assertions.assertThat(factory.newCopyObjectRequest(path, path2, md) - .getCannedAccessControlList()) + Assertions.assertThat(factory.newCopyObjectRequestBuilder(path, path2, md) + .build() + .acl() + .toString()) .describedAs("ACL of COPY") .isEqualTo(acl); - Assertions.assertThat(factory.newMultipartUploadRequest(path, - null) - .getCannedACL()) + Assertions.assertThat(factory.newMultipartUploadRequestBuilder(path, null) + .build() + .acl() + .toString()) .describedAs("ACL of MPU") .isEqualTo(acl); } @@ -132,21 +135,18 @@ private final class CountRequests private final AtomicLong counter = new AtomicLong(); @Override - public T prepareRequest(final T t) { + public void prepareRequest(final SdkRequest.Builder t) { counter.addAndGet(1); - return t; } } /** * Analyze the request, log the output, return the info. - * @param request request. - * @param type of request. + * @param builder request builder. * @return value */ - private AWSRequestAnalyzer.RequestInfo - a(T request) { - AWSRequestAnalyzer.RequestInfo info = analyzer.analyze(request); + private AWSRequestAnalyzer.RequestInfo a(AwsRequest.Builder builder) { + AWSRequestAnalyzer.RequestInfo info = analyzer.analyze(builder.build()); LOG.info("{}", info); requestsAnalyzed++; return info; @@ -160,27 +160,25 @@ private void createFactoryObjects(RequestFactory factory) throws IOException { String path = "path"; String path2 = "path2"; String id = "1"; - ObjectMetadata md = factory.newObjectMetadata(128); - a(factory.newAbortMultipartUploadRequest(path, id)); - a(factory.newCompleteMultipartUploadRequest(path, id, + a(factory.newAbortMultipartUploadRequestBuilder(path, id)); + a(factory.newCompleteMultipartUploadRequestBuilder(path, id, new ArrayList<>())); - a(factory.newCopyObjectRequest(path, path2, md)); - a(factory.newDeleteObjectRequest(path)); - a(factory.newBulkDeleteRequest(new ArrayList<>())); + a(factory.newCopyObjectRequestBuilder(path, path2, + HeadObjectResponse.builder().build())); + a(factory.newDeleteObjectRequestBuilder(path)); + a(factory.newBulkDeleteRequestBuilder(new ArrayList<>())); a(factory.newDirectoryMarkerRequest(path)); - a(factory.newGetObjectRequest(path)); - a(factory.newGetObjectMetadataRequest(path)); - a(factory.newListMultipartUploadsRequest(path)); - a(factory.newListObjectsV1Request(path, "/", 1)); - a(factory.newListNextBatchOfObjectsRequest(new ObjectListing())); - a(factory.newListObjectsV2Request(path, "/", 1)); - a(factory.newMultipartUploadRequest(path, null)); - File srcfile = new File("/tmp/a"); - a(factory.newPutObjectRequest(path, - factory.newObjectMetadata(-1), null, srcfile)); - ByteArrayInputStream stream = new ByteArrayInputStream(new byte[0]); - a(factory.newPutObjectRequest(path, md, null, stream)); - a(factory.newSelectRequest(path)); + a(factory.newGetObjectRequestBuilder(path)); + a(factory.newHeadObjectRequestBuilder(path)); + a(factory.newListMultipartUploadsRequestBuilder(path)); + a(factory.newListObjectsV1RequestBuilder(path, "/", 1)); + a(factory.newListObjectsV2RequestBuilder(path, "/", 1)); + a(factory.newMultipartUploadRequestBuilder(path, null)); + a(factory.newPutObjectRequestBuilder(path, + PutObjectOptions.keepingDirs(), -1, true)); + a(factory.newPutObjectRequestBuilder(path, + PutObjectOptions.deletingDirs(), 1024, false)); + a(factory.newSelectRequestBuilder(path)); } /** @@ -193,25 +191,18 @@ public void testMultipartUploadRequest() throws Throwable { RequestFactory factory = RequestFactoryImpl.builder() .withBucket("bucket") .withRequestPreparer(countRequests) + .withMultipartPartCountLimit(2) .build(); String path = "path"; - String path2 = "path2"; String id = "1"; - File srcfile = File.createTempFile("file", ""); - try { - ByteArrayInputStream stream = new ByteArrayInputStream(new byte[0]); - - a(factory.newUploadPartRequest(path, id, 1, 0, stream, null, 0)); - a(factory.newUploadPartRequest(path, id, 2, 128_000_000, - null, srcfile, 0)); - // offset is past the EOF - intercept(IllegalArgumentException.class, () -> - factory.newUploadPartRequest(path, id, 3, 128_000_000, - null, srcfile, 128)); - } finally { - srcfile.delete(); - } + + a(factory.newUploadPartRequestBuilder(path, id, 1, 0)); + a(factory.newUploadPartRequestBuilder(path, id, 2, 128_000_000)); + // partNumber is past the limit + intercept(PathIOException.class, () -> + factory.newUploadPartRequestBuilder(path, id, 3, 128_000_000)); + assertThat(countRequests.counter.get()) .describedAs("request preparation count") .isEqualTo(requestsAnalyzed); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestSDKStreamDrainer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestSDKStreamDrainer.java index 33a44a9ad78f7..7042737b31085 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestSDKStreamDrainer.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestSDKStreamDrainer.java @@ -19,13 +19,15 @@ package org.apache.hadoop.fs.s3a.impl; import java.io.IOException; +import java.io.InputStream; -import com.amazonaws.internal.SdkFilterInputStream; +import software.amazon.awssdk.http.Abortable; import org.assertj.core.api.Assertions; import org.junit.Test; import org.apache.hadoop.test.HadoopTestBase; + import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DRAIN_BUFFER_SIZE; import static org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext.EMPTY_INPUT_STREAM_STATISTICS; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -127,7 +129,6 @@ public void testStreamUnderflow() throws Throwable { public void testReadFailure() throws Throwable { int threshold = 50; SDKStreamDrainer drainer = new SDKStreamDrainer("s3://example/", - null, new FakeSDKInputStream(BYTES, threshold), false, BYTES, @@ -145,7 +146,6 @@ public void testReadFailure() throws Throwable { public void testReadFailureDoesNotSurfaceInAbort() throws Throwable { int threshold = 50; SDKStreamDrainer drainer = new SDKStreamDrainer("s3://example/", - null, new FakeSDKInputStream(BYTES, threshold), true, BYTES, @@ -183,7 +183,6 @@ private SDKStreamDrainer drainer(int remaining, boolean shouldAbort, FakeSDKInputStream in) throws Throwable { SDKStreamDrainer drainer = new SDKStreamDrainer("s3://example/", - null, in, shouldAbort, remaining, @@ -246,7 +245,8 @@ private static SDKStreamDrainer assertBytesRead(final SDKStreamDrainer drainer, * Fake stream; generates data dynamically. * Only overrides the methods used in stream draining. */ - private static final class FakeSDKInputStream extends SdkFilterInputStream { + private static final class FakeSDKInputStream extends InputStream + implements Abortable { private final int capacity; @@ -264,7 +264,6 @@ private static final class FakeSDKInputStream extends SdkFilterInputStream { * @param readToRaiseIOE position to raise an IOE, or -1 */ private FakeSDKInputStream(final int capacity, final int readToRaiseIOE) { - super(null); this.capacity = capacity; this.readToRaiseIOE = readToRaiseIOE; } @@ -282,11 +281,6 @@ public void abort() { aborted = true; } - @Override - protected boolean isAborted() { - return aborted; - } - @Override public int read() throws IOException { if (bytesRead >= capacity) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index 8bbf52b578e1a..e37717bfa1e34 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -32,7 +32,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; -import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.Statistic; @@ -92,6 +91,13 @@ protected AbstractS3ACostTest( this.keepMarkers = keepMarkers; } + /** + * Constructor with markers kept. + */ + public AbstractS3ACostTest() { + this(true); + } + @Override public Configuration createConfiguration() { Configuration conf = super.createConfiguration(); @@ -135,9 +141,7 @@ public void setup() throws Exception { setupCostValidator(); // determine bulk delete settings - final Configuration fsConf = getFileSystem().getConf(); - isBulkDelete = fsConf.getBoolean(Constants.ENABLE_MULTI_DELETE, - true); + isBulkDelete = isBulkDeleteEnabled(getFileSystem()); deleteMarkerStatistic = isBulkDelete() ? OBJECT_BULK_DELETE_REQUEST : OBJECT_DELETE_REQUEST; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java index ccb0c0e79ec7e..cd4ee44406676 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java @@ -27,9 +27,10 @@ import java.util.concurrent.Callable; import java.util.stream.Collectors; -import com.amazonaws.AmazonClientException; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.ObjectMetadata; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -48,6 +49,7 @@ import org.apache.hadoop.fs.s3a.S3AUtils; import org.apache.hadoop.fs.store.audit.AuditSpan; + import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE; @@ -156,7 +158,7 @@ public static Collection params() { /** * S3 Client of the FS. */ - private AmazonS3 s3client; + private S3Client s3client; /** * Path to a file under the marker. @@ -212,7 +214,7 @@ protected Configuration createConfiguration() { public void setup() throws Exception { super.setup(); S3AFileSystem fs = getFileSystem(); - s3client = fs.getAmazonS3ClientForTesting("markers"); + s3client = getS3AInternals().getAmazonS3Client("markers"); bucket = fs.getBucket(); Path base = new Path(methodPath(), "base"); @@ -453,6 +455,7 @@ public void testRenameBase() throws Throwable { Path src = basePath; Path dest = new Path(methodPath(), "dest"); + getFileSystem().delete(dest, true); assertRenamed(src, dest); assertPathDoesNotExist("source", src); @@ -604,18 +607,22 @@ private void assertTestObjectsExist() throws Exception { */ private void put(final String key, final String content) throws Exception { exec("PUT " + key, () -> - s3client.putObject(bucket, key, content)); + s3client.putObject(b -> b.bucket(bucket).key(key), + RequestBody.fromString(content))); } /** - * Delete an object. + * Delete an object; exceptions are swallowed. * @param key key - * @param content string */ private void deleteObject(final String key) throws Exception { - exec("DELETE " + key, () -> { - s3client.deleteObject(bucket, key); - return "deleted " + key; - }); + try { + exec("DELETE " + key, () -> { + s3client.deleteObject(b -> b.bucket(bucket).key(key)); + return "deleted " + key; + }); + } catch (IOException ignored) { + + } } /** @@ -624,10 +631,10 @@ private void deleteObject(final String key) throws Exception { * @return a description of the object. */ private String head(final String key) throws Exception { - ObjectMetadata md = exec("HEAD " + key, () -> - s3client.getObjectMetadata(bucket, key)); + HeadObjectResponse response = exec("HEAD " + key, () -> + s3client.headObject(b -> b.bucket(bucket).key(key))); return String.format("Object %s of length %d", - key, md.getInstanceLength()); + key, response.contentLength()); } /** @@ -655,7 +662,7 @@ private T exec(String op, Callable call) throws Exception { ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); try (AuditSpan span = getSpanSource().createSpan(op, null, null)) { return call.call(); - } catch (AmazonClientException ex) { + } catch (SdkException ex) { throw S3AUtils.translateException(op, "", ex); } finally { timer.end(op); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java index be4de7942f7b9..fae2df973af7f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -99,16 +99,17 @@ public void testDeleteSingleFileInDir() throws Throwable { // still be there Path simpleFile = file(new Path(dir, "simple.txt")); - boolean rawAndKeeping = !isDeleting(); - boolean rawAndDeleting = isDeleting(); + boolean keeping = !isDeleting(); + boolean deleting = isDeleting(); + boolean bulkDelete = isBulkDelete(); verifyMetrics(() -> { fs.delete(simpleFile, false); return "after fs.delete(simpleFile) " + getMetricSummary(); }, - probe(rawAndKeeping, OBJECT_METADATA_REQUESTS, + probe(keeping, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H), // if deleting markers, look for the parent too - probe(rawAndDeleting, OBJECT_METADATA_REQUESTS, + probe(deleting, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H), with(OBJECT_LIST_REQUEST, FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), @@ -116,7 +117,9 @@ public void testDeleteSingleFileInDir() throws Throwable { with(FILES_DELETED, 1), // a single DELETE call is made to delete the object - with(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST), + probe(bulkDelete, OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST), + probe(!bulkDelete, OBJECT_DELETE_REQUEST, + DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST), // keeping: create no parent dirs or delete parents withWhenKeeping(DIRECTORIES_CREATED, 0), @@ -127,7 +130,8 @@ public void testDeleteSingleFileInDir() throws Throwable { // a bulk delete for all parents is issued. // the number of objects in it depends on the depth of the tree; // don't worry about that - withWhenDeleting(OBJECT_BULK_DELETE_REQUEST, DELETE_MARKER_REQUEST) + probe(deleting && bulkDelete, OBJECT_BULK_DELETE_REQUEST, + DELETE_MARKER_REQUEST) ); // there is an empty dir for a parent diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java index 6e2f547a22ec1..5fbbc3a127997 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java @@ -21,11 +21,12 @@ import java.io.ByteArrayInputStream; import java.io.IOException; -import java.io.InputStream; import java.util.concurrent.CompletableFuture; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.S3Object; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.http.AbortableInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; import org.apache.hadoop.fs.impl.prefetch.Validate; import org.apache.hadoop.fs.s3a.S3AInputStream; @@ -55,7 +56,7 @@ class MockS3ARemoteObject extends S3ARemoteObject { super( S3APrefetchFakes.createReadContext(null, KEY, size, 1, 1), S3APrefetchFakes.createObjectAttributes(BUCKET, KEY, size), - S3APrefetchFakes.createInputStreamCallbacks(BUCKET, KEY), + S3APrefetchFakes.createInputStreamCallbacks(BUCKET), EmptyS3AStatisticsContext.EMPTY_INPUT_STREAM_STATISTICS, S3APrefetchFakes.createChangeTracker(BUCKET, KEY, size) ); @@ -68,7 +69,8 @@ class MockS3ARemoteObject extends S3ARemoteObject { } @Override - public InputStream openForRead(long offset, int size) throws IOException { + public ResponseInputStream openForRead(long offset, int size) + throws IOException { Validate.checkLessOrEqual(offset, "offset", size(), "size()"); Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset"); @@ -77,11 +79,15 @@ public InputStream openForRead(long offset, int size) throws IOException { throw new IOException("Throwing because throwExceptionOnOpen is true "); } int bufSize = (int) Math.min(size, size() - offset); - return new ByteArrayInputStream(contents, (int) offset, bufSize); + GetObjectResponse objectResponse = GetObjectResponse.builder().build(); + return new ResponseInputStream(objectResponse, + AbortableInputStream.create(new ByteArrayInputStream(contents, + (int) offset, bufSize), () -> {})); } @Override - public void close(InputStream inputStream, int numRemainingBytes) { + public void close(ResponseInputStream inputStream, + int numRemainingBytes) { // do nothing since we do not use a real S3 stream. } @@ -92,7 +98,8 @@ public static byte byteAtOffset(int offset) { public static S3AInputStream.InputStreamCallbacks createClient(String bucketName) { return new S3AInputStream.InputStreamCallbacks() { @Override - public S3Object getObject(GetObjectRequest request) { + public ResponseInputStream getObject( + GetObjectRequest request) { return null; } @@ -102,8 +109,8 @@ public CompletableFuture submit(CallableRaisingIOE operation) { } @Override - public GetObjectRequest newGetRequest(String key) { - return new GetObjectRequest(bucketName, key); + public GetObjectRequest.Builder newGetRequestBuilder(String key) { + return GetObjectRequest.builder().bucket(bucketName).key(key); } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java index 9616c574f37a1..89477a9071973 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java @@ -31,10 +31,10 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.S3ObjectInputStream; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.http.AbortableInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -64,6 +64,7 @@ import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.util.functional.CallableRaisingIOE; + import static org.apache.hadoop.fs.s3a.Constants.BUFFER_DIR; import static org.apache.hadoop.fs.s3a.Constants.HADOOP_TMP_DIR; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatisticsStore; @@ -177,32 +178,26 @@ public static ChangeTracker createChangeTracker( createObjectAttributes(bucket, key, fileSize)); } - public static S3ObjectInputStream createS3ObjectInputStream(byte[] buffer) { - return new S3ObjectInputStream(new ByteArrayInputStream(buffer), null); + public static ResponseInputStream createS3ObjectInputStream( + GetObjectResponse objectResponse, byte[] buffer) { + return new ResponseInputStream(objectResponse, + AbortableInputStream.create(new ByteArrayInputStream(buffer), () -> {})); } public static S3AInputStream.InputStreamCallbacks createInputStreamCallbacks( - String bucket, - String key) { + String bucket) { - S3Object object = new S3Object() { - @Override - public S3ObjectInputStream getObjectContent() { - return createS3ObjectInputStream(new byte[8]); - } + GetObjectResponse objectResponse = GetObjectResponse.builder() + .eTag(E_TAG) + .build(); - @Override - public ObjectMetadata getObjectMetadata() { - ObjectMetadata metadata = new ObjectMetadata(); - metadata.setHeader("ETag", E_TAG); - return metadata; - } - }; + ResponseInputStream responseInputStream = + createS3ObjectInputStream(objectResponse, new byte[8]); return new S3AInputStream.InputStreamCallbacks() { @Override - public S3Object getObject(GetObjectRequest request) { - return object; + public ResponseInputStream getObject(GetObjectRequest request) { + return responseInputStream; } @Override @@ -211,8 +206,8 @@ public CompletableFuture submit(CallableRaisingIOE operation) { } @Override - public GetObjectRequest newGetRequest(String key) { - return new GetObjectRequest(bucket, key); + public GetObjectRequest.Builder newGetRequestBuilder(String key) { + return GetObjectRequest.builder().bucket(bucket).key(key); } @Override @@ -231,9 +226,6 @@ public static S3ARemoteInputStream createInputStream( int prefetchBlockSize, int prefetchBlockCount) { - org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key); - - S3AFileStatus fileStatus = createFileStatus(key, fileSize); S3ObjectAttributes s3ObjectAttributes = createObjectAttributes(bucket, key, fileSize); S3AReadOpContext s3AReadOpContext = createReadContext( @@ -244,7 +236,7 @@ public static S3ARemoteInputStream createInputStream( prefetchBlockCount); S3AInputStream.InputStreamCallbacks callbacks = - createInputStreamCallbacks(bucket, key); + createInputStreamCallbacks(bucket); S3AInputStreamStatistics stats = s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardTool.java index 23b14fd3792ca..daa4c2c8483a6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardTool.java @@ -70,6 +70,7 @@ public void testLandsatBucketRequireGuarded() throws Throwable { @Test public void testLandsatBucketRequireUnencrypted() throws Throwable { + skipIfClientSideEncryption(); run(BucketInfo.NAME, "-" + BucketInfo.ENCRYPTION_FLAG, "none", getLandsatCSVFile(getConfiguration())); @@ -94,22 +95,22 @@ public void testStoreInfo() throws Throwable { LOG.info("Exec output=\n{}", output); } - private final static String UPLOAD_PREFIX = "test-upload-prefix"; private final static String UPLOAD_NAME = "test-upload"; @Test public void testUploads() throws Throwable { S3AFileSystem fs = getFileSystem(); - Path path = path(UPLOAD_PREFIX + "/" + UPLOAD_NAME); + Path path = methodPath(); + Path file = new Path(path, UPLOAD_NAME); describe("Cleaning up any leftover uploads from previous runs."); - final String key = fs.pathToKey(path); + final String key = fs.pathToKey(file); try { // 1. Make sure key doesn't already exist clearAnyUploads(fs, path); // 2. Confirm no uploads are listed via API - assertNoUploadsAt(fs, path.getParent()); + assertNoUploadsAt(fs, path); // 3. Confirm no uploads are listed via CLI describe("Confirming CLI lists nothing."); @@ -124,8 +125,6 @@ public void testUploads() throws Throwable { // 6. Confirm part exists via CLI, direct path and parent path describe("Confirming CLI lists one part"); assertNumUploads(path, 1); - assertNumUploads(path.getParent(), 1); - // 7. Use CLI to delete part, assert it worked describe("Deleting part via CLI"); assertNumDeleted(fs, path, 1); @@ -147,22 +146,23 @@ public void testUploads() throws Throwable { @Test public void testUploadListByAge() throws Throwable { S3AFileSystem fs = getFileSystem(); - Path path = path(UPLOAD_PREFIX + "/" + UPLOAD_NAME); + Path path = methodPath(); + Path file = new Path(path, UPLOAD_NAME); describe("Cleaning up any leftover uploads from previous runs."); + // 1. Make sure key doesn't already exist clearAnyUploads(fs, path); // 2. Create a upload part describe("Uploading single part."); - final String key = fs.pathToKey(path); + final String key = fs.pathToKey(file); createPartUpload(fs, key, 128, 1); //try (AuditSpan span = fs.startOperation("multipart", key, null)) { try { - // 3. Confirm it exists via API.. may want to wrap with - // LambdaTestUtils.eventually() ? + // 3. Confirm it exists via API assertEquals("Should be one upload", 1, countUploadsAt(fs, path)); // 4. Confirm part does appear in listing with long age filter @@ -178,8 +178,9 @@ public void testUploadListByAge() throws Throwable { // least a second old describe("Sleeping 1 second then confirming upload still there"); Thread.sleep(1000); - LambdaTestUtils.eventually(5000, 1000, - () -> { assertNumUploadsAge(path, 1, 1); }); + LambdaTestUtils.eventually(5000, 1000, () -> { + assertNumUploadsAge(path, 1, 1); + }); // 7. Assert deletion works when age filter matches describe("Doing aged deletion"); @@ -231,8 +232,8 @@ private void assertNumDeleted(S3AFileSystem fs, Path path, int numDeleted) * search all parts * @throws Exception on failure */ - private void uploadCommandAssertCount(S3AFileSystem fs, String options[], - Path path, int numUploads, int ageSeconds) + private void uploadCommandAssertCount(S3AFileSystem fs, String[] options, Path path, + int numUploads, int ageSeconds) throws Exception { List allOptions = new ArrayList<>(); List output = new ArrayList<>(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index c4949375b7685..628b567041827 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java @@ -26,9 +26,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.IntFunction; -import com.amazonaws.event.ProgressEvent; -import com.amazonaws.event.ProgressEventType; -import com.amazonaws.event.ProgressListener; import org.assertj.core.api.Assertions; import org.junit.FixMethodOrder; import org.junit.Test; @@ -48,6 +45,8 @@ 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.fs.s3a.impl.ProgressListener; +import org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent; import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.util.DurationInfo; @@ -356,10 +355,10 @@ public long getFilesize() { /** * Is this expected to be a multipart upload? * Assertions will change if not. - * @return true by default. + * @return what the filesystem expects. */ protected boolean expectMultipartUpload() { - return true; + return getFileSystem().getS3AInternals().isMultipartCopyEnabled(); } /** @@ -377,10 +376,9 @@ protected int getPartitionSize() { } /** - * Progress callback from AWS. Likely to come in on a different thread. + * Progress callback. */ - private final class ProgressCallback implements Progressable, - ProgressListener { + private final class ProgressCallback implements Progressable, ProgressListener { private AtomicLong bytesTransferred = new AtomicLong(0); private AtomicLong uploadEvents = new AtomicLong(0); private AtomicInteger failures = new AtomicInteger(0); @@ -395,11 +393,8 @@ public void progress() { } @Override - public void progressChanged(ProgressEvent progressEvent) { - ProgressEventType eventType = progressEvent.getEventType(); - if (eventType.isByteCountEvent()) { - bytesTransferred.addAndGet(progressEvent.getBytesTransferred()); - } + public void progressChanged(ProgressListenerEvent eventType, long transferredBytes) { + switch (eventType) { case TRANSFER_PART_FAILED_EVENT: // failure @@ -408,6 +403,7 @@ public void progressChanged(ProgressEvent progressEvent) { break; case TRANSFER_PART_COMPLETED_EVENT: // completion + bytesTransferred.addAndGet(transferredBytes); long elapsedTime = timer.elapsedTime(); double elapsedTimeS = elapsedTime / 1.0e9; long written = bytesTransferred.get(); @@ -415,21 +411,18 @@ public void progressChanged(ProgressEvent progressEvent) { LOG.info(String.format( "Event %s; total uploaded=%d MB in %.1fs;" + " effective upload bandwidth = %.2f MB/s", - progressEvent, + eventType, writtenMB, elapsedTimeS, writtenMB / elapsedTimeS)); break; case REQUEST_BYTE_TRANSFER_EVENT: uploadEvents.incrementAndGet(); break; default: - if (!eventType.isByteCountEvent()) { - LOG.info("Event {}", progressEvent); - } + // nothing break; } } - @Override public String toString() { String sb = "ProgressCallback{" + "bytesTransferred=" + bytesTransferred.get() + diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java index 2738dd9284a14..8d118e5a4fafd 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ILoadTestS3ABulkDeleteThrottling.java @@ -29,7 +29,7 @@ import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.assertj.core.api.Assertions; @@ -52,6 +52,7 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.concurrent.HadoopExecutors; + import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING; import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE; import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT; @@ -228,7 +229,7 @@ private File deleteFiles(final int requestCount, Path basePath = path("testDeleteObjectThrottling"); final S3AFileSystem fs = getFileSystem(); final String base = fs.pathToKey(basePath); - final List fileList + final List fileList = buildDeleteRequest(base, entries); final FileWriter out = new FileWriter(csvFile); Csvout csvout = new Csvout(out, "\t", "\n"); @@ -304,12 +305,12 @@ private File deleteFiles(final int requestCount, } - private List buildDeleteRequest( + private List buildDeleteRequest( String base, int count) { - List request = new ArrayList<>(count); + List request = new ArrayList<>(count); for (int i = 0; i < count; i++) { - request.add(new DeleteObjectsRequest.KeyVersion( - String.format("%s/file-%04d", base, i))); + request.add(ObjectIdentifier.builder().key( + String.format("%s/file-%04d", base, i)).build()); } return request; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java index de903b3d75a57..173099bb2ca71 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.fs.s3a.S3ADataBlocks; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; @@ -41,6 +42,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; @@ -49,9 +51,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.PutObjectResult; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; @@ -249,18 +250,19 @@ public void testMultiPagesListingPerformanceAndCorrectness() = fs.getWriteOperationHelper(); final RequestFactory requestFactory = writeOperationHelper.getRequestFactory(); - List> futures = + List> futures = new ArrayList<>(numOfPutRequests); for (int i=0; i - writeOperationHelper.putObject(put, PutObjectOptions.keepingDirs(), null))); + PutObjectRequest.Builder putObjectRequestBuilder = requestFactory + .newPutObjectRequestBuilder(fs.pathToKey(file), + null, 0, false); + futures.add(submit(executorService, + () -> writeOperationHelper.putObject(putObjectRequestBuilder.build(), + PutObjectOptions.keepingDirs(), + new S3ADataBlocks.BlockUploadData(new FailingInputStream()), false, null))); } LOG.info("Waiting for PUTs to complete"); waitForCompletion(futures); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesEncryption.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesEncryption.java index 0448f9dcc1533..324738b303ab7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesEncryption.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesEncryption.java @@ -27,6 +27,8 @@ import org.apache.hadoop.fs.s3a.EncryptionTestUtils; import org.apache.hadoop.fs.s3a.S3AFileSystem; +import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_ALGORITHM; +import static org.apache.hadoop.fs.s3a.S3AEncryptionMethods.DSSE_KMS; import static org.apache.hadoop.fs.s3a.S3AEncryptionMethods.SSE_KMS; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfEncryptionNotSet; @@ -43,7 +45,7 @@ public class ITestS3AHugeFilesEncryption extends AbstractSTestS3AHugeFiles { @Override public void setup() throws Exception { Configuration c = new Configuration(); - skipIfEncryptionNotSet(c, SSE_KMS); + skipIfEncryptionNotSet(c, SSE_KMS, DSSE_KMS); super.setup(); } @@ -67,7 +69,12 @@ protected boolean isEncrypted(S3AFileSystem fileSystem) { protected void assertEncrypted(Path hugeFile) throws IOException { Configuration c = new Configuration(); String kmsKey = getS3EncryptionKey(getTestBucketName(c), c); - EncryptionTestUtils.assertEncrypted(getFileSystem(), hugeFile, - SSE_KMS, kmsKey); + if (SSE_KMS.getMethod().equals(c.get(S3_ENCRYPTION_ALGORITHM))) { + EncryptionTestUtils.assertEncrypted(getFileSystem(), hugeFile, SSE_KMS, kmsKey); + } else if (DSSE_KMS.getMethod().equals(c.get(S3_ENCRYPTION_ALGORITHM))) { + EncryptionTestUtils.assertEncrypted(getFileSystem(), hugeFile, DSSE_KMS, kmsKey); + } else { + throw new AssertionError("Invalid encryption configured"); + } } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesNoMultipart.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesNoMultipart.java index ed300dba01ea3..e154ab5676f81 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesNoMultipart.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesNoMultipart.java @@ -18,13 +18,10 @@ package org.apache.hadoop.fs.s3a.scale; +import org.assertj.core.api.Assertions; + import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Constants; -import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.S3ATestUtils; -import org.apache.hadoop.fs.s3a.api.UnsupportedRequestException; import static org.apache.hadoop.fs.contract.ContractTestUtils.IO_CHUNK_BUFFER_SIZE; import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD; @@ -33,7 +30,6 @@ import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_UPLOADS_ENABLED; import static org.apache.hadoop.fs.s3a.Constants.REQUEST_TIMEOUT; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Use a single PUT for the whole upload/rename/delete workflow; include verification @@ -41,11 +37,6 @@ */ public class ITestS3AHugeFilesNoMultipart extends AbstractSTestS3AHugeFiles { - /** - * Size to ensure MPUs don't happen in transfer manager. - */ - public static final String S_1T = "1T"; - public static final String SINGLE_PUT_REQUEST_TIMEOUT = "1h"; /** @@ -56,11 +47,23 @@ protected String getBlockOutputBufferName() { return Constants.FAST_UPLOAD_BUFFER_DISK; } + /** + * Multipart upload is always disabled. + * @return false + */ @Override protected boolean expectMultipartUpload() { return false; } + /** + * Is multipart copy enabled? + * @return true if the transfer manager is used to copy files. + */ + private boolean isMultipartCopyEnabled() { + return getFileSystem().getS3AInternals().isMultipartCopyEnabled(); + } + /** * Create a configuration without multipart upload, * and a long request timeout to allow for a very slow @@ -77,35 +80,21 @@ protected Configuration createScaleConfiguration() { MULTIPART_SIZE, REQUEST_TIMEOUT); conf.setInt(IO_CHUNK_BUFFER_SIZE, 655360); - conf.set(MIN_MULTIPART_THRESHOLD, S_1T); - conf.set(MULTIPART_SIZE, S_1T); + conf.setInt(MIN_MULTIPART_THRESHOLD, MULTIPART_MIN_SIZE); + conf.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE); conf.setBoolean(MULTIPART_UPLOADS_ENABLED, false); conf.set(REQUEST_TIMEOUT, SINGLE_PUT_REQUEST_TIMEOUT); return conf; } /** - * After the file is created, attempt a rename with an FS - * instance with a small multipart threshold; - * this MUST be rejected. + * Verify multipart copy is disabled. */ @Override public void test_030_postCreationAssertions() throws Throwable { - assumeHugeFileExists(); - final Path hugefile = getHugefile(); - final Path hugefileRenamed = getHugefileRenamed(); - describe("renaming %s to %s", hugefile, hugefileRenamed); - S3AFileSystem fs = getFileSystem(); - fs.delete(hugefileRenamed, false); - // create a new fs with a small multipart threshold; expect rename failure. - final Configuration conf = new Configuration(fs.getConf()); - conf.setInt(MIN_MULTIPART_THRESHOLD, MULTIPART_MIN_SIZE); - conf.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE); - S3ATestUtils.disableFilesystemCaching(conf); - - try (FileSystem fs2 = FileSystem.get(fs.getUri(), conf)) { - intercept(UnsupportedRequestException.class, () -> - fs2.rename(hugefile, hugefileRenamed)); - } + super.test_030_postCreationAssertions(); + Assertions.assertThat(isMultipartCopyEnabled()) + .describedAs("Multipart copy should be disabled in %s", getFileSystem()) + .isFalse(); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesSSECDiskBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesSSECDiskBlocks.java index 3dc64ea3b45fd..6f19ba15c1c9a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesSSECDiskBlocks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesSSECDiskBlocks.java @@ -19,9 +19,9 @@ package org.apache.hadoop.fs.s3a.scale; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.AWSUnsupportedFeatureException; import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; -import org.apache.hadoop.fs.s3a.S3ATestUtils; import java.nio.file.AccessDeniedException; @@ -54,8 +54,7 @@ public class ITestS3AHugeFilesSSECDiskBlocks public void setup() throws Exception { try { super.setup(); - skipIfEncryptionTestsDisabled(getConfiguration()); - } catch (AccessDeniedException e) { + } catch (AccessDeniedException | AWSUnsupportedFeatureException e) { skip("Bucket does not allow " + S3AEncryptionMethods.SSE_C + " encryption method"); } } @@ -67,7 +66,7 @@ protected Configuration createScaleConfiguration() { removeBaseAndBucketOverrides(conf, S3_ENCRYPTION_KEY, S3_ENCRYPTION_ALGORITHM, SERVER_SIDE_ENCRYPTION_ALGORITHM, SERVER_SIDE_ENCRYPTION_KEY); - S3ATestUtils.disableFilesystemCaching(conf); + skipIfEncryptionTestsDisabled(conf); conf.set(Constants.S3_ENCRYPTION_ALGORITHM, getSSEAlgorithm().getMethod()); conf.set(Constants.S3_ENCRYPTION_KEY, KEY_1); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java index 99407467df56d..58988c9c41bf8 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java @@ -46,15 +46,10 @@ public class ITestS3AHugeFilesStorageClass extends AbstractSTestS3AHugeFiles { private static final Logger LOG = LoggerFactory.getLogger(ITestS3AHugeFilesStorageClass.class); - @Override - public void setup() throws Exception { - super.setup(); - skipIfStorageClassTestsDisabled(getConfiguration()); - } - @Override protected Configuration createScaleConfiguration() { Configuration conf = super.createScaleConfiguration(); + skipIfStorageClassTestsDisabled(conf); disableFilesystemCaching(conf); removeBaseAndBucketOverrides(conf, STORAGE_CLASS); @@ -125,8 +120,8 @@ private void skipQuietly(String text) { } protected void assertStorageClass(Path hugeFile) throws IOException { - S3AFileSystem fs = getFileSystem(); - String actual = fs.getObjectMetadata(hugeFile).getStorageClass(); + + String actual = getS3AInternals().getObjectMetadata(hugeFile).storageClassAsString(); assertTrue( "Storage class of object is " + actual + ", expected " + STORAGE_CLASS_REDUCED_REDUNDANCY, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/AbstractS3SelectTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/AbstractS3SelectTest.java index 2c1a10a21d0ab..a3d41116182e5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/AbstractS3SelectTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/AbstractS3SelectTest.java @@ -263,9 +263,9 @@ private static CsvFile writeStandardHeader(final CsvFile csv, protected static AWSServiceIOException verifyErrorCode(final String code, final AWSServiceIOException ex) { logIntercepted(ex); - if (!code.equals(ex.getErrorCode())) { + if (!code.equals(ex.awsErrorDetails().errorCode())) { throw new AssertionError("Expected Error code" + code - + " actual " + ex.getErrorCode(), + + " actual " + ex.awsErrorDetails().errorCode(), ex); } return ex; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java index 51da971fb7063..4d4af822ee50b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java @@ -415,7 +415,7 @@ public void testSelectSeekFullLandsat() throws Throwable { long increment = 64 * _1KB; // seek forward, comparing bytes - for(offset = 32 * _1KB; offset < _1MB; offset += increment) { + for(offset = 32 * _1KB; offset < 256 * _1KB; offset += increment) { seek(seekStream, offset); assertEquals("Seek position in " + seekStream, offset, seekStream.getPos()); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/StreamPublisher.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/StreamPublisher.java new file mode 100644 index 0000000000000..461aef726876c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/StreamPublisher.java @@ -0,0 +1,89 @@ +/* + * 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.select; + +import java.util.Iterator; +import java.util.concurrent.Executor; +import java.util.stream.Stream; + +import org.reactivestreams.Subscriber; +import org.reactivestreams.Subscription; +import software.amazon.awssdk.core.async.SdkPublisher; + +/** + * Publisher used to test the handling of asynchronous responses. + * @param The type of published elements. + */ +final class StreamPublisher implements SdkPublisher { + private final Executor executor; + private final Iterator iterator; + private Boolean done = false; + + StreamPublisher(Stream data, Executor executor) { + this.iterator = data.iterator(); + this.executor = executor; + } + + StreamPublisher(Stream data) { + this(data, Runnable::run); + } + + @Override + public void subscribe(Subscriber subscriber) { + subscriber.onSubscribe(new Subscription() { + @Override + public void request(long n) { + if (done) { + return; + } + + if (n < 1) { + done = true; + executor.execute(() -> subscriber.onError(new IllegalArgumentException())); + return; + } + + for (long i = 0; i < n; i++) { + final T value; + try { + synchronized (iterator) { + value = iterator.hasNext() ? iterator.next() : null; + } + } catch (Throwable e) { + executor.execute(() -> subscriber.onError(e)); + break; + } + + if (value == null) { + done = true; + executor.execute(subscriber::onComplete); + break; + } else { + executor.execute(() -> subscriber.onNext(value)); + } + } + } + + @Override + public void cancel() { + done = true; + } + }); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestBlockingEnumeration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestBlockingEnumeration.java new file mode 100644 index 0000000000000..43bdcb062f0a3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestBlockingEnumeration.java @@ -0,0 +1,200 @@ +/* + * 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.select; + +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import org.junit.Assert; +import org.junit.Test; +import software.amazon.awssdk.core.async.SdkPublisher; +import software.amazon.awssdk.core.exception.SdkException; + +/** + * Unit tests for {@link BlockingEnumeration}. + */ +public final class TestBlockingEnumeration extends Assert { + + @Test + public void containsElement() { + SdkPublisher publisher = new StreamPublisher<>(Stream.of("foo")); + + BlockingEnumeration enumeration = + new BlockingEnumeration<>(publisher, 1); + + assertTrue(enumeration.hasMoreElements()); + assertEquals("foo", enumeration.nextElement()); + assertFalse(enumeration.hasMoreElements()); + } + + @Test + public void containsInjectedElement() { + SdkPublisher publisher = new StreamPublisher<>(Stream.of("foo")); + + BlockingEnumeration enumeration = + new BlockingEnumeration<>(publisher, 1, "bar"); + + assertTrue(enumeration.hasMoreElements()); + assertEquals("bar", enumeration.nextElement()); + assertTrue(enumeration.hasMoreElements()); + assertEquals("foo", enumeration.nextElement()); + assertFalse(enumeration.hasMoreElements()); + } + + @Test + public void throwsExceptionOnFirstElement() { + SdkPublisher publisher = new StreamPublisher<>( + Stream.of(0, 1) + .map(i -> { + throw SdkException.create("error!", null); + }), + Executors.newSingleThreadExecutor()); + + BlockingEnumeration enumeration = + new BlockingEnumeration<>(publisher, 1); + assertThrows(SdkException.class, enumeration::hasMoreElements); + } + + @Test + public void throwsExceptionAfterInjectedElement() { + SdkPublisher publisher = new StreamPublisher<>( + Stream.of(0, 1) + .peek(i -> { + throw SdkException.create("error!", null); + }), + Executors.newSingleThreadExecutor()); + + BlockingEnumeration enumeration = + new BlockingEnumeration<>(publisher, 1, 99); + assertTrue(enumeration.hasMoreElements()); + assertEquals(99, enumeration.nextElement().intValue()); + assertThrows(SdkException.class, enumeration::hasMoreElements); + } + + @Test + public void throwsNonSdkException() { + SdkPublisher publisher = new StreamPublisher<>( + Stream.of(0, 1) + .peek(i -> { + throw new RuntimeException("error!", null); + }), + Executors.newSingleThreadExecutor()); + + BlockingEnumeration enumeration = + new BlockingEnumeration<>(publisher, 1); + SdkException exception = Assert.assertThrows(SdkException.class, enumeration::hasMoreElements); + assertEquals(RuntimeException.class, exception.getCause().getClass()); + } + + @Test + public void throwsError() { + SdkPublisher publisher = new StreamPublisher<>( + Stream.of(0, 1) + .peek(i -> { + throw new Error("error!", null); + }), + Executors.newSingleThreadExecutor()); + + BlockingEnumeration enumeration = + new BlockingEnumeration<>(publisher, 1); + assertThrows(Error.class, enumeration::hasMoreElements); + } + + @Test + public void throwsExceptionOnSecondElement() { + SdkPublisher publisher = new StreamPublisher<>( + Stream.of(0, 1) + .peek(i -> { + if (i == 1) { + throw SdkException.create("error!", null); + } + }), + Executors.newSingleThreadExecutor()); + + BlockingEnumeration enumeration = + new BlockingEnumeration<>(publisher, 1); + assertTrue(enumeration.hasMoreElements()); + assertEquals(0, enumeration.nextElement().intValue()); + assertThrows(SdkException.class, enumeration::hasMoreElements); + } + + @Test + public void noMoreElementsAfterThrow() { + SdkPublisher publisher = new StreamPublisher<>( + Stream.of(0, 1) + .map(i -> { + throw SdkException.create("error!", null); + }), + Executors.newSingleThreadExecutor()); + + BlockingEnumeration enumeration = + new BlockingEnumeration<>(publisher, 1); + assertThrows(SdkException.class, enumeration::hasMoreElements); + assertFalse(enumeration.hasMoreElements()); + } + + @Test + public void buffersOnSameThread() { + verifyBuffering(10, 3, Runnable::run); + } + + @Test + public void publisherOnDifferentThread() { + verifyBuffering(5, 1, Executors.newSingleThreadExecutor()); + } + + @Test + public void publisherOnDifferentThreadWithBuffer() { + verifyBuffering(30, 10, Executors.newSingleThreadExecutor()); + } + + private static void verifyBuffering(int length, int bufferSize, Executor executor) { + AtomicInteger emitted = new AtomicInteger(); + SdkPublisher publisher = new StreamPublisher<>( + IntStream.range(0, length).boxed().peek(i -> emitted.incrementAndGet()), + executor); + + BlockingEnumeration enumeration = + new BlockingEnumeration<>(publisher, bufferSize); + + int pulled = 0; + while (true) { + try { + int expected = Math.min(length, pulled + bufferSize); + if (expected != emitted.get()) { + Thread.sleep(10); + } + assertEquals(expected, emitted.get()); + } catch (InterruptedException e) { + fail("Interrupted: " + e); + } + + if (!enumeration.hasMoreElements()) { + break; + } + + int i = enumeration.nextElement(); + assertEquals(pulled, i); + pulled++; + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestSelectEventStreamPublisher.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestSelectEventStreamPublisher.java new file mode 100644 index 0000000000000..fdf3b5b725376 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestSelectEventStreamPublisher.java @@ -0,0 +1,190 @@ +/* + * 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.select; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Stream; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.core.async.SdkPublisher; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.http.AbortableInputStream; +import software.amazon.awssdk.services.s3.model.SelectObjectContentEventStream; +import software.amazon.awssdk.services.s3.model.SelectObjectContentResponse; + +/** + * Unit tests for {@link SelectEventStreamPublisher}. + */ +@RunWith(Parameterized.class) +public final class TestSelectEventStreamPublisher extends Assert { + + @Parameterized.Parameters(name = "threading-{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {"main"}, + {"background"} + }); + } + + private final String threading; + + public TestSelectEventStreamPublisher(String threading) { + this.threading = threading; + } + + private Executor createExecutor() { + if (threading.equals("main")) { + return Runnable::run; + } else if (threading.equals("background")) { + return Executors.newSingleThreadExecutor(); + } else { + throw new IllegalArgumentException("Unknown: " + threading); + } + } + + @Test + public void emptyRecordsInputStream() throws IOException { + SelectEventStreamPublisher selectEventStreamPublisher = + createSelectPublisher(Stream.of( + SelectObjectContentEventStream.recordsBuilder() + .payload(SdkBytes.fromByteArray(new byte[0])) + .build())); + + try (AbortableInputStream inputStream = + selectEventStreamPublisher.toRecordsInputStream(e -> {})) { + assertEquals(-1, inputStream.read()); + } + } + + @Test + public void multipleRecords() throws IOException { + SelectEventStreamPublisher selectEventStreamPublisher = + createSelectPublisher(Stream.of( + SelectObjectContentEventStream.recordsBuilder() + .payload(SdkBytes.fromUtf8String("foo")) + .build(), + SelectObjectContentEventStream.recordsBuilder() + .payload(SdkBytes.fromUtf8String("bar")) + .build())); + + try (AbortableInputStream inputStream = + selectEventStreamPublisher.toRecordsInputStream(e -> {})) { + String result = readAll(inputStream); + assertEquals("foobar", result); + } + } + + @Test + public void skipsOtherEvents() throws IOException { + SelectEventStreamPublisher selectEventStreamPublisher = + createSelectPublisher(Stream.of( + SelectObjectContentEventStream.recordsBuilder() + .payload(SdkBytes.fromUtf8String("foo")) + .build(), + SelectObjectContentEventStream.progressBuilder() + .build(), + SelectObjectContentEventStream.statsBuilder() + .build(), + SelectObjectContentEventStream.recordsBuilder() + .payload(SdkBytes.fromUtf8String("bar")) + .build(), + SelectObjectContentEventStream.endBuilder() + .build())); + + try (AbortableInputStream inputStream = + selectEventStreamPublisher.toRecordsInputStream(e -> {})) { + String result = readAll(inputStream); + assertEquals("foobar", result); + } + } + + @Test + public void callsOnEndEvent() throws IOException { + SelectEventStreamPublisher selectEventStreamPublisher = + createSelectPublisher(Stream.of( + SelectObjectContentEventStream.recordsBuilder() + .payload(SdkBytes.fromUtf8String("foo")) + .build(), + SelectObjectContentEventStream.endBuilder() + .build())); + + AtomicBoolean endEvent = new AtomicBoolean(false); + try (AbortableInputStream inputStream = + selectEventStreamPublisher.toRecordsInputStream(e -> endEvent.set(true))) { + String result = readAll(inputStream); + assertEquals("foo", result); + } + + assertTrue(endEvent.get()); + } + + @Test + public void handlesErrors() throws IOException { + SelectEventStreamPublisher selectEventStreamPublisher = + createSelectPublisher(Stream.of( + SelectObjectContentEventStream.recordsBuilder() + .payload(SdkBytes.fromUtf8String("foo")) + .build(), + SelectObjectContentEventStream.recordsBuilder() + .payload(SdkBytes.fromUtf8String("bar")) + .build()) + .map(e -> { + throw SdkException.create("error!", null); + })); + + try (AbortableInputStream inputStream = + selectEventStreamPublisher.toRecordsInputStream(e -> {})) { + assertThrows(SdkException.class, () -> readAll(inputStream)); + } + } + + private SelectEventStreamPublisher createSelectPublisher( + Stream stream) { + SdkPublisher sdkPublisher = + new StreamPublisher<>(stream, createExecutor()); + CompletableFuture future = + CompletableFuture.completedFuture(null); + SelectObjectContentResponse response = + SelectObjectContentResponse.builder().build(); + return new SelectEventStreamPublisher(future, response, sdkPublisher); + } + + private static String readAll(InputStream inputStream) throws IOException { + try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { + byte[] buffer = new byte[8096]; + int read; + while ((read = inputStream.read(buffer, 0, buffer.length)) != -1) { + outputStream.write(buffer, 0, read); + } + return outputStream.toString(); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java index e7696996dbd1a..8c97d896edbde 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java @@ -53,7 +53,7 @@ public void testLandsatStatistics() throws Throwable { conf.unset("fs.s3a.bucket.landsat-pds.endpoint"); try (S3AFileSystem fs = (S3AFileSystem) path.getFileSystem(conf)) { - fs.getObjectMetadata(path); + fs.getS3AInternals().getObjectMetadata(path); IOStatistics iostats = fs.getIOStatistics(); assertThatStatisticCounter(iostats, STORE_IO_REQUEST.getSymbol()) @@ -71,7 +71,7 @@ public void testCommonCrawlStatistics() throws Throwable { conf.set(ENDPOINT, DEFAULT_ENDPOINT); try (S3AFileSystem fs = (S3AFileSystem) path.getFileSystem(conf)) { - fs.getObjectMetadata(path); + fs.getS3AInternals().getObjectMetadata(path); IOStatistics iostats = fs.getIOStatistics(); assertThatStatisticCounter(iostats, STORE_IO_REQUEST.getSymbol()) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java index 77c7736575c39..fdf5eb53e187e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java @@ -147,7 +147,7 @@ public static T extractCause(Class expected, */ protected void assertStatusCode(AWSServiceIOException e, int code) throws AWSServiceIOException { - if (e.getStatusCode() != code) { + if (e.statusCode() != code) { throw e; } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalOperationCallbacks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalOperationCallbacks.java index fa1ad2db62af7..004e15676a04a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalOperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalOperationCallbacks.java @@ -21,10 +21,9 @@ import java.io.IOException; import java.util.List; -import com.amazonaws.AmazonClientException; -import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.MultiObjectDeleteException; -import com.amazonaws.services.s3.transfer.model.CopyResult; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.s3.model.CopyObjectResponse; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -33,6 +32,7 @@ import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; import org.apache.hadoop.fs.s3a.S3AReadOpContext; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; +import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteException; import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; /** @@ -88,7 +88,7 @@ public RemoteIterator listFilesAndDirectoryMarkers( } @Override - public CopyResult copyFile( + public CopyObjectResponse copyFile( String srcKey, String destKey, S3ObjectAttributes srcAttributes, @@ -99,9 +99,9 @@ public CopyResult copyFile( @Override public void removeKeys( - List keysToDelete, + List keysToDelete, boolean deleteFakeDir) - throws MultiObjectDeleteException, AmazonClientException, + throws MultiObjectDeleteException, AwsServiceException, IOException { } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalWriteOperationHelperCallbacks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalWriteOperationHelperCallbacks.java index ffba558d11fd0..eee0c71950566 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalWriteOperationHelperCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalWriteOperationHelperCallbacks.java @@ -18,10 +18,12 @@ package org.apache.hadoop.fs.s3a.test; -import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; -import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; -import com.amazonaws.services.s3.model.SelectObjectContentRequest; -import com.amazonaws.services.s3.model.SelectObjectContentResult; +import java.util.concurrent.CompletableFuture; + +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; +import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse; +import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; +import software.amazon.awssdk.services.s3.model.SelectObjectContentResponseHandler; import org.apache.hadoop.fs.s3a.WriteOperationHelper; @@ -32,12 +34,14 @@ public class MinimalWriteOperationHelperCallbacks implements WriteOperationHelper.WriteOperationHelperCallbacks { @Override - public SelectObjectContentResult selectObjectContent(SelectObjectContentRequest request) { + public CompletableFuture selectObjectContent( + SelectObjectContentRequest request, + SelectObjectContentResponseHandler th) { return null; } @Override - public CompleteMultipartUploadResult completeMultipartUpload( + public CompleteMultipartUploadResponse completeMultipartUpload( CompleteMultipartUploadRequest request) { return null; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java index 127fcbab75023..ab22c51f28b7b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -224,13 +224,6 @@ public void testRunNoArgs() throws Throwable { runToFailure(EXIT_USAGE, MARKERS); } - @Test - public void testRunWrongBucket() throws Throwable { - runToFailure(EXIT_NOT_FOUND, MARKERS, - AUDIT, - "s3a://this-bucket-does-not-exist-hopefully"); - } - /** * Run with a path that doesn't exist. */ diff --git a/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml b/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml index 1533477229642..e6c3eeed3d7e9 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml +++ b/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml @@ -31,11 +31,10 @@
    - - fs.s3a.bucket.landsat-pds.endpoint - ${central.endpoint} - The endpoint for s3a://landsat-pds URLs + fs.s3a.bucket.landsat-pds.endpoint.region + us-west-2 + The region for s3a://landsat-pds @@ -58,10 +57,9 @@ - - fs.s3a.bucket.usgs-landsat.endpoint - ${central.endpoint} + fs.s3a.bucket.usgs-landsat.endpoint.region + us-west-2 @@ -82,6 +80,12 @@ false + + + fs.s3a.bucket.osm-pds.endpoint.region + us-east-1 + The region for s3a://osm-pds +