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,
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 fa7de69140fcf..a56f1db0b4556 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
@@ -334,6 +334,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
/** Exactly once log to warn about setting the region in config to avoid probe. */
private static final LogExactlyOnce SET_REGION_WARNING = new LogExactlyOnce(LOG);
+
+ /** 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;
@@ -1073,7 +1077,8 @@ private Region getS3Region(String region) throws IOException {
if (exception.statusCode() == SC_404_NOT_FOUND) {
throw new UnknownStoreException("s3a://" + bucket + "/",
- " Bucket does " + "not exist");
+ " Bucket does not exist: " + exception,
+ exception);
}
throw exception;
@@ -1174,6 +1179,9 @@ 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, "")
@@ -1181,10 +1189,11 @@ protected RequestFactory createRequestFactory() {
StorageClass storageClass = null;
if (!storageClassConf.isEmpty()) {
storageClass = StorageClass.fromValue(storageClassConf);
-
+ LOG.debug("Using storage class {}", storageClass);
if (storageClass.equals(StorageClass.UNKNOWN_TO_SDK_VERSION)) {
- LOG.warn("Unknown storage class property {}: {}; falling back to default storage class",
- STORAGE_CLASS, storageClassConf);
+ STORAGE_CLASS_WARNING.warn("Unknown storage class \"{}\" from option: {};"
+ + " falling back to default storage class",
+ storageClassConf, STORAGE_CLASS);
storageClass = null;
}
@@ -1431,7 +1440,7 @@ public String getBucketLocation() throws IOException {
public String getBucketLocation(String bucketName) throws IOException {
final String region = trackDurationAndSpan(
STORE_EXISTS_PROBE, bucketName, null, () ->
- invoker.retry("getBucketLocation()", bucketName, true, () ->
+ once("getBucketLocation()", bucketName, () ->
// If accessPoint then region is known from Arn
accessPoint != null
? accessPoint.getRegion()
@@ -2993,7 +3002,7 @@ 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(),
@@ -3002,6 +3011,12 @@ protected void deleteObject(String 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;
+ }
}
}
@@ -4287,13 +4302,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);
}
}
@@ -4443,7 +4458,6 @@ private CopyObjectResponse 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,
@@ -4454,6 +4468,8 @@ private CopyObjectResponse copyFile(String srcKey, String dstKey, long size,
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
@@ -4468,7 +4484,7 @@ private CopyObjectResponse copyFile(String srcKey, String dstKey, long size,
Copy copy = transferManager.copy(
CopyRequest.builder()
- .copyObjectRequest(copyObjectRequestBuilder.build())
+ .copyObjectRequest(copyRequest)
.build());
try {
@@ -4477,6 +4493,8 @@ private CopyObjectResponse copyFile(String srcKey, String dstKey, long size,
} 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;
@@ -4493,7 +4511,15 @@ private CopyObjectResponse copyFile(String srcKey, String dstKey, long size,
() -> {
LOG.debug("copyFile: single part copy {} -> {} of size {}", srcKey, dstKey, size);
incrementStatistic(OBJECT_COPY_REQUESTS);
- return s3Client.copyObject(copyObjectRequestBuilder.build());
+ 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;
+ }
});
}
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 fdb4591476c56..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;
@@ -31,7 +34,6 @@
import java.util.concurrent.TimeUnit;
import software.amazon.awssdk.core.exception.SdkException;
-import org.apache.hadoop.util.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -45,13 +47,14 @@
import org.apache.hadoop.net.ConnectTimeoutException;
+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.
@@ -66,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, SdkException)}.
* Because the {@link #shouldRetry(Exception, int, int, boolean)} method
* 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(
@@ -122,8 +130,7 @@ public class S3ARetryPolicy implements RetryPolicy {
* @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);
@@ -188,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.
@@ -224,27 +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);
- // use specific retry policy for aws client exceptions
- policyMap.put(AWSClientIOException.class, retryAwsClientExceptions);
- policyMap.put(AWSServiceIOException.class, retryAwsClientExceptions);
+ // network routing.
+ policyMap.put(NoRouteToHostException.class, fail);
- // other operations
- policyMap.put(AWSS3IOException.class, retryIdempotentCalls);
- policyMap.put(SocketTimeoutException.class, retryIdempotentCalls);
+ // 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;
}
@@ -253,14 +293,19 @@ public RetryAction shouldRetry(Exception exception,
int retries,
int failovers,
boolean idempotent) throws Exception {
- Preconditions.checkArgument(exception != null, "Null exception");
- Exception ex = exception;
+ Exception ex = requireNonNull(exception, "Null exception");
if (exception instanceof SdkException) {
// update the sdk exception for the purpose of exception
// processing.
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;
}
/**
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 093608fb528c7..6798a99c19e1e 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
@@ -80,6 +80,7 @@
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.maybeExtractNetworkException;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator;
@@ -191,10 +192,14 @@ public static IOException translateException(@Nullable String operation,
ioe = maybeTranslateCredentialException(path, exception);
if (ioe != null) {
return ioe;
- } else {
- // no custom handling.
- return new AWSClientIOException(message, exception);
}
+ // network problems covered by an IOE inside the exception chain.
+ ioe = maybeExtractNetworkException(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
@@ -209,6 +214,8 @@ public static IOException translateException(@Nullable String operation,
if (ase.awsErrorDetails() != null) {
message = message + ":" + ase.awsErrorDetails().errorCode();
}
+
+ // big switch on the HTTP status code.
switch (status) {
case SC_301_MOVED_PERMANENTLY:
@@ -242,7 +249,8 @@ public static IOException translateException(@Nullable String operation,
// 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);
}
@@ -255,10 +263,13 @@ public static IOException translateException(@Nullable String operation,
ioe.initCause(ase);
break;
- // method not allowed; seen on S3 Select.
- // treated as a bad request
+ // errors which stores can return from requests which
+ // the store does not support.
case SC_405_METHOD_NOT_ALLOWED:
- ioe = new AWSBadRequestException(message, s3Exception);
+ 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
@@ -277,7 +288,8 @@ public static IOException translateException(@Nullable String operation,
break;
// throttling
- case SC_503_SERVICE_UNAVAILABLE:
+ case SC_429_TOO_MANY_REQUESTS_GCS: // google cloud through this connector
+ case SC_503_SERVICE_UNAVAILABLE: // AWS
ioe = new AWSServiceThrottledException(message, ase);
break;
@@ -295,8 +307,15 @@ public static IOException translateException(@Nullable String operation,
// 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);
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 c77311211abcb..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
@@ -29,6 +29,8 @@
import org.slf4j.Logger;
+import static java.util.Objects.requireNonNull;
+
/**
* API version-independent container for S3 List responses.
*/
@@ -47,7 +49,7 @@ protected S3ListResult(ListObjectsResponse v1, ListObjectsV2Response v2) {
* @return new list result container
*/
public static S3ListResult v1(ListObjectsResponse result) {
- return new S3ListResult(result, null);
+ return new S3ListResult(requireNonNull(result), null);
}
/**
@@ -56,7 +58,7 @@ public static S3ListResult v1(ListObjectsResponse result) {
* @return new list result container
*/
public static S3ListResult v2(ListObjectsV2Response result) {
- return new S3ListResult(null, result);
+ return new S3ListResult(null, requireNonNull(result));
}
/**
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 f7eaf825b9c94..c8fa97a9b9f9e 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
@@ -90,7 +90,7 @@ public AwsCredentials resolveCredentials() {
public String toString() {
return "SimpleAWSCredentialsProvider{" +
"accessKey.empty=" + accessKey.isEmpty() +
- ", secretKey.empty'" + secretKey.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 b5f7fcbcd016d..f54113af6c5c1 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
@@ -582,7 +582,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/auth/SignerFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerFactory.java
index c786086947fac..5d34688cebe14 100644
--- 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
@@ -31,7 +31,10 @@
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.
@@ -44,6 +47,9 @@ public final class SignerFactory {
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<>();
@@ -69,12 +75,8 @@ public static void registerSigner(
final String signerType,
final Class extends Signer> signerClass) {
- if (signerType == null) {
- throw new IllegalArgumentException("signerType cannot be null");
- }
- if (signerClass == null) {
- throw new IllegalArgumentException("signerClass cannot be null");
- }
+ checkArgument(signerType != null, "signerType cannot be null");
+ checkArgument(signerClass != null, "signerClass cannot be null");
SIGNERS.put(signerType, signerClass);
}
@@ -82,14 +84,21 @@ public static void registerSigner(
/**
* Check if the signer has already been registered.
* @param signerType signer to get
+ * @throws IllegalArgumentException if the signer type is unknown.
*/
public static void verifySignerRegistered(String signerType) {
- Class extends Signer> signerClass = SIGNERS.get(signerType);
- if (signerClass == null) {
- throw new IllegalArgumentException("unknown signer type: " + signerType);
- }
+ checkArgument(isSignerRegistered(signerType),
+ "unknown signer type: %s", signerType);
}
+ /**
+ * 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.
@@ -97,13 +106,22 @@ public static void verifySignerRegistered(String signerType) {
* @param signerType The signer type.
* @param configKey Config key used to configure the signer.
* @return The new signer instance.
- * @throws IOException on any problem.
+ * @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 is {}", className);
+ LOG.debug("Signer class from {} and key {} is {}", signerType, configKey, className);
Signer signer =
S3AUtils.getInstanceFromReflection(className, null, null, Signer.class, "create",
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 198bc66133a95..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
@@ -109,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.verifySignerRegistered(signerName);
- } catch (IllegalArgumentException e) {
+
+ if (!SignerFactory.isSignerRegistered(signerName)) {
// Signer is not registered with the AWS SDK.
// Load the class and register the signer.
- Class extends Signer> clazz = null;
+ Class extends Signer> clazz;
try {
clazz = (Class extends Signer>) conf.getClassByName(signerClassName);
} catch (ClassNotFoundException cnfe) {
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
index 4ff2ec0b0b9f4..3400daff509b2 100644
--- 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
@@ -117,14 +117,14 @@ public static ApacheHttpClient.Builder createHttpClientBuilder(Configuration con
S3AUtils.intOption(conf, ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT, 0);
int socketTimeout = S3AUtils.intOption(conf, SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT, 0);
- httpClientBuilder.connectionTimeout(Duration.ofSeconds(connectionEstablishTimeout));
- httpClientBuilder.socketTimeout(Duration.ofSeconds(socketTimeout));
+ 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.ofSeconds(ttl));
+ httpClientBuilder.connectionTimeToLive(Duration.ofMillis(ttl));
NetworkBinding.bindSSLChannelMode(conf, httpClientBuilder);
@@ -148,15 +148,15 @@ public static NettyNioAsyncHttpClient.Builder createAsyncHttpClientBuilder(Confi
S3AUtils.intOption(conf, ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT, 0);
int socketTimeout = S3AUtils.intOption(conf, SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT, 0);
- httpClientBuilder.connectionTimeout(Duration.ofSeconds(connectionEstablishTimeout));
- httpClientBuilder.readTimeout(Duration.ofSeconds(socketTimeout));
- httpClientBuilder.writeTimeout(Duration.ofSeconds(socketTimeout));
+ 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.ofSeconds(ttl));
+ httpClientBuilder.connectionTimeToLive(Duration.ofMillis(ttl));
// TODO: Don't think you can set a socket factory for the netty client.
// NetworkBinding.bindSSLChannelMode(conf, awsConf);
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 54a91323bc2e2..7b5190becc487 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,8 +18,13 @@
package org.apache.hadoop.fs.s3a.impl;
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+
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;
/**
@@ -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.
@@ -66,6 +71,69 @@ 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 maybeExtractNetworkException(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.
+ return wrapWithInnerIOE(path, thrown, (IOException) cause);
+
+ }
+
+ /**
+ * 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 extends Throwable> clazz = inner.getClass();
+ try {
+ Constructor extends Throwable> ctor = clazz.getConstructor(String.class);
+ Throwable t = ctor.newInstance(msg);
+ return (T) (t.initCause(outer));
+ } catch (Throwable e) {
+ return new PathIOException(path, msg, outer);
+ }
+ }
+
/**
* AWS error codes explicitly recognized and processes specially;
* kept in their own class for isolation.
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
index 435db879fabf8..e09571fc338d6 100644
--- 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
@@ -73,12 +73,14 @@ public enum Kind {
public InstantiationIOException(
Kind kind,
- @Nullable URI uri, String classname,
+ @Nullable URI uri,
+ @Nullable String classname,
@Nullable String key,
String message,
- Throwable cause) {
+ @Nullable Throwable cause) {
super(uri!= null ? uri.toString() : "",
- "Class " + classname + " " + message
+ (classname != null ? ("Class " + classname + " ") : "")
+ + message
+ (key != null ? (" (configuration key " + key + ")") : ""),
cause);
this.kind = kind;
@@ -137,8 +139,8 @@ public static InstantiationIOException isNotInstanceOf(
*/
public static InstantiationIOException unavailable(
@Nullable URI uri,
- String classname,
- String key,
+ @Nullable String classname,
+ @Nullable String key,
String text) {
return new InstantiationIOException(Kind.Unavailable,
uri, classname, key, text, null);
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 7af82f70aebf6..e7fcbe0351042 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
@@ -148,9 +148,15 @@ private InternalConstants() {
/** 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;
@@ -160,7 +166,10 @@ private InternalConstants() {
/** 500 status code: Internal Server Error. */
public static final int SC_500_INTERNAL_SERVER_ERROR = 500;
- /** 503 status code: Service Unavailable. */
+ /** 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}. */
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 cacbee381bebc..ca36b658d70f3 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
@@ -63,6 +63,7 @@
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT;
import static org.apache.hadoop.util.Preconditions.checkArgument;
import static org.apache.hadoop.util.Preconditions.checkNotNull;
+import static software.amazon.awssdk.services.s3.model.StorageClass.UNKNOWN_TO_SDK_VERSION;
/**
* The standard implementation of the request factory.
@@ -242,7 +243,7 @@ public CopyObjectRequest.Builder newCopyObjectRequestBuilder(String srcKey,
.metadataDirective(MetadataDirective.REPLACE)
.acl(cannedACL);
- if (srcom.storageClass() != null) {
+ if (srcom.storageClass() != null && srcom.storageClass() != UNKNOWN_TO_SDK_VERSION) {
copyObjectRequestBuilder.storageClass(srcom.storageClass());
}
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 ec68168bd0ffd..330b92186dde1 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;
@@ -400,9 +399,10 @@ 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);
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 ef8413ccf0a64..f54ab18650150 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
@@ -35,6 +35,7 @@
import software.amazon.awssdk.awscore.exception.AwsServiceException;
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.s3a.AWSBadRequestException;
import org.apache.hadoop.util.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -679,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();
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 ea53b2e1fa9e3..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
@@ -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/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 382ae36c1bf5b..0730f86cd1a2f 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,11 +235,6 @@ 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
@@ -369,13 +360,13 @@ There are a number of AWS Credential Providers inside the `hadoop-aws` JAR:
| `org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` | EC2/k8s instance credentials |
-There are also many in the Amazon SDKs, with the common ones being.
+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 |
+|----------------------------------------------------------------------------------|------------------------------|
+| `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 |
@@ -716,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.
@@ -736,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
@@ -788,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.
+
@@ -804,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
@@ -830,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
@@ -839,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.
+
@@ -854,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).
@@ -874,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.
+
@@ -900,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.
+
@@ -930,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.
@@ -958,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
- 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.
+ 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.object.content.encoding
-
+ fs.s3a.change.detection.source
+ etag
- 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.
+ 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.create.storage.class
-
+ fs.s3a.change.detection.mode
+ server
- 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 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.prefetch.enabled
- false
+ fs.s3a.change.detection.version.required
+ true
+
+ 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.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;
@@ -1089,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
@@ -1141,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
@@ -1657,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
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 bfec94b19c101..6100bc0ae5c95 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
@@ -379,6 +379,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
@@ -521,7 +534,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.
@@ -547,6 +560,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.
@@ -587,7 +626,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
@@ -620,19 +659,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
@@ -699,6 +739,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;
@@ -945,7 +999,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.
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..a7ea7b2e59024
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md
@@ -0,0 +1,415 @@
+
+
+# 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`)
+
+## 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.
+
+# 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 41351cdf1bdf0..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
@@ -71,28 +71,38 @@ An exception reporting this class as missing means that this JAR is not on
the classpath.
-### `NoClassDefFoundError: software/amazon/awssdk/crt/s3/S3MetaRequest`
-
-The library `aws-crt.jar` is not on the classpath. Its classes
-are not in the AWS `bundle.jar` file, yet may be needed by some uses made
-of the SDK.
-
-Fix: add.
-
-```
-java.lang.BootstrapMethodError: java.lang.NoClassDefFoundError: software/amazon/awssdk/crt/s3/S3MetaRequest
-at software.amazon.awssdk.services.s3.internal.crt.S3MetaRequestPauseObservable.(S3MetaRequestPauseObservable.java:33)
-at software.amazon.awssdk.transfer.s3.internal.DefaultS3TransferManager.uploadFile(DefaultS3TransferManager.java:205)
-at org.apache.hadoop.fs.s3a.S3AFileSystem.putObject(S3AFileSystem.java:3064)
-at org.apache.hadoop.fs.s3a.S3AFileSystem.executePut(S3AFileSystem.java:4054)
-
-```
-### `ClassNotFoundException: software.amazon.awssdk.services.s3.S3Client`
-
-(or other `software.amazon` class.)
-
-This means that the AWS V2 SDK `bundle.jar` JAR is not on the classpath:
-add it.
+### `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`
@@ -152,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,
@@ -229,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.
@@ -291,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;
```
@@ -314,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
@@ -357,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
+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"
@@ -383,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
```
@@ -445,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.
@@ -500,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".
@@ -539,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=),
@@ -561,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.
@@ -589,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
```
@@ -598,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
@@ -614,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.
```
@@ -624,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}
```
@@ -672,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`
@@ -690,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
@@ -725,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
@@ -778,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
@@ -813,12 +646,12 @@ than `fs.s3a.threads.max`.
```xml
fs.s3a.threads.max
- 20
+ 64
fs.s3a.connection.maximum
- 30
+ 64
```
@@ -836,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
@@ -922,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.
@@ -944,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.
@@ -1031,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
@@ -1065,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
@@ -1133,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:
@@ -1186,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=:
@@ -1214,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.
@@ -1266,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:
@@ -1283,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:
@@ -1307,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.
@@ -1360,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
@@ -1528,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.
@@ -1547,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
@@ -1663,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"
@@ -1738,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
@@ -1839,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 "/".
@@ -1883,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:
@@ -1980,8 +1480,9 @@ The number of retries and interval between each retry can be configured: