, ClientT> Build
* Create an override configuration for an S3 client.
* @param parameters parameter object
* @param conf configuration object
+ * @throws IOException any IOE raised, or translated exception
* @return the override configuration
*/
protected ClientOverrideConfiguration createClientOverrideConfiguration(
- S3ClientCreationParameters parameters, Configuration conf) {
+ S3ClientCreationParameters parameters, Configuration conf) throws IOException {
final ClientOverrideConfiguration.Builder clientOverrideConfigBuilder =
- AWSClientConfig.createClientConfigBuilder(conf);
+ AWSClientConfig.createClientConfigBuilder(conf, AWS_SERVICE_IDENTIFIER_S3);
// add any headers
parameters.getHeaders().forEach((h, v) -> clientOverrideConfigBuilder.putHeader(h, v));
@@ -311,231 +192,6 @@ protected ClientOverrideConfiguration createClientOverrideConfiguration(
return clientOverrideConfigBuilder.build();
}
- /**
- * Create an {@link AmazonS3} client of type
- * {@link AmazonS3EncryptionV2} if CSE is enabled.
- *
- * @param awsConf AWS configuration.
- * @param bucket bucket name.
- * @param parameters parameters.
- * @return new AmazonS3 client.
- * @throws IOException if lookupPassword() has any problem.
- */
- protected AmazonS3 buildAmazonS3EncryptionClient(
- final ClientConfiguration awsConf,
- final String bucket,
- final S3ClientCreationParameters parameters) throws IOException {
-
- AmazonS3 client;
- AmazonS3EncryptionClientV2Builder builder =
- new AmazonS3EncryptionClientV2Builder();
- Configuration conf = getConf();
-
- // CSE-KMS Method
- String kmsKeyId = getS3EncryptionKey(bucket, conf, true);
- // Check if kmsKeyID is not null
- Preconditions.checkArgument(!StringUtils.isBlank(kmsKeyId), "CSE-KMS "
- + "method requires KMS key ID. Use " + S3_ENCRYPTION_KEY
- + " property to set it. ");
-
- EncryptionMaterialsProvider materialsProvider =
- new KMSEncryptionMaterialsProvider(kmsKeyId);
- builder.withEncryptionMaterialsProvider(materialsProvider);
- //Configure basic params of a S3 builder.
- configureBasicParams(builder, awsConf, parameters);
-
- // Configuring endpoint.
- AmazonS3EncryptionClientV2Builder.EndpointConfiguration epr
- = createEndpointConfiguration(parameters.getEndpoint(),
- awsConf, getConf().getTrimmed(AWS_REGION));
- configureEndpoint(builder, epr);
-
- // Create cryptoConfig.
- CryptoConfigurationV2 cryptoConfigurationV2 =
- new CryptoConfigurationV2(CryptoMode.AuthenticatedEncryption)
- .withRangeGetMode(CryptoRangeGetMode.ALL);
- if (epr != null) {
- cryptoConfigurationV2
- .withAwsKmsRegion(RegionUtils.getRegion(epr.getSigningRegion()));
- LOG.debug("KMS region used: {}", cryptoConfigurationV2.getAwsKmsRegion());
- }
- builder.withCryptoConfiguration(cryptoConfigurationV2);
- client = builder.build();
- IGNORE_CSE_WARN.info("S3 client-side encryption enabled: Ignore S3-CSE "
- + "Warnings.");
-
- return client;
- }
-
- /**
- * Use the Builder API to create an AWS S3 client.
- *
- * This has a more complex endpoint configuration mechanism
- * which initially caused problems; the
- * {@code withForceGlobalBucketAccessEnabled(true)}
- * command is critical here.
- * @param awsConf AWS configuration
- * @param parameters parameters
- * @return new AmazonS3 client
- * @throws SdkClientException if the configuration is invalid.
- */
- protected AmazonS3 buildAmazonS3Client(
- final ClientConfiguration awsConf,
- final S3ClientCreationParameters parameters) {
- AmazonS3ClientBuilder b = AmazonS3Client.builder();
- configureBasicParams(b, awsConf, parameters);
-
- // endpoint set up is a PITA
- AwsClientBuilder.EndpointConfiguration epr
- = createEndpointConfiguration(parameters.getEndpoint(),
- awsConf, getConf().getTrimmed(AWS_REGION));
- configureEndpoint(b, epr);
- final AmazonS3 client = b.build();
- return client;
- }
-
- /**
- * A method to configure basic AmazonS3Builder parameters.
- *
- * @param builder Instance of AmazonS3Builder used.
- * @param awsConf ClientConfiguration used.
- * @param parameters Parameters used to set in the builder.
- */
- private void configureBasicParams(AmazonS3Builder builder,
- ClientConfiguration awsConf, S3ClientCreationParameters parameters) {
- // TODO: This whole block will be removed when we remove the V1 client.
- // builder.withCredentials(parameters.getCredentialSet());
- builder.withClientConfiguration(awsConf);
- builder.withPathStyleAccessEnabled(parameters.isPathStyleAccess());
-
- if (parameters.getMonitoringListener() != null) {
- builder.withMonitoringListener(parameters.getMonitoringListener());
- }
-
- }
-
- /**
- * A method to configure endpoint and Region for an AmazonS3Builder.
- *
- * @param builder Instance of AmazonS3Builder used.
- * @param epr EndpointConfiguration used to set in builder.
- */
- private void configureEndpoint(
- AmazonS3Builder builder,
- AmazonS3Builder.EndpointConfiguration epr) {
- if (epr != null) {
- // an endpoint binding was constructed: use it.
- builder.withEndpointConfiguration(epr);
- } else {
- // no idea what the endpoint is, so tell the SDK
- // to work it out at the cost of an extra HEAD request
- builder.withForceGlobalBucketAccessEnabled(true);
- // HADOOP-17771 force set the region so the build process doesn't halt.
- String region = getConf().getTrimmed(AWS_REGION, AWS_S3_CENTRAL_REGION);
- LOG.debug("fs.s3a.endpoint.region=\"{}\"", region);
- if (!region.isEmpty()) {
- // there's either an explicit region or we have fallen back
- // to the central one.
- LOG.debug("Using default endpoint; setting region to {}", region);
- builder.setRegion(region);
- } else {
- // no region.
- // allow this if people really want it; it is OK to rely on this
- // when deployed in EC2.
- WARN_OF_DEFAULT_REGION_CHAIN.warn(SDK_REGION_CHAIN_IN_USE);
- LOG.debug(SDK_REGION_CHAIN_IN_USE);
- }
- }
- }
-
- /**
- * Configure classic S3 client.
- *
- * This includes: endpoint, Path Access and possibly other
- * options.
- *
- * @param s3 S3 Client.
- * @param endPoint s3 endpoint, may be empty
- * @param pathStyleAccess enable path style access?
- * @return S3 client
- * @throws IllegalArgumentException if misconfigured
- */
- protected static AmazonS3 configureAmazonS3Client(AmazonS3 s3,
- final String endPoint,
- final boolean pathStyleAccess)
- throws IllegalArgumentException {
- if (!endPoint.isEmpty()) {
- try {
- s3.setEndpoint(endPoint);
- } catch (IllegalArgumentException e) {
- String msg = "Incorrect endpoint: " + e.getMessage();
- LOG.error(msg);
- throw new IllegalArgumentException(msg, e);
- }
- }
- if (pathStyleAccess) {
- LOG.debug("Enabling path style access!");
- s3.setS3ClientOptions(S3ClientOptions.builder()
- .setPathStyleAccess(true)
- .build());
- }
- return s3;
- }
-
- /**
- * Given an endpoint string, return an endpoint config, or null, if none
- * is needed.
- *
- * This is a pretty painful piece of code. It is trying to replicate
- * what AwsClient.setEndpoint() does, because you can't
- * call that setter on an AwsClient constructed via
- * the builder, and you can't pass a metrics collector
- * down except through the builder.
- *
- * Note also that AWS signing is a mystery which nobody fully
- * understands, especially given all problems surface in a
- * "400 bad request" response, which, like all security systems,
- * provides minimal diagnostics out of fear of leaking
- * secrets.
- *
- * @param endpoint possibly null endpoint.
- * @param awsConf config to build the URI from.
- * @param awsRegion AWS S3 Region if the corresponding config is set.
- * @return a configuration for the S3 client builder.
- */
- @VisibleForTesting
- public static AwsClientBuilder.EndpointConfiguration
- createEndpointConfiguration(
- final String endpoint, final ClientConfiguration awsConf,
- String awsRegion) {
- LOG.debug("Creating endpoint configuration for \"{}\"", endpoint);
- if (endpoint == null || endpoint.isEmpty()) {
- // the default endpoint...we should be using null at this point.
- LOG.debug("Using default endpoint -no need to generate a configuration");
- return null;
- }
-
- final URI epr = RuntimeHttpUtils.toUri(endpoint, awsConf);
- LOG.debug("Endpoint URI = {}", epr);
- String region = awsRegion;
- if (StringUtils.isBlank(region)) {
- if (!ServiceUtils.isS3USStandardEndpoint(endpoint)) {
- LOG.debug("Endpoint {} is not the default; parsing", epr);
- region = AwsHostNameUtils.parseRegion(
- epr.getHost(),
- S3_SERVICE_NAME);
- } else {
- // US-east, set region == null.
- LOG.debug("Endpoint {} is the standard one; declare region as null",
- epr);
- region = null;
- }
- }
- LOG.debug("Region for endpoint {}, URI {} is determined as {}",
- endpoint, epr, region);
- return new AwsClientBuilder.EndpointConfiguration(endpoint, region);
- }
-
/**
* Given a endpoint string, create the endpoint URI.
*
@@ -550,8 +206,8 @@ private static URI getS3Endpoint(String endpoint, final Configuration conf) {
String protocol = secureConnections ? "https" : "http";
if (endpoint == null || endpoint.isEmpty()) {
- // the default endpoint
- endpoint = CENTRAL_ENDPOINT;
+ // don't set an endpoint if none is configured, instead let the SDK figure it out.
+ return null;
}
if (!endpoint.contains("://")) {
@@ -564,45 +220,4 @@ private static URI getS3Endpoint(String endpoint, final Configuration conf) {
throw new IllegalArgumentException(e);
}
}
-
- /**
- * Get the bucket region.
- *
- * @param region AWS S3 Region set in the config. This property may not be set, in which case
- * ask S3 for the region.
- * @param bucket Bucket name.
- * @param credentialsProvider Credentials provider to be used with the default s3 client.
- * @return region of the bucket.
- */
- private static Region getS3Region(String region, String bucket,
- AwsCredentialsProvider credentialsProvider) {
-
- if (!StringUtils.isBlank(region)) {
- return Region.of(region);
- }
-
- try {
- // build a s3 client with region eu-west-1 that can be used to get the region of the bucket.
- // Using eu-west-1, as headBucket() doesn't work with us-east-1. This is because
- // us-east-1 uses the endpoint s3.amazonaws.com, which resolves bucket.s3.amazonaws.com to
- // the actual region the bucket is in. As the request is signed with us-east-1 and not the
- // bucket's region, it fails.
- S3Client s3Client = S3Client.builder().region(Region.EU_WEST_1)
- .credentialsProvider(credentialsProvider)
- .build();
-
- HeadBucketResponse headBucketResponse =
- s3Client.headBucket(HeadBucketRequest.builder().bucket(bucket).build());
- return Region.of(
- headBucketResponse.sdkHttpResponse().headers().get(BUCKET_REGION_HEADER).get(0));
- } catch (S3Exception exception) {
- if (exception.statusCode() == SC_301_MOVED_PERMANENTLY) {
- List bucketRegion =
- exception.awsErrorDetails().sdkHttpResponse().headers().get(BUCKET_REGION_HEADER);
- return Region.of(bucketRegion.get(0));
- }
- }
-
- return Region.US_EAST_1;
- }
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
index e9946e7e85c34..d519c1c0763d8 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentS3ClientFactory.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.fs.s3a;
+import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
import software.amazon.awssdk.awscore.exception.AwsServiceException;
@@ -45,7 +46,7 @@ public class InconsistentS3ClientFactory extends DefaultS3ClientFactory {
@Override
protected ClientOverrideConfiguration createClientOverrideConfiguration(
- S3ClientCreationParameters parameters, Configuration conf) {
+ S3ClientCreationParameters parameters, Configuration conf) throws IOException {
LOG.warn("** FAILURE INJECTION ENABLED. Do not run in production! **");
LOG.warn("List inconsistency is no longer emulated; only throttling and read errors");
return super.createClientOverrideConfiguration(parameters, conf)
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java
index 1c0fd76c6b107..b614b379bd6b2 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.fs.s3a;
-import software.amazon.awssdk.transfer.s3.ObjectTransfer;
+import software.amazon.awssdk.transfer.s3.model.ObjectTransfer;
import software.amazon.awssdk.transfer.s3.progress.TransferListener;
import org.apache.hadoop.util.Progressable;
import org.slf4j.Logger;
@@ -66,7 +66,7 @@ public void bytesTransferred(TransferListener.Context.BytesTransferred context)
progress.progress();
}
- long transferred = context.progressSnapshot().bytesTransferred();
+ long transferred = context.progressSnapshot().transferredBytes();
long delta = transferred - lastBytesTransferred;
fs.incrementPutProgressStatistics(key, delta);
lastBytesTransferred = transferred;
@@ -80,7 +80,7 @@ public void bytesTransferred(TransferListener.Context.BytesTransferred context)
public long uploadCompleted(ObjectTransfer upload) {
long delta =
- upload.progress().snapshot().bytesTransferred() - lastBytesTransferred;
+ upload.progress().snapshot().transferredBytes() - lastBytesTransferred;
if (delta > 0) {
LOG.debug("S3A write delta changed after finished: {} bytes", delta);
fs.incrementPutProgressStatistics(key, delta);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
index 2ec4976071af3..4b858d88b7f5a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java
@@ -38,10 +38,9 @@
import software.amazon.awssdk.services.s3.model.PutObjectResponse;
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
-import com.amazonaws.event.ProgressEvent;
-import com.amazonaws.event.ProgressEventType;
-import com.amazonaws.event.ProgressListener;
+import org.apache.hadoop.fs.s3a.impl.ProgressListener;
+import org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent;
import org.apache.hadoop.fs.s3a.impl.PutObjectOptions;
import org.apache.hadoop.fs.statistics.IOStatisticsAggregator;
import org.apache.hadoop.util.Preconditions;
@@ -71,6 +70,7 @@
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.apache.hadoop.fs.s3a.Statistic.*;
+import static org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent.*;
import static org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext.EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
@@ -192,7 +192,7 @@ class S3ABlockOutputStream extends OutputStream implements
this.executorService = MoreExecutors.listeningDecorator(
builder.executorService);
this.multiPartUpload = null;
- final Progressable progress = builder.progress;
+ Progressable progress = builder.progress;
this.progressListener = (progress instanceof ProgressListener) ?
(ProgressListener) progress
: new ProgressableListener(progress);
@@ -586,21 +586,19 @@ private int putObject() throws IOException {
builder.putOptions,
false);
- // TODO: You cannot currently add progress listeners to requests not via the TM.
- // There is an open ticket for this with the SDK team. But need to check how important
- // this is for us?
- // BlockUploadProgress callback =
- // new BlockUploadProgress(
- // block, progressListener, now());
- // putObjectRequest.setGeneralProgressListener(callback);
+ BlockUploadProgress progressCallback =
+ new BlockUploadProgress(block, progressListener, now());
statistics.blockUploadQueued(size);
ListenableFuture putObjectResult =
executorService.submit(() -> {
try {
// the putObject call automatically closes the input
// stream afterwards.
- return writeOperationHelper.putObject(putObjectRequest, builder.putOptions, uploadData,
- uploadData.hasFile());
+ PutObjectResponse response =
+ writeOperationHelper.putObject(putObjectRequest, builder.putOptions, uploadData,
+ uploadData.hasFile());
+ progressCallback.progressChanged(REQUEST_BYTE_TRANSFER_EVENT);
+ return response;
} finally {
cleanupWithLogger(LOG, uploadData, block);
}
@@ -872,12 +870,8 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block,
throw e;
}
- // TODO: You cannot currently add progress listeners to requests not via the TM.
- // See also putObject
- // BlockUploadProgress callback =
- // new BlockUploadProgress(
- // block, progressListener, now());
- // request.setGeneralProgressListener(callback);
+ BlockUploadProgress progressCallback =
+ new BlockUploadProgress(block, progressListener, now());
statistics.blockUploadQueued(block.dataSize());
ListenableFuture partETagFuture =
@@ -887,12 +881,18 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block,
try {
LOG.debug("Uploading part {} for id '{}'",
currentPartNumber, uploadId);
+
+ progressCallback.progressChanged(TRANSFER_PART_STARTED_EVENT);
+
UploadPartResponse response = writeOperationHelper
.uploadPart(request, requestBody);
LOG.debug("Completed upload of {} to part {}",
block, response.eTag());
LOG.debug("Stream statistics of {}", statistics);
partsUploaded++;
+
+ progressCallback.progressChanged(TRANSFER_PART_COMPLETED_EVENT);
+
return CompletedPart.builder()
.eTag(response.eTag())
.partNumber(currentPartNumber)
@@ -900,6 +900,7 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block,
} catch (IOException e) {
// save immediately.
noteUploadFailure(e);
+ progressCallback.progressChanged(TRANSFER_PART_FAILED_EVENT);
throw e;
} finally {
// close the stream and block
@@ -995,22 +996,24 @@ private IOException abort() {
}
}
+
/**
* The upload progress listener registered for events returned
* during the upload of a single block.
* It updates statistics and handles the end of the upload.
* Transfer failures are logged at WARN.
*/
- private final class BlockUploadProgress implements ProgressListener {
+ private final class BlockUploadProgress {
+
private final S3ADataBlocks.DataBlock block;
private final ProgressListener nextListener;
private final Instant transferQueueTime;
private Instant transferStartTime;
+ private int size;
/**
* Track the progress of a single block upload.
* @param block block to monitor
- * @param nextListener optional next progress listener
* @param transferQueueTime time the block was transferred
* into the queue
*/
@@ -1019,20 +1022,17 @@ private BlockUploadProgress(S3ADataBlocks.DataBlock block,
Instant transferQueueTime) {
this.block = block;
this.transferQueueTime = transferQueueTime;
+ this.size = block.dataSize();
this.nextListener = nextListener;
}
- @Override
- public void progressChanged(ProgressEvent progressEvent) {
- ProgressEventType eventType = progressEvent.getEventType();
- long bytesTransferred = progressEvent.getBytesTransferred();
+ public void progressChanged(ProgressListenerEvent eventType) {
- int size = block.dataSize();
switch (eventType) {
case REQUEST_BYTE_TRANSFER_EVENT:
// bytes uploaded
- statistics.bytesTransferred(bytesTransferred);
+ statistics.bytesTransferred(size);
break;
case TRANSFER_PART_STARTED_EVENT:
@@ -1047,6 +1047,7 @@ public void progressChanged(ProgressEvent progressEvent) {
statistics.blockUploadCompleted(
Duration.between(transferStartTime, now()),
size);
+ statistics.bytesTransferred(size);
break;
case TRANSFER_PART_FAILED_EVENT:
@@ -1061,13 +1062,13 @@ public void progressChanged(ProgressEvent progressEvent) {
}
if (nextListener != null) {
- nextListener.progressChanged(progressEvent);
+ nextListener.progressChanged(eventType, size);
}
}
}
/**
- * Bridge from AWS {@code ProgressListener} to Hadoop {@link Progressable}.
+ * Bridge from {@link ProgressListener} to Hadoop {@link Progressable}.
*/
private static class ProgressableListener implements ProgressListener {
private final Progressable progress;
@@ -1076,7 +1077,7 @@ private static class ProgressableListener implements ProgressListener {
this.progress = progress;
}
- public void progressChanged(ProgressEvent progressEvent) {
+ public void progressChanged(ProgressListenerEvent eventType, int bytesTransferred) {
if (progress != null) {
progress.progress();
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 5875a4b6c96f9..2efacb47bb997 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
@@ -34,6 +34,7 @@
import java.util.Collections;
import java.util.Date;
import java.util.EnumSet;
+import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
@@ -58,11 +59,11 @@
import software.amazon.awssdk.services.s3.S3Client;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
-import software.amazon.awssdk.services.s3.model.GetBucketAclRequest;
import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest;
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
import software.amazon.awssdk.services.s3.model.GetObjectResponse;
import software.amazon.awssdk.services.s3.model.HeadBucketRequest;
+import software.amazon.awssdk.services.s3.model.HeadBucketResponse;
import software.amazon.awssdk.services.s3.model.MultipartUpload;
import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest;
import software.amazon.awssdk.services.s3.model.CreateMultipartUploadResponse;
@@ -78,30 +79,30 @@
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
import software.amazon.awssdk.services.s3.model.NoSuchBucketException;
-import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
import software.amazon.awssdk.services.s3.model.PutObjectResponse;
import software.amazon.awssdk.services.s3.model.S3Error;
+import software.amazon.awssdk.services.s3.model.S3Exception;
import software.amazon.awssdk.services.s3.model.S3Object;
import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest;
import software.amazon.awssdk.services.s3.model.SelectObjectContentResponseHandler;
import software.amazon.awssdk.services.s3.model.StorageClass;
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
-import software.amazon.awssdk.transfer.s3.CompletedCopy;
-import software.amazon.awssdk.transfer.s3.CompletedFileUpload;
-import software.amazon.awssdk.transfer.s3.Copy;
-import software.amazon.awssdk.transfer.s3.CopyRequest;
-import software.amazon.awssdk.transfer.s3.FileUpload;
+import software.amazon.awssdk.transfer.s3.model.CompletedCopy;
+import software.amazon.awssdk.transfer.s3.model.CompletedFileUpload;
+import software.amazon.awssdk.transfer.s3.model.Copy;
import software.amazon.awssdk.transfer.s3.S3TransferManager;
-import software.amazon.awssdk.transfer.s3.UploadFileRequest;
+import software.amazon.awssdk.transfer.s3.model.CopyRequest;
+import software.amazon.awssdk.transfer.s3.model.FileUpload;
+import software.amazon.awssdk.transfer.s3.model.UploadFileRequest;
+import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@@ -156,6 +157,7 @@
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import org.apache.hadoop.fs.statistics.IOStatisticsContext;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
+import org.apache.hadoop.fs.store.LogExactlyOnce;
import org.apache.hadoop.fs.store.audit.AuditEntryPoint;
import org.apache.hadoop.fs.store.audit.ActiveThreadSpanSource;
import org.apache.hadoop.fs.store.audit.AuditSpan;
@@ -228,6 +230,7 @@
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.apache.hadoop.fs.s3a.Statistic.*;
import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.INITIALIZE_SPAN;
+import static org.apache.hadoop.fs.s3a.auth.AwsCredentialListProvider.createAWSCredentialProviderSet;
import static org.apache.hadoop.fs.s3a.auth.RolePolicies.STATEMENT_ALLOW_SSE_KMS_RW;
import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3Operations;
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.TokenIssuingPolicy.NoTokensAvailable;
@@ -239,12 +242,12 @@
import static org.apache.hadoop.fs.s3a.impl.CreateFileBuilder.OPTIONS_CREATE_FILE_OVERWRITE;
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound;
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket;
-import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AP_INACCESSIBLE;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AP_REQUIRED_EXCEPTION;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.ARN_BUCKET_OPTION;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_301_MOVED_PERMANENTLY;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_403_FORBIDDEN;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404_NOT_FOUND;
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.UPLOAD_PART_COUNT_LIMIT;
@@ -329,10 +332,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
private int executorCapacity;
private long multiPartThreshold;
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);
private static final Logger PROGRESS =
LoggerFactory.getLogger("org.apache.hadoop.fs.s3a.S3AFileSystem.Progress");
private LocalDirAllocator directoryAllocator;
- private ObjectCannedACL cannedACL;
+ private String cannedACL;
/**
* This must never be null; until initialized it just declares that there
@@ -435,6 +440,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
*/
private final Set deleteOnExit = new TreeSet<>();
+ private final static Map BUCKET_REGIONS = new HashMap<>();
+
/** Add any deprecated keys. */
@SuppressWarnings("deprecation")
private static void addDeprecatedKeys() {
@@ -708,7 +715,7 @@ private void setCSEGauge() {
* bucket existence check is not done to improve performance of
* S3AFileSystem initialization. When set to 1 or 2, bucket existence check
* will be performed which is potentially slow.
- * If 3 or higher: warn and use the v2 check.
+ * If 3 or higher: warn and skip check.
* Also logging DNS address of the s3 endpoint if the bucket probe value is
* greater than 0 else skipping it for increased performance.
* @throws UnknownStoreException the bucket is absent
@@ -725,18 +732,14 @@ private void doBucketProbing() throws IOException {
LOG.debug("skipping check for bucket existence");
break;
case 1:
- logDnsLookup(getConf());
- verifyBucketExists();
- break;
case 2:
logDnsLookup(getConf());
- verifyBucketExistsV2();
+ verifyBucketExists();
break;
default:
// we have no idea what this is, assume it is from a later release.
- LOG.warn("Unknown bucket probe option {}: {}; falling back to check #2",
+ LOG.warn("Unknown bucket probe option {}: {}; skipping check for bucket existence",
S3A_BUCKET_PROBE, bucketProbe);
- verifyBucketExistsV2();
break;
}
}
@@ -834,58 +837,32 @@ protected static S3AStorageStatistics createStorageStatistics(
* @throws UnknownStoreException the bucket is absent
* @throws IOException any other problem talking to S3
*/
- // TODO: Review: this used to call doesBucketExist in v1, which does not check permissions,
- // not even read access.
@Retries.RetryTranslated
protected void verifyBucketExists() throws UnknownStoreException, IOException {
- if (!invoker.retry("doesBucketExist", bucket, true,
- trackDurationOfOperation(getDurationTrackerFactory(), STORE_EXISTS_PROBE.getSymbol(),
- () -> {
+
+ if(!trackDurationAndSpan(
+ STORE_EXISTS_PROBE, bucket, null, () ->
+ invoker.retry("doestBucketExist", bucket, true, () -> {
try {
+ if (BUCKET_REGIONS.containsKey(bucket)) {
+ return true;
+ }
s3Client.headBucket(HeadBucketRequest.builder().bucket(bucket).build());
return true;
- } catch (NoSuchBucketException e) {
- return false;
- }
- }))) {
- throw new UnknownStoreException("s3a://" + bucket + "/", " Bucket does " + "not exist");
- }
- }
-
- /**
- * Verify that the bucket exists. This will correctly throw an exception
- * when credentials are invalid.
- * TODO: Review. May be redundant in v2.
- * Retry policy: retrying, translated.
- * @throws UnknownStoreException the bucket is absent
- * @throws IOException any other problem talking to S3
- */
- @Retries.RetryTranslated
- protected void verifyBucketExistsV2()
- throws UnknownStoreException, IOException {
- if (!invoker.retry("doesBucketExistV2", bucket, true,
- trackDurationOfOperation(getDurationTrackerFactory(),
- STORE_EXISTS_PROBE.getSymbol(),
- () -> {
- // Bug in SDK always returns `true` for AccessPoint ARNs with `doesBucketExistV2()`
- // expanding implementation to use ARNs and buckets correctly
- try {
- s3Client.getBucketAcl(GetBucketAclRequest.builder()
- .bucket(bucket)
- .build());
} catch (AwsServiceException ex) {
int statusCode = ex.statusCode();
if (statusCode == SC_404_NOT_FOUND ||
- (statusCode == SC_403_FORBIDDEN &&
- ex.getMessage().contains(AP_INACCESSIBLE))) {
+ (statusCode == SC_403_FORBIDDEN && accessPoint != null)) {
return false;
}
}
return true;
}))) {
- throw new UnknownStoreException("s3a://" + bucket + "/", " Bucket does "
- + "not exist");
+
+ throw new UnknownStoreException("s3a://" + bucket + "/",
+ " Bucket does " + "not exist. " + "Accessing with " + ENDPOINT + " set to "
+ + getConf().getTrimmed(ENDPOINT, null));
}
}
@@ -976,6 +953,12 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException {
? conf.getTrimmed(ENDPOINT, DEFAULT_ENDPOINT)
: accessPoint.getEndpoint();
+ String configuredRegion = accessPoint == null
+ ? conf.getTrimmed(AWS_REGION)
+ : accessPoint.getRegion();
+
+ Region region = getS3Region(configuredRegion);
+
S3ClientFactory.S3ClientCreationParameters parameters =
new S3ClientFactory.S3ClientCreationParameters()
.withCredentialSet(credentials)
@@ -987,12 +970,94 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException {
.withRequesterPays(conf.getBoolean(ALLOW_REQUESTER_PAYS, DEFAULT_ALLOW_REQUESTER_PAYS))
.withExecutionInterceptors(auditManager.createExecutionInterceptors())
.withMinimumPartSize(partSize)
- .withTransferManagerExecutor(unboundedThreadPool);
+ .withTransferManagerExecutor(unboundedThreadPool)
+ .withRegion(region);
S3ClientFactory clientFactory = ReflectionUtils.newInstance(s3ClientFactoryClass, conf);
- s3Client = clientFactory.createS3ClientV2(getUri(), parameters);
+ s3Client = clientFactory.createS3Client(getUri(), parameters);
+ createS3AsyncClient(clientFactory, parameters);
+ transferManager = clientFactory.createS3TransferManager(s3AsyncClient);
+ }
+
+ /**
+ * Creates and configures the S3AsyncClient.
+ * Uses synchronized method to suppress spotbugs error.
+ *
+ * @param clientFactory factory used to create S3AsyncClient
+ * @param parameters parameter object
+ * @throws IOException on any IO problem
+ */
+ private synchronized void createS3AsyncClient(S3ClientFactory clientFactory,
+ S3ClientFactory.S3ClientCreationParameters parameters) throws IOException {
s3AsyncClient = clientFactory.createS3AsyncClient(getUri(), parameters);
- transferManager = clientFactory.createS3TransferManager(getUri(), parameters);
+ }
+
+ /**
+ * Get the bucket region.
+ *
+ * @param region AWS S3 Region set in the config. This property may not be set, in which case
+ * ask S3 for the region.
+ * @return region of the bucket.
+ */
+ private Region getS3Region(String region) throws IOException {
+
+ if (!StringUtils.isBlank(region)) {
+ return Region.of(region);
+ }
+
+ Region cachedRegion = BUCKET_REGIONS.get(bucket);
+
+ if (cachedRegion != null) {
+ LOG.debug("Got region {} for bucket {} from cache", cachedRegion, bucket);
+ return cachedRegion;
+ }
+
+ Region s3Region = trackDurationAndSpan(STORE_REGION_PROBE, bucket, null,
+ () -> invoker.retry("getS3Region", bucket, true, () -> {
+ try {
+
+ SET_REGION_WARNING.warn(
+ "Getting region for bucket {} from S3, this will slow down FS initialisation. "
+ + "To avoid this, set the region using property {}", bucket,
+ FS_S3A_BUCKET_PREFIX + bucket + ".endpoint.region");
+
+ // build a s3 client with region eu-west-1 that can be used to get the region of the
+ // bucket. Using eu-west-1, as headBucket() doesn't work with us-east-1. This is because
+ // us-east-1 uses the endpoint s3.amazonaws.com, which resolves bucket.s3.amazonaws.com
+ // to the actual region the bucket is in. As the request is signed with us-east-1 and
+ // not the bucket's region, it fails.
+ S3Client getRegionS3Client =
+ S3Client.builder().region(Region.EU_WEST_1).credentialsProvider(credentials)
+ .build();
+
+ HeadBucketResponse headBucketResponse =
+ getRegionS3Client.headBucket(HeadBucketRequest.builder().bucket(bucket).build());
+
+ Region bucketRegion = Region.of(
+ headBucketResponse.sdkHttpResponse().headers().get(BUCKET_REGION_HEADER).get(0));
+ BUCKET_REGIONS.put(bucket, bucketRegion);
+
+ return bucketRegion;
+ } catch (S3Exception exception) {
+ if (exception.statusCode() == SC_301_MOVED_PERMANENTLY) {
+ Region bucketRegion = Region.of(
+ exception.awsErrorDetails().sdkHttpResponse().headers().get(BUCKET_REGION_HEADER)
+ .get(0));
+ BUCKET_REGIONS.put(bucket, bucketRegion);
+
+ return bucketRegion;
+ }
+
+ if (exception.statusCode() == SC_404_NOT_FOUND) {
+ throw new UnknownStoreException("s3a://" + bucket + "/",
+ " Bucket does " + "not exist");
+ }
+
+ throw exception;
+ }
+ }));
+
+ return s3Region;
}
/**
@@ -1168,7 +1233,7 @@ public EncryptionSecrets getEncryptionSecrets() {
private void initCannedAcls(Configuration conf) {
String cannedACLName = conf.get(CANNED_ACL, DEFAULT_CANNED_ACL);
if (!cannedACLName.isEmpty()) {
- cannedACL = ObjectCannedACL.valueOf(AWSCannedACL.valueOf(cannedACLName).toString());
+ cannedACL = AWSCannedACL.valueOf(cannedACLName).toString();
} else {
cannedACL = null;
}
@@ -1272,7 +1337,7 @@ public int getDefaultPort() {
* @return S3Client
*/
@VisibleForTesting
- public S3Client getAmazonS3V2ClientForTesting(String reason) {
+ public S3Client getAmazonS3ClientForTesting(String reason) {
LOG.warn("Access to S3 client requested, reason {}", reason);
return s3Client;
}
@@ -1404,7 +1469,7 @@ protected void setBucket(String bucket) {
* Get the canned ACL of this FS.
* @return an ACL, if any
*/
- ObjectCannedACL getCannedACL() {
+ String getCannedACL() {
return cannedACL;
}
@@ -2657,6 +2722,26 @@ protected HeadObjectResponse getObjectMetadata(String key,
return response;
}
+ /**
+ * Request bucket metadata.
+ * @return the metadata
+ * @throws UnknownStoreException the bucket is absent
+ * @throws IOException any other problem talking to S3
+ */
+ @Retries.RetryRaw
+ protected HeadBucketResponse getBucketMetadata() throws IOException {
+ final HeadBucketResponse response = trackDurationAndSpan(STORE_EXISTS_PROBE, bucket, null,
+ () -> invoker.retry("getBucketMetadata()", bucket, true, () -> {
+ try {
+ return s3Client.headBucket(
+ getRequestFactory().newHeadBucketRequestBuilder(bucket).build());
+ } catch (NoSuchBucketException e) {
+ throw new UnknownStoreException("s3a://" + bucket + "/", " Bucket does " + "not exist");
+ }
+ }));
+ return response;
+ }
+
/**
* Initiate a {@code listObjects} operation, incrementing metrics
* in the process.
@@ -2934,10 +3019,12 @@ public UploadInfo putObject(PutObjectRequest putObjectRequest, File file,
LOG.debug("PUT {} bytes to {} via transfer manager ", len, putObjectRequest.key());
incrementPutStartStatistics(len);
- // TODO: Something not right with the TM listener, fix
FileUpload upload = transferManager.uploadFile(
- UploadFileRequest.builder().putObjectRequest(putObjectRequest).source(file).build());
- // .overrideConfiguration(o -> o.addListener(listener)).build());
+ UploadFileRequest.builder()
+ .putObjectRequest(putObjectRequest)
+ .source(file)
+ .addTransferListener(listener)
+ .build());
return new UploadInfo(upload, len);
}
@@ -2993,16 +3080,7 @@ PutObjectResponse putObjectDirect(PutObjectRequest putObjectRequest,
* @throws IllegalArgumentException if the length is negative
*/
private long getPutRequestLength(PutObjectRequest putObjectRequest) {
- long len;
-
- // TODO: Check why this exists. Content length is set before. Why can't that be used directly?
-// if (putObjectRequest.getFile() != null) {
-// len = putObjectRequest.getFile().length();
-// } else {
-// len = putObjectRequest.getMetadata().getContentLength();
-// }
-
- len = putObjectRequest.contentLength();
+ long len = putObjectRequest.contentLength();
Preconditions.checkState(len >= 0, "Cannot PUT object of unknown length");
return len;
@@ -4211,19 +4289,9 @@ public List listAWSPolicyRules(
@Retries.RetryTranslated
private CopyObjectResponse copyFile(String srcKey, String dstKey, long size,
S3ObjectAttributes srcAttributes, S3AReadOpContext readContext)
- throws IOException, InterruptedIOException {
+ throws IOException {
LOG.debug("copyFile {} -> {} ", srcKey, dstKey);
- // TODO: Transfer manager currently only provides transfer listeners for upload,
- // add progress listener for copy when this is supported.
-// TODO: Is the above still valid? Try to enable when logger issue is resolved.
-// TransferListener progressListener = new TransferListener() {
-// @Override
-// public void transferComplete(Context.TransferComplete context) {
-// incrementWriteOperations();
-// }
-// };
-
ChangeTracker changeTracker = new ChangeTracker(
keyToQualifiedPath(srcKey).toString(),
changeDetectionPolicy,
@@ -4266,10 +4334,6 @@ private CopyObjectResponse copyFile(String srcKey, String dstKey, long size,
Copy copy = transferManager.copy(
CopyRequest.builder()
.copyObjectRequest(copyObjectRequestBuilder.build())
-// TODO: Enable when logger issue is resolved.
-// .overrideConfiguration(c -> c
-// .addListener(getAuditManager().createTransferListener())
-// .addListener(progressListener))
.build());
try {
@@ -4785,6 +4849,13 @@ public HeadObjectResponse getObjectMetadata(final String key)
return once("getObjectMetadata", key, () ->
S3AFileSystem.this.getObjectMetadata(key));
}
+
+ @Override
+ public HeadBucketResponse getBucketMetadata()
+ throws IOException {
+ return once("getBucketMetadata", bucket, () ->
+ S3AFileSystem.this.getBucketMetadata());
+ }
}
/**
* {@inheritDoc}.
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 f7bfee729582f..54186ea6aa1fd 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -18,11 +18,6 @@
package org.apache.hadoop.fs.s3a;
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.Protocol;
-import com.amazonaws.auth.AWSCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider;
import software.amazon.awssdk.awscore.exception.AwsServiceException;
import software.amazon.awssdk.core.exception.AbortedException;
import software.amazon.awssdk.core.exception.SdkException;
@@ -31,7 +26,6 @@
import software.amazon.awssdk.services.s3.model.S3Object;
import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@@ -42,14 +36,11 @@
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.util.functional.RemoteIterators;
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
-import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider;
import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteException;
-import org.apache.hadoop.fs.s3a.impl.NetworkBinding;
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
import org.apache.hadoop.net.ConnectTimeoutException;
import org.apache.hadoop.security.ProviderUtils;
import org.apache.hadoop.util.Preconditions;
-import org.apache.hadoop.util.VersionInfo;
import org.apache.hadoop.util.Lists;
import org.slf4j.Logger;
@@ -70,19 +61,13 @@
import java.net.URI;
import java.nio.file.AccessDeniedException;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collection;
-import java.util.Collections;
import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
-import java.util.Set;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
import static org.apache.commons.lang3.StringUtils.isEmpty;
import static org.apache.hadoop.fs.s3a.Constants.*;
@@ -104,12 +89,7 @@ public final class S3AUtils {
static final String CONSTRUCTOR_EXCEPTION = "constructor exception";
static final String INSTANTIATION_EXCEPTION
= "instantiation exception";
- static final String NOT_AWS_PROVIDER =
- "does not implement AWSCredentialsProvider";
- static final String NOT_AWS_V2_PROVIDER =
- "does not implement AwsCredentialsProvider";
- static final String ABSTRACT_PROVIDER =
- "is abstract and therefore cannot be created";
+
static final String ENDPOINT_KEY = "Endpoint";
/** Filesystem is closed; kept here to keep the errors close. */
@@ -144,14 +124,6 @@ public final class S3AUtils {
private static final String BUCKET_PATTERN = FS_S3A_BUCKET_PREFIX + "%s.%s";
- /**
- * Error message when the AWS provider list built up contains a forbidden
- * entry.
- */
- @VisibleForTesting
- public static final String E_FORBIDDEN_AWS_PROVIDER
- = "AWS provider class cannot be used";
-
private S3AUtils() {
}
@@ -228,16 +200,11 @@ public static IOException translateException(@Nullable String operation,
case SC_301_MOVED_PERMANENTLY:
case SC_307_TEMPORARY_REDIRECT:
if (s3Exception != null) {
- // TODO: Can we get the endpoint in v2?
- // Maybe not: https://github.com/aws/aws-sdk-java-v2/issues/3048
-// if (s3Exception.getAdditionalDetails() != null &&
-// s3Exception.getAdditionalDetails().containsKey(ENDPOINT_KEY)) {
-// message = String.format("Received permanent redirect response to "
-// + "endpoint %s. This likely indicates that the S3 endpoint "
-// + "configured in %s does not match the AWS region containing "
-// + "the bucket.",
-// s3Exception.getAdditionalDetails().get(ENDPOINT_KEY), ENDPOINT);
-// }
+ message = String.format("Received permanent redirect response to "
+ + "region %s. This likely indicates that the S3 region "
+ + "configured in %s does not match the AWS region containing " + "the bucket.",
+ s3Exception.awsErrorDetails().sdkHttpResponse().headers().get(BUCKET_REGION_HEADER),
+ AWS_REGION);
ioe = new AWSRedirectException(message, s3Exception);
} else {
ioe = new AWSRedirectException(message, ase);
@@ -562,135 +529,8 @@ public static long dateToLong(final Date date) {
return date.getTime();
}
- /**
- * The standard AWS provider list for AWS connections.
- */
- public static final List>
- STANDARD_AWS_PROVIDERS = Collections.unmodifiableList(
- Arrays.asList(
- TemporaryAWSCredentialsProvider.class,
- SimpleAWSCredentialsProvider.class,
- EnvironmentVariableCredentialsProvider.class,
- IAMInstanceCredentialsProvider.class));
-
- /**
- * Create the AWS credentials from the providers, the URI and
- * the key {@link Constants#AWS_CREDENTIALS_PROVIDER} in the configuration.
- * @param binding Binding URI -may be null
- * @param conf filesystem configuration
- * @return a credentials provider list
- * @throws IOException Problems loading the providers (including reading
- * secrets from credential files).
- */
- public static AWSCredentialProviderList createAWSCredentialProviderSet(
- @Nullable URI binding,
- Configuration conf) throws IOException {
- // this will reject any user:secret entries in the URI
- S3xLoginHelper.rejectSecretsInURIs(binding);
- AWSCredentialProviderList credentials =
- buildAWSProviderList(binding,
- conf,
- AWS_CREDENTIALS_PROVIDER,
- STANDARD_AWS_PROVIDERS,
- new HashSet<>());
- // make sure the logging message strips out any auth details
- LOG.debug("For URI {}, using credentials {}",
- binding, credentials);
- return credentials;
- }
-
- /**
- * Load list of AWS credential provider/credential provider factory classes.
- * @param conf configuration
- * @param key key
- * @param defaultValue list of default values
- * @return the list of classes, possibly empty
- * @throws IOException on a failure to load the list.
- */
- public static List> loadAWSProviderClasses(Configuration conf,
- String key,
- Class>... defaultValue) throws IOException {
- try {
- return Arrays.asList(conf.getClasses(key, defaultValue));
- } catch (RuntimeException e) {
- Throwable c = e.getCause() != null ? e.getCause() : e;
- throw new IOException("From option " + key + ' ' + c, c);
- }
- }
-
- /**
- * Maps V1 credential providers to either their equivalent SDK V2 class or hadoop provider.
- */
- private static Map initCredentialProvidersMap() {
- Map v1v2CredentialProviderMap = new HashMap<>();
-
- v1v2CredentialProviderMap.put("EnvironmentVariableCredentialsProvider",
- EnvironmentVariableCredentialsProvider.class);
- v1v2CredentialProviderMap.put("EC2ContainerCredentialsProviderWrapper",
- IAMInstanceCredentialsProvider.class);
- v1v2CredentialProviderMap.put("InstanceProfileCredentialsProvider",
- IAMInstanceCredentialsProvider.class);
-
- return v1v2CredentialProviderMap;
- }
-
- /**
- * Load list of AWS credential provider/credential provider factory classes;
- * support a forbidden list to prevent loops, mandate full secrets, etc.
- * @param binding Binding URI -may be null
- * @param conf configuration
- * @param key key
- * @param forbidden a possibly empty set of forbidden classes.
- * @param defaultValues list of default providers.
- * @return the list of classes, possibly empty
- * @throws IOException on a failure to load the list.
- */
- public static AWSCredentialProviderList buildAWSProviderList(
- @Nullable final URI binding,
- final Configuration conf,
- final String key,
- final List> defaultValues,
- final Set> forbidden) throws IOException {
-
- // build up the base provider
- List> awsClasses = loadAWSProviderClasses(conf,
- key,
- defaultValues.toArray(new Class[defaultValues.size()]));
-
- Map v1v2CredentialProviderMap = initCredentialProvidersMap();
- // and if the list is empty, switch back to the defaults.
- // this is to address the issue that configuration.getClasses()
- // doesn't return the default if the config value is just whitespace.
- if (awsClasses.isEmpty()) {
- awsClasses = defaultValues;
- }
- // iterate through, checking for blacklists and then instantiating
- // each provider
- AWSCredentialProviderList providers = new AWSCredentialProviderList();
- for (Class> aClass : awsClasses) {
-
- if (forbidden.contains(aClass)) {
- throw new IOException(E_FORBIDDEN_AWS_PROVIDER
- + " in option " + key + ": " + aClass);
- }
-
- if (v1v2CredentialProviderMap.containsKey(aClass.getSimpleName()) &&
- aClass.getName().contains(AWS_AUTH_CLASS_PREFIX)){
- providers.add(createAWSV2CredentialProvider(conf,
- v1v2CredentialProviderMap.get(aClass.getSimpleName()), binding));
- } else if (AWSCredentialsProvider.class.isAssignableFrom(aClass)) {
- providers.add(createAWSV1CredentialProvider(conf,
- aClass, binding));
- } else {
- providers.add(createAWSV2CredentialProvider(conf, aClass, binding));
- }
-
- }
- return providers;
- }
-
- /**
- * Create an AWS credential provider from its class by using reflection. The
+ /***
+ * Creates an instance of a class using reflection. The
* class must implement one of the following means of construction, which are
* attempted in order:
*
@@ -699,187 +539,79 @@ public static AWSCredentialProviderList buildAWSProviderList(
* org.apache.hadoop.conf.Configuration
* a public constructor accepting
* org.apache.hadoop.conf.Configuration
- * a public static method named getInstance that accepts no
+ * a public static method named as per methodName, that accepts no
* arguments and returns an instance of
- * com.amazonaws.auth.AWSCredentialsProvider, or
+ * specified type, or
* a public default constructor.
*
*
+ * @param instanceClass Class for which instance is to be created
* @param conf configuration
- * @param credClass credential class
* @param uri URI of the FS
- * @return the instantiated class
- * @throws IOException on any instantiation failure.
+ * @param interfaceImplemented interface that this class implements
+ * @param methodName name of factory method to be invoked
+ * @param configKey config key under which this class is specified
+ * @param Instance of class
+ * @return instance of the specified class
+ * @throws IOException on any problem
*/
- private static AWSCredentialsProvider createAWSV1CredentialProvider(
- Configuration conf,
- Class> credClass,
- @Nullable URI uri) throws IOException {
- AWSCredentialsProvider credentials = null;
- String className = credClass.getName();
- if (!AWSCredentialsProvider.class.isAssignableFrom(credClass)) {
- throw new IOException("Class " + credClass + " " + NOT_AWS_PROVIDER);
- }
- if (Modifier.isAbstract(credClass.getModifiers())) {
- throw new IOException("Class " + credClass + " " + ABSTRACT_PROVIDER);
- }
- LOG.debug("Credential provider class is {}", className);
+ @SuppressWarnings("unchecked")
+ public static InstanceT getInstanceFromReflection(Class> instanceClass,
+ Configuration conf, @Nullable URI uri, Class> interfaceImplemented, String methodName,
+ String configKey) throws IOException {
+
+ String className = instanceClass.getName();
try {
- // new X(uri, conf)
- Constructor cons = getConstructor(credClass, URI.class,
- Configuration.class);
- if (cons != null) {
- credentials = (AWSCredentialsProvider)cons.newInstance(uri, conf);
- return credentials;
- }
- // new X(conf)
- cons = getConstructor(credClass, Configuration.class);
- if (cons != null) {
- credentials = (AWSCredentialsProvider)cons.newInstance(conf);
- return credentials;
+ Constructor cons = null;
+ if (conf != null) {
+ // new X(uri, conf)
+ cons = getConstructor(instanceClass, URI.class, Configuration.class);
+
+ if (cons != null) {
+ return (InstanceT) cons.newInstance(uri, conf);
+ }
+ // new X(conf)
+ cons = getConstructor(instanceClass, Configuration.class);
+ if (cons != null) {
+ return (InstanceT) cons.newInstance(conf);
+ }
}
- // X.getInstance()
- Method factory = getFactoryMethod(credClass, AWSCredentialsProvider.class,
- "getInstance");
+ // X.methodName()
+ Method factory = getFactoryMethod(instanceClass, interfaceImplemented, methodName);
if (factory != null) {
- credentials = (AWSCredentialsProvider)factory.invoke(null);
- return credentials;
+ return (InstanceT) factory.invoke(null);
}
// new X()
- cons = getConstructor(credClass);
+ cons = getConstructor(instanceClass);
if (cons != null) {
- credentials = (AWSCredentialsProvider)cons.newInstance();
- return credentials;
+ return (InstanceT) cons.newInstance();
}
// no supported constructor or factory method found
throw new IOException(String.format("%s " + CONSTRUCTOR_EXCEPTION
+ ". A class specified in %s must provide a public constructor "
+ "of a supported signature, or a public factory method named "
- + "getInstance that accepts no arguments.",
- className, AWS_CREDENTIALS_PROVIDER));
+ + "create that accepts no arguments.", className, configKey));
} catch (InvocationTargetException e) {
Throwable targetException = e.getTargetException();
if (targetException == null) {
- targetException = e;
+ targetException = e;
}
if (targetException instanceof IOException) {
throw (IOException) targetException;
} else if (targetException instanceof SdkException) {
- throw translateException("Instantiate " + className, "",
- (SdkException) targetException);
+ throw translateException("Instantiate " + className, "", (SdkException) targetException);
} else {
// supported constructor or factory method found, but the call failed
- throw new IOException(className + " " + INSTANTIATION_EXCEPTION
- + ": " + targetException,
+ throw new IOException(className + " " + INSTANTIATION_EXCEPTION + ": " + targetException,
targetException);
}
} catch (ReflectiveOperationException | IllegalArgumentException e) {
// supported constructor or factory method found, but the call failed
- throw new IOException(className + " " + INSTANTIATION_EXCEPTION
- + ": " + e,
- e);
- }
- }
-
- /**
- * Create an AWS credential provider from its class by using reflection. The
- * class must implement one of the following means of construction, which are
- * attempted in order:
- *
- *
- * - a public constructor accepting java.net.URI and
- * org.apache.hadoop.conf.Configuration
- * - a public constructor accepting
- * org.apache.hadoop.conf.Configuration
- * - a public static method named getInstance that accepts no
- * arguments and returns an instance of
- * software.amazon.awssdk.auth.credentials.AwsCredentialsProvider, or
- * - a public default constructor.
- *
- *
- * @param conf configuration
- * @param credClass credential class
- * @param uri URI of the FS
- * @return the instantiated class
- * @throws IOException on any instantiation failure.
- */
- private static AwsCredentialsProvider createAWSV2CredentialProvider(
- Configuration conf,
- Class> credClass,
- @Nullable URI uri) throws IOException {
- AwsCredentialsProvider credentials = null;
- String className = credClass.getName();
- if (!AwsCredentialsProvider.class.isAssignableFrom(credClass)) {
- throw new IOException("Class " + credClass + " " + NOT_AWS_V2_PROVIDER);
- }
- if (Modifier.isAbstract(credClass.getModifiers())) {
- throw new IOException("Class " + credClass + " " + ABSTRACT_PROVIDER);
- }
- LOG.debug("Credential provider class is {}", className);
-
- try {
- // new X(uri, conf)
- Constructor cons = getConstructor(credClass, URI.class,
- Configuration.class);
- if (cons != null) {
- credentials = (AwsCredentialsProvider)cons.newInstance(uri, conf);
- return credentials;
- }
- // new X(conf)
- cons = getConstructor(credClass, Configuration.class);
- if (cons != null) {
- credentials = (AwsCredentialsProvider)cons.newInstance(conf);
- return credentials;
- }
-
- // X.getInstance()
- Method factory = getFactoryMethod(credClass, AwsCredentialsProvider.class,
- "create");
- if (factory != null) {
- credentials = (AwsCredentialsProvider)factory.invoke(null);
- return credentials;
- }
-
- // new X()
- cons = getConstructor(credClass);
- if (cons != null) {
- credentials = (AwsCredentialsProvider)cons.newInstance();
- return credentials;
- }
-
- // no supported constructor or factory method found
- throw new IOException(String.format("%s " + CONSTRUCTOR_EXCEPTION
- + ". A class specified in %s must provide a public constructor "
- + "of a supported signature, or a public factory method named "
- + "create that accepts no arguments.",
- className, AWS_CREDENTIALS_PROVIDER));
- } catch (InvocationTargetException e) {
- // TODO: Can probably be moved to a common method, but before doing this, check if we still
- // want to extend V2 providers the same way v1 providers are.
- Throwable targetException = e.getTargetException();
- if (targetException == null) {
- targetException = e;
- }
- if (targetException instanceof IOException) {
- throw (IOException) targetException;
- } else if (targetException instanceof SdkException) {
- throw translateException("Instantiate " + className, "",
- (SdkException) targetException);
- } else {
- // supported constructor or factory method found, but the call failed
- throw new IOException(className + " " + INSTANTIATION_EXCEPTION
- + ": " + targetException,
- targetException);
- }
- } catch (ReflectiveOperationException | IllegalArgumentException e) {
- // supported constructor or factory method found, but the call failed
- throw new IOException(className + " " + INSTANTIATION_EXCEPTION
- + ": " + e,
- e);
+ throw new IOException(className + " " + INSTANTIATION_EXCEPTION + ": " + e, e);
}
}
@@ -1317,216 +1049,6 @@ public static void deleteWithWarning(FileSystem fs,
}
}
- /**
- * Create a new AWS {@code ClientConfiguration}.
- * All clients to AWS services MUST use this for consistent setup
- * of connectivity, UA, proxy settings.
- * @param conf The Hadoop configuration
- * @param bucket Optional bucket to use to look up per-bucket proxy secrets
- * @return new AWS client configuration
- * @throws IOException problem creating AWS client configuration
- *
- * @deprecated use {@link #createAwsConf(Configuration, String, String)}
- */
- @Deprecated
- public static ClientConfiguration createAwsConf(Configuration conf,
- String bucket)
- throws IOException {
- return createAwsConf(conf, bucket, null);
- }
-
- /**
- * Create a new AWS {@code ClientConfiguration}. All clients to AWS services
- * MUST use this or the equivalents for the specific service for
- * consistent setup of connectivity, UA, proxy settings.
- *
- * @param conf The Hadoop configuration
- * @param bucket Optional bucket to use to look up per-bucket proxy secrets
- * @param awsServiceIdentifier a string representing the AWS service (S3,
- * etc) for which the ClientConfiguration is being created.
- * @return new AWS client configuration
- * @throws IOException problem creating AWS client configuration
- */
- public static ClientConfiguration createAwsConf(Configuration conf,
- String bucket, String awsServiceIdentifier)
- throws IOException {
- final ClientConfiguration awsConf = new ClientConfiguration();
- initConnectionSettings(conf, awsConf);
- initProxySupport(conf, bucket, awsConf);
- initUserAgent(conf, awsConf);
- if (StringUtils.isNotEmpty(awsServiceIdentifier)) {
- String configKey = null;
- switch (awsServiceIdentifier) {
- case AWS_SERVICE_IDENTIFIER_S3:
- configKey = SIGNING_ALGORITHM_S3;
- break;
- case AWS_SERVICE_IDENTIFIER_STS:
- configKey = SIGNING_ALGORITHM_STS;
- break;
- default:
- // Nothing to do. The original signer override is already setup
- }
- if (configKey != null) {
- String signerOverride = conf.getTrimmed(configKey, "");
- if (!signerOverride.isEmpty()) {
- LOG.debug("Signer override for {}} = {}", awsServiceIdentifier,
- signerOverride);
- awsConf.setSignerOverride(signerOverride);
- }
- }
- }
- return awsConf;
- }
-
- /**
- * Initializes all AWS SDK settings related to connection management.
- *
- * @param conf Hadoop configuration
- * @param awsConf AWS SDK configuration
- *
- * @throws IOException if there was an error initializing the protocol
- * settings
- */
- public static void initConnectionSettings(Configuration conf,
- ClientConfiguration awsConf) throws IOException {
- awsConf.setMaxConnections(intOption(conf, MAXIMUM_CONNECTIONS,
- DEFAULT_MAXIMUM_CONNECTIONS, 1));
- initProtocolSettings(conf, awsConf);
- awsConf.setMaxErrorRetry(intOption(conf, MAX_ERROR_RETRIES,
- DEFAULT_MAX_ERROR_RETRIES, 0));
- awsConf.setConnectionTimeout(intOption(conf, ESTABLISH_TIMEOUT,
- DEFAULT_ESTABLISH_TIMEOUT, 0));
- awsConf.setSocketTimeout(intOption(conf, SOCKET_TIMEOUT,
- DEFAULT_SOCKET_TIMEOUT, 0));
- int sockSendBuffer = intOption(conf, SOCKET_SEND_BUFFER,
- DEFAULT_SOCKET_SEND_BUFFER, 2048);
- int sockRecvBuffer = intOption(conf, SOCKET_RECV_BUFFER,
- DEFAULT_SOCKET_RECV_BUFFER, 2048);
- long requestTimeoutMillis = conf.getTimeDuration(REQUEST_TIMEOUT,
- DEFAULT_REQUEST_TIMEOUT, TimeUnit.SECONDS, TimeUnit.MILLISECONDS);
-
- if (requestTimeoutMillis > Integer.MAX_VALUE) {
- LOG.debug("Request timeout is too high({} ms). Setting to {} ms instead",
- requestTimeoutMillis, Integer.MAX_VALUE);
- requestTimeoutMillis = Integer.MAX_VALUE;
- }
- awsConf.setRequestTimeout((int) requestTimeoutMillis);
- awsConf.setSocketBufferSizeHints(sockSendBuffer, sockRecvBuffer);
- String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, "");
- if (!signerOverride.isEmpty()) {
- LOG.debug("Signer override = {}", signerOverride);
- awsConf.setSignerOverride(signerOverride);
- }
- }
-
- /**
- * Initializes the connection protocol settings when connecting to S3 (e.g.
- * either HTTP or HTTPS). If secure connections are enabled, this method
- * will load the configured SSL providers.
- *
- * @param conf Hadoop configuration
- * @param awsConf AWS SDK configuration
- *
- * @throws IOException if there is an error initializing the configured
- * {@link javax.net.ssl.SSLSocketFactory}
- */
- private static void initProtocolSettings(Configuration conf,
- ClientConfiguration awsConf) throws IOException {
- boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS,
- DEFAULT_SECURE_CONNECTIONS);
- awsConf.setProtocol(secureConnections ? Protocol.HTTPS : Protocol.HTTP);
- if (secureConnections) {
- NetworkBinding.bindSSLChannelMode(conf, awsConf);
- }
- }
-
- /**
- * Initializes AWS SDK proxy support in the AWS client configuration
- * if the S3A settings enable it.
- *
- * Note: LimitedPrivate to provide proxy support in ranger repository.
- *
- * @param conf Hadoop configuration
- * @param bucket Optional bucket to use to look up per-bucket proxy secrets
- * @param awsConf AWS SDK configuration to update
- * @throws IllegalArgumentException if misconfigured
- * @throws IOException problem getting username/secret from password source.
- */
- @InterfaceAudience.LimitedPrivate("Ranger")
- public static void initProxySupport(Configuration conf,
- String bucket,
- ClientConfiguration awsConf) throws IllegalArgumentException,
- IOException {
- String proxyHost = conf.getTrimmed(PROXY_HOST, "");
- int proxyPort = conf.getInt(PROXY_PORT, -1);
- if (!proxyHost.isEmpty()) {
- awsConf.setProxyHost(proxyHost);
- if (proxyPort >= 0) {
- awsConf.setProxyPort(proxyPort);
- } else {
- if (conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS)) {
- LOG.warn("Proxy host set without port. Using HTTPS default 443");
- awsConf.setProxyPort(443);
- } else {
- LOG.warn("Proxy host set without port. Using HTTP default 80");
- awsConf.setProxyPort(80);
- }
- }
- final String proxyUsername = lookupPassword(bucket, conf, PROXY_USERNAME,
- null, null);
- final String proxyPassword = lookupPassword(bucket, conf, PROXY_PASSWORD,
- null, null);
- if ((proxyUsername == null) != (proxyPassword == null)) {
- String msg = "Proxy error: " + PROXY_USERNAME + " or " +
- PROXY_PASSWORD + " set without the other.";
- LOG.error(msg);
- throw new IllegalArgumentException(msg);
- }
- boolean isProxySecured = conf.getBoolean(PROXY_SECURED, false);
- awsConf.setProxyUsername(proxyUsername);
- awsConf.setProxyPassword(proxyPassword);
- awsConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN));
- awsConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION));
- awsConf.setProxyProtocol(isProxySecured ? Protocol.HTTPS : Protocol.HTTP);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Using proxy server {}://{}:{} as user {} with password {} "
- + "on domain {} as workstation {}",
- awsConf.getProxyProtocol(),
- awsConf.getProxyHost(),
- awsConf.getProxyPort(),
- String.valueOf(awsConf.getProxyUsername()),
- awsConf.getProxyPassword(), awsConf.getProxyDomain(),
- awsConf.getProxyWorkstation());
- }
- } else if (proxyPort >= 0) {
- String msg =
- "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST;
- LOG.error(msg);
- throw new IllegalArgumentException(msg);
- }
- }
-
- /**
- * Initializes the User-Agent header to send in HTTP requests to AWS
- * services. We always include the Hadoop version number. The user also
- * may set an optional custom prefix to put in front of the Hadoop version
- * number. The AWS SDK internally appends its own information, which seems
- * to include the AWS SDK version, OS and JVM version.
- *
- * @param conf Hadoop configuration
- * @param awsConf AWS SDK configuration to update
- */
- private static void initUserAgent(Configuration conf,
- ClientConfiguration awsConf) {
- String userAgent = "Hadoop " + VersionInfo.getVersion();
- String userAgentPrefix = conf.getTrimmed(USER_AGENT_PREFIX, "");
- if (!userAgentPrefix.isEmpty()) {
- userAgent = userAgentPrefix + ", " + userAgent;
- }
- LOG.debug("Using User-Agent: {}", userAgent);
- awsConf.setUserAgentPrefix(userAgent);
- }
-
/**
* Convert the data of an iterator of {@link S3AFileStatus} to
* an array.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
index c046f1e12633d..fa2c0769d26a3 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java
@@ -26,10 +26,9 @@
import java.util.Map;
import java.util.concurrent.Executor;
-import com.amazonaws.monitoring.MonitoringListener;
-import com.amazonaws.services.s3.AmazonS3;
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
+import software.amazon.awssdk.regions.Region;
import software.amazon.awssdk.services.s3.S3AsyncClient;
import software.amazon.awssdk.services.s3.S3Client;
import software.amazon.awssdk.transfer.s3.S3TransferManager;
@@ -41,7 +40,7 @@
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ENDPOINT;
/**
- * Factory for creation of {@link AmazonS3} client instances.
+ * Factory for creation of {@link S3Client} client instances.
* Important: HBase's HBoss module implements this interface in its
* tests.
* Take care when updating this interface to ensure that a client
@@ -55,17 +54,6 @@
@InterfaceStability.Evolving
public interface S3ClientFactory {
- /**
- * Creates a new {@link AmazonS3} client.
- *
- * @param uri S3A file system URI
- * @param parameters parameter object
- * @return S3 client
- * @throws IOException IO problem
- */
- AmazonS3 createS3Client(URI uri,
- S3ClientCreationParameters parameters) throws IOException;
-
/**
* Creates a new {@link S3Client}.
* The client returned supports synchronous operations. For
@@ -77,14 +65,14 @@ AmazonS3 createS3Client(URI uri,
* @return S3 client
* @throws IOException on any IO problem
*/
- S3Client createS3ClientV2(URI uri,
+ S3Client createS3Client(URI uri,
S3ClientCreationParameters parameters) throws IOException;
/**
* Creates a new {@link S3AsyncClient}.
* The client returned supports asynchronous operations. For
* synchronous operations, use
- * {@link #createS3ClientV2(URI, S3ClientCreationParameters)}.
+ * {@link #createS3Client(URI, S3ClientCreationParameters)}.
*
* @param uri S3A file system URI
* @param parameters parameter object
@@ -97,13 +85,10 @@ S3AsyncClient createS3AsyncClient(URI uri,
/**
* Creates a new {@link S3TransferManager}.
*
- * @param uri S3A file system URI
- * @param parameters parameter object
+ * @param s3AsyncClient the async client to be used by the TM.
* @return S3 transfer manager
- * @throws IOException on any IO problem
*/
- S3TransferManager createS3TransferManager(URI uri,
- S3ClientCreationParameters parameters) throws IOException;
+ S3TransferManager createS3TransferManager(S3AsyncClient s3AsyncClient);
/**
* Settings for the S3 Client.
@@ -128,11 +113,6 @@ final class S3ClientCreationParameters {
*/
private final Map headers = new HashMap<>();
- /**
- * Monitoring listener.
- */
- private MonitoringListener monitoringListener;
-
/**
* RequestMetricCollector metrics...if not-null will be wrapped
* with an {@code AwsStatisticsCollector} and passed to
@@ -176,6 +156,12 @@ final class S3ClientCreationParameters {
*/
private Executor transferManagerExecutor;
+ /**
+ * Region of the S3 bucket.
+ */
+ private Region region;
+
+
/**
* List of execution interceptors to include in the chain
* of interceptors in the SDK.
@@ -196,21 +182,6 @@ public S3ClientCreationParameters withExecutionInterceptors(
return this;
}
- public MonitoringListener getMonitoringListener() {
- return monitoringListener;
- }
-
- /**
- * listener for AWS monitoring events.
- * @param listener listener
- * @return this object
- */
- public S3ClientCreationParameters withMonitoringListener(
- @Nullable final MonitoringListener listener) {
- monitoringListener = listener;
- return this;
- }
-
public StatisticsFromAwsSdk getMetrics() {
return metrics;
}
@@ -384,5 +355,25 @@ public S3ClientCreationParameters withTransferManagerExecutor(
transferManagerExecutor = value;
return this;
}
+
+ /**
+ * Set region.
+ *
+ * @param value new value
+ * @return the builder
+ */
+ public S3ClientCreationParameters withRegion(
+ final Region value) {
+ region = value;
+ return this;
+ }
+
+ /**
+ * Get the region.
+ * @return invoker
+ */
+ public Region getRegion() {
+ return region;
+ }
}
}
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 651769ff283bd..ae761fe270f46 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
@@ -538,6 +538,11 @@ public enum Statistic {
StoreStatisticNames.STORE_IO_THROTTLE_RATE,
"Rate of S3 request throttling",
TYPE_QUANTILE),
+ STORE_REGION_PROBE(
+ StoreStatisticNames.STORE_REGION_PROBE,
+ "Store Region Probe",
+ TYPE_DURATION
+ ),
/*
* Delegation Token Operations.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java
index ee3a7c8a7e557..85ed69e14bc79 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.fs.s3a;
-import software.amazon.awssdk.transfer.s3.FileUpload;
+import software.amazon.awssdk.transfer.s3.model.FileUpload;
/**
* Simple struct that contains information about a S3 upload.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
index 7a452f7f25040..be1c24c985b3a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
@@ -728,6 +728,7 @@ public interface WriteOperationHelperCallbacks {
/**
* Initiates a select request.
* @param request selectObjectContent request
+ * @param t selectObjectContent request handler
* @return selectObjectContentResult
*/
CompletableFuture selectObjectContent(SelectObjectContentRequest request,
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java
index de65922c02e53..7b6934d4a335e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java
@@ -29,12 +29,12 @@
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
+import software.amazon.awssdk.services.s3.model.HeadBucketRequest;
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest;
import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
-import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest;
@@ -73,7 +73,7 @@ public interface RequestFactory {
* Get the canned ACL of this FS.
* @return an ACL, if any
*/
- ObjectCannedACL getCannedACL();
+ String getCannedACL();
/**
* Get the encryption algorithm of this endpoint.
@@ -172,12 +172,19 @@ CompleteMultipartUploadRequest.Builder newCompleteMultipartUploadRequestBuilder(
List partETags);
/**
- * Create a HEAD request builder.
+ * Create a HEAD object request builder.
* @param key key, may have trailing /
* @return the request builder.
*/
HeadObjectRequest.Builder newHeadObjectRequestBuilder(String key);
+ /**
+ * Create a HEAD bucket request builder.
+ * @param bucket bucket to get metadata for
+ * @return the request builder.
+ */
+ HeadBucketRequest.Builder newHeadBucketRequestBuilder(String bucket);
+
/**
* Create a GET request builder.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java
index aeb55d6ce2e61..3b4dbc8482d8e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/impl/LoggingAuditor.java
@@ -252,8 +252,9 @@ private class LoggingAuditSpan extends AbstractAuditSpanImpl {
private void attachRangeFromRequest(SdkHttpRequest request,
ExecutionAttributes executionAttributes) {
- if (executionAttributes.getAttribute(AwsExecutionAttribute.OPERATION_NAME)
- .equals("GetObject")) {
+ String operationName = executionAttributes.getAttribute(AwsExecutionAttribute.OPERATION_NAME);
+
+ if (operationName != null && operationName.equals("GetObject")) {
if (request.headers() != null && request.headers().get("Range") != null) {
String[] rangeHeader = request.headers().get("Range").get(0).split("=");
// only set header if range unit is bytes
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java
index e493154d85535..3517fabb9579e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java
@@ -53,7 +53,7 @@
import org.apache.hadoop.security.UserGroupInformation;
import static org.apache.hadoop.fs.s3a.Constants.*;
-import static org.apache.hadoop.fs.s3a.S3AUtils.buildAWSProviderList;
+import static org.apache.hadoop.fs.s3a.auth.AwsCredentialListProvider.buildAWSProviderList;
/**
* Support IAM Assumed roles by instantiating an instance of
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AwsCredentialListProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AwsCredentialListProvider.java
new file mode 100644
index 0000000000000..d94f8c25bad6b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AwsCredentialListProvider.java
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.auth;
+
+import java.io.IOException;
+import java.lang.reflect.Modifier;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.fs.s3a.S3AUtils;
+import org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider;
+import org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
+
+import static org.apache.hadoop.fs.s3a.Constants.AWS_AUTH_CLASS_PREFIX;
+import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
+
+/**
+ * This class provides methods to create the list of AWS credential providers.
+ */
+public final class AwsCredentialListProvider {
+
+ private AwsCredentialListProvider() {
+ }
+
+ private static final Logger LOG = LoggerFactory.getLogger(AwsCredentialListProvider.class);
+
+ public static final String NOT_AWS_PROVIDER =
+ "does not implement AWSCredentialsProvider";
+ public static final String NOT_AWS_V2_PROVIDER =
+ "does not implement AwsCredentialsProvider";
+ public static final String ABSTRACT_PROVIDER =
+ "is abstract and therefore cannot be created";
+
+ /**
+ * Error message when the AWS provider list built up contains a forbidden
+ * entry.
+ */
+ @VisibleForTesting
+ public static final String E_FORBIDDEN_AWS_PROVIDER
+ = "AWS provider class cannot be used";
+
+ /**
+ * The standard AWS provider list for AWS connections.
+ */
+ public static final List>
+ STANDARD_AWS_PROVIDERS = Collections.unmodifiableList(
+ Arrays.asList(
+ TemporaryAWSCredentialsProvider.class,
+ SimpleAWSCredentialsProvider.class,
+ EnvironmentVariableCredentialsProvider.class,
+ IAMInstanceCredentialsProvider.class));
+
+ /**
+ * Create the AWS credentials from the providers, the URI and
+ * the key {@link Constants#AWS_CREDENTIALS_PROVIDER} in the configuration.
+ * @param binding Binding URI -may be null
+ * @param conf filesystem configuration
+ * @return a credentials provider list
+ * @throws IOException Problems loading the providers (including reading
+ * secrets from credential files).
+ */
+ public static AWSCredentialProviderList createAWSCredentialProviderSet(
+ @Nullable URI binding,
+ Configuration conf) throws IOException {
+ // this will reject any user:secret entries in the URI
+ S3xLoginHelper.rejectSecretsInURIs(binding);
+ AWSCredentialProviderList credentials =
+ buildAWSProviderList(binding,
+ conf,
+ AWS_CREDENTIALS_PROVIDER,
+ STANDARD_AWS_PROVIDERS,
+ new HashSet<>());
+ // make sure the logging message strips out any auth details
+ LOG.debug("For URI {}, using credentials {}",
+ binding, credentials);
+ return credentials;
+ }
+
+ /**
+ * Load list of AWS credential provider/credential provider factory classes.
+ * @param conf configuration
+ * @param key key
+ * @param defaultValue list of default values
+ * @return the list of classes, possibly empty
+ * @throws IOException on a failure to load the list.
+ */
+ private static List> loadAWSProviderClasses(Configuration conf,
+ String key,
+ Class>... defaultValue) throws IOException {
+ try {
+ return Arrays.asList(conf.getClasses(key, defaultValue));
+ } catch (RuntimeException e) {
+ Throwable c = e.getCause() != null ? e.getCause() : e;
+ throw new IOException("From option " + key + ' ' + c, c);
+ }
+ }
+
+ /**
+ * Maps V1 credential providers to either their equivalent SDK V2 class or hadoop provider.
+ */
+ private static Map initCredentialProvidersMap() {
+ Map v1v2CredentialProviderMap = new HashMap<>();
+
+ v1v2CredentialProviderMap.put("EnvironmentVariableCredentialsProvider",
+ EnvironmentVariableCredentialsProvider.class);
+ v1v2CredentialProviderMap.put("EC2ContainerCredentialsProviderWrapper",
+ IAMInstanceCredentialsProvider.class);
+ v1v2CredentialProviderMap.put("InstanceProfileCredentialsProvider",
+ IAMInstanceCredentialsProvider.class);
+
+ return v1v2CredentialProviderMap;
+ }
+
+ /**
+ * Load list of AWS credential provider/credential provider factory classes;
+ * support a forbidden list to prevent loops, mandate full secrets, etc.
+ * @param binding Binding URI -may be null
+ * @param conf configuration
+ * @param key key
+ * @param forbidden a possibly empty set of forbidden classes.
+ * @param defaultValues list of default providers.
+ * @return the list of classes, possibly empty
+ * @throws IOException on a failure to load the list.
+ */
+ public static AWSCredentialProviderList buildAWSProviderList(
+ @Nullable final URI binding,
+ final Configuration conf,
+ final String key,
+ final List> defaultValues,
+ final Set> forbidden) throws IOException {
+
+ // build up the base provider
+ List> awsClasses = loadAWSProviderClasses(conf,
+ key,
+ defaultValues.toArray(new Class[defaultValues.size()]));
+
+ Map v1v2CredentialProviderMap = initCredentialProvidersMap();
+ // and if the list is empty, switch back to the defaults.
+ // this is to address the issue that configuration.getClasses()
+ // doesn't return the default if the config value is just whitespace.
+ if (awsClasses.isEmpty()) {
+ awsClasses = defaultValues;
+ }
+ // iterate through, checking for blacklists and then instantiating
+ // each provider
+ AWSCredentialProviderList providers = new AWSCredentialProviderList();
+ for (Class> aClass : awsClasses) {
+
+ if (forbidden.contains(aClass)) {
+ throw new IOException(E_FORBIDDEN_AWS_PROVIDER
+ + " in option " + key + ": " + aClass);
+ }
+
+ if (v1v2CredentialProviderMap.containsKey(aClass.getSimpleName()) &&
+ aClass.getName().contains(AWS_AUTH_CLASS_PREFIX)){
+ providers.add(createAWSV2CredentialProvider(conf,
+ v1v2CredentialProviderMap.get(aClass.getSimpleName()), binding));
+ } else if (AWSCredentialsProvider.class.isAssignableFrom(aClass)) {
+ providers.add(createAWSV1CredentialProvider(conf,
+ aClass, binding));
+ } else {
+ providers.add(createAWSV2CredentialProvider(conf, aClass, binding));
+ }
+
+ }
+ return providers;
+ }
+
+ /**
+ * Create an AWS credential provider from its class by using reflection. The
+ * class must implement one of the following means of construction, which are
+ * attempted in order:
+ *
+ *
+ * - a public constructor accepting java.net.URI and
+ * org.apache.hadoop.conf.Configuration
+ * - a public constructor accepting
+ * org.apache.hadoop.conf.Configuration
+ * - a public static method named getInstance that accepts no
+ * arguments and returns an instance of
+ * com.amazonaws.auth.AWSCredentialsProvider, or
+ * - a public default constructor.
+ *
+ *
+ * @param conf configuration
+ * @param credClass credential class
+ * @param uri URI of the FS
+ * @return the instantiated class
+ * @throws IOException on any instantiation failure.
+ */
+ private static AWSCredentialsProvider createAWSV1CredentialProvider(Configuration conf,
+ Class> credClass, @Nullable URI uri) throws IOException {
+ AWSCredentialsProvider credentials = null;
+ String className = credClass.getName();
+ if (!AWSCredentialsProvider.class.isAssignableFrom(credClass)) {
+ throw new IOException("Class " + credClass + " " + NOT_AWS_PROVIDER);
+ }
+ if (Modifier.isAbstract(credClass.getModifiers())) {
+ throw new IOException("Class " + credClass + " " + ABSTRACT_PROVIDER);
+ }
+ LOG.debug("Credential provider class is {}", className);
+
+ credentials =
+ S3AUtils.getInstanceFromReflection(credClass, conf, uri, AWSCredentialsProvider.class,
+ "getInstance", AWS_CREDENTIALS_PROVIDER);
+ return credentials;
+
+ }
+
+ /**
+ * Create an AWS credential provider from its class by using reflection. The
+ * class must implement one of the following means of construction, which are
+ * attempted in order:
+ *
+ *
+ * - a public constructor accepting java.net.URI and
+ * org.apache.hadoop.conf.Configuration
+ * - a public constructor accepting
+ * org.apache.hadoop.conf.Configuration
+ * - a public static method named getInstance that accepts no
+ * arguments and returns an instance of
+ * software.amazon.awssdk.auth.credentials.AwsCredentialsProvider, or
+ * - a public default constructor.
+ *
+ *
+ * @param conf configuration
+ * @param credClass credential class
+ * @param uri URI of the FS
+ * @return the instantiated class
+ * @throws IOException on any instantiation failure.
+ */
+ private static AwsCredentialsProvider createAWSV2CredentialProvider(Configuration conf,
+ Class> credClass, @Nullable URI uri) throws IOException {
+ AwsCredentialsProvider credentials = null;
+ String className = credClass.getName();
+ if (!AwsCredentialsProvider.class.isAssignableFrom(credClass)) {
+ throw new IOException("Class " + credClass + " " + NOT_AWS_V2_PROVIDER);
+ }
+ if (Modifier.isAbstract(credClass.getModifiers())) {
+ throw new IOException("Class " + credClass + " " + ABSTRACT_PROVIDER);
+ }
+ LOG.debug("Credential provider class is {}", className);
+ credentials =
+ S3AUtils.getInstanceFromReflection(credClass, conf, uri, AwsCredentialsProvider.class,
+ "create", AWS_CREDENTIALS_PROVIDER);
+ return credentials;
+ }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java
index 62f6ea845e6bf..dcfc0a7ee8416 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java
@@ -51,6 +51,7 @@
import static org.apache.commons.lang3.StringUtils.isEmpty;
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
+import static org.apache.hadoop.fs.s3a.Constants.AWS_SERVICE_IDENTIFIER_STS;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.*;
/**
@@ -117,7 +118,9 @@ public static StsClientBuilder builder(
* @param credentials AWS credential chain to use
* @param stsEndpoint optional endpoint "https://sns.us-west-1.amazonaws.com"
* @param stsRegion the region, e.g "us-west-1". Must be set if endpoint is.
+ * @param bucket bucket name
* @return the builder to call {@code build()}
+ * @throws IOException problem reading proxy secrets
*/
public static StsClientBuilder builder(final AwsCredentialsProvider credentials,
final Configuration conf, final String stsEndpoint, final String stsRegion,
@@ -127,7 +130,7 @@ public static StsClientBuilder builder(final AwsCredentialsProvider credentials,
Preconditions.checkArgument(credentials != null, "No credentials");
final ClientOverrideConfiguration.Builder clientOverrideConfigBuilder =
- AWSClientConfig.createClientConfigBuilder(conf);
+ AWSClientConfig.createClientConfigBuilder(conf, AWS_SERVICE_IDENTIFIER_STS);
final ApacheHttpClient.Builder httpClientBuilder =
AWSClientConfig.createHttpClientBuilder(conf);
@@ -143,7 +146,6 @@ public static StsClientBuilder builder(final AwsCredentialsProvider credentials,
.overrideConfiguration(clientOverrideConfigBuilder.build())
.credentialsProvider(credentials);
- // TODO: SIGNERS NOT ADDED YET.
boolean destIsStandardEndpoint = STS_STANDARD.equals(stsEndpoint);
if (isNotEmpty(stsEndpoint) && !destIsStandardEndpoint) {
Preconditions.checkArgument(isNotEmpty(stsRegion),
@@ -165,8 +167,6 @@ public static StsClientBuilder builder(final AwsCredentialsProvider credentials,
*/
private static URI getSTSEndpoint(String endpoint) {
try {
- // TODO: The URI builder is currently imported via a shaded dependency. This is due to TM
- // preview dependency causing some issues.
return new URIBuilder().setScheme("https").setHost(endpoint).build();
} catch (URISyntaxException e) {
throw new IllegalArgumentException(e);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerFactory.java
new file mode 100644
index 0000000000000..7beabb9fa3c84
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerFactory.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.auth;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.signer.Aws4Signer;
+import software.amazon.awssdk.auth.signer.Aws4UnsignedPayloadSigner;
+import software.amazon.awssdk.auth.signer.AwsS3V4Signer;
+import software.amazon.awssdk.core.signer.NoOpSigner;
+import software.amazon.awssdk.core.signer.Signer;
+
+import org.apache.hadoop.fs.s3a.S3AUtils;
+
+
+/**
+ * Signer factory used to register and create signers.
+ */
+public final class SignerFactory {
+
+ private static final Logger LOG = LoggerFactory.getLogger(SignerFactory.class);
+ public static final String VERSION_FOUR_SIGNER = "AWS4SignerType";
+ public static final String VERSION_FOUR_UNSIGNED_PAYLOAD_SIGNER = "AWS4UnsignedPayloadSignerType";
+ public static final String NO_OP_SIGNER = "NoOpSignerType";
+ private static final String S3_V4_SIGNER = "AWSS3V4SignerType";
+
+ private static final Map> SIGNERS
+ = new ConcurrentHashMap<>();
+
+ static {
+ // Register the standard signer types.
+ SIGNERS.put(VERSION_FOUR_SIGNER, Aws4Signer.class);
+ SIGNERS.put(VERSION_FOUR_UNSIGNED_PAYLOAD_SIGNER, Aws4UnsignedPayloadSigner.class);
+ SIGNERS.put(NO_OP_SIGNER, NoOpSigner.class);
+ SIGNERS.put(S3_V4_SIGNER, AwsS3V4Signer.class);
+ }
+
+
+ private SignerFactory() {
+ }
+
+ /**
+ * Register an implementation class for the given signer type.
+ *
+ * @param signerType The name of the signer type to register.
+ * @param signerClass The class implementing the given signature protocol.
+ */
+ public static void registerSigner(
+ final String signerType,
+ final Class extends Signer> signerClass) {
+
+ if (signerType == null) {
+ throw new IllegalArgumentException("signerType cannot be null");
+ }
+ if (signerClass == null) {
+ throw new IllegalArgumentException("signerClass cannot be null");
+ }
+
+ SIGNERS.put(signerType, signerClass);
+ }
+
+ /**
+ * Check if the signer has already been registered.
+ * @param signerType signer to get
+ */
+ public static void verifySignerRegistered(String signerType) {
+ Class extends Signer> signerClass = SIGNERS.get(signerType);
+ if (signerClass == null) {
+ throw new IllegalArgumentException("unknown signer type: " + signerType);
+ }
+ }
+
+
+ /**
+ * Create an instance of the given signer.
+ *
+ * @param signerType The signer type.
+ * @param configKey Config key used to configure the signer.
+ * @return The new signer instance.
+ * @throws IOException on any problem.
+ */
+ public static Signer createSigner(String signerType, String configKey) throws IOException {
+ Class> signerClass = SIGNERS.get(signerType);
+ String className = signerClass.getName();
+
+ LOG.debug("Signer class is {}", className);
+
+ Signer signer =
+ S3AUtils.getInstanceFromReflection(signerClass, null, null, Signer.class, "create",
+ configKey);
+
+ return signer;
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerManager.java
index e162428787cc4..198bc66133a95 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerManager.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/SignerManager.java
@@ -22,14 +22,12 @@
import java.util.LinkedList;
import java.util.List;
-import com.amazonaws.auth.Signer;
-import com.amazonaws.auth.SignerFactory;
+import software.amazon.awssdk.core.signer.Signer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.auth.delegation.DelegationTokenProvider;
-import org.apache.hadoop.fs.s3a.impl.V2Migration;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.ReflectionUtils;
@@ -71,8 +69,6 @@ public void initCustomSigners() {
return;
}
- V2Migration.v1CustomSignerUsed();
-
for (String customSigner : customSigners) {
String[] parts = customSigner.split(":");
if (!(parts.length == 1 || parts.length == 2 || parts.length == 3)) {
@@ -119,7 +115,7 @@ public void initCustomSigners() {
private static void maybeRegisterSigner(String signerName,
String signerClassName, Configuration conf) {
try {
- SignerFactory.getSignerByTypeAndService(signerName, null);
+ SignerFactory.verifySignerRegistered(signerName);
} catch (IllegalArgumentException e) {
// Signer is not registered with the AWS SDK.
// Load the class and register the signer.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java
index 0a73411156b6d..bfb7e6966457b 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java
@@ -42,6 +42,7 @@
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.DelegationTokenIssuer;
import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.service.ServiceOperations;
import org.apache.hadoop.util.DurationInfo;
import static org.apache.hadoop.fs.s3a.Statistic.DELEGATION_TOKENS_ISSUED;
@@ -227,10 +228,7 @@ protected void serviceStop() throws Exception {
try {
super.serviceStop();
} finally {
- // TODO: Importing the transfer manager preview outside of the bundle causes some
- // issues. Over here, it can no longer find the LOG. We expect this to be fixed with the
- // release of the TM.
- // ServiceOperations.stopQuietly(LOG, tokenBinding);
+ ServiceOperations.stopQuietly(LOG, tokenBinding);
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java
index 440f5305af2c2..4b9fd517b264b 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java
@@ -47,8 +47,8 @@
import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
import static org.apache.hadoop.fs.s3a.Invoker.once;
-import static org.apache.hadoop.fs.s3a.S3AUtils.STANDARD_AWS_PROVIDERS;
-import static org.apache.hadoop.fs.s3a.S3AUtils.buildAWSProviderList;
+import static org.apache.hadoop.fs.s3a.auth.AwsCredentialListProvider.STANDARD_AWS_PROVIDERS;
+import static org.apache.hadoop.fs.s3a.auth.AwsCredentialListProvider.buildAWSProviderList;
import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.fromAWSCredentials;
import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.fromSTSCredentials;
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.*;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSCannedACL.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSCannedACL.java
index 2f52f3ae9a832..12a89d50f6b0d 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSCannedACL.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/AWSCannedACL.java
@@ -22,13 +22,14 @@
* Enum to map AWS SDK V1 Acl values to SDK V2.
*/
public enum AWSCannedACL {
- Private("PRIVATE"),
- PublicRead("PUBLIC_READ"),
- PublicReadWrite("PUBLIC_READ_WRITE"),
- AuthenticatedRead("AUTHENTICATED_READ"),
- AwsExecRead("AWS_EXEC_READ"),
- BucketOwnerRead("BUCKET_OWNER_READ"),
- BucketOwnerFullControl("BUCKET_OWNER_FULL_CONTROL");
+ Private("private"),
+ PublicRead("public-read"),
+ PublicReadWrite("public-read-write"),
+ AuthenticatedRead("authenticated-read"),
+ AwsExecRead("aws-exec-read"),
+ BucketOwnerRead("bucket-owner-read"),
+ BucketOwnerFullControl("bucket-owner-full-control"),
+ LogDeliveryWrite("log-delivery-write");
private final String value;
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 55fb1132bb0e7..a69907755eeac 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
@@ -32,19 +32,19 @@
import software.amazon.awssdk.http.apache.ApacheHttpClient;
import software.amazon.awssdk.http.apache.ProxyConfiguration;
import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient;
-// TODO: Update to use the non shaded dependency. There is an issue with the preview version of TM
-// which is preventing this, should be resolve with the TM release.
-import software.amazon.awssdk.thirdparty.org.apache.http.client.utils.URIBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.S3AUtils;
+import org.apache.hadoop.fs.s3a.auth.SignerFactory;
import org.apache.hadoop.util.VersionInfo;
+import org.apache.http.client.utils.URIBuilder;
+import static org.apache.hadoop.fs.s3a.Constants.AWS_SERVICE_IDENTIFIER_S3;
+import static org.apache.hadoop.fs.s3a.Constants.AWS_SERVICE_IDENTIFIER_STS;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ESTABLISH_TIMEOUT;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAXIMUM_CONNECTIONS;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAX_ERROR_RETRIES;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_REQUEST_TIMEOUT;
-import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_SECURE_CONNECTIONS;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_SOCKET_TIMEOUT;
import static org.apache.hadoop.fs.s3a.Constants.ESTABLISH_TIMEOUT;
import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS;
@@ -53,10 +53,13 @@
import static org.apache.hadoop.fs.s3a.Constants.PROXY_HOST;
import static org.apache.hadoop.fs.s3a.Constants.PROXY_PASSWORD;
import static org.apache.hadoop.fs.s3a.Constants.PROXY_PORT;
+import static org.apache.hadoop.fs.s3a.Constants.PROXY_SECURED;
import static org.apache.hadoop.fs.s3a.Constants.PROXY_USERNAME;
import static org.apache.hadoop.fs.s3a.Constants.PROXY_WORKSTATION;
import static org.apache.hadoop.fs.s3a.Constants.REQUEST_TIMEOUT;
-import static org.apache.hadoop.fs.s3a.Constants.SECURE_CONNECTIONS;
+import static org.apache.hadoop.fs.s3a.Constants.SIGNING_ALGORITHM;
+import static org.apache.hadoop.fs.s3a.Constants.SIGNING_ALGORITHM_S3;
+import static org.apache.hadoop.fs.s3a.Constants.SIGNING_ALGORITHM_STS;
import static org.apache.hadoop.fs.s3a.Constants.SOCKET_TIMEOUT;
import static org.apache.hadoop.fs.s3a.Constants.USER_AGENT_PREFIX;
@@ -71,7 +74,8 @@ public final class AWSClientConfig {
private AWSClientConfig() {
}
- public static ClientOverrideConfiguration.Builder createClientConfigBuilder(Configuration conf) {
+ public static ClientOverrideConfiguration.Builder createClientConfigBuilder(Configuration conf,
+ String awsServiceIdentifier) throws IOException {
ClientOverrideConfiguration.Builder overrideConfigBuilder =
ClientOverrideConfiguration.builder();
@@ -79,12 +83,14 @@ public static ClientOverrideConfiguration.Builder createClientConfigBuilder(Conf
initUserAgent(conf, overrideConfigBuilder);
- // TODO: Look at signers. See issue https://github.com/aws/aws-sdk-java-v2/issues/1024
- // String signerOverride = conf.getTrimmed(SIGNING_ALGORITHM, "");
- // if (!signerOverride.isEmpty()) {
- // LOG.debug("Signer override = {}", signerOverride);
- // overrideConfigBuilder.putAdvancedOption(SdkAdvancedClientOption.SIGNER)
- // }
+ String signer = conf.getTrimmed(SIGNING_ALGORITHM, "");
+ if (!signer.isEmpty()) {
+ LOG.debug("Signer override = {}", signer);
+ overrideConfigBuilder.putAdvancedOption(SdkAdvancedClientOption.SIGNER,
+ SignerFactory.createSigner(signer, SIGNING_ALGORITHM));
+ }
+
+ initSigner(conf, overrideConfigBuilder, awsServiceIdentifier);
return overrideConfigBuilder;
}
@@ -94,8 +100,10 @@ public static ClientOverrideConfiguration.Builder createClientConfigBuilder(Conf
*
* @param conf The Hadoop configuration
* @return Http client builder
+ * @throws IOException on any problem
*/
- public static ApacheHttpClient.Builder createHttpClientBuilder(Configuration conf) {
+ public static ApacheHttpClient.Builder createHttpClientBuilder(Configuration conf)
+ throws IOException {
ApacheHttpClient.Builder httpClientBuilder =
ApacheHttpClient.builder();
@@ -109,8 +117,7 @@ public static ApacheHttpClient.Builder createHttpClientBuilder(Configuration con
httpClientBuilder.connectionTimeout(Duration.ofSeconds(connectionEstablishTimeout));
httpClientBuilder.socketTimeout(Duration.ofSeconds(socketTimeout));
- // TODO: Need to set ssl socket factory, as done in
- // NetworkBinding.bindSSLChannelMode(conf, awsConf);
+ NetworkBinding.bindSSLChannelMode(conf, httpClientBuilder);
return httpClientBuilder;
}
@@ -136,7 +143,7 @@ public static NettyNioAsyncHttpClient.Builder createAsyncHttpClientBuilder(Confi
httpClientBuilder.readTimeout(Duration.ofSeconds(socketTimeout));
httpClientBuilder.writeTimeout(Duration.ofSeconds(socketTimeout));
- // TODO: Need to set ssl socket factory, as done in
+ // TODO: Don't think you can set a socket factory for the netty client.
// NetworkBinding.bindSSLChannelMode(conf, awsConf);
return httpClientBuilder;
@@ -176,14 +183,15 @@ public static ProxyConfiguration createProxyConfiguration(Configuration conf,
if (!proxyHost.isEmpty()) {
if (proxyPort >= 0) {
- proxyConfigBuilder.endpoint(buildURI(proxyHost, proxyPort));
+ String scheme = conf.getBoolean(PROXY_SECURED, false) ? "https" : "http";
+ proxyConfigBuilder.endpoint(buildURI(scheme, proxyHost, proxyPort));
} else {
- if (conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS)) {
+ if (conf.getBoolean(PROXY_SECURED, false)) {
LOG.warn("Proxy host set without port. Using HTTPS default 443");
- proxyConfigBuilder.endpoint(buildURI(proxyHost, 443));
+ proxyConfigBuilder.endpoint(buildURI("https", proxyHost, 443));
} else {
LOG.warn("Proxy host set without port. Using HTTP default 80");
- proxyConfigBuilder.endpoint(buildURI(proxyHost, 80));
+ proxyConfigBuilder.endpoint(buildURI("http", proxyHost, 80));
}
}
final String proxyUsername = S3AUtils.lookupPassword(bucket, conf, PROXY_USERNAME,
@@ -235,17 +243,21 @@ public static ProxyConfiguration createProxyConfiguration(Configuration conf,
if (!proxyHost.isEmpty()) {
if (proxyPort >= 0) {
+ String scheme = conf.getBoolean(PROXY_SECURED, false) ? "https" : "http";
proxyConfigBuilder.host(proxyHost);
proxyConfigBuilder.port(proxyPort);
+ proxyConfigBuilder.scheme(scheme);
} else {
- if (conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS)) {
+ if (conf.getBoolean(PROXY_SECURED, false)) {
LOG.warn("Proxy host set without port. Using HTTPS default 443");
proxyConfigBuilder.host(proxyHost);
proxyConfigBuilder.port(443);
+ proxyConfigBuilder.scheme("https");
} else {
LOG.warn("Proxy host set without port. Using HTTP default 80");
proxyConfigBuilder.host(proxyHost);
proxyConfigBuilder.port(80);
+ proxyConfigBuilder.scheme("http");
}
}
final String proxyUsername = S3AUtils.lookupPassword(bucket, conf, PROXY_USERNAME,
@@ -287,9 +299,9 @@ public static ProxyConfiguration createProxyConfiguration(Configuration conf,
* @param port proxy port
* @return uri with host and port
*/
- private static URI buildURI(String host, int port) {
+ private static URI buildURI(String scheme, String host, int port) {
try {
- return new URIBuilder().setHost(host).setPort(port).build();
+ return new URIBuilder().setScheme(scheme).setHost(host).setPort(port).build();
} catch (URISyntaxException e) {
String msg =
"Proxy error: incorrect " + PROXY_HOST + " or " + PROXY_PORT;
@@ -319,6 +331,30 @@ private static void initUserAgent(Configuration conf,
clientConfig.putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX, userAgent);
}
+ private static void initSigner(Configuration conf,
+ ClientOverrideConfiguration.Builder clientConfig, String awsServiceIdentifier)
+ throws IOException {
+ String configKey = null;
+ switch (awsServiceIdentifier) {
+ case AWS_SERVICE_IDENTIFIER_S3:
+ configKey = SIGNING_ALGORITHM_S3;
+ break;
+ case AWS_SERVICE_IDENTIFIER_STS:
+ configKey = SIGNING_ALGORITHM_STS;
+ break;
+ default:
+ // Nothing to do. The original signer override is already setup
+ }
+ if (configKey != null) {
+ String signerOverride = conf.getTrimmed(configKey, "");
+ if (!signerOverride.isEmpty()) {
+ LOG.debug("Signer override for {}} = {}", awsServiceIdentifier, signerOverride);
+ clientConfig.putAdvancedOption(SdkAdvancedClientOption.SIGNER,
+ SignerFactory.createSigner(signerOverride, configKey));
+ }
+ }
+ }
+
/**
* Configures request timeout.
*
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java
index e36842c39b731..2c9d6857b46a2 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java
@@ -207,15 +207,13 @@ public void processResponse(final CopyObjectResponse copyObjectResponse)
// be the same on the copy. As such, there isn't really anything that
// can be verified on the response, except that a revision ID is present
// if required.
- // TODO: Commenting out temporarily, due to the TM not returning copyObjectResult
- // in the response.
-// String newRevisionId = policy.getRevisionId(copyObjectResponse);
-// LOG.debug("Copy result {}: {}", policy.getSource(), newRevisionId);
-// if (newRevisionId == null && policy.isRequireVersion()) {
-// throw new NoVersionAttributeException(uri, String.format(
-// "Change detection policy requires %s",
-// policy.getSource()));
-// }
+ String newRevisionId = policy.getRevisionId(copyObjectResponse);
+ LOG.debug("Copy result {}: {}", policy.getSource(), newRevisionId);
+ if (newRevisionId == null && policy.isRequireVersion()) {
+ throw new NoVersionAttributeException(uri, String.format(
+ "Change detection policy requires %s",
+ policy.getSource()));
+ }
}
/**
@@ -232,9 +230,6 @@ public void processException(SdkException e, String operation) throws
RemoteFileChangedException {
if (e instanceof AwsServiceException) {
AwsServiceException serviceException = (AwsServiceException)e;
- // TODO: Verify whether this is fixed in SDK v2.
- // In SDK v1, this wasn't really going to be hit due to
- // https://github.com/aws/aws-sdk-java/issues/1644
if (serviceException.statusCode() == SC_412_PRECONDITION_FAILED) {
versionMismatches.versionMismatchError();
throw new RemoteFileChangedException(uri, operation, String.format(
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ConfigureShadedAWSSocketFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ConfigureShadedAWSSocketFactory.java
index 8205668e8f354..ba21f6028a5f8 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ConfigureShadedAWSSocketFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ConfigureShadedAWSSocketFactory.java
@@ -21,8 +21,8 @@
import javax.net.ssl.HostnameVerifier;
import java.io.IOException;
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.thirdparty.apache.http.conn.ssl.SSLConnectionSocketFactory;
+import software.amazon.awssdk.http.apache.ApacheHttpClient;
+import software.amazon.awssdk.thirdparty.org.apache.http.conn.ssl.SSLConnectionSocketFactory;
import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
@@ -35,13 +35,12 @@ public class ConfigureShadedAWSSocketFactory implements
NetworkBinding.ConfigureAWSSocketFactory {
@Override
- public void configureSocketFactory(final ClientConfiguration awsConf,
+ public void configureSocketFactory(final ApacheHttpClient.Builder httpClientBuilder,
final DelegatingSSLSocketFactory.SSLChannelMode channelMode)
throws IOException {
DelegatingSSLSocketFactory.initializeDefaultFactory(channelMode);
- awsConf.getApacheHttpClientConfig().setSslSocketFactory(
- new SSLConnectionSocketFactory(
- DelegatingSSLSocketFactory.getDefaultFactory(),
- (HostnameVerifier) null));
+ httpClientBuilder.socketFactory(new SSLConnectionSocketFactory(
+ DelegatingSSLSocketFactory.getDefaultFactory(),
+ (HostnameVerifier) null));
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java
index 4926ff13f8d0c..d42dda59caa5f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java
@@ -30,6 +30,7 @@
import java.util.TreeMap;
import software.amazon.awssdk.services.s3.model.CopyObjectRequest;
+import software.amazon.awssdk.services.s3.model.HeadBucketResponse;
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -276,9 +277,28 @@ private Map retrieveHeaders(
final Statistic statistic) throws IOException {
StoreContext context = getStoreContext();
String objectKey = context.pathToKey(path);
- HeadObjectResponse md;
String symbol = statistic.getSymbol();
S3AStatisticsContext instrumentation = context.getInstrumentation();
+ Map headers = new TreeMap<>();
+ HeadObjectResponse md;
+
+ // Attempting to get metadata for the root, so use head bucket.
+ if (objectKey.isEmpty()) {
+ HeadBucketResponse headBucketResponse =
+ trackDuration(instrumentation, symbol, () -> callbacks.getBucketMetadata());
+
+ if (headBucketResponse.sdkHttpResponse() != null
+ && headBucketResponse.sdkHttpResponse().headers() != null
+ && headBucketResponse.sdkHttpResponse().headers().get(AWSHeaders.CONTENT_TYPE) != null) {
+ maybeSetHeader(headers, XA_CONTENT_TYPE,
+ headBucketResponse.sdkHttpResponse().headers().get(AWSHeaders.CONTENT_TYPE).get(0));
+ }
+
+ maybeSetHeader(headers, XA_CONTENT_LENGTH, 0);
+
+ return headers;
+ }
+
try {
md = trackDuration(instrumentation, symbol, () ->
callbacks.getObjectMetadata(objectKey));
@@ -289,7 +309,6 @@ private Map retrieveHeaders(
}
// all user metadata
Map rawHeaders = md.metadata();
- Map headers = new TreeMap<>();
rawHeaders.forEach((key, value) ->
headers.put(XA_HEADER_PREFIX + key, encodeBytes(value)));
@@ -346,9 +365,7 @@ private Map retrieveHeaders(
md.serverSideEncryptionAsString());
maybeSetHeader(headers, XA_STORAGE_CLASS,
md.storageClassAsString());
- // TODO: check this, looks wrong.
- // maybeSetHeader(headers, XA_STORAGE_CLASS,
-// md.getReplicationStatus());
+
return headers;
}
@@ -525,5 +542,14 @@ public interface HeaderProcessingCallbacks {
*/
@Retries.RetryTranslated
HeadObjectResponse getObjectMetadata(String key) throws IOException;
+
+ /**
+ * Retrieve the bucket metadata.
+ *
+ * @return metadata
+ * @throws IOException IO and object access problems.
+ */
+ @Retries.RetryTranslated
+ HeadBucketResponse getBucketMetadata() throws IOException;
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java
index 575a3d1b2de81..34b4049b06123 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java
@@ -23,9 +23,9 @@
import java.net.URI;
import java.net.URISyntaxException;
-import com.amazonaws.ClientConfiguration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.http.apache.ApacheHttpClient;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.net.NetUtils;
@@ -56,13 +56,12 @@ private NetworkBinding() {
* so as to avoid
* @param conf the {@link Configuration} used to get the client specified
* value of {@code SSL_CHANNEL_MODE}
- * @param awsConf the {@code ClientConfiguration} to set the
- * SSLConnectionSocketFactory for.
+ * @param httpClientBuilder the http client builder.
* @throws IOException if there is an error while initializing the
* {@code SSLSocketFactory} other than classloader problems.
*/
public static void bindSSLChannelMode(Configuration conf,
- ClientConfiguration awsConf) throws IOException {
+ ApacheHttpClient.Builder httpClientBuilder) throws IOException {
// Validate that SSL_CHANNEL_MODE is set to a valid value.
String channelModeString = conf.getTrimmed(
@@ -89,7 +88,7 @@ public static void bindSSLChannelMode(Configuration conf,
(Class extends ConfigureAWSSocketFactory>) Class.forName(BINDING_CLASSNAME);
clazz.getConstructor()
.newInstance()
- .configureSocketFactory(awsConf, channelMode);
+ .configureSocketFactory(httpClientBuilder, channelMode);
} catch (ClassNotFoundException | NoSuchMethodException |
IllegalAccessException | InstantiationException |
InvocationTargetException | LinkageError e) {
@@ -103,7 +102,7 @@ public static void bindSSLChannelMode(Configuration conf,
* works with the shaded AWS libraries to exist in their own class.
*/
interface ConfigureAWSSocketFactory {
- void configureSocketFactory(ClientConfiguration awsConf,
+ void configureSocketFactory(ApacheHttpClient.Builder httpClientBuilder,
DelegatingSSLSocketFactory.SSLChannelMode channelMode)
throws IOException;
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListener.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListener.java
new file mode 100644
index 0000000000000..c26ffa1699ba7
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListener.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl;
+
+/**
+ * Interface for progress listeners to implement.
+ */
+public interface ProgressListener {
+ default void progressChanged(ProgressListenerEvent eventType, int bytesTransferred) {};
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java
new file mode 100644
index 0000000000000..f3f9fb61e434d
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ProgressListenerEvent.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl;
+
+/**
+ * Enum for progress listener events.
+ */
+public enum ProgressListenerEvent {
+ REQUEST_BYTE_TRANSFER_EVENT,
+ TRANSFER_PART_STARTED_EVENT,
+ TRANSFER_PART_COMPLETED_EVENT,
+ TRANSFER_PART_FAILED_EVENT;
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java
index 17cd60890d28d..5d5aa5be71afc 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
@@ -34,13 +34,13 @@
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest;
+import software.amazon.awssdk.services.s3.model.HeadBucketRequest;
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest;
import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
import software.amazon.awssdk.services.s3.model.MetadataDirective;
-import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest;
@@ -99,7 +99,7 @@ public class RequestFactoryImpl implements RequestFactory {
/**
* ACL For new objects.
*/
- private final ObjectCannedACL cannedACL;
+ private final String cannedACL;
/**
* Max number of multipart entries allowed in a large
@@ -155,7 +155,7 @@ private T prepareRequest(T t) {
* @return an ACL, if any
*/
@Override
- public ObjectCannedACL getCannedACL() {
+ public String getCannedACL() {
return cannedACL;
}
@@ -201,7 +201,6 @@ public StorageClass getStorageClass() {
*/
protected void uploadPartEncryptionParameters(
UploadPartRequest.Builder builder) {
- // TODO: review/refactor together with similar methods for other requests.
// need to set key to get objects encrypted with SSE_C
EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets).ifPresent(base64customerKey -> {
builder.sseCustomerAlgorithm(ServerSideEncryption.AES256.name())
@@ -230,7 +229,7 @@ public CopyObjectRequest.Builder newCopyObjectRequestBuilder(String srcKey,
Map dstom = new HashMap<>();
HeaderProcessing.cloneObjectMetadata(srcom, dstom, copyObjectRequestBuilder);
- copyEncryptionParameters(copyObjectRequestBuilder);
+ copyEncryptionParameters(srcom, copyObjectRequestBuilder);
copyObjectRequestBuilder
.metadata(dstom)
@@ -251,11 +250,22 @@ public CopyObjectRequest.Builder newCopyObjectRequestBuilder(String srcKey,
* Propagate encryption parameters from source file if set else use the
* current filesystem encryption settings.
* @param copyObjectRequestBuilder copy object request builder.
+ * @param srcom source object metadata.
*/
- protected void copyEncryptionParameters(CopyObjectRequest.Builder copyObjectRequestBuilder) {
+ protected void copyEncryptionParameters(HeadObjectResponse srcom,
+ CopyObjectRequest.Builder copyObjectRequestBuilder) {
final S3AEncryptionMethods algorithm = getServerSideEncryptionAlgorithm();
+ String sourceKMSId = srcom.ssekmsKeyId();
+ if (isNotEmpty(sourceKMSId)) {
+ // source KMS ID is propagated
+ LOG.debug("Propagating SSE-KMS settings from source {}",
+ sourceKMSId);
+ copyObjectRequestBuilder.ssekmsKeyId(sourceKMSId);
+ return;
+ }
+
if (S3AEncryptionMethods.SSE_S3 == algorithm) {
copyObjectRequestBuilder.serverSideEncryption(algorithm.getMethod());
} else if (S3AEncryptionMethods.SSE_KMS == algorithm) {
@@ -465,6 +475,15 @@ public HeadObjectRequest.Builder newHeadObjectRequestBuilder(String key) {
return prepareRequest(headObjectRequestBuilder);
}
+ @Override
+ public HeadBucketRequest.Builder newHeadBucketRequestBuilder(String bucketName) {
+
+ HeadBucketRequest.Builder headBucketRequestBuilder =
+ HeadBucketRequest.builder().bucket(bucketName);
+
+ return prepareRequest(headBucketRequestBuilder);
+ }
+
@Override
public GetObjectRequest.Builder newGetObjectRequestBuilder(String key) {
GetObjectRequest.Builder builder = GetObjectRequest.builder()
@@ -604,7 +623,7 @@ public static final class RequestFactoryBuilder {
/**
* ACL For new objects.
*/
- private ObjectCannedACL cannedACL = null;
+ private String cannedACL = null;
/** Content Encoding. */
private String contentEncoding;
@@ -682,7 +701,7 @@ public RequestFactoryBuilder withEncryptionSecrets(
* @return the builder
*/
public RequestFactoryBuilder withCannedACL(
- final ObjectCannedACL value) {
+ final String value) {
cannedACL = value;
return this;
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
index d1647fb3b2f54..d233081ee6851 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
@@ -64,7 +64,7 @@ public void setup() throws Exception {
// unset S3CSE property from config to avoid pathIOE.
conf.unset(Constants.S3_ENCRYPTION_ALGORITHM);
fs.initialize(uri, conf);
- s3 = fs.getAmazonS3V2ClientForTesting("mocking");
+ s3 = fs.getAmazonS3ClientForTesting("mocking");
}
public Configuration createConfiguration() {
@@ -79,6 +79,8 @@ public Configuration createConfiguration() {
// assertions to be safely made without worrying
// about any race conditions
conf.setInt(ASYNC_DRAIN_THRESHOLD, Integer.MAX_VALUE);
+ // set the region to avoid the getBucketLocation on FS init.
+ conf.set(AWS_REGION, "eu-west-1");
return conf;
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java
index fb295f3f09ff4..3080da8b689e2 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java
@@ -34,6 +34,7 @@
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION;
import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_ACCESSPOINT_REQUIRED;
import static org.apache.hadoop.fs.s3a.Constants.FS_S3A;
import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE;
@@ -124,11 +125,12 @@ private Configuration createConfigurationWithProbe(final int probe) {
Configuration conf = new Configuration(getFileSystem().getConf());
S3ATestUtils.disableFilesystemCaching(conf);
conf.setInt(S3A_BUCKET_PROBE, probe);
+ conf.set(AWS_REGION, "eu-west-1");
return conf;
}
@Test
- public void testBucketProbingV1() throws Exception {
+ public void testBucketProbing() throws Exception {
describe("Test the V1 bucket probe");
Configuration configuration = createConfigurationWithProbe(1);
expectUnknownStore(
@@ -136,18 +138,24 @@ public void testBucketProbingV1() throws Exception {
}
@Test
- public void testBucketProbingV2() throws Exception {
- describe("Test the V2 bucket probe");
+ public void testBucketProbing2() throws Exception {
+ describe("Test the bucket probe with probe value set to 2");
Configuration configuration = createConfigurationWithProbe(2);
+
expectUnknownStore(
() -> FileSystem.get(uri, configuration));
- /*
- * Bucket probing should also be done when value of
- * S3A_BUCKET_PROBE is greater than 2.
- */
- configuration.setInt(S3A_BUCKET_PROBE, 3);
- expectUnknownStore(
- () -> FileSystem.get(uri, configuration));
+ }
+
+ @Test
+ public void testBucketProbing3() throws Exception {
+ describe("Test the bucket probe with probe value set to 3");
+ Configuration configuration = createConfigurationWithProbe(3);
+ fs = FileSystem.get(uri, configuration);
+ Path root = new Path(uri);
+
+ assertTrue("root path should always exist", fs.exists(root));
+ assertTrue("getFileStatus on root should always return a directory",
+ fs.getFileStatus(root).isDirectory());
}
@Test
@@ -161,8 +169,8 @@ public void testBucketProbingParameterValidation() throws Exception {
}
@Test
- public void testAccessPointProbingV2() throws Exception {
- describe("Test V2 bucket probing using an AccessPoint ARN");
+ public void testAccessPointProbing2() throws Exception {
+ describe("Test bucket probing using probe value 2, and an AccessPoint ARN");
Configuration configuration = createConfigurationWithProbe(2);
String accessPointArn = "arn:aws:s3:eu-west-1:123456789012:accesspoint/" + randomBucket;
configuration.set(String.format(InternalConstants.ARN_BUCKET_OPTION, randomBucket),
@@ -174,7 +182,7 @@ public void testAccessPointProbingV2() throws Exception {
@Test
public void testAccessPointRequired() throws Exception {
- describe("Test V2 bucket probing with 'fs.s3a.accesspoint.required' property.");
+ describe("Test bucket probing with 'fs.s3a.accesspoint.required' property.");
Configuration configuration = createConfigurationWithProbe(2);
configuration.set(AWS_S3_ACCESSPOINT_REQUIRED, "true");
intercept(PathIOException.class,
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACannedACLs.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACannedACLs.java
index 1b65b5ded9ae9..7ffb16833e044 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACannedACLs.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACannedACLs.java
@@ -57,8 +57,6 @@ protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
removeBaseAndBucketOverrides(conf,
CANNED_ACL);
- // TODO: Check why we need this ACL? V2 does not have a LOG_DELIVERY_WRITE ACL which causes
- // this test to fail.
conf.set(CANNED_ACL, LOG_DELIVERY_WRITE);
// needed because of direct calls made
conf.setBoolean(S3AAuditConstants.REJECT_OUT_OF_SPAN_OPERATIONS, false);
@@ -92,7 +90,7 @@ private void assertObjectHasLoggingGrant(Path path, boolean isFile) {
S3AFileSystem fs = getFileSystem();
StoreContext storeContext = fs.createStoreContext();
- S3Client s3 = fs.getAmazonS3V2ClientForTesting("acls");
+ S3Client s3 = fs.getAmazonS3ClientForTesting("acls");
String key = storeContext.pathToKey(path);
if (!isFile) {
key = key + "/";
@@ -105,8 +103,6 @@ private void assertObjectHasLoggingGrant(Path path, boolean isFile) {
for (Grant grant : grants) {
LOG.info("{}", grant.toString());
}
- // TODO: Review whether this test is required in v2.
- // Reproduces v1's GroupGrantee.LogDelivery
Grant loggingGrant = Grant.builder()
.grantee(Grantee.builder()
.type(Type.GROUP)
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
index ad2c16bae1d9e..f7bdaa62422ed 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
@@ -18,11 +18,17 @@
package org.apache.hadoop.fs.s3a;
-import com.amazonaws.ClientConfiguration;
import software.amazon.awssdk.core.client.config.SdkClientConfiguration;
import software.amazon.awssdk.core.client.config.SdkClientOption;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.core.signer.Signer;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
import software.amazon.awssdk.services.s3.S3Client;
import software.amazon.awssdk.services.s3.S3Configuration;
+import software.amazon.awssdk.services.s3.model.HeadBucketRequest;
+import software.amazon.awssdk.services.s3.model.S3Exception;
+import software.amazon.awssdk.services.sts.StsClient;
+import software.amazon.awssdk.services.sts.model.StsException;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.reflect.FieldUtils;
@@ -30,11 +36,11 @@
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3a.auth.STSClientFactory;
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
import org.apache.hadoop.test.GenericTestUtils;
import org.assertj.core.api.Assertions;
-import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
@@ -54,7 +60,6 @@
import org.apache.http.HttpStatus;
import org.junit.rules.TemporaryFolder;
-
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
@@ -113,7 +118,7 @@ public void testEndpoint() throws Exception {
} else {
conf.set(Constants.ENDPOINT, endpoint);
fs = S3ATestUtils.createTestFileSystem(conf);
- S3Client s3 = fs.getAmazonS3V2ClientForTesting("test endpoint");
+ S3Client s3 = fs.getAmazonS3ClientForTesting("test endpoint");
String endPointRegion = "";
// Differentiate handling of "s3-" and "s3." based endpoint identifiers
String[] endpointParts = StringUtils.split(endpoint, '.');
@@ -353,7 +358,7 @@ public void shouldBeAbleToSwitchOnS3PathStyleAccessViaConfigProperty()
try {
fs = S3ATestUtils.createTestFileSystem(conf);
assertNotNull(fs);
- S3Client s3 = fs.getAmazonS3V2ClientForTesting("configuration");
+ S3Client s3 = fs.getAmazonS3ClientForTesting("configuration");
assertNotNull(s3);
SdkClientConfiguration clientConfiguration = getField(s3, SdkClientConfiguration.class,
@@ -388,7 +393,7 @@ public void testDefaultUserAgent() throws Exception {
conf = new Configuration();
fs = S3ATestUtils.createTestFileSystem(conf);
assertNotNull(fs);
- S3Client s3 = fs.getAmazonS3V2ClientForTesting("User Agent");
+ S3Client s3 = fs.getAmazonS3ClientForTesting("User Agent");
assertNotNull(s3);
SdkClientConfiguration clientConfiguration = getField(s3, SdkClientConfiguration.class,
"clientConfiguration");
@@ -403,7 +408,7 @@ public void testCustomUserAgent() throws Exception {
conf.set(Constants.USER_AGENT_PREFIX, "MyApp");
fs = S3ATestUtils.createTestFileSystem(conf);
assertNotNull(fs);
- S3Client s3 = fs.getAmazonS3V2ClientForTesting("User agent");
+ S3Client s3 = fs.getAmazonS3ClientForTesting("User agent");
assertNotNull(s3);
SdkClientConfiguration clientConfiguration = getField(s3, SdkClientConfiguration.class,
"clientConfiguration");
@@ -417,7 +422,7 @@ public void testRequestTimeout() throws Exception {
conf = new Configuration();
conf.set(REQUEST_TIMEOUT, "120");
fs = S3ATestUtils.createTestFileSystem(conf);
- S3Client s3 = fs.getAmazonS3V2ClientForTesting("Request timeout (ms)");
+ S3Client s3 = fs.getAmazonS3ClientForTesting("Request timeout (ms)");
SdkClientConfiguration clientConfiguration = getField(s3, SdkClientConfiguration.class,
"clientConfiguration");
assertEquals("Configured " + REQUEST_TIMEOUT +
@@ -525,37 +530,74 @@ public void testConfOptionPropagationToFS() throws Exception {
@Test(timeout = 10_000L)
public void testS3SpecificSignerOverride() throws IOException {
- ClientConfiguration clientConfiguration = null;
- Configuration config;
-
- String signerOverride = "testSigner";
- String s3SignerOverride = "testS3Signer";
-
- // Default SIGNING_ALGORITHM, overridden for S3 only
- config = new Configuration();
- config.set(SIGNING_ALGORITHM_S3, s3SignerOverride);
-
- // TODO: update during signer work.
- clientConfiguration = S3AUtils
- .createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_S3);
- Assert.assertEquals(s3SignerOverride,
- clientConfiguration.getSignerOverride());
- clientConfiguration = S3AUtils
- .createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_STS);
- Assert.assertNull(clientConfiguration.getSignerOverride());
-
- // Configured base SIGNING_ALGORITHM, overridden for S3 only
- config = new Configuration();
- config.set(SIGNING_ALGORITHM, signerOverride);
- config.set(SIGNING_ALGORITHM_S3, s3SignerOverride);
- clientConfiguration = S3AUtils
- .createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_S3);
- Assert.assertEquals(s3SignerOverride,
- clientConfiguration.getSignerOverride());
- clientConfiguration = S3AUtils
- .createAwsConf(config, "dontcare", AWS_SERVICE_IDENTIFIER_STS);
- Assert
- .assertEquals(signerOverride, clientConfiguration.getSignerOverride());
+ Configuration config = new Configuration();
+
+ config.set(CUSTOM_SIGNERS,
+ "CustomS3Signer:" + CustomS3Signer.class.getName() + ",CustomSTSSigner:"
+ + CustomSTSSigner.class.getName());
+
+ config.set(SIGNING_ALGORITHM_S3, "CustomS3Signer");
+ config.set(SIGNING_ALGORITHM_STS, "CustomSTSSigner");
+
+ config.set(AWS_REGION, "eu-west-1");
+ fs = S3ATestUtils.createTestFileSystem(config);
+
+ S3Client s3Client = fs.getAmazonS3ClientForTesting("testS3SpecificSignerOverride");
+
+ StsClient stsClient =
+ STSClientFactory.builder(config, fs.getBucket(), new AnonymousAWSCredentialsProvider(), "",
+ "").build();
+
+ try {
+ stsClient.getSessionToken();
+ } catch (StsException exception) {
+ // Expected 403, as credentials are not provided.
+ }
+
+ try {
+ s3Client.headBucket(HeadBucketRequest.builder().bucket(fs.getBucket()).build());
+ } catch (S3Exception exception) {
+ // Expected 403, as credentials are not provided.
+ }
+
+ Assertions.assertThat(CustomS3Signer.isS3SignerCalled())
+ .describedAs("Custom S3 signer not called").isTrue();
+
+ Assertions.assertThat(CustomSTSSigner.isSTSSignerCalled())
+ .describedAs("Custom STS signer not called").isTrue();
+ }
+
+ public static final class CustomS3Signer implements Signer {
+
+ private static boolean s3SignerCalled = false;
+
+ @Override
+ public SdkHttpFullRequest sign(SdkHttpFullRequest request,
+ ExecutionAttributes executionAttributes) {
+ LOG.debug("Custom S3 signer called");
+ s3SignerCalled = true;
+ return request;
+ }
+
+ public static boolean isS3SignerCalled() {
+ return s3SignerCalled;
+ }
}
+ public static final class CustomSTSSigner implements Signer {
+
+ private static boolean stsSignerCalled = false;
+
+ @Override
+ public SdkHttpFullRequest sign(SdkHttpFullRequest request,
+ ExecutionAttributes executionAttributes) {
+ LOG.debug("Custom STS signer called");
+ stsSignerCalled = true;
+ return request;
+ }
+
+ public static boolean isSTSSignerCalled() {
+ return stsSignerCalled;
+ }
+ }
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
index 64e37bf832b87..45b0c6c206f2d 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
@@ -63,7 +63,7 @@
public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
private static final String SERVICE_AMAZON_S3_STATUS_CODE_403
- = "Service: Amazon S3; Status Code: 403;";
+ = "Service: S3, Status Code: 403";
private static final String KEY_1
= "4niV/jPK5VFRHY+KNb6wtqYd4xXyMgdJ9XQJpcQUVbs=";
private static final String KEY_2
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java
index add6502d7da71..c957ab7b6a438 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java
@@ -21,21 +21,27 @@
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.client.builder.AwsClientBuilder;
-import com.amazonaws.services.s3.AmazonS3;
-import com.amazonaws.util.AwsHostNameUtils;
import org.assertj.core.api.Assertions;
+import org.junit.Assert;
import org.junit.Test;
+import software.amazon.awssdk.awscore.AwsExecutionAttribute;
+import software.amazon.awssdk.awscore.exception.AwsServiceException;
+import software.amazon.awssdk.core.interceptor.Context;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.HeadBucketRequest;
+import software.amazon.awssdk.services.s3.model.S3Exception;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION;
-import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_CENTRAL_REGION;
-import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ENDPOINT;
-import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AWS_REGION_SYSPROP;
+import static org.apache.hadoop.fs.s3a.Statistic.STORE_REGION_PROBE;
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_301_MOVED_PERMANENTLY;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
/**
@@ -44,114 +50,104 @@
*/
public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
- private static final String AWS_REGION_TEST = "test-region";
private static final String AWS_ENDPOINT_TEST = "test-endpoint";
- private static final String AWS_ENDPOINT_TEST_WITH_REGION =
- "test-endpoint.some-region.amazonaws.com";
- public static final String MARS_NORTH_2 = "mars-north-2";
- /**
- * Test to verify that setting a region with the config would bypass the
- * construction of region from endpoint.
- */
- @Test
- public void testWithRegionConfig() {
- getFileSystem().getConf().set(AWS_REGION, AWS_REGION_TEST);
-
- //Creating an endpoint config with a custom endpoint.
- AwsClientBuilder.EndpointConfiguration epr = createEpr(AWS_ENDPOINT_TEST,
- getFileSystem().getConf().getTrimmed(AWS_REGION));
- //Checking if setting region config bypasses the endpoint region.
- Assertions.assertThat(epr.getSigningRegion())
- .describedAs("There is a region mismatch")
- .isEqualTo(getFileSystem().getConf().get(AWS_REGION));
- }
/**
- * Test to verify that not setting the region config, would lead to using
- * endpoint to construct the region.
+ * Test to verify that not setting the region config, will lead to the client factory making
+ * a HEAD bucket call to configure the correct region. If an incorrect region is set, the HEAD
+ * bucket call in this test will raise an exception.
*/
@Test
- public void testWithoutRegionConfig() {
- getFileSystem().getConf().unset(AWS_REGION);
-
- //Creating an endpoint config with a custom endpoint containing a region.
- AwsClientBuilder.EndpointConfiguration eprRandom =
- createEpr(AWS_ENDPOINT_TEST_WITH_REGION,
- getFileSystem().getConf().getTrimmed(AWS_REGION));
- String regionFromEndpoint =
- AwsHostNameUtils
- .parseRegionFromAwsPartitionPattern(AWS_ENDPOINT_TEST_WITH_REGION);
- //Checking if not setting region config leads to constructing the region
- // from endpoint.
- Assertions.assertThat(eprRandom.getSigningRegion())
- .describedAs("There is a region mismatch")
- .isNotEqualTo(getFileSystem().getConf().get(AWS_REGION))
- .isEqualTo(regionFromEndpoint);
- }
+ public void testWithoutRegionConfig() throws IOException {
+ Configuration conf = getConfiguration();
+ String bucket = getFileSystem().getBucket();
+ conf.unset(String.format("fs.s3a.bucket.%s.endpoint.region", bucket));
+ conf.unset(AWS_REGION);
- /**
- * Method to create EndpointConfiguration using an endpoint.
- *
- * @param endpoint the endpoint to be used for EndpointConfiguration creation.
- * @return an instance of EndpointConfiguration.
- */
- private AwsClientBuilder.EndpointConfiguration createEpr(String endpoint,
- String awsRegion) {
- return DefaultS3ClientFactory.createEndpointConfiguration(endpoint,
- new ClientConfiguration(), awsRegion);
+ S3AFileSystem fs = new S3AFileSystem();
+ fs.initialize(getFileSystem().getUri(), conf);
+
+ try {
+ fs.getBucketMetadata();
+ } catch (S3Exception exception) {
+ if (exception.statusCode() == SC_301_MOVED_PERMANENTLY) {
+ Assert.fail(exception.toString());
+ }
+ }
+
+ Assertions.assertThat(fs.getInstrumentation().getCounterValue(STORE_REGION_PROBE))
+ .describedAs("Region is not configured, region probe should have been made").isEqualTo(1);
}
@Test
- public void testInvalidRegionDefaultEndpoint() throws Throwable {
- describe("Create a client with an invalid region and the default endpoint");
+ public void testWithRegionConfig() throws IOException, URISyntaxException {
Configuration conf = getConfiguration();
- // we are making a big assumption about the timetable for AWS
- // region rollout.
- // if this test ever fails because this region now exists
- // -congratulations!
- conf.set(AWS_REGION, MARS_NORTH_2);
- createMarsNorth2Client(conf);
+ conf.set(AWS_REGION, "us-east-2");
+
+ S3AFileSystem fs = new S3AFileSystem();
+ fs.initialize(new URI("s3a://landsat-pds"), conf);
+
+ Assertions.assertThat(fs.getInstrumentation().getCounterValue(STORE_REGION_PROBE))
+ .describedAs("Region is configured, region probe should not have been made").isEqualTo(0);
}
@Test
- public void testUnsetRegionDefaultEndpoint() throws Throwable {
- describe("Create a client with no region and the default endpoint");
+ public void testRegionCache() throws IOException, URISyntaxException {
Configuration conf = getConfiguration();
conf.unset(AWS_REGION);
- createS3Client(conf, DEFAULT_ENDPOINT, AWS_S3_CENTRAL_REGION);
+ conf.unset("fs.s3a.bucket.landsat-pds.endpoint.region");
+ S3AFileSystem fs = new S3AFileSystem();
+
+ fs.initialize(new URI("s3a://landsat-pds"), conf);
+
+ Assertions.assertThat(fs.getInstrumentation().getCounterValue(STORE_REGION_PROBE))
+ .describedAs("Incorrect number of calls made to get bucket region").isEqualTo(1);
+
+ fs.initialize(new URI("s3a://landsat-pds"), conf);
+
+ // value should already be cached.
+ Assertions.assertThat(fs.getInstrumentation().getCounterValue(STORE_REGION_PROBE))
+ .describedAs("Incorrect number of calls made to get bucket region").isEqualTo(0);
}
- /**
- * By setting the system property {@code "aws.region"} we can
- * guarantee that the SDK region resolution chain will always succeed
- * (and fast).
- * Clearly there is no validation of the region during the build process.
- */
@Test
- public void testBlankRegionTriggersSDKResolution() throws Throwable {
- describe("Create a client with a blank region and the default endpoint."
- + " This will trigger the SDK Resolution chain");
+ public void testEndpointOverride() throws Throwable {
+ describe("Create a client with no region and the default endpoint");
Configuration conf = getConfiguration();
- conf.set(AWS_REGION, "");
- System.setProperty(AWS_REGION_SYSPROP, MARS_NORTH_2);
- try {
- createMarsNorth2Client(conf);
- } finally {
- System.clearProperty(AWS_REGION_SYSPROP);
- }
+
+ S3Client client = createS3Client(conf, AWS_ENDPOINT_TEST);
+
+ intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket(
+ HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build()));
}
- /**
- * Create an S3 client bonded to an invalid region;
- * verify that calling {@code getRegion()} triggers
- * a failure.
- * @param conf configuration to use in the building.
- */
- private void createMarsNorth2Client(Configuration conf) throws Exception {
- AmazonS3 client = createS3Client(conf, DEFAULT_ENDPOINT, MARS_NORTH_2);
- intercept(IllegalArgumentException.class, MARS_NORTH_2, client::getRegion);
+
+ class RegionInterceptor implements ExecutionInterceptor {
+ private boolean endpointOverridden;
+
+ RegionInterceptor(boolean endpointOverridden) {
+ this.endpointOverridden = endpointOverridden;
+ }
+
+ @Override
+ public void beforeExecution(Context.BeforeExecution context,
+ ExecutionAttributes executionAttributes) {
+
+ if (endpointOverridden) {
+ Assertions.assertThat(
+ executionAttributes.getAttribute(AwsExecutionAttribute.ENDPOINT_OVERRIDDEN))
+ .describedAs("Endpoint not overridden").isTrue();
+
+ Assertions.assertThat(
+ executionAttributes.getAttribute(AwsExecutionAttribute.CLIENT_ENDPOINT).toString())
+ .describedAs("There is an endpoint mismatch").isEqualTo("https://" + AWS_ENDPOINT_TEST);
+ }
+
+ // We don't actually want to make a request, so exit early.
+ throw AwsServiceException.builder().message("Exception thrown by interceptor").build();
+ }
}
/**
@@ -160,16 +156,23 @@ private void createMarsNorth2Client(Configuration conf) throws Exception {
* value.
* @param conf configuration to use.
* @param endpoint endpoint.
- * @param expectedRegion expected region
* @return the client.
* @throws URISyntaxException parse problems.
* @throws IOException IO problems
*/
@SuppressWarnings("deprecation")
- private AmazonS3 createS3Client(Configuration conf,
- String endpoint,
- String expectedRegion)
- throws URISyntaxException, IOException {
+ private S3Client createS3Client(Configuration conf,
+ String endpoint)
+ throws IOException {
+
+ boolean endpointOverridden = false;
+
+ if (endpoint != null && !endpoint.isEmpty()) {
+ endpointOverridden = true;
+ }
+
+ List interceptors = new ArrayList<>();
+ interceptors.add(new RegionInterceptor(endpointOverridden));
DefaultS3ClientFactory factory
= new DefaultS3ClientFactory();
@@ -177,16 +180,14 @@ private AmazonS3 createS3Client(Configuration conf,
S3ClientFactory.S3ClientCreationParameters parameters
= new S3ClientFactory.S3ClientCreationParameters()
.withCredentialSet(new AnonymousAWSCredentialsProvider())
- .withPathUri(new URI("s3a://localhost/"))
.withEndpoint(endpoint)
.withMetrics(new EmptyS3AStatisticsContext()
- .newStatisticsFromAwsSdk());
- AmazonS3 client = factory.createS3Client(
- new URI("s3a://localhost/"),
+ .newStatisticsFromAwsSdk())
+ .withExecutionInterceptors(interceptors);
+
+ S3Client client = factory.createS3Client(
+ getFileSystem().getUri(),
parameters);
- Assertions.assertThat(client.getRegionName())
- .describedAs("Client region name")
- .isEqualTo(expectedRegion);
return client;
}
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
index cb010bc09c5b3..53ccffcfbec98 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
@@ -411,7 +411,7 @@ private static T verifyNoTrailingSlash(String role, T o) {
*/
private GetBucketEncryptionResponse getDefaultEncryption() throws IOException {
S3AFileSystem fs = getFileSystem();
- S3Client s3 = fs.getAmazonS3V2ClientForTesting("check default encryption");
+ S3Client s3 = fs.getAmazonS3ClientForTesting("check default encryption");
try {
return Invoker.once("getBucketEncryption()",
fs.getBucket(),
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
index e16a99be8e7c8..0c61caacd055b 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java
@@ -20,44 +20,38 @@
import static org.mockito.Mockito.*;
-import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
-import com.amazonaws.services.s3.AmazonS3;
-import com.amazonaws.services.s3.model.MultipartUploadListing;
-import com.amazonaws.services.s3.model.Region;
+import software.amazon.awssdk.regions.Region;
import software.amazon.awssdk.services.s3.S3AsyncClient;
import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest;
+import software.amazon.awssdk.services.s3.model.GetBucketLocationResponse;
+import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest;
+import software.amazon.awssdk.services.s3.model.ListMultipartUploadsResponse;
import software.amazon.awssdk.transfer.s3.S3TransferManager;
/**
- * An {@link S3ClientFactory} that returns Mockito mocks of the {@link AmazonS3}
+ * An {@link S3ClientFactory} that returns Mockito mocks of the {@link S3Client}
* interface suitable for unit testing.
*/
public class MockS3ClientFactory implements S3ClientFactory {
- // TODO: This will be removed when we remove this method for the client factory.
- @Override
- public AmazonS3 createS3Client(URI uri,
- final S3ClientCreationParameters parameters) {
- AmazonS3 s3 = mock(AmazonS3.class);
- String bucket = uri.getHost();
- when(s3.doesBucketExist(bucket)).thenReturn(true);
- when(s3.doesBucketExistV2(bucket)).thenReturn(true);
- // this listing is used in startup if purging is enabled, so
- // return a stub value
- MultipartUploadListing noUploads = new MultipartUploadListing();
- noUploads.setMultipartUploads(new ArrayList<>(0));
- when(s3.listMultipartUploads(any())).thenReturn(noUploads);
- when(s3.getBucketLocation(anyString())).thenReturn(Region.US_West.toString());
- return s3;
- }
- //TODO: This is incomplete, add in mocks as we update operations
@Override
- public S3Client createS3ClientV2(URI uri, final S3ClientCreationParameters parameters) {
+ public S3Client createS3Client(URI uri, final S3ClientCreationParameters parameters) {
S3Client s3 = mock(S3Client.class);
+ // this listing is used in startup if purging is enabled, so
+ // return a stub value
+ ListMultipartUploadsResponse noUploads = ListMultipartUploadsResponse.builder()
+ .uploads(new ArrayList<>(0))
+ .isTruncated(false)
+ .build();
+ when(s3.listMultipartUploads((ListMultipartUploadsRequest) any())).thenReturn(noUploads);
+ when(s3.getBucketLocation((GetBucketLocationRequest) any())).thenReturn(
+ GetBucketLocationResponse.builder().locationConstraint(Region.US_WEST_2.toString())
+ .build());
return s3;
}
@@ -68,8 +62,7 @@ public S3AsyncClient createS3AsyncClient(URI uri, final S3ClientCreationParamete
}
@Override
- public S3TransferManager createS3TransferManager(URI uri, S3ClientCreationParameters parameters)
- throws IOException {
+ public S3TransferManager createS3TransferManager(S3AsyncClient s3AsyncClient) {
S3TransferManager tm = mock(S3TransferManager.class);
return tm;
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
index 957db1a038f4b..1eee096180b16 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
@@ -34,11 +34,14 @@
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.ContainerCredentialsProvider;
import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
import software.amazon.awssdk.auth.credentials.AwsCredentials;
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
import software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider;
import software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider;
+
+import org.apache.hadoop.fs.s3a.adapter.V1V2AwsCredentialProviderAdapter;
import org.apache.hadoop.util.Sets;
import org.junit.Rule;
import org.junit.Test;
@@ -55,6 +58,11 @@
import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.auth.AwsCredentialListProvider.ABSTRACT_PROVIDER;
+import static org.apache.hadoop.fs.s3a.auth.AwsCredentialListProvider.NOT_AWS_V2_PROVIDER;
+import static org.apache.hadoop.fs.s3a.auth.AwsCredentialListProvider.STANDARD_AWS_PROVIDERS;
+import static org.apache.hadoop.fs.s3a.auth.AwsCredentialListProvider.buildAWSProviderList;
+import static org.apache.hadoop.fs.s3a.auth.AwsCredentialListProvider.createAWSCredentialProviderSet;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
import static org.junit.Assert.*;
@@ -62,7 +70,6 @@
/**
* Unit tests for {@link Constants#AWS_CREDENTIALS_PROVIDER} logic.
*/
-// TODO: Add new tests that use a mix of V1 and V2 providers and assert that everything works ok.
public class TestS3AAWSCredentialsProvider {
/**
@@ -149,6 +156,27 @@ public void testDefaultChainNoURI() throws Exception {
createAWSCredentialProviderSet(null, conf));
}
+ @Test
+ public void testConfiguredChainV1V2() throws Exception {
+ URI uri1 = new URI("s3a://bucket1"), uri2 = new URI("s3a://bucket2");
+ List> credentialProviders =
+ Arrays.asList(
+ ContainerCredentialsProvider.class,
+ AnonymousAWSCredentialsProvider.class);
+ List> expectedClasses =
+ Arrays.asList(
+ V1V2AwsCredentialProviderAdapter.class,
+ AnonymousAWSCredentialsProvider.class);
+ Configuration conf =
+ createProviderConfiguration(buildClassListString(credentialProviders));
+ AWSCredentialProviderList list1 = createAWSCredentialProviderSet(
+ uri1, conf);
+ AWSCredentialProviderList list2 = createAWSCredentialProviderSet(
+ uri2, conf);
+ assertCredentialProviders(expectedClasses, list1);
+ assertCredentialProviders(expectedClasses, list2);
+ }
+
@Test
public void testConfiguredChain() throws Exception {
URI uri1 = new URI("s3a://bucket1"), uri2 = new URI("s3a://bucket2");
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java
index a89f1744fd2f9..1f85c8fdef304 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java
@@ -75,7 +75,7 @@ public void testDeleteOnExit() throws Exception {
// unset S3CSE property from config to avoid pathIOE.
conf.unset(Constants.S3_ENCRYPTION_ALGORITHM);
testFs.initialize(uri, conf);
- S3Client testS3 = testFs.getAmazonS3V2ClientForTesting("mocking");
+ S3Client testS3 = testFs.getAmazonS3ClientForTesting("mocking");
Path path = new Path("/file");
String key = path.toUri().getPath().substring(1);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java
index fd186e51427cc..33e6b4a08145c 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AExceptionTranslation.java
@@ -30,14 +30,13 @@
import java.io.InterruptedIOException;
import java.net.SocketTimeoutException;
import java.nio.file.AccessDeniedException;
-import java.util.Collections;
-import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.function.Consumer;
import software.amazon.awssdk.awscore.exception.AwsErrorDetails;
import software.amazon.awssdk.awscore.exception.AwsServiceException;
import software.amazon.awssdk.core.exception.SdkException;
+import software.amazon.awssdk.http.SdkHttpResponse;
import software.amazon.awssdk.services.s3.model.S3Exception;
import org.junit.Test;
@@ -61,21 +60,26 @@ public class TestS3AExceptionTranslation {
= new SocketTimeoutException("socket");
@Test
- public void test301ContainsEndpoint() throws Exception {
- String bucket = "bucket.s3-us-west-2.amazonaws.com";
- S3Exception s3Exception = createS3Exception("wrong endpoint",
+ public void test301ContainsRegion() throws Exception {
+ String region = "us-west-1";
+
+ AwsErrorDetails redirectError = AwsErrorDetails.builder()
+ .sdkHttpResponse(
+ SdkHttpResponse.builder().putHeader(BUCKET_REGION_HEADER, region).build())
+ .build();
+
+ S3Exception s3Exception = createS3Exception("wrong region",
SC_301_MOVED_PERMANENTLY,
- Collections.singletonMap(S3AUtils.ENDPOINT_KEY,
- bucket));
+ redirectError);
AWSRedirectException ex = verifyTranslated(
AWSRedirectException.class, s3Exception);
assertStatusCode(SC_301_MOVED_PERMANENTLY, ex);
assertNotNull(ex.getMessage());
- assertContained(ex.getMessage(), bucket);
- assertContained(ex.getMessage(), ENDPOINT);
- assertExceptionContains(ENDPOINT, ex, "endpoint");
- assertExceptionContains(bucket, ex, "bucket name");
+ assertContained(ex.getMessage(), region);
+ assertContained(ex.getMessage(), AWS_REGION);
+ assertExceptionContains(AWS_REGION, ex, "region");
+ assertExceptionContains(region, ex, "region name");
}
protected void assertContained(String text, String contained) {
@@ -189,16 +193,13 @@ private static S3Exception createS3Exception(int code) {
}
private static S3Exception createS3Exception(String message, int code,
- Map additionalDetails) {
+ AwsErrorDetails additionalDetails) {
+
S3Exception source = (S3Exception) S3Exception.builder()
.message(message)
.statusCode(code)
+ .awsErrorDetails(additionalDetails)
.build();
- // TODO: is there an equivalent for v2?
- // currently used to retrieve endpoint on redirect
- // see S3AUtils.translateException and
- // https://github.com/aws/aws-sdk-java-v2/issues/3048
- // source.setAdditionalDetails(additionalDetails);
return source;
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AProxy.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AProxy.java
index e05ee25adfa74..0982c8cbd4761 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AProxy.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AProxy.java
@@ -20,18 +20,17 @@
import java.io.IOException;
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.Protocol;
import org.assertj.core.api.Assertions;
import org.junit.Test;
+import software.amazon.awssdk.http.apache.ProxyConfiguration;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.impl.AWSClientConfig;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.fs.s3a.Constants.PROXY_HOST;
import static org.apache.hadoop.fs.s3a.Constants.PROXY_PORT;
import static org.apache.hadoop.fs.s3a.Constants.PROXY_SECURED;
-import static org.apache.hadoop.fs.s3a.S3AUtils.initProxySupport;
/**
* Tests to verify {@link S3AUtils} translates the proxy configurations
@@ -79,11 +78,16 @@ public void testProxyDefault() throws IOException {
private void verifyProxy(Configuration proxyConfig,
boolean isExpectedSecured)
throws IOException {
- ClientConfiguration awsConf = new ClientConfiguration();
- initProxySupport(proxyConfig, "test-bucket", awsConf);
- Assertions.assertThat(awsConf.getProxyProtocol())
+ ProxyConfiguration config =
+ AWSClientConfig.createProxyConfiguration(proxyConfig, "testBucket");
+ ProxyConfiguration asyncConfig =
+ AWSClientConfig.createProxyConfiguration(proxyConfig, "testBucket");
+ Assertions.assertThat(config.scheme())
.describedAs("Proxy protocol not as expected")
- .isEqualTo(isExpectedSecured ? Protocol.HTTPS : Protocol.HTTP);
+ .isEqualTo(isExpectedSecured ? "https" : "http");
+ Assertions.assertThat(asyncConfig.scheme())
+ .describedAs("Proxy protocol not as expected")
+ .isEqualTo(isExpectedSecured ? "https" : "http");
}
/**
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestWildflyAndOpenSSLBinding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestWildflyAndOpenSSLBinding.java
index a2b013f468a79..9e903fd85ff49 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestWildflyAndOpenSSLBinding.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestWildflyAndOpenSSLBinding.java
@@ -20,10 +20,9 @@
import java.io.IOException;
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.Protocol;
import org.junit.Before;
import org.junit.Test;
+import software.amazon.awssdk.http.apache.ApacheHttpClient;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
@@ -74,7 +73,7 @@ public void testUnknownMode() throws Throwable {
Configuration conf = new Configuration(false);
conf.set(SSL_CHANNEL_MODE, "no-such-mode ");
intercept(IllegalArgumentException.class, () ->
- bindSSLChannelMode(conf, new ClientConfiguration()));
+ bindSSLChannelMode(conf, ApacheHttpClient.builder()));
}
@Test
@@ -143,9 +142,7 @@ private DelegatingSSLSocketFactory.SSLChannelMode bindSocketFactory(
DelegatingSSLSocketFactory.resetDefaultFactory();
Configuration conf = new Configuration(false);
conf.set(SSL_CHANNEL_MODE, channelMode.name());
- ClientConfiguration awsConf = new ClientConfiguration();
- awsConf.setProtocol(Protocol.HTTPS);
- bindSSLChannelMode(conf, awsConf);
+ bindSSLChannelMode(conf, ApacheHttpClient.builder());
return DelegatingSSLSocketFactory.getDefaultFactory().getChannelMode();
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
index 70d91ba7b113f..1c6e00655acb2 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
@@ -58,7 +58,7 @@
import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
-import static org.apache.hadoop.fs.s3a.S3AUtils.*;
+import static org.apache.hadoop.fs.s3a.auth.AwsCredentialListProvider.E_FORBIDDEN_AWS_PROVIDER;
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.*;
import static org.apache.hadoop.fs.s3a.auth.RoleModel.*;
import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*;
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java
index a829d470e7a66..cdf89211fd7fc 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java
@@ -25,12 +25,12 @@
import java.util.Objects;
import java.util.concurrent.atomic.AtomicInteger;
-import com.amazonaws.SignableRequest;
-import com.amazonaws.auth.AWS4Signer;
-import com.amazonaws.arn.Arn;
-import com.amazonaws.auth.AWSCredentials;
-import com.amazonaws.auth.Signer;
-import com.amazonaws.services.s3.internal.AWSS3V4Signer;
+import software.amazon.awssdk.arns.Arn;
+import software.amazon.awssdk.auth.signer.Aws4Signer;
+import software.amazon.awssdk.auth.signer.AwsS3V4Signer;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.core.signer.Signer;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
import org.assertj.core.api.Assertions;
import org.junit.Test;
import org.slf4j.Logger;
@@ -183,14 +183,15 @@ public CustomSigner() {
* request because the signature calculated by the service doesn't match
* what we sent.
* @param request the request to sign.
- * @param credentials credentials used to sign the request.
+ * @param executionAttributes request executionAttributes which contain the credentials.
*/
@Override
- public void sign(SignableRequest> request, AWSCredentials credentials) {
+ public SdkHttpFullRequest sign(SdkHttpFullRequest request,
+ ExecutionAttributes executionAttributes) {
int c = INVOCATION_COUNT.incrementAndGet();
LOG.info("Signing request #{}", c);
- String host = request.getEndpoint().getHost();
+ String host = request.host();
String bucketName = parseBucketFromHost(host);
try {
lastStoreValue = CustomSignerInitializer
@@ -199,19 +200,11 @@ public void sign(SignableRequest> request, AWSCredentials credentials) {
throw new RuntimeException("Failed to get current Ugi", e);
}
if (bucketName.equals("kms")) {
- AWS4Signer realKMSSigner = new AWS4Signer();
- realKMSSigner.setServiceName("kms");
- if (lastStoreValue != null) {
- realKMSSigner.setRegionName(lastStoreValue.conf.get(TEST_REGION_KEY));
- }
- realKMSSigner.sign(request, credentials);
+ Aws4Signer realKMSSigner = Aws4Signer.create();
+ return realKMSSigner.sign(request, executionAttributes);
} else {
- AWSS3V4Signer realSigner = new AWSS3V4Signer();
- realSigner.setServiceName("s3");
- if (lastStoreValue != null) {
- realSigner.setRegionName(lastStoreValue.conf.get(TEST_REGION_KEY));
- }
- realSigner.sign(request, credentials);
+ AwsS3V4Signer realSigner = AwsS3V4Signer.create();
+ return realSigner.sign(request, executionAttributes);
}
}
@@ -235,11 +228,11 @@ private String parseBucketFromHost(String host) {
String accessPointName =
bucketName.substring(0, bucketName.length() - (accountId.length() + 1));
Arn arn = Arn.builder()
- .withAccountId(accountId)
- .withPartition("aws")
- .withRegion(hostBits[2])
- .withResource("accesspoint" + "/" + accessPointName)
- .withService("s3").build();
+ .accountId(accountId)
+ .partition("aws")
+ .region(hostBits[2])
+ .resource("accesspoint" + "/" + accessPointName)
+ .service("s3").build();
bucketName = arn.toString();
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java
index ca87b5c1b34a6..595e2687276b1 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/TestSignerManager.java
@@ -19,8 +19,6 @@
import java.io.Closeable;
import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
import java.nio.charset.StandardCharsets;
import java.security.PrivilegedExceptionAction;
import java.util.HashMap;
@@ -28,12 +26,10 @@
import java.util.Objects;
import java.util.concurrent.TimeUnit;
-import com.amazonaws.AmazonWebServiceRequest;
-import com.amazonaws.DefaultRequest;
-import com.amazonaws.SignableRequest;
-import com.amazonaws.auth.AWSCredentials;
-import com.amazonaws.auth.Signer;
-import com.amazonaws.auth.SignerFactory;
+import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
+import software.amazon.awssdk.core.signer.Signer;
+import software.amazon.awssdk.http.SdkHttpFullRequest;
+import software.amazon.awssdk.http.SdkHttpMethod;
import org.assertj.core.api.Assertions;
import org.junit.Before;
import org.junit.Rule;
@@ -284,7 +280,7 @@ private void attemptSignAndVerify(String identifier, String bucket,
throws IOException, InterruptedException {
ugi.doAs((PrivilegedExceptionAction) () -> {
Signer signer = new SignerForInitializerTest();
- SignableRequest> signableRequest = constructSignableRequest(bucket);
+ SdkHttpFullRequest signableRequest = constructSignableRequest(bucket);
signer.sign(signableRequest, null);
verifyStoreValueInSigner(expectNullStoreInfo, bucket, identifier);
return null;
@@ -336,8 +332,10 @@ public static class SignerForTest1 implements Signer {
private static boolean initialized = false;
@Override
- public void sign(SignableRequest> request, AWSCredentials credentials) {
+ public SdkHttpFullRequest sign(SdkHttpFullRequest sdkHttpFullRequest,
+ ExecutionAttributes executionAttributes) {
initialized = true;
+ return sdkHttpFullRequest;
}
public static void reset() {
@@ -354,8 +352,10 @@ public static class SignerForTest2 implements Signer {
private static boolean initialized = false;
@Override
- public void sign(SignableRequest> request, AWSCredentials credentials) {
+ public SdkHttpFullRequest sign(SdkHttpFullRequest sdkHttpFullRequest,
+ ExecutionAttributes executionAttributes) {
initialized = true;
+ return sdkHttpFullRequest;
}
public static void reset() {
@@ -472,11 +472,15 @@ public static class SignerForInitializerTest implements Signer {
private static StoreValue retrievedStoreValue;
@Override
- public void sign(SignableRequest> request, AWSCredentials credentials) {
- String bucketName = request.getEndpoint().getHost();
+ public SdkHttpFullRequest sign(SdkHttpFullRequest sdkHttpFullRequest,
+ ExecutionAttributes executionAttributes) {
+ String bucket = sdkHttpFullRequest.host().split("//")[1];
+ // remove trailing slash
+ String bucketName = bucket.substring(0, bucket.length() - 1);
try {
retrievedStoreValue = SignerInitializerForTest
.getStoreInfo(bucketName, UserGroupInformation.getCurrentUser());
+ return sdkHttpFullRequest;
} catch (IOException e) {
throw new RuntimeException("Failed to get current ugi", e);
}
@@ -579,12 +583,9 @@ private String createTokenIdentifierString(String identifier,
return identifier + "_" + bucketName + "_" + user;
}
- private SignableRequest> constructSignableRequest(String bucketName)
- throws URISyntaxException {
- DefaultRequest signableRequest = new DefaultRequest(
- AmazonWebServiceRequest.NOOP, "fakeservice");
- URI uri = new URI("s3://" + bucketName + "/");
- signableRequest.setEndpoint(uri);
- return signableRequest;
+ private SdkHttpFullRequest constructSignableRequest(String bucketName) {
+ String host = "s3://" + bucketName + "/";
+ return SdkHttpFullRequest.builder().host(host).protocol("https").method(SdkHttpMethod.GET)
+ .build();
}
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java
index 39d0e34ee807b..728de38f7d204 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFileystem.java
@@ -26,6 +26,7 @@
import java.net.URI;
import java.nio.file.AccessDeniedException;
+import software.amazon.awssdk.regions.Region;
import software.amazon.awssdk.services.s3.S3Client;
import software.amazon.awssdk.services.s3.model.HeadBucketResponse;
import org.junit.AfterClass;
@@ -176,8 +177,7 @@ protected Configuration createConfiguration() {
conf.set(YarnConfiguration.RM_PRINCIPAL, YARN_RM);
// turn on ACLs so as to verify role DT permissions include
// write access.
- // TODO: Why do we need this? Can we get rid of ACLs?
- // conf.set(CANNED_ACL, LOG_DELIVERY_WRITE);
+ conf.set(CANNED_ACL, LOG_DELIVERY_WRITE);
return conf;
}
@@ -330,10 +330,9 @@ public void testDelegatedFileSystem() throws Throwable {
+ " if role restricted, permissions are tightened.");
S3AFileSystem fs = getFileSystem();
// force a probe of the remote FS to make sure its endpoint is valid
- // TODO: Previously a call to getObjectMetadata for a base path, ie with an empty key would
- // return some metadata. (bucket region, content type). headObject() fails without a key, check
- // how this can be fixed.
- // fs.getObjectMetadata(new Path("/"));
+ // TODO: Check what should happen here. Calling headObject() on the root path fails in V2,
+ // with the error that key cannot be empty.
+ // fs.getObjectMetadata(new Path("/"));
readLandsatMetadata(fs);
URI uri = fs.getUri();
@@ -588,17 +587,18 @@ protected HeadBucketResponse readLandsatMetadata(final S3AFileSystem delegatedFS
URI landsat = new URI(DEFAULT_CSVTEST_FILE);
DefaultS3ClientFactory factory
= new DefaultS3ClientFactory();
- factory.setConf(new Configuration(delegatedFS.getConf()));
+ Configuration conf = delegatedFS.getConf();
+ factory.setConf(conf);
String host = landsat.getHost();
S3ClientFactory.S3ClientCreationParameters parameters = null;
parameters = new S3ClientFactory.S3ClientCreationParameters()
.withCredentialSet(testingCreds)
.withPathUri(new URI("s3a://localhost/"))
- .withEndpoint(DEFAULT_ENDPOINT)
.withMetrics(new EmptyS3AStatisticsContext()
.newStatisticsFromAwsSdk())
- .withUserAgentSuffix("ITestSessionDelegationInFileystem");
- S3Client s3 = factory.createS3ClientV2(landsat, parameters);
+ .withUserAgentSuffix("ITestSessionDelegationInFileystem")
+ .withRegion(Region.US_WEST_2);
+ S3Client s3 = factory.createS3Client(landsat, parameters);
return Invoker.once("HEAD", host,
() -> s3.headBucket(b -> b.bucket(host)));
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
index d28f4279f1ba0..fbad671e1fa66 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java
@@ -16,7 +16,6 @@
import java.io.IOException;
import java.net.URI;
-import com.amazonaws.services.s3.model.CryptoStorageMode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -77,7 +76,7 @@ protected void verifyReadBytes(FileSystem.Statistics stats) {
*
* NOTE: if Client side encryption is enabled, expected bytes written
* should increase by 16(padding of data) + bytes for the key ID set + 94(KMS
- * key generation) in case of storage type{@link CryptoStorageMode} as
+ * key generation) in case of storage type CryptoStorageMode as
* ObjectMetadata(Default). If Crypto Storage mode is instruction file then
* add additional bytes as that file is stored separately and would account
* for bytes written.
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java
index b521a81a94942..3a390e34ecad2 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java
@@ -64,9 +64,6 @@ public ITestXAttrCost() {
@Test
public void testXAttrRoot() throws Throwable {
describe("Test xattr on root");
- // TODO: Previously a call to getObjectMetadata for a base path, ie with an empty key would
- // return some metadata. (bucket region, content type). headObject() fails without a key, check
- // how this can be fixed.
Path root = new Path("/");
S3AFileSystem fs = getFileSystem();
Map xAttrs = verifyMetrics(
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java
index 9d026fd90ee50..81bd8a5efe2e4 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestHeaderProcessing.java
@@ -27,6 +27,7 @@
import java.util.Map;
import software.amazon.awssdk.services.s3.model.CopyObjectRequest;
+import software.amazon.awssdk.services.s3.model.HeadBucketResponse;
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
import org.assertj.core.api.Assertions;
import org.assertj.core.util.Lists;
@@ -321,6 +322,11 @@ public HeadObjectResponse getObjectMetadata(final String key)
}
+ @Override
+ public HeadBucketResponse getBucketMetadata() throws IOException {
+ return HeadBucketResponse.builder().build();
+ }
+
public void setHeader(String key, String val) {
userHeaders.put(key, val);
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java
index 7f51d2b45362c..919a89b8c1dd0 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNetworkBinding.java
@@ -18,14 +18,10 @@
package org.apache.hadoop.fs.s3a.impl;
-import com.amazonaws.ClientConfiguration;
-import com.amazonaws.client.builder.AwsClientBuilder;
-import org.junit.Ignore;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
-import static org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createEndpointConfiguration;
import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
import static org.assertj.core.api.Assertions.assertThat;
@@ -63,43 +59,4 @@ private static void assertRegionFixup(String region, String expected) {
.describedAs("Fixup of %s", region)
.isEqualTo(expected);
}
-
- @Test
- public void testNull() throws Throwable {
- expectEndpoint("", true, "unused");
- }
-
- @Test
- @Ignore("disabled until endpoint logic works for S3 client builder API")
- public void testUSEastEndpoint() throws Throwable {
- expectEndpoint(US_EAST_1, false, US_EAST_1);
- }
-
- @Test
- @Ignore("disabled until endpoint logic works for S3 client builder API")
- public void testUSWestEndpoint() throws Throwable {
- expectEndpoint(US_WEST_2, false, US_WEST_2);
- }
-
- public void expectEndpoint(final String src,
- final boolean expectNull,
- final String expectRegion) {
- AwsClientBuilder.EndpointConfiguration epr =
- createEndpointConfiguration(src, new ClientConfiguration(), src);
- String eprStr = epr == null
- ? "(empty)"
- : ("(" + epr.getServiceEndpoint() + " " + epr.getSigningRegion());
- if (expectNull) {
- assertThat(epr)
- .describedAs("Endpoint configuration of %s =",
- src, eprStr)
- .isNull();
- } else {
- assertThat(epr)
- .describedAs("Endpoint configuration of %s =",
- src, eprStr)
- .hasFieldOrPropertyWithValue("serviceEndpoint", src)
- .hasFieldOrPropertyWithValue("signingRegion", expectRegion);
- }
- }
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java
index e1e9cc5aeee5c..a2194ee5f4a23 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java
@@ -24,7 +24,6 @@
import software.amazon.awssdk.awscore.AwsRequest;
import software.amazon.awssdk.core.SdkRequest;
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
-import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
import org.assertj.core.api.Assertions;
import org.junit.Test;
import org.slf4j.Logger;
@@ -80,7 +79,7 @@ public void testRequestFactoryWithEncryption() throws Throwable {
*/
@Test
public void testRequestFactoryWithCannedACL() throws Throwable {
- ObjectCannedACL acl = ObjectCannedACL.BUCKET_OWNER_FULL_CONTROL;
+ String acl = "bucket-owner-full-control";
RequestFactory factory = RequestFactoryImpl.builder()
.withBucket("bucket")
.withCannedACL(acl)
@@ -91,17 +90,20 @@ public void testRequestFactoryWithCannedACL() throws Throwable {
Assertions.assertThat(factory.newPutObjectRequestBuilder(path, null, 128, false)
.build()
- .acl())
+ .acl()
+ .toString())
.describedAs("ACL of PUT")
.isEqualTo(acl);
Assertions.assertThat(factory.newCopyObjectRequestBuilder(path, path2, md)
.build()
- .acl())
+ .acl()
+ .toString())
.describedAs("ACL of COPY")
.isEqualTo(acl);
Assertions.assertThat(factory.newMultipartUploadRequestBuilder(path, null)
.build()
- .acl())
+ .acl()
+ .toString())
.describedAs("ACL of MPU")
.isEqualTo(acl);
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java
index 9555e8316380c..de0048c25581c 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java
@@ -214,7 +214,7 @@ protected Configuration createConfiguration() {
public void setup() throws Exception {
super.setup();
S3AFileSystem fs = getFileSystem();
- s3client = fs.getAmazonS3V2ClientForTesting("markers");
+ s3client = fs.getAmazonS3ClientForTesting("markers");
bucket = fs.getBucket();
Path base = new Path(methodPath(), "base");
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
index f8d47011de3f0..dd30e72cba237 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
@@ -27,9 +27,6 @@
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.IntFunction;
-import com.amazonaws.event.ProgressEvent;
-import com.amazonaws.event.ProgressEventType;
-import com.amazonaws.event.ProgressListener;
import org.assertj.core.api.Assertions;
import org.junit.FixMethodOrder;
import org.junit.Test;
@@ -49,6 +46,8 @@
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.Statistic;
+import org.apache.hadoop.fs.s3a.impl.ProgressListener;
+import org.apache.hadoop.fs.s3a.impl.ProgressListenerEvent;
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.util.Progressable;
@@ -299,10 +298,9 @@ protected int getPartitionSize() {
}
/**
- * Progress callback from AWS. Likely to come in on a different thread.
+ * Progress callback.
*/
- private final class ProgressCallback implements Progressable,
- ProgressListener {
+ private final class ProgressCallback implements Progressable, ProgressListener {
private AtomicLong bytesTransferred = new AtomicLong(0);
private AtomicInteger failures = new AtomicInteger(0);
private final ContractTestUtils.NanoTimer timer;
@@ -316,11 +314,8 @@ public void progress() {
}
@Override
- public void progressChanged(ProgressEvent progressEvent) {
- ProgressEventType eventType = progressEvent.getEventType();
- if (eventType.isByteCountEvent()) {
- bytesTransferred.addAndGet(progressEvent.getBytesTransferred());
- }
+ public void progressChanged(ProgressListenerEvent eventType, int transferredBytes) {
+
switch (eventType) {
case TRANSFER_PART_FAILED_EVENT:
// failure
@@ -329,6 +324,7 @@ public void progressChanged(ProgressEvent progressEvent) {
break;
case TRANSFER_PART_COMPLETED_EVENT:
// completion
+ bytesTransferred.addAndGet(transferredBytes);
long elapsedTime = timer.elapsedTime();
double elapsedTimeS = elapsedTime / 1.0e9;
long written = bytesTransferred.get();
@@ -336,20 +332,15 @@ public void progressChanged(ProgressEvent progressEvent) {
LOG.info(String.format(
"Event %s; total uploaded=%d MB in %.1fs;" +
" effective upload bandwidth = %.2f MB/s",
- progressEvent,
+ eventType,
writtenMB, elapsedTimeS, writtenMB / elapsedTimeS));
break;
default:
- if (eventType.isByteCountEvent()) {
- LOG.debug("Event {}", progressEvent);
- } else {
- LOG.info("Event {}", progressEvent);
- }
+ // nothing
break;
}
}
- @Override
public String toString() {
String sb = "ProgressCallback{"
+ "bytesTransferred=" + bytesTransferred +
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
index 8b7cf21bef010..968292aefc5a3 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java
@@ -258,7 +258,7 @@ public void testMultiPagesListingPerformanceAndCorrectness()
originalListOfFiles.add(file.toString());
PutObjectRequest.Builder putObjectRequestBuilder = requestFactory
.newPutObjectRequestBuilder(fs.pathToKey(file),
- null, 128, false);
+ null, 0, false);
futures.add(submit(executorService,
() -> writeOperationHelper.putObject(putObjectRequestBuilder.build(),
PutObjectOptions.keepingDirs(),
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java
index 51da971fb7063..4d4af822ee50b 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java
@@ -415,7 +415,7 @@ public void testSelectSeekFullLandsat() throws Throwable {
long increment = 64 * _1KB;
// seek forward, comparing bytes
- for(offset = 32 * _1KB; offset < _1MB; offset += increment) {
+ for(offset = 32 * _1KB; offset < 256 * _1KB; offset += increment) {
seek(seekStream, offset);
assertEquals("Seek position in " + seekStream,
offset, seekStream.getPos());
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java
index 127fcbab75023..ab22c51f28b7b 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java
@@ -224,13 +224,6 @@ public void testRunNoArgs() throws Throwable {
runToFailure(EXIT_USAGE, MARKERS);
}
- @Test
- public void testRunWrongBucket() throws Throwable {
- runToFailure(EXIT_NOT_FOUND, MARKERS,
- AUDIT,
- "s3a://this-bucket-does-not-exist-hopefully");
- }
-
/**
* Run with a path that doesn't exist.
*/
diff --git a/hadoop-tools/hadoop-aws/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/hadoop-tools/hadoop-aws/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
deleted file mode 100644
index 3b308f19255c3..0000000000000
--- a/hadoop-tools/hadoop-aws/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
+++ /dev/null
@@ -1,13 +0,0 @@
-# Licensed under the Apache License, Version 2.0 (the "License");
-# you may not use this file except in compliance with the License.
-# You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-mock-maker-inline
\ No newline at end of file