From 77c5cf04dc148fca7b1c5874be69b1a2cd47ed21 Mon Sep 17 00:00:00 2001 From: Ahmar Suhail Date: Thu, 22 Sep 2022 13:47:30 +0100 Subject: [PATCH 1/3] configures s3 client, updates list operation --- hadoop-project/pom.xml | 12 + hadoop-tools/hadoop-aws/pom.xml | 5 + .../apache/hadoop/fs/s3a/AWSClientConfig.java | 247 ++++++++++++++++++ .../org/apache/hadoop/fs/s3a/Constants.java | 10 + .../hadoop/fs/s3a/DefaultS3ClientFactory.java | 167 ++++++++++++ .../org/apache/hadoop/fs/s3a/Listing.java | 53 ++-- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 54 +++- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 23 +- .../apache/hadoop/fs/s3a/S3ClientFactory.java | 13 + .../apache/hadoop/fs/s3a/S3ListRequest.java | 16 +- .../apache/hadoop/fs/s3a/S3ListResult.java | 61 ++--- .../V1ToV2AwsCredentialProviderAdapter.java | 70 +++++ .../V1V2AwsCredentialProviderAdapter.java | 36 +++ .../hadoop/fs/s3a/adapter/package-info.java | 27 ++ .../hadoop/fs/s3a/api/RequestFactory.java | 4 +- .../hadoop/fs/s3a/impl/ChangeTracker.java | 5 +- .../fs/s3a/impl/RequestFactoryImpl.java | 44 +++- .../hadoop/fs/s3a/AbstractS3AMockTest.java | 7 + .../hadoop/fs/s3a/MockS3ClientFactory.java | 14 +- .../hadoop/fs/s3a/TestS3AGetFileStatus.java | 59 +++-- .../fs/s3a/impl/TestRequestFactory.java | 6 +- .../org.mockito.plugins.MockMaker | 13 + 22 files changed, 809 insertions(+), 137 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientConfig.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1ToV2AwsCredentialProviderAdapter.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1V2AwsCredentialProviderAdapter.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/package-info.java create mode 100644 hadoop-tools/hadoop-aws/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 6d5509b2fa68e..bbd2953e3ba3c 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -186,6 +186,7 @@ 900 1.12.262 2.5.2 + 2.17.196 1.11.2 2.1 0.7 @@ -1116,6 +1117,17 @@ + + software.amazon.awssdk + bundle + ${aws-java-sdk-v2.version} + + + io.netty + * + + + org.apache.mina mina-core diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 6ebf1c71f0d5b..4eb9508fc68ae 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -492,6 +492,11 @@ aws-java-sdk-bundle compile + + software.amazon.awssdk + bundle + compile + org.assertj assertj-core diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientConfig.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientConfig.java new file mode 100644 index 0000000000000..40ee14d86daa2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientConfig.java @@ -0,0 +1,247 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.time.Duration; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.http.apache.ProxyConfiguration; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.VersionInfo; +import org.apache.http.client.utils.URIBuilder; + +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; +import static org.apache.hadoop.fs.s3a.Constants.MAX_ERROR_RETRIES; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_DOMAIN; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_HOST; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_PASSWORD; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_PORT; +import static org.apache.hadoop.fs.s3a.Constants.PROXY_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.SOCKET_TIMEOUT; +import static org.apache.hadoop.fs.s3a.Constants.USER_AGENT_PREFIX; + +/** + * Methods for configuring the S3 client. + * These methods are used when creating and configuring + * {@link software.amazon.awssdk.services.s3.S3Client} which communicates with the S3 service. + */ +public final class AWSClientConfig { + private static final Logger LOG = LoggerFactory.getLogger(AWSClientConfig.class); + + private AWSClientConfig() { + } + + public static ClientOverrideConfiguration.Builder createClientConfigBuilder(Configuration conf) { + ClientOverrideConfiguration.Builder overrideConfigBuilder = + ClientOverrideConfiguration.builder(); + + initRequestTimeout(conf, overrideConfigBuilder); + + 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) + // } + + return overrideConfigBuilder; + } + + /** + * Configures the http client. + * + * @param conf The Hadoop configuration + * @return Http client builder + */ + public static ApacheHttpClient.Builder createHttpClientBuilder(Configuration conf) { + ApacheHttpClient.Builder httpClientBuilder = + ApacheHttpClient.builder(); + + httpClientBuilder.maxConnections(S3AUtils.intOption(conf, MAXIMUM_CONNECTIONS, + DEFAULT_MAXIMUM_CONNECTIONS, 1)); + + int connectionEstablishTimeout = + S3AUtils.intOption(conf, ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT, 0); + int socketTimeout = S3AUtils.intOption(conf, SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT, 0); + + httpClientBuilder.connectionTimeout(Duration.ofSeconds(connectionEstablishTimeout)); + httpClientBuilder.socketTimeout(Duration.ofSeconds(socketTimeout)); + + // TODO: Need to set ssl socket factory, as done in + // NetworkBinding.bindSSLChannelMode(conf, awsConf); + + return httpClientBuilder; + } + + /** + * Configures the retry policy. + * + * @param conf The Hadoop configuration + * @return Retry policy builder + */ + public static RetryPolicy.Builder createRetryPolicyBuilder(Configuration conf) { + + RetryPolicy.Builder retryPolicyBuilder = RetryPolicy.builder(); + + retryPolicyBuilder.numRetries(S3AUtils.intOption(conf, MAX_ERROR_RETRIES, + DEFAULT_MAX_ERROR_RETRIES, 0)); + + return retryPolicyBuilder; + } + + /** + * Configures the proxy. + * + * @param conf The Hadoop configuration + * @param bucket Optional bucket to use to look up per-bucket proxy secrets + * @return Proxy configuration builder + * @throws IOException on any IO problem + */ + public static ProxyConfiguration.Builder createProxyConfigurationBuilder(Configuration conf, + String bucket) throws IOException { + + ProxyConfiguration.Builder proxyConfigBuilder = ProxyConfiguration.builder(); + + String proxyHost = conf.getTrimmed(PROXY_HOST, ""); + int proxyPort = conf.getInt(PROXY_PORT, -1); + + if (!proxyHost.isEmpty()) { + if (proxyPort >= 0) { + proxyConfigBuilder.endpoint(buildURI(proxyHost, proxyPort)); + } else { + if (conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS)) { + LOG.warn("Proxy host set without port. Using HTTPS default 443"); + proxyConfigBuilder.endpoint(buildURI(proxyHost, 443)); + } else { + LOG.warn("Proxy host set without port. Using HTTP default 80"); + proxyConfigBuilder.endpoint(buildURI(proxyHost, 80)); + } + } + final String proxyUsername = S3AUtils.lookupPassword(bucket, conf, PROXY_USERNAME, + null, null); + final String proxyPassword = S3AUtils.lookupPassword(bucket, conf, PROXY_PASSWORD, + null, null); + if ((proxyUsername == null) != (proxyPassword == null)) { + String msg = "Proxy error: " + PROXY_USERNAME + " or " + + PROXY_PASSWORD + " set without the other."; + LOG.error(msg); + throw new IllegalArgumentException(msg); + } + proxyConfigBuilder.username(proxyUsername); + proxyConfigBuilder.password(proxyPassword); + proxyConfigBuilder.ntlmDomain(conf.getTrimmed(PROXY_DOMAIN)); + proxyConfigBuilder.ntlmWorkstation(conf.getTrimmed(PROXY_WORKSTATION)); + if (LOG.isDebugEnabled()) { + LOG.debug("Using proxy server {}:{} as user {} with password {} on " + + "domain {} as workstation {}", proxyHost, proxyPort, proxyUsername, proxyPassword, + PROXY_DOMAIN, PROXY_WORKSTATION); + } + } else if (proxyPort >= 0) { + String msg = + "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST; + LOG.error(msg); + throw new IllegalArgumentException(msg); + } + + return proxyConfigBuilder; + } + + /*** + * Builds a URI, throws an IllegalArgumentException in case of errors. + * + * @param host proxy host + * @param port proxy port + * @return uri with host and port + */ + private static URI buildURI(String host, int port) { + try { + return new URIBuilder().setHost(host).setPort(port).build(); + } catch (URISyntaxException e) { + String msg = + "Proxy error: incorrect " + PROXY_HOST + " or " + PROXY_PORT; + LOG.error(msg); + throw new IllegalArgumentException(msg); + } + } + + /** + * Initializes the User-Agent header to send in HTTP requests to AWS + * services. We always include the Hadoop version number. The user also + * may set an optional custom prefix to put in front of the Hadoop version + * number. The AWS SDK internally appends its own information, which seems + * to include the AWS SDK version, OS and JVM version. + * + * @param conf Hadoop configuration + * @param clientConfig AWS SDK configuration to update + */ + private static void initUserAgent(Configuration conf, + ClientOverrideConfiguration.Builder clientConfig) { + String userAgent = "Hadoop " + VersionInfo.getVersion(); + String userAgentPrefix = conf.getTrimmed(USER_AGENT_PREFIX, ""); + if (!userAgentPrefix.isEmpty()) { + userAgent = userAgentPrefix + ", " + userAgent; + } + LOG.debug("Using User-Agent: {}", userAgent); + clientConfig.putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX, userAgent); + } + + /** + * Configures request timeout. + * + * @param conf Hadoop configuration + * @param clientConfig AWS SDK configuration to update + */ + private static void initRequestTimeout(Configuration conf, + ClientOverrideConfiguration.Builder clientConfig) { + long requestTimeoutMillis = conf.getTimeDuration(REQUEST_TIMEOUT, + DEFAULT_REQUEST_TIMEOUT, TimeUnit.SECONDS, TimeUnit.MILLISECONDS); + + if (requestTimeoutMillis > Integer.MAX_VALUE) { + LOG.debug("Request timeout is too high({} ms). Setting to {} ms instead", + requestTimeoutMillis, Integer.MAX_VALUE); + requestTimeoutMillis = Integer.MAX_VALUE; + } + + if(requestTimeoutMillis > 0) { + clientConfig.apiCallAttemptTimeout(Duration.ofMillis(requestTimeoutMillis)); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index c6d21f2e554ac..d3076f71265c3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -1235,4 +1235,14 @@ private Constants() { */ public static final String PREFETCH_BLOCK_COUNT_KEY = "fs.s3a.prefetch.block.count"; public static final int PREFETCH_BLOCK_DEFAULT_COUNT = 8; + + /** + * The bucket region header. + */ + public static final String BUCKET_REGION_HEADER = "x-amz-bucket-region"; + + /** + * Status code for moved permanently. + */ + public static final int HTTP_STATUS_CODE_MOVED_PERMANENTLY = 301; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index f724f86e4afcd..d48991296ed4c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -20,9 +20,12 @@ import java.io.IOException; import java.net.URI; +import java.net.URISyntaxException; +import java.util.List; import com.amazonaws.ClientConfiguration; import com.amazonaws.SdkClientException; +import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.handlers.RequestHandler2; import com.amazonaws.regions.RegionUtils; @@ -41,11 +44,26 @@ import com.amazonaws.services.s3.model.KMSEncryptionMaterialsProvider; import com.amazonaws.util.AwsHostNameUtils; import com.amazonaws.util.RuntimeHttpUtils; + +import org.apache.hadoop.fs.s3a.adapter.V1V2AwsCredentialProviderAdapter; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.classification.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.http.apache.ProxyConfiguration; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.S3ClientBuilder; +import software.amazon.awssdk.services.s3.S3Configuration; +import software.amazon.awssdk.services.s3.model.HeadBucketRequest; +import software.amazon.awssdk.services.s3.model.HeadBucketResponse; +import software.amazon.awssdk.services.s3.model.S3Exception; + import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -57,9 +75,14 @@ import static com.amazonaws.services.s3.Headers.REQUESTER_PAYS_HEADER; import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION; import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_CENTRAL_REGION; +import static org.apache.hadoop.fs.s3a.Constants.BUCKET_REGION_HEADER; +import static org.apache.hadoop.fs.s3a.Constants.CENTRAL_ENDPOINT; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_SECURE_CONNECTIONS; import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING; import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING_DEFAULT; +import static org.apache.hadoop.fs.s3a.Constants.HTTP_STATUS_CODE_MOVED_PERMANENTLY; import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_KEY; +import static org.apache.hadoop.fs.s3a.Constants.SECURE_CONNECTIONS; import static org.apache.hadoop.fs.s3a.S3AUtils.getEncryptionAlgorithm; import static org.apache.hadoop.fs.s3a.S3AUtils.getS3EncryptionKey; import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; @@ -161,6 +184,82 @@ public AmazonS3 createS3Client( } } + /** + * Creates a new {@link S3Client}. + * + * @param uri S3A file system URI + * @param parameters parameter object + * @return S3 client + * @throws IOException on any IO problem + */ + @Override + public S3Client createS3ClientV2( + final URI uri, + final S3ClientCreationParameters parameters) throws IOException { + + Configuration conf = getConf(); + bucket = uri.getHost(); + + final ClientOverrideConfiguration.Builder clientOverrideConfigBuilder = + AWSClientConfig.createClientConfigBuilder(conf); + + final ApacheHttpClient.Builder httpClientBuilder = + AWSClientConfig.createHttpClientBuilder(conf); + + final RetryPolicy.Builder retryPolicyBuilder = AWSClientConfig.createRetryPolicyBuilder(conf); + + final ProxyConfiguration.Builder proxyConfigBuilder = + AWSClientConfig.createProxyConfigurationBuilder(conf, bucket); + + S3ClientBuilder s3ClientBuilder = S3Client.builder(); + + // add any headers + parameters.getHeaders().forEach((h, v) -> clientOverrideConfigBuilder.putHeader(h, v)); + + if (parameters.isRequesterPays()) { + // All calls must acknowledge requester will pay via header. + clientOverrideConfigBuilder.putHeader(REQUESTER_PAYS_HEADER, REQUESTER_PAYS_HEADER_VALUE); + } + + if (!StringUtils.isEmpty(parameters.getUserAgentSuffix())) { + clientOverrideConfigBuilder.putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_SUFFIX, + parameters.getUserAgentSuffix()); + } + + clientOverrideConfigBuilder.retryPolicy(retryPolicyBuilder.build()); + httpClientBuilder.proxyConfiguration(proxyConfigBuilder.build()); + + s3ClientBuilder.httpClientBuilder(httpClientBuilder) + .overrideConfiguration(clientOverrideConfigBuilder.build()); + + // use adapter classes so V1 credential providers continue to work. This will be moved to + // AWSCredentialProviderList.add() when that class is updated. + s3ClientBuilder.credentialsProvider( + V1V2AwsCredentialProviderAdapter.adapt(parameters.getCredentialSet())); + + URI endpoint = getS3Endpoint(parameters.getEndpoint(), conf); + + Region region = + getS3Region(conf.getTrimmed(AWS_REGION), parameters.getCredentialSet()); + + LOG.debug("Using endpoint {}; and region {}", endpoint, region); + + s3ClientBuilder.endpointOverride(endpoint).region(region); + + S3Configuration s3Configuration = S3Configuration.builder() + .pathStyleAccessEnabled(parameters.isPathStyleAccess()) + .build(); + + s3ClientBuilder.serviceConfiguration(s3Configuration); + + // TODO: Some configuration done in configureBasicParams is not done yet. + // Request handlers will be added during auditor work. Need to verify how metrics collection + // can be done, as SDK V2 only seems to have a metrics publisher. + + return s3ClientBuilder.build(); + } + + /** * Create an {@link AmazonS3} client of type * {@link AmazonS3EncryptionV2} if CSE is enabled. @@ -391,4 +490,72 @@ protected static AmazonS3 configureAmazonS3Client(AmazonS3 s3, endpoint, epr, region); return new AwsClientBuilder.EndpointConfiguration(endpoint, region); } + + /** + * Given a endpoint string, create the endpoint URI. + * + * @param endpoint possibly null endpoint. + * @param conf config to build the URI from. + * @return an endpoint uri + */ + private static URI getS3Endpoint(String endpoint, final Configuration conf) { + + boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS); + + String protocol = secureConnections ? "https" : "http"; + + if (endpoint == null || endpoint.isEmpty()) { + // the default endpoint + endpoint = CENTRAL_ENDPOINT; + } + + if (!endpoint.contains("://")) { + endpoint = String.format("%s://%s", protocol, endpoint); + } + + try { + return new URI(endpoint); + } catch (URISyntaxException e) { + 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 credentialsProvider Credentials provider to be used with the default s3 client. + * @return region of the bucket. + */ + private Region getS3Region(String region, 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(V1V2AwsCredentialProviderAdapter.adapt(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() == HTTP_STATUS_CODE_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/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java index 6c39cc4b64240..b4674159ea473 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.services.s3.model.S3ObjectSummary; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.VisibleForTesting; @@ -39,6 +38,8 @@ import org.apache.hadoop.util.functional.RemoteIterators; import org.slf4j.Logger; +import software.amazon.awssdk.services.s3.model.CommonPrefix; +import software.amazon.awssdk.services.s3.model.S3Object; import java.io.Closeable; import java.io.IOException; @@ -277,19 +278,19 @@ public S3ListRequest createListObjectsRequest(String key, } /** - * Interface to implement by the logic deciding whether to accept a summary + * Interface to implement the logic deciding whether to accept a s3Object * entry or path as a valid file or directory. */ interface FileStatusAcceptor { /** - * Predicate to decide whether or not to accept a summary entry. + * Predicate to decide whether or not to accept a s3Object entry. * @param keyPath qualified path to the entry - * @param summary summary entry + * @param s3Object s3Object entry * @return true if the entry is accepted (i.e. that a status entry * should be generated. */ - boolean accept(Path keyPath, S3ObjectSummary summary); + boolean accept(Path keyPath, S3Object s3Object); /** * Predicate to decide whether or not to accept a prefix. @@ -451,21 +452,21 @@ private boolean buildNextStatusBatch(S3ListResult objects) { int added = 0, ignored = 0; // list to fill in with results. Initial size will be list maximum. List stats = new ArrayList<>( - objects.getObjectSummaries().size() + + objects.getS3Objects().size() + objects.getCommonPrefixes().size()); // objects - for (S3ObjectSummary summary : objects.getObjectSummaries()) { - String key = summary.getKey(); + for (S3Object s3Object : objects.getS3Objects()) { + String key = s3Object.key(); Path keyPath = getStoreContext().getContextAccessors().keyToPath(key); if (LOG.isDebugEnabled()) { - LOG.debug("{}: {}", keyPath, stringify(summary)); + LOG.debug("{}: {}", keyPath, stringify(s3Object)); } // Skip over keys that are ourselves and old S3N _$folder$ files - if (acceptor.accept(keyPath, summary) && filter.accept(keyPath)) { - S3AFileStatus status = createFileStatus(keyPath, summary, + if (acceptor.accept(keyPath, s3Object) && filter.accept(keyPath)) { + S3AFileStatus status = createFileStatus(keyPath, s3Object, listingOperationCallbacks.getDefaultBlockSize(keyPath), getStoreContext().getUsername(), - summary.getETag(), null, isCSEEnabled); + s3Object.eTag(), null, isCSEEnabled); LOG.debug("Adding: {}", status); stats.add(status); added++; @@ -476,11 +477,11 @@ private boolean buildNextStatusBatch(S3ListResult objects) { } // prefixes: always directories - for (String prefix : objects.getCommonPrefixes()) { + for (CommonPrefix prefix : objects.getCommonPrefixes()) { Path keyPath = getStoreContext() .getContextAccessors() - .keyToPath(prefix); - if (acceptor.accept(keyPath, prefix) && filter.accept(keyPath)) { + .keyToPath(prefix.prefix()); + if (acceptor.accept(keyPath, prefix.prefix()) && filter.accept(keyPath)) { S3AFileStatus status = new S3AFileStatus(Tristate.FALSE, keyPath, getStoreContext().getUsername()); LOG.debug("Adding directory: {}", status); @@ -731,18 +732,18 @@ public AcceptFilesOnly(Path qualifiedPath) { } /** - * Reject a summary entry if the key path is the qualified Path, or + * Reject a s3Object entry if the key path is the qualified Path, or * it ends with {@code "_$folder$"}. * @param keyPath key path of the entry - * @param summary summary entry + * @param s3Object s3Object entry * @return true if the entry is accepted (i.e. that a status entry * should be generated. */ @Override - public boolean accept(Path keyPath, S3ObjectSummary summary) { + public boolean accept(Path keyPath, S3Object s3Object) { return !keyPath.equals(qualifiedPath) - && !summary.getKey().endsWith(S3N_FOLDER_SUFFIX) - && !objectRepresentsDirectory(summary.getKey()); + && !s3Object.key().endsWith(S3N_FOLDER_SUFFIX) + && !objectRepresentsDirectory(s3Object.key()); } /** @@ -767,8 +768,8 @@ public boolean accept(FileStatus status) { */ static class AcceptAllButS3nDirs implements FileStatusAcceptor { - public boolean accept(Path keyPath, S3ObjectSummary summary) { - return !summary.getKey().endsWith(S3N_FOLDER_SUFFIX); + public boolean accept(Path keyPath, S3Object s3Object) { + return !s3Object.key().endsWith(S3N_FOLDER_SUFFIX); } public boolean accept(Path keyPath, String prefix) { @@ -799,17 +800,17 @@ public AcceptAllButSelfAndS3nDirs(Path qualifiedPath) { } /** - * Reject a summary entry if the key path is the qualified Path, or + * Reject a s3Object entry if the key path is the qualified Path, or * it ends with {@code "_$folder$"}. * @param keyPath key path of the entry - * @param summary summary entry + * @param s3Object s3Object entry * @return true if the entry is accepted (i.e. that a status entry * should be generated.) */ @Override - public boolean accept(Path keyPath, S3ObjectSummary summary) { + public boolean accept(Path keyPath, S3Object s3Object) { return !keyPath.equals(qualifiedPath) && - !summary.getKey().endsWith(S3N_FOLDER_SUFFIX); + !s3Object.key().endsWith(S3N_FOLDER_SUFFIX); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index f8a014c8933c0..83fe3ca49921d 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 @@ -64,8 +64,6 @@ import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadResult; import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsV2Request; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import com.amazonaws.services.s3.model.MultipartUpload; import com.amazonaws.services.s3.model.ObjectMetadata; @@ -89,6 +87,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; + import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -277,6 +279,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private Path workingDir; private String username; private AmazonS3 s3; + private S3Client s3V2; // initial callback policy is fail-once; it's there just to assist // some mock tests and other codepaths trying to call the low level // APIs on an uninitialized filesystem. @@ -955,6 +958,11 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf) .createS3Client(getUri(), parameters); + + s3V2 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf) + .createS3ClientV2(getUri(), + parameters); + } /** @@ -1242,6 +1250,7 @@ private AmazonS3 getAmazonS3Client() { * @param reason a justification for requesting access. * @return AmazonS3Client */ + // TODO: Remove when we remove S3V1 client @VisibleForTesting public AmazonS3 getAmazonS3ClientForTesting(String reason) { LOG.warn("Access to S3A client requested, reason {}", reason); @@ -1249,6 +1258,19 @@ public AmazonS3 getAmazonS3ClientForTesting(String reason) { return s3; } + /** + * Returns the S3 client used by this filesystem. + * Warning: this must only be used for testing, as it bypasses core + * S3A operations. + * @param reason a justification for requesting access. + * @return S3Client + */ + @VisibleForTesting + public S3Client getAmazonS3V2ClientForTesting(String reason) { + LOG.warn("Access to S3 client requested, reason {}", reason); + return s3V2; + } + /** * Set the client -used in mocking tests to force in a different client. * @param client client. @@ -2653,9 +2675,9 @@ protected S3ListResult listObjects(S3ListRequest request, OBJECT_LIST_REQUEST, () -> { if (useListV1) { - return S3ListResult.v1(s3.listObjects(request.getV1())); + return S3ListResult.v1(s3V2.listObjects(request.getV1())); } else { - return S3ListResult.v2(s3.listObjectsV2(request.getV2())); + return S3ListResult.v2(s3V2.listObjectsV2(request.getV2())); } })); } @@ -2698,15 +2720,23 @@ protected S3ListResult continueListObjects(S3ListRequest request, OBJECT_CONTINUE_LIST_REQUEST, () -> { if (useListV1) { - return S3ListResult.v1( - s3.listNextBatchOfObjects( - getRequestFactory() - .newListNextBatchOfObjectsRequest( - prevResult.getV1()))); + //TODO: Update to List once we can get rid of the other S3Object import + List + prevListResult = prevResult.getV1().contents(); + + // Next markers are only present when a delimiter is specified. + String nextMarker; + if (prevResult.getV1().nextMarker() != null) { + nextMarker = prevResult.getV1().nextMarker(); + } else { + nextMarker = prevListResult.get(prevListResult.size() - 1).key(); + } + + return S3ListResult.v1(s3V2.listObjects( + request.getV1().toBuilder().marker(nextMarker).build())); } else { - request.getV2().setContinuationToken(prevResult.getV2() - .getNextContinuationToken()); - return S3ListResult.v2(s3.listObjectsV2(request.getV2())); + return S3ListResult.v2(s3V2.listObjectsV2(request.getV2().toBuilder() + .continuationToken(prevResult.getV2().nextContinuationToken()).build())); } })); } 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 7cc7d635c51f0..88d69e9c3d5f7 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 @@ -29,7 +29,6 @@ import com.amazonaws.retry.RetryUtils; import com.amazonaws.services.s3.model.AmazonS3Exception; import com.amazonaws.services.s3.model.MultiObjectDeleteException; -import com.amazonaws.services.s3.model.S3ObjectSummary; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.util.Preconditions; @@ -53,8 +52,10 @@ import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.util.Lists; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.model.S3Object; import javax.annotation.Nullable; import java.io.Closeable; @@ -467,7 +468,7 @@ public static String stringify(AmazonS3Exception e) { /** * Create a files status instance from a listing. * @param keyPath path to entry - * @param summary summary from AWS + * @param s3Object s3Object entry * @param blockSize block size to declare. * @param owner owner of the file * @param eTag S3 object eTag or null if unavailable @@ -476,20 +477,20 @@ public static String stringify(AmazonS3Exception e) { * @return a status entry */ public static S3AFileStatus createFileStatus(Path keyPath, - S3ObjectSummary summary, + S3Object s3Object, long blockSize, String owner, String eTag, String versionId, boolean isCSEEnabled) { - long size = summary.getSize(); + long size = s3Object.size(); // check if cse is enabled; strip out constant padding length. if (isCSEEnabled && size >= CSE_PADDING_LENGTH) { size -= CSE_PADDING_LENGTH; } return createFileStatus(keyPath, - objectRepresentsDirectory(summary.getKey()), - size, summary.getLastModified(), blockSize, owner, eTag, versionId); + objectRepresentsDirectory(s3Object.key()), + size, Date.from(s3Object.lastModified()), blockSize, owner, eTag, versionId); } /** @@ -936,13 +937,13 @@ static String lookupPassword(Configuration conf, String key, String defVal) /** * String information about a summary entry for debug messages. - * @param summary summary object + * @param s3Object s3Object entry * @return string value */ - public static String stringify(S3ObjectSummary summary) { - StringBuilder builder = new StringBuilder(summary.getKey().length() + 100); - builder.append(summary.getKey()).append(' '); - builder.append("size=").append(summary.getSize()); + public static String stringify(S3Object s3Object) { + StringBuilder builder = new StringBuilder(s3Object.key().length() + 100); + builder.append(s3Object.key()).append(' '); + builder.append("size=").append(s3Object.size()); return builder.toString(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java index 9010f34dc259c..ec94edd6f3b8a 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 @@ -29,6 +29,7 @@ import com.amazonaws.handlers.RequestHandler2; import com.amazonaws.monitoring.MonitoringListener; import com.amazonaws.services.s3.AmazonS3; +import software.amazon.awssdk.services.s3.S3Client; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -63,6 +64,18 @@ public interface S3ClientFactory { AmazonS3 createS3Client(URI uri, S3ClientCreationParameters parameters) throws IOException; + /** + * Creates a new {@link S3Client}. + * + * @param uri S3A file system URI + * @param parameters parameter object + * @return S3 client + * @throws IOException on any IO problem + */ + S3Client createS3ClientV2(URI uri, + S3ClientCreationParameters parameters) throws IOException; + + /** * Settings for the S3 Client. * Implemented as a class to pass in so that adding diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java index d51211516f251..c729f3de15f08 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java @@ -18,8 +18,8 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; /** * API version-independent container for S3 List requests. @@ -78,14 +78,14 @@ public ListObjectsV2Request getV2() { public String toString() { if (isV1()) { return String.format(DESCRIPTION, - v1Request.getBucketName(), v1Request.getPrefix(), - v1Request.getDelimiter(), v1Request.getMaxKeys(), - v1Request.isRequesterPays()); + v1Request.bucket(), v1Request.prefix(), + v1Request.delimiter(), v1Request.maxKeys(), + v1Request.requestPayerAsString()); } else { return String.format(DESCRIPTION, - v2Request.getBucketName(), v2Request.getPrefix(), - v2Request.getDelimiter(), v2Request.getMaxKeys(), - v2Request.isRequesterPays()); + v2Request.bucket(), v2Request.prefix(), + v2Request.delimiter(), v2Request.maxKeys(), + v2Request.requestPayerAsString()); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java index 69c42bfe1471a..c77311211abcb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java @@ -22,19 +22,21 @@ import java.util.List; import java.util.stream.Collectors; -import com.amazonaws.services.s3.model.ListObjectsV2Result; -import com.amazonaws.services.s3.model.ObjectListing; -import com.amazonaws.services.s3.model.S3ObjectSummary; +import software.amazon.awssdk.services.s3.model.CommonPrefix; +import software.amazon.awssdk.services.s3.model.ListObjectsResponse; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; +import software.amazon.awssdk.services.s3.model.S3Object; + import org.slf4j.Logger; /** * API version-independent container for S3 List responses. */ public class S3ListResult { - private ObjectListing v1Result; - private ListObjectsV2Result v2Result; + private ListObjectsResponse v1Result; + private ListObjectsV2Response v2Result; - protected S3ListResult(ObjectListing v1, ListObjectsV2Result v2) { + protected S3ListResult(ListObjectsResponse v1, ListObjectsV2Response v2) { v1Result = v1; v2Result = v2; } @@ -44,7 +46,7 @@ protected S3ListResult(ObjectListing v1, ListObjectsV2Result v2) { * @param result v1 result * @return new list result container */ - public static S3ListResult v1(ObjectListing result) { + public static S3ListResult v1(ListObjectsResponse result) { return new S3ListResult(result, null); } @@ -53,7 +55,7 @@ public static S3ListResult v1(ObjectListing result) { * @param result v2 result * @return new list result container */ - public static S3ListResult v2(ListObjectsV2Result result) { + public static S3ListResult v2(ListObjectsV2Response result) { return new S3ListResult(null, result); } @@ -65,19 +67,19 @@ public boolean isV1() { return v1Result != null; } - public ObjectListing getV1() { + public ListObjectsResponse getV1() { return v1Result; } - public ListObjectsV2Result getV2() { + public ListObjectsV2Response getV2() { return v2Result; } - public List getObjectSummaries() { + public List getS3Objects() { if (isV1()) { - return v1Result.getObjectSummaries(); + return v1Result.contents(); } else { - return v2Result.getObjectSummaries(); + return v2Result.contents(); } } @@ -89,21 +91,21 @@ public boolean isTruncated() { } } - public List getCommonPrefixes() { + public List getCommonPrefixes() { if (isV1()) { - return v1Result.getCommonPrefixes(); + return v1Result.commonPrefixes(); } else { - return v2Result.getCommonPrefixes(); + return v2Result.commonPrefixes(); } } /** - * Get the list of keys in the object summary. + * Get the list of keys in the list result. * @return a possibly empty list */ - private List objectSummaryKeys() { - return getObjectSummaries().stream() - .map(S3ObjectSummary::getKey) + private List objectKeys() { + return getS3Objects().stream() + .map(S3Object::key) .collect(Collectors.toList()); } @@ -112,9 +114,8 @@ private List objectSummaryKeys() { * @return true if the result is non-empty */ public boolean hasPrefixesOrObjects() { - return !(getCommonPrefixes()).isEmpty() - || !getObjectSummaries().isEmpty(); + || !getS3Objects().isEmpty(); } /** @@ -128,7 +129,7 @@ public boolean representsEmptyDirectory( // no children. // So the listing must contain the marker entry only as an object, // and prefixes is null - List keys = objectSummaryKeys(); + List keys = objectKeys(); return keys.size() == 1 && keys.contains(dirKey) && getCommonPrefixes().isEmpty(); } @@ -138,15 +139,15 @@ public boolean representsEmptyDirectory( * @param log log to use */ public void logAtDebug(Logger log) { - Collection prefixes = getCommonPrefixes(); - Collection summaries = getObjectSummaries(); + Collection prefixes = getCommonPrefixes(); + Collection s3Objects = getS3Objects(); log.debug("Prefix count = {}; object count={}", - prefixes.size(), summaries.size()); - for (S3ObjectSummary summary : summaries) { - log.debug("Summary: {} {}", summary.getKey(), summary.getSize()); + prefixes.size(), s3Objects.size()); + for (S3Object s3Object : s3Objects) { + log.debug("Summary: {} {}", s3Object.key(), s3Object.size()); } - for (String prefix : prefixes) { - log.debug("Prefix: {}", prefix); + for (CommonPrefix prefix : prefixes) { + log.debug("Prefix: {}", prefix.prefix()); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1ToV2AwsCredentialProviderAdapter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1ToV2AwsCredentialProviderAdapter.java new file mode 100644 index 0000000000000..efea6540ef9ac --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1ToV2AwsCredentialProviderAdapter.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.adapter; + +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSSessionCredentials; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; + +/** + * Adapts a V1 {@link AWSCredentialsProvider} to the V2 {@link AwsCredentialsProvider} interface. + * Implements both interfaces so can be used with either the V1 or V2 AWS SDK. + */ +final class V1ToV2AwsCredentialProviderAdapter implements V1V2AwsCredentialProviderAdapter { + + private final AWSCredentialsProvider v1CredentialsProvider; + + private V1ToV2AwsCredentialProviderAdapter(AWSCredentialsProvider v1CredentialsProvider) { + this.v1CredentialsProvider = v1CredentialsProvider; + } + + @Override + public AwsCredentials resolveCredentials() { + AWSCredentials toAdapt = v1CredentialsProvider.getCredentials(); + if (toAdapt instanceof AWSSessionCredentials) { + return AwsSessionCredentials.create(toAdapt.getAWSAccessKeyId(), + toAdapt.getAWSSecretKey(), + ((AWSSessionCredentials) toAdapt).getSessionToken()); + } else { + return AwsBasicCredentials.create(toAdapt.getAWSAccessKeyId(), toAdapt.getAWSSecretKey()); + } + } + + @Override + public AWSCredentials getCredentials() { + return v1CredentialsProvider.getCredentials(); + } + + @Override + public void refresh() { + v1CredentialsProvider.refresh(); + } + + /** + * @param v1CredentialsProvider V1 credential provider to adapt. + * @return A new instance of the credentials provider adapter. + */ + static V1ToV2AwsCredentialProviderAdapter create(AWSCredentialsProvider v1CredentialsProvider) { + return new V1ToV2AwsCredentialProviderAdapter(v1CredentialsProvider); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1V2AwsCredentialProviderAdapter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1V2AwsCredentialProviderAdapter.java new file mode 100644 index 0000000000000..f27166a9ef91d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1V2AwsCredentialProviderAdapter.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.adapter; + +import com.amazonaws.auth.AWSCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; + +public interface V1V2AwsCredentialProviderAdapter extends AWSCredentialsProvider, + AwsCredentialsProvider { + + /** + * Creates a two-way adapter from a V1 {@link AWSCredentialsProvider} interface. + * + * @param v1CredentialsProvider V1 credentials provider. + * @return Two-way credential provider adapter. + */ + static V1V2AwsCredentialProviderAdapter adapt(AWSCredentialsProvider v1CredentialsProvider) { + return V1ToV2AwsCredentialProviderAdapter.create(v1CredentialsProvider); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/package-info.java new file mode 100644 index 0000000000000..8d03c915e171a --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/package-info.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Adapter classes for allowing V1 credential providers to be used with SDKV2. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.adapter; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java index cae4d3ef034e8..c0c691741b280 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 @@ -35,8 +35,8 @@ import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import com.amazonaws.services.s3.model.ObjectListing; import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PartETag; 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 e7dd75c581131..8b8cd0a9588c9 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 @@ -281,7 +281,10 @@ private void processNewRevision(final String newRevisionId, LOG.debug("Setting revision ID for object at {}: {}", uri, newRevisionId); revisionId = newRevisionId; - } else if (!revisionId.equals(newRevisionId)) { + //TODO: Remove this. This is a temporary fix to prevent tests from failing. Needed because + // SDKV2 returns etag with quotation marks, and V1 does not use quotations so this equality + // fails. Regex removes quotation marks. + } else if (!revisionId.replaceAll("^\"|\"$", "").equals(newRevisionId)) { LOG.debug("Revision ID changed from {} to {}", revisionId, newRevisionId); ImmutablePair pair = 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 ce11df0383929..fba75bb4bcc82 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 @@ -39,8 +39,6 @@ import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsV2Request; import com.amazonaws.services.s3.model.ObjectListing; import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PartETag; @@ -53,6 +51,9 @@ import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.awscore.AwsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.Retries; @@ -151,6 +152,17 @@ private T prepareRequest(T t) { : t; } + /** + * Preflight preparation of V2 AWS request. + * @param web service request + * @return prepared entry. + */ + // TODO: Currently this is a NOOP, will be completed separately as part of auditor work. + @Retries.OnceRaw + private T prepareV2Request(T t) { + return t; + } + /** * Get the canned ACL of this FS. * @return an ACL, if any @@ -569,14 +581,15 @@ public ListObjectsRequest newListObjectsV1Request( final String key, final String delimiter, final int maxKeys) { - ListObjectsRequest request = new ListObjectsRequest() - .withBucketName(bucket) - .withMaxKeys(maxKeys) - .withPrefix(key); + + ListObjectsRequest.Builder requestBuilder = + ListObjectsRequest.builder().bucket(bucket).maxKeys(maxKeys).prefix(key); + if (delimiter != null) { - request.setDelimiter(delimiter); + requestBuilder.delimiter(delimiter); } - return prepareRequest(request); + + return prepareV2Request(requestBuilder.build()); } @Override @@ -590,14 +603,17 @@ public ListObjectsV2Request newListObjectsV2Request( final String key, final String delimiter, final int maxKeys) { - final ListObjectsV2Request request = new ListObjectsV2Request() - .withBucketName(bucket) - .withMaxKeys(maxKeys) - .withPrefix(key); + + final ListObjectsV2Request.Builder requestBuilder = ListObjectsV2Request.builder() + .bucket(bucket) + .maxKeys(maxKeys) + .prefix(key); + if (delimiter != null) { - request.setDelimiter(delimiter); + requestBuilder.delimiter(delimiter); } - return prepareRequest(request); + + return prepareV2Request(requestBuilder.build()); } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java index a46303f339678..d2fbd4def4151 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 @@ -31,6 +31,7 @@ import org.junit.Before; import org.junit.Rule; import org.junit.rules.ExpectedException; +import software.amazon.awssdk.services.s3.S3Client; /** * Abstract base class for S3A unit tests using a mock S3 client and a null @@ -50,6 +51,7 @@ public abstract class AbstractS3AMockTest { protected S3AFileSystem fs; protected AmazonS3 s3; + private S3Client s3V2; @Before public void setup() throws Exception { @@ -60,6 +62,7 @@ public void setup() throws Exception { conf.unset(Constants.S3_ENCRYPTION_ALGORITHM); fs.initialize(uri, conf); s3 = fs.getAmazonS3ClientForTesting("mocking"); + s3V2 = fs.getAmazonS3V2ClientForTesting("mocking"); } @SuppressWarnings("deprecation") @@ -78,6 +81,10 @@ public Configuration createConfiguration() { return conf; } + public S3Client getS3Client() { + return s3V2; + } + @After public void teardown() throws Exception { if (fs != null) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3ClientFactory.java index 3240309aef971..7397adc73734b 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 @@ -26,6 +26,7 @@ import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.MultipartUploadListing; import com.amazonaws.services.s3.model.Region; +import software.amazon.awssdk.services.s3.S3Client; /** * An {@link S3ClientFactory} that returns Mockito mocks of the {@link AmazonS3} @@ -45,10 +46,15 @@ public AmazonS3 createS3Client(URI uri, // 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()); + 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) { + S3Client s3 = mock(S3Client.class); return s3; } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java index 34a275b580f25..cc61ef1fe2ab7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java @@ -25,17 +25,13 @@ import static org.mockito.Mockito.when; import java.io.FileNotFoundException; +import java.util.ArrayList; import java.util.Collections; import java.util.Date; import java.util.List; import com.amazonaws.services.s3.model.GetObjectMetadataRequest; -import com.amazonaws.services.s3.model.ListObjectsRequest; -import com.amazonaws.services.s3.model.ListObjectsV2Request; -import com.amazonaws.services.s3.model.ListObjectsV2Result; -import com.amazonaws.services.s3.model.ObjectListing; import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.S3ObjectSummary; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -43,6 +39,13 @@ import org.apache.hadoop.fs.contract.ContractTestUtils; import org.junit.Test; import org.mockito.ArgumentMatcher; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.CommonPrefix; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsResponse; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; +import software.amazon.awssdk.services.s3.model.S3Object; /** * S3A tests for getFileStatus using mock S3 client. @@ -74,17 +77,17 @@ public void testFile() throws Exception { public void testFakeDirectory() throws Exception { Path path = new Path("/dir"); String key = path.toUri().getPath().substring(1); + S3Client s3V2 = getS3Client(); when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) .thenThrow(NOT_FOUND); String keyDir = key + "/"; - ListObjectsV2Result listResult = new ListObjectsV2Result(); - S3ObjectSummary objectSummary = new S3ObjectSummary(); - objectSummary.setKey(keyDir); - objectSummary.setSize(0L); - listResult.getObjectSummaries().add(objectSummary); - when(s3.listObjectsV2(argThat( + List s3Objects = new ArrayList<>(1); + s3Objects.add(S3Object.builder().key(keyDir).size(0L).build()); + ListObjectsV2Response listObjectsV2Response = + ListObjectsV2Response.builder().contents(s3Objects).build(); + when(s3V2.listObjectsV2(argThat( matchListV2Request(BUCKET, keyDir)) - )).thenReturn(listResult); + )).thenReturn(listObjectsV2Response); FileStatus stat = fs.getFileStatus(path); assertNotNull(stat); assertEquals(fs.makeQualified(path), stat.getPath()); @@ -100,7 +103,8 @@ public void testImplicitDirectory() throws Exception { when(s3.getObjectMetadata(argThat( correctGetMetadataRequest(BUCKET, key + "/")) )).thenThrow(NOT_FOUND); - setupListMocks(Collections.singletonList("dir/"), Collections.emptyList()); + setupListMocks(Collections.singletonList(CommonPrefix.builder().prefix("dir/").build()), + Collections.emptyList()); FileStatus stat = fs.getFileStatus(path); assertNotNull(stat); assertEquals(fs.makeQualified(path), stat.getPath()); @@ -142,21 +146,22 @@ public void testNotFound() throws Exception { fs.getFileStatus(path); } - private void setupListMocks(List prefixes, - List summaries) { - + private void setupListMocks(List prefixes, + List s3Objects) { + S3Client s3V2 = getS3Client(); // V1 list API mock - ObjectListing objects = mock(ObjectListing.class); - when(objects.getCommonPrefixes()).thenReturn(prefixes); - when(objects.getObjectSummaries()).thenReturn(summaries); - when(s3.listObjects(any(ListObjectsRequest.class))).thenReturn(objects); + ListObjectsResponse v1Response = mock(ListObjectsResponse.class); + when(v1Response.commonPrefixes()).thenReturn(prefixes); + when(v1Response.contents()).thenReturn(s3Objects); + when(s3V2.listObjects(any(ListObjectsRequest.class))).thenReturn(v1Response); // V2 list API mock - ListObjectsV2Result v2Result = mock(ListObjectsV2Result.class); - when(v2Result.getCommonPrefixes()).thenReturn(prefixes); - when(v2Result.getObjectSummaries()).thenReturn(summaries); - when(s3.listObjectsV2(any(ListObjectsV2Request.class))) - .thenReturn(v2Result); + ListObjectsV2Response v2Result = mock(ListObjectsV2Response.class); + when(v2Result.commonPrefixes()).thenReturn(prefixes); + when(v2Result.contents()).thenReturn(s3Objects); + when(s3V2.listObjectsV2( + any(software.amazon.awssdk.services.s3.model.ListObjectsV2Request.class))).thenReturn( + v2Result); } private ArgumentMatcher correctGetMetadataRequest( @@ -170,8 +175,8 @@ private ArgumentMatcher matchListV2Request( String bucket, String key) { return (ListObjectsV2Request request) -> { return request != null - && request.getBucketName().equals(bucket) - && request.getPrefix().equals(key); + && request.bucket().equals(bucket) + && request.prefix().equals(key); }; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java index 5c243bb820f02..03f03c46f97b8 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 @@ -170,9 +170,11 @@ private void createFactoryObjects(RequestFactory factory) { a(factory.newGetObjectRequest(path)); a(factory.newGetObjectMetadataRequest(path)); a(factory.newListMultipartUploadsRequest(path)); - a(factory.newListObjectsV1Request(path, "/", 1)); + //TODO: Commenting out for now, new request extends AwsRequest, this can be updated once all + // request factory operations are updated. + //a(factory.newListObjectsV1Request(path, "/", 1)); a(factory.newListNextBatchOfObjectsRequest(new ObjectListing())); - a(factory.newListObjectsV2Request(path, "/", 1)); + // a(factory.newListObjectsV2Request(path, "/", 1)); a(factory.newMultipartUploadRequest(path, null)); File srcfile = new File("/tmp/a"); a(factory.newPutObjectRequest(path, 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 new file mode 100644 index 0000000000000..3b308f19255c3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1,13 @@ +# 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 From 4085a8d8104798ca4ed1f47d576f8996eb593e70 Mon Sep 17 00:00:00 2001 From: Ahmar Suhail Date: Thu, 22 Sep 2022 15:26:54 +0100 Subject: [PATCH 2/3] updates getObjectMetadata, putObject & copyObject operations --- hadoop-project/pom.xml | 5 + hadoop-tools/hadoop-aws/pom.xml | 5 + .../apache/hadoop/fs/s3a/AWSClientConfig.java | 2 +- .../org/apache/hadoop/fs/s3a/Invoker.java | 25 +- .../fs/s3a/ProgressableProgressListener.java | 48 ++- .../hadoop/fs/s3a/S3ABlockOutputStream.java | 31 +- .../apache/hadoop/fs/s3a/S3ADataBlocks.java | 8 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 340 ++++++++++------- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 49 +++ .../org/apache/hadoop/fs/s3a/UploadInfo.java | 12 +- .../hadoop/fs/s3a/WriteOperationHelper.java | 82 ++-- .../apache/hadoop/fs/s3a/WriteOperations.java | 43 +-- .../V1ToV2AwsCredentialProviderAdapter.java | 5 + .../hadoop/fs/s3a/api/RequestFactory.java | 75 ++-- .../EncryptionSecretOperations.java | 29 ++ .../auth/delegation/S3ADelegationTokens.java | 2 +- .../s3a/commit/magic/MagicCommitTracker.java | 24 +- .../fs/s3a/impl/ChangeDetectionPolicy.java | 77 ++-- .../hadoop/fs/s3a/impl/ChangeTracker.java | 49 +-- .../hadoop/fs/s3a/impl/ErrorTranslation.java | 18 +- .../hadoop/fs/s3a/impl/HeaderProcessing.java | 140 ++++--- .../fs/s3a/impl/OperationCallbacks.java | 3 +- .../fs/s3a/impl/RequestFactoryImpl.java | 355 ++++++++---------- .../hadoop/fs/s3a/AbstractS3AMockTest.java | 2 +- .../hadoop/fs/s3a/EncryptionTestUtils.java | 19 +- .../ITestS3AEncryptionSSEKMSDefaultKey.java | 7 +- ...estS3AEncryptionWithDefaultS3Settings.java | 2 +- .../hadoop/fs/s3a/ITestS3AMiscOperations.java | 20 +- .../hadoop/fs/s3a/ITestS3ARequesterPays.java | 2 +- .../hadoop/fs/s3a/TestS3ADeleteOnExit.java | 21 +- .../hadoop/fs/s3a/TestS3AGetFileStatus.java | 38 +- .../apache/hadoop/fs/s3a/TestS3AUnbuffer.java | 21 +- .../fs/s3a/TestStreamChangeTracker.java | 34 +- .../fs/s3a/audit/AbstractAuditingTest.java | 6 +- .../fs/s3a/audit/TestAuditIntegration.java | 54 +-- .../ITestSessionDelegationInFileystem.java | 5 +- .../s3a/commit/AbstractITCommitProtocol.java | 2 +- .../hadoop/fs/s3a/impl/ITestXAttrCost.java | 3 + .../fs/s3a/impl/TestHeaderProcessing.java | 25 +- .../fs/s3a/impl/TestRequestFactory.java | 38 +- .../scale/ITestS3ADirectoryPerformance.java | 23 +- .../scale/ITestS3AHugeFilesStorageClass.java | 2 +- .../s3a/test/MinimalOperationCallbacks.java | 3 +- 43 files changed, 934 insertions(+), 820 deletions(-) diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index bbd2953e3ba3c..9132e111ca270 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -1128,6 +1128,11 @@ + + software.amazon.awssdk + s3-transfer-manager + 2.17.196-PREVIEW + org.apache.mina mina-core diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 4eb9508fc68ae..cdb1d0924e151 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -497,6 +497,11 @@ bundle compile + + software.amazon.awssdk + s3-transfer-manager + compile + org.assertj assertj-core diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientConfig.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientConfig.java index 40ee14d86daa2..43f5b136cfccc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientConfig.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AWSClientConfig.java @@ -31,10 +31,10 @@ import software.amazon.awssdk.core.retry.RetryPolicy; import software.amazon.awssdk.http.apache.ApacheHttpClient; import software.amazon.awssdk.http.apache.ProxyConfiguration; +import software.amazon.awssdk.thirdparty.org.apache.http.client.utils.URIBuilder; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.VersionInfo; -import org.apache.http.client.utils.URIBuilder; import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ESTABLISH_TIMEOUT; import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAXIMUM_CONNECTIONS; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java index 279bfeba98769..2f91861e35446 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java @@ -28,6 +28,8 @@ import com.amazonaws.SdkBaseException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkException; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; @@ -120,8 +122,13 @@ public static T once(String action, String path, throws IOException { try (DurationInfo ignored = new DurationInfo(LOG, false, "%s", action)) { return operation.apply(); - } catch (AmazonClientException e) { - throw S3AUtils.translateException(action, path, e); + } catch (AmazonClientException | AwsServiceException e) { + // TODO: This is temporary, and will be updated during error translation work. + if (e instanceof AmazonClientException) { + throw S3AUtils.translateException(action, path, (SdkBaseException) e); + } else { + throw S3AUtils.translateExceptionV2(action, path, (SdkException) e); + } } } @@ -466,7 +473,7 @@ public T retryUntranslated( } // execute the operation, returning if successful return operation.apply(); - } catch (IOException | SdkBaseException e) { + } catch (IOException | SdkBaseException | AwsServiceException e) { caught = e; } // you only get here if the operation didn't complete @@ -474,11 +481,16 @@ public T retryUntranslated( // translate the exception into an IOE for the retry logic IOException translated; + // TODO: Update during error translation work. This is a temporary fix to allow + // getObjectMetadata to throw FNFE. if (caught instanceof IOException) { translated = (IOException) caught; - } else { + } else if (caught instanceof SdkBaseException) { translated = S3AUtils.translateException(text, "", (SdkBaseException)caught); + } else { + translated = S3AUtils.translateExceptionV2(text, "", + (AwsServiceException)caught); } try { @@ -516,8 +528,11 @@ public T retryUntranslated( if (caught instanceof IOException) { throw (IOException) caught; - } else { + // TODO: This is temporary, and will be updated during error translation work. + } else if (caught instanceof SdkBaseException) { throw (SdkBaseException) caught; + } else { + throw (AwsServiceException) caught; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java index 0ce022aa88588..a42120288b81a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/ProgressableProgressListener.java @@ -18,59 +18,56 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.event.ProgressEvent; -import com.amazonaws.event.ProgressEventType; -import com.amazonaws.event.ProgressListener; -import com.amazonaws.services.s3.transfer.Upload; -import org.apache.hadoop.util.Progressable; import org.slf4j.Logger; -import static com.amazonaws.event.ProgressEventType.TRANSFER_COMPLETED_EVENT; -import static com.amazonaws.event.ProgressEventType.TRANSFER_PART_STARTED_EVENT; +import software.amazon.awssdk.transfer.s3.ObjectTransfer; +import software.amazon.awssdk.transfer.s3.progress.TransferListener; + +import org.apache.hadoop.util.Progressable; /** * Listener to progress from AWS regarding transfers. */ -public class ProgressableProgressListener implements ProgressListener { +public class ProgressableProgressListener implements TransferListener { private static final Logger LOG = S3AFileSystem.LOG; private final S3AFileSystem fs; private final String key; private final Progressable progress; private long lastBytesTransferred; - private final Upload upload; /** * Instantiate. * @param fs filesystem: will be invoked with statistics updates * @param key key for the upload - * @param upload source of events * @param progress optional callback for progress. */ public ProgressableProgressListener(S3AFileSystem fs, String key, - Upload upload, Progressable progress) { this.fs = fs; this.key = key; - this.upload = upload; this.progress = progress; this.lastBytesTransferred = 0; } @Override - public void progressChanged(ProgressEvent progressEvent) { - if (progress != null) { - progress.progress(); - } + public void transferInitiated(TransferListener.Context.TransferInitiated context) { + fs.incrementWriteOperations(); + } - // There are 3 http ops here, but this should be close enough for now - ProgressEventType pet = progressEvent.getEventType(); - if (pet == TRANSFER_PART_STARTED_EVENT || - pet == TRANSFER_COMPLETED_EVENT) { - fs.incrementWriteOperations(); + @Override + public void transferComplete(TransferListener.Context.TransferComplete context) { + fs.incrementWriteOperations(); + } + + @Override + public void bytesTransferred(TransferListener.Context.BytesTransferred context) { + + if(progress != null) { + progress.progress(); } - long transferred = upload.getProgress().getBytesTransferred(); + long transferred = context.progressSnapshot().bytesTransferred(); long delta = transferred - lastBytesTransferred; fs.incrementPutProgressStatistics(key, delta); lastBytesTransferred = transferred; @@ -81,9 +78,10 @@ public void progressChanged(ProgressEvent progressEvent) { * This can handle race conditions in setup/teardown. * @return the number of bytes which were transferred after the notification */ - public long uploadCompleted() { - long delta = upload.getProgress().getBytesTransferred() - - lastBytesTransferred; + public long uploadCompleted(ObjectTransfer upload) { + + long delta = + upload.progress().snapshot().bytesTransferred() - 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 19943ff2f70da..c8d6e23f6dc40 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 @@ -36,8 +36,6 @@ import com.amazonaws.event.ProgressEventType; import com.amazonaws.event.ProgressListener; import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.PutObjectResult; import com.amazonaws.services.s3.model.UploadPartRequest; import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; @@ -50,6 +48,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.Abortable; @@ -575,24 +576,30 @@ private int putObject() throws IOException { final PutObjectRequest putObjectRequest = uploadData.hasFile() ? writeOperationHelper.createPutObjectRequest( key, - uploadData.getFile(), - builder.putOptions) + uploadData.getFile().length(), + builder.putOptions, + true) : writeOperationHelper.createPutObjectRequest( key, - uploadData.getUploadStream(), size, - builder.putOptions); - BlockUploadProgress callback = - new BlockUploadProgress( - block, progressListener, now()); - putObjectRequest.setGeneralProgressListener(callback); + builder.putOptions, + false); + + // 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); statistics.blockUploadQueued(size); - ListenableFuture putObjectResult = + ListenableFuture putObjectResult = executorService.submit(() -> { try { // the putObject call automatically closes the input // stream afterwards. - return writeOperationHelper.putObject(putObjectRequest, builder.putOptions); + return writeOperationHelper.putObject(putObjectRequest, builder.putOptions, uploadData, + uploadData.hasFile()); } finally { cleanupWithLogger(LOG, uploadData, block); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java index 03b5bd96162af..e175972d94c6a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java @@ -47,7 +47,7 @@ * Set of classes to support output streaming into blocks which are then * uploaded as to S3 as a single PUT, or as part of a multipart request. */ -final class S3ADataBlocks { +public final class S3ADataBlocks { private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class); @@ -101,7 +101,7 @@ static BlockFactory createFactory(S3AFileSystem owner, * It can be one of a file or an input stream. * When closed, any stream is closed. Any source file is untouched. */ - static final class BlockUploadData implements Closeable { +public static final class BlockUploadData implements Closeable { private final File file; private final InputStream uploadStream; @@ -109,7 +109,7 @@ static final class BlockUploadData implements Closeable { * File constructor; input stream will be null. * @param file file to upload */ - BlockUploadData(File file) { + public BlockUploadData(File file) { Preconditions.checkArgument(file.exists(), "No file: " + file); this.file = file; this.uploadStream = null; @@ -119,7 +119,7 @@ static final class BlockUploadData implements Closeable { * Stream constructor, file field will be null. * @param uploadStream stream to upload */ - BlockUploadData(InputStream uploadStream) { + public BlockUploadData(InputStream uploadStream) { Preconditions.checkNotNull(uploadStream, "rawUploadStream"); this.uploadStream = uploadStream; this.file = null; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 83fe3ca49921d..59a974d71fd09 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -21,6 +21,7 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; +import java.io.InputStream; import java.io.InterruptedIOException; import java.io.UncheckedIOException; import java.net.URI; @@ -56,32 +57,22 @@ import com.amazonaws.services.s3.model.CannedAccessControlList; import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; -import com.amazonaws.services.s3.model.CopyObjectRequest; import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.DeleteObjectsResult; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadResult; import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import com.amazonaws.services.s3.model.MultipartUpload; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.PutObjectResult; import com.amazonaws.services.s3.model.S3Object; import com.amazonaws.services.s3.model.SelectObjectContentRequest; import com.amazonaws.services.s3.model.SelectObjectContentResult; import com.amazonaws.services.s3.model.StorageClass; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; -import com.amazonaws.services.s3.transfer.Copy; import com.amazonaws.services.s3.transfer.TransferManager; import com.amazonaws.services.s3.transfer.TransferManagerConfiguration; -import com.amazonaws.services.s3.transfer.Upload; -import com.amazonaws.services.s3.transfer.model.CopyResult; -import com.amazonaws.services.s3.transfer.model.UploadResult; -import com.amazonaws.event.ProgressListener; import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool; import org.slf4j.Logger; @@ -90,6 +81,21 @@ import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.ListObjectsRequest; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.sync.RequestBody; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.CopyObjectResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; +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.S3TransferManager; +import software.amazon.awssdk.transfer.s3.UploadFileRequest; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.classification.InterfaceAudience; @@ -114,7 +120,6 @@ import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ContextAccessors; import org.apache.hadoop.fs.s3a.impl.CopyFromLocalOperation; -import org.apache.hadoop.fs.s3a.impl.CopyOutcome; import org.apache.hadoop.fs.s3a.impl.CreateFileBuilder; import org.apache.hadoop.fs.s3a.impl.DeleteOperation; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; @@ -227,6 +232,7 @@ 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.ErrorTranslation.isUnknownBucketV2; 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; @@ -299,6 +305,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private long partSize; private boolean enableMultiObjectsDelete; private TransferManager transfers; + private S3TransferManager transferManagerV2; private ExecutorService boundedThreadPool; private ThreadPoolExecutor unboundedThreadPool; @@ -1141,6 +1148,18 @@ private void initTransferManager() { transferConfiguration.setMultipartCopyPartSize(partSize); transferConfiguration.setMultipartCopyThreshold(multiPartThreshold); + transferManagerV2 = S3TransferManager.builder() + .s3ClientConfiguration(clientConfiguration -> { + // TODO: This partSize check is required temporarily as some of the unit tests + // (TestStagingCommitter) set the S3Client using setAmazonS3Client() at which point + // partSize = 0, which gives a validation error with the new TM. The fix for this is + // probably in the tests and will be updated separately. + if (partSize > 0) { + clientConfiguration.minimumPartSizeInBytes(partSize); + } + }) + .build(); + transfers = new TransferManager(s3, unboundedThreadPool); transfers.setConfiguration(transferConfiguration); } @@ -2257,7 +2276,7 @@ public RemoteIterator listFilesAndDirectoryMarkers( } @Override - public CopyResult copyFile(final String srcKey, + public CopyObjectResponse copyFile(final String srcKey, final String destKey, final S3ObjectAttributes srcAttributes, final S3AReadOpContext readContext) throws IOException { @@ -2379,7 +2398,7 @@ public int getMaxKeys() { @InterfaceAudience.LimitedPrivate("utilities") @Retries.RetryTranslated @InterfaceStability.Evolving - public ObjectMetadata getObjectMetadata(Path path) throws IOException { + public HeadObjectResponse getObjectMetadata(Path path) throws IOException { V2Migration.v1GetObjectMetadataCalled(); return trackDurationAndSpan(INVOCATION_GET_FILE_STATUS, path, () -> getObjectMetadata(makeQualified(path), null, invoker, @@ -2396,7 +2415,7 @@ public ObjectMetadata getObjectMetadata(Path path) throws IOException { * @throws IOException IO and object access problems. */ @Retries.RetryTranslated - private ObjectMetadata getObjectMetadata(Path path, + private HeadObjectResponse getObjectMetadata(Path path, ChangeTracker changeTracker, Invoker changeInvoker, String operation) throws IOException { String key = pathToKey(path); @@ -2594,7 +2613,7 @@ protected DurationTrackerFactory getDurationTrackerFactory() { @Retries.RetryRaw @VisibleForTesting @InterfaceAudience.LimitedPrivate("external utilities") - ObjectMetadata getObjectMetadata(String key) throws IOException { + HeadObjectResponse getObjectMetadata(String key) throws IOException { return getObjectMetadata(key, null, invoker, "getObjectMetadata"); } @@ -2611,28 +2630,28 @@ ObjectMetadata getObjectMetadata(String key) throws IOException { * @throws RemoteFileChangedException if an unexpected version is detected */ @Retries.RetryRaw - protected ObjectMetadata getObjectMetadata(String key, + protected HeadObjectResponse getObjectMetadata(String key, ChangeTracker changeTracker, Invoker changeInvoker, String operation) throws IOException { - ObjectMetadata meta = changeInvoker.retryUntranslated("GET " + key, true, + HeadObjectResponse response = changeInvoker.retryUntranslated("GET " + key, true, () -> { - GetObjectMetadataRequest request - = getRequestFactory().newGetObjectMetadataRequest(key); + HeadObjectRequest.Builder requestBuilder = + getRequestFactory().newHeadObjectRequestBuilder(key); incrementStatistic(OBJECT_METADATA_REQUESTS); DurationTracker duration = getDurationTrackerFactory() .trackDuration(ACTION_HTTP_HEAD_REQUEST.getSymbol()); try { LOG.debug("HEAD {} with change tracker {}", key, changeTracker); if (changeTracker != null) { - changeTracker.maybeApplyConstraint(request); + changeTracker.maybeApplyConstraint(requestBuilder); } - ObjectMetadata objectMetadata = s3.getObjectMetadata(request); + HeadObjectResponse headObjectResponse = s3V2.headObject(requestBuilder.build()); if (changeTracker != null) { - changeTracker.processMetadata(objectMetadata, operation); + changeTracker.processMetadata(headObjectResponse, operation); } - return objectMetadata; - } catch(AmazonServiceException ase) { + return headObjectResponse; + } catch(AwsServiceException ase) { if (!isObjectNotFound(ase)) { // file not found is not considered a failure of the call, // so only switch the duration tracker to update failure @@ -2646,7 +2665,7 @@ protected ObjectMetadata getObjectMetadata(String key, } }); incrementReadOperations(); - return meta; + return response; } /** @@ -2886,28 +2905,17 @@ private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest) } /** - * Create a putObject request. + * Create a putObject request builder. * Adds the ACL and metadata * @param key key of object - * @param metadata metadata header - * @param srcfile source file + * @param length length of object to be uploaded + * @param isDirectoryMarker true if object to be uploaded is a directory marker * @return the request */ - public PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, File srcfile) { - return requestFactory.newPutObjectRequest(key, metadata, null, srcfile); - } - - /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * - * @param length length of data to set in header. - * @return a new metadata instance - */ - public ObjectMetadata newObjectMetadata(long length) { - return requestFactory.newObjectMetadata(length); + public PutObjectRequest.Builder newPutObjectRequestBuilder(String key, + long length, + boolean isDirectoryMarker) { + return requestFactory.newPutObjectRequestBuilder(key, null, length, isDirectoryMarker); } /** @@ -2924,15 +2932,22 @@ public ObjectMetadata newObjectMetadata(long length) { * Retry policy: N/A: the transfer manager is performing the upload. * Auditing: must be inside an audit span. * @param putObjectRequest the request + * @param file the file to be uploaded + * @param listener the progress listener for the request * @return the upload initiated */ @Retries.OnceRaw - public UploadInfo putObject(PutObjectRequest putObjectRequest) { + public UploadInfo putObject(PutObjectRequest putObjectRequest, File file, + ProgressableProgressListener listener) { long len = getPutRequestLength(putObjectRequest); - LOG.debug("PUT {} bytes to {} via transfer manager ", - len, putObjectRequest.getKey()); + LOG.debug("PUT {} bytes to {} via transfer manager ", len, putObjectRequest.key()); incrementPutStartStatistics(len); - Upload upload = transfers.upload(putObjectRequest); + + // TODO: Something not right with the TM listener, fix + FileUpload upload = transferManagerV2.uploadFile( + UploadFileRequest.builder().putObjectRequest(putObjectRequest).source(file).build()); + // .overrideConfiguration(o -> o.addListener(listener)).build()); + return new UploadInfo(upload, len); } @@ -2946,28 +2961,34 @@ public UploadInfo putObject(PutObjectRequest putObjectRequest) { * Important: this call will close any input stream in the request. * @param putObjectRequest the request * @param putOptions put object options + * @param uploadData data to be uploaded + * @param isFile represents if data to be uploaded is a file * @return the upload initiated * @throws AmazonClientException on problems */ @VisibleForTesting @Retries.OnceRaw("For PUT; post-PUT actions are RetryExceptionsSwallowed") - PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest, - PutObjectOptions putOptions) + PutObjectResponse putObjectDirect(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions, + S3ADataBlocks.BlockUploadData uploadData, boolean isFile) throws AmazonClientException { long len = getPutRequestLength(putObjectRequest); - LOG.debug("PUT {} bytes to {}", len, putObjectRequest.getKey()); + LOG.debug("PUT {} bytes to {}", len, putObjectRequest.key()); incrementPutStartStatistics(len); try { - PutObjectResult result = trackDurationOfSupplier( - getDurationTrackerFactory(), - OBJECT_PUT_REQUESTS.getSymbol(), () -> - s3.putObject(putObjectRequest)); + PutObjectResponse response = + trackDurationOfSupplier(getDurationTrackerFactory(), OBJECT_PUT_REQUESTS.getSymbol(), + () -> isFile ? + s3V2.putObject(putObjectRequest, RequestBody.fromFile(uploadData.getFile())) : + s3V2.putObject(putObjectRequest, + RequestBody.fromInputStream(uploadData.getUploadStream(), + putObjectRequest.contentLength()))); incrementPutCompletedStatistics(true, len); // apply any post-write actions. - finishedWrite(putObjectRequest.getKey(), len, - result.getETag(), result.getVersionId(), + finishedWrite(putObjectRequest.key(), len, + response.eTag(), response.versionId(), putOptions); - return result; + return response; } catch (SdkBaseException e) { incrementPutCompletedStatistics(false, len); throw e; @@ -2982,11 +3003,16 @@ PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest, */ private long getPutRequestLength(PutObjectRequest putObjectRequest) { long len; - if (putObjectRequest.getFile() != null) { - len = putObjectRequest.getFile().length(); - } else { - len = putObjectRequest.getMetadata().getContentLength(); - } + + // 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(); + Preconditions.checkState(len >= 0, "Cannot PUT object of unknown length"); return len; } @@ -3391,15 +3417,15 @@ public S3ListRequest createListObjectsRequest(String key, private S3ListRequest createListObjectsRequest(String key, String delimiter, int limit) { if (!useListV1) { - ListObjectsV2Request request = - getRequestFactory().newListObjectsV2Request( + ListObjectsV2Request.Builder requestBuilder = + getRequestFactory().newListObjectsV2RequestBuilder( key, delimiter, limit); - return S3ListRequest.v2(request); + return S3ListRequest.v2(requestBuilder.build()); } else { - ListObjectsRequest request = - getRequestFactory().newListObjectsV1Request( + ListObjectsRequest.Builder requestBuilder = + getRequestFactory().newListObjectsV1RequestBuilder( key, delimiter, limit); - return S3ListRequest.v1(request); + return S3ListRequest.v1(requestBuilder.build()); } } @@ -3687,22 +3713,22 @@ S3AFileStatus s3GetFileStatus(final Path path, && probes.contains(StatusProbeEnum.Head)) { try { // look for the simple file - ObjectMetadata meta = getObjectMetadata(key); + HeadObjectResponse meta = getObjectMetadata(key); LOG.debug("Found exact file: normal file {}", key); - long contentLength = meta.getContentLength(); + long contentLength = meta.contentLength(); // check if CSE is enabled, then strip padded length. - if (isCSEEnabled - && meta.getUserMetaDataOf(Headers.CRYPTO_CEK_ALGORITHM) != null + if (isCSEEnabled && + meta.metadata().get(Headers.CRYPTO_CEK_ALGORITHM) != null && contentLength >= CSE_PADDING_LENGTH) { contentLength -= CSE_PADDING_LENGTH; } return new S3AFileStatus(contentLength, - dateToLong(meta.getLastModified()), + meta.lastModified().toEpochMilli(), path, getDefaultBlockSize(path), username, - meta.getETag(), - meta.getVersionId()); + meta.eTag(), + meta.versionId()); } catch (AmazonServiceException e) { // if the response is a 404 error, it just means that there is // no file at that path...the remaining checks will be needed. @@ -3713,6 +3739,12 @@ S3AFileStatus s3GetFileStatus(final Path path, } } catch (AmazonClientException e) { throw translateException("getFileStatus", path, e); + } catch (AwsServiceException e) { + // TODO: This and above exception handling blocks will be updated during exception + // handling work. + if (e.statusCode() != SC_404 || isUnknownBucketV2(e)) { + throw translateExceptionV2("getFileStatus", path.toString(), e); + } } } @@ -3760,6 +3792,10 @@ S3AFileStatus s3GetFileStatus(final Path path, } } catch (AmazonClientException e) { throw translateException("getFileStatus", path, e); + } catch (AwsServiceException e) { + if (e.statusCode() != SC_404 || isUnknownBucketV2(e)) { + throw translateExceptionV2("getFileStatus", path.toString(), e); + } } } @@ -3857,13 +3893,12 @@ public void copyLocalFileFromTo(File file, Path from, Path to) throws IOExceptio to, () -> { final String key = pathToKey(to); - final ObjectMetadata om = newObjectMetadata(file.length()); Progressable progress = null; - PutObjectRequest putObjectRequest = newPutObjectRequest(key, om, file); - S3AFileSystem.this.invoker.retry( - "putObject(" + "" + ")", to.toString(), - true, - () -> executePut(putObjectRequest, progress, putOptionsForPath(to))); + PutObjectRequest.Builder putObjectRequestBuilder = + newPutObjectRequestBuilder(key, file.length(), false); + S3AFileSystem.this.invoker.retry("putObject(" + "" + ")", to.toString(), true, + () -> executePut(putObjectRequestBuilder.build(), progress, putOptionsForPath(to), + file)); return null; }); @@ -3897,24 +3932,23 @@ public boolean createEmptyDir(Path path, StoreContext storeContext) * @throws InterruptedIOException if the blocking was interrupted. */ @Retries.OnceRaw("For PUT; post-PUT actions are RetrySwallowed") - UploadResult executePut( + PutObjectResponse executePut( final PutObjectRequest putObjectRequest, final Progressable progress, - final PutObjectOptions putOptions) + final PutObjectOptions putOptions, + final File file) throws InterruptedIOException { - String key = putObjectRequest.getKey(); + String key = putObjectRequest.key(); long len = getPutRequestLength(putObjectRequest); - UploadInfo info = putObject(putObjectRequest); - Upload upload = info.getUpload(); - ProgressableProgressListener listener = new ProgressableProgressListener( - this, key, upload, progress); - upload.addProgressListener(listener); - UploadResult result = waitForUploadCompletion(key, info); - listener.uploadCompleted(); + ProgressableProgressListener listener = + new ProgressableProgressListener(this, putObjectRequest.key(), progress); + UploadInfo info = putObject(putObjectRequest, file, listener); + PutObjectResponse result = waitForUploadCompletion(key, info).response(); + listener.uploadCompleted(info.getFileUpload()); // post-write actions finishedWrite(key, len, - result.getETag(), result.getVersionId(), putOptions); + result.eTag(), result.versionId(), putOptions); return result; } @@ -3932,22 +3966,24 @@ UploadResult executePut( * @throws InterruptedIOException if the blocking was interrupted. */ @Retries.OnceRaw - UploadResult waitForUploadCompletion(String key, UploadInfo uploadInfo) + CompletedFileUpload waitForUploadCompletion(String key, UploadInfo uploadInfo) throws InterruptedIOException { - Upload upload = uploadInfo.getUpload(); - try { - UploadResult result = upload.waitForUploadResult(); + FileUpload upload = uploadInfo.getFileUpload(); + // TODO: Check what this logic should be updated to. + // this no longer throws an interrupted exception. + // try { + CompletedFileUpload result = upload.completionFuture().join(); incrementPutCompletedStatistics(true, uploadInfo.getLength()); return result; - } catch (InterruptedException e) { - LOG.info("Interrupted: aborting upload"); - incrementPutCompletedStatistics(false, uploadInfo.getLength()); - upload.abort(); - throw (InterruptedIOException) - new InterruptedIOException("Interrupted in PUT to " - + keyToQualifiedPath(key)) - .initCause(e); - } +// } catch (InterruptedException e) { +// LOG.info("Interrupted: aborting upload"); +// incrementPutCompletedStatistics(false, uploadInfo.getLength()); +// upload.completionFuture().cancel(true); +// throw (InterruptedIOException) +// new InterruptedIOException("Interrupted in PUT to " +// + keyToQualifiedPath(key)) +// .initCause(e); +// } } /** @@ -4208,20 +4244,22 @@ public List listAWSPolicyRules( * @throws IOException Other IO problems */ @Retries.RetryTranslated - private CopyResult copyFile(String srcKey, String dstKey, long size, + private CopyObjectResponse copyFile(String srcKey, String dstKey, long size, S3ObjectAttributes srcAttributes, S3AReadOpContext readContext) throws IOException, InterruptedIOException { LOG.debug("copyFile {} -> {} ", srcKey, dstKey); - ProgressListener progressListener = progressEvent -> { - switch (progressEvent.getEventType()) { - case TRANSFER_PART_COMPLETED_EVENT: - incrementWriteOperations(); - break; - default: - break; - } - }; + // TODO: Transfer manager currently only provides transfer listeners for upload, + // add progress listener for copy when this is supported. +// ProgressListener progressListener = progressEvent -> { +// switch (progressEvent.getEventType()) { +// case TRANSFER_PART_COMPLETED_EVENT: +// incrementWriteOperations(); +// break; +// default: +// break; +// } +// }; ChangeTracker changeTracker = new ChangeTracker( keyToQualifiedPath(srcKey).toString(), @@ -4234,7 +4272,7 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, String action = "copyFile(" + srcKey + ", " + dstKey + ")"; Invoker readInvoker = readContext.getReadInvoker(); - ObjectMetadata srcom; + HeadObjectResponse srcom; try { srcom = once(action, srcKey, () -> @@ -4257,29 +4295,32 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, action, srcKey, true, () -> { - CopyObjectRequest copyObjectRequest = - getRequestFactory().newCopyObjectRequest(srcKey, dstKey, srcom); - changeTracker.maybeApplyConstraint(copyObjectRequest); + CopyObjectRequest.Builder copyObjectRequestBuilder = + getRequestFactory().newCopyObjectRequestBuilder(srcKey, dstKey, srcom); + changeTracker.maybeApplyConstraint(copyObjectRequestBuilder); incrementStatistic(OBJECT_COPY_REQUESTS); - Copy copy = transfers.copy(copyObjectRequest, - getAuditManager().createStateChangeListener()); - copy.addProgressListener(progressListener); - CopyOutcome copyOutcome = CopyOutcome.waitForCopy(copy); - InterruptedException interruptedException = - copyOutcome.getInterruptedException(); - if (interruptedException != null) { - // copy interrupted: convert to an IOException. - throw (IOException)new InterruptedIOException( - "Interrupted copying " + srcKey - + " to " + dstKey + ", cancelling") - .initCause(interruptedException); - } - SdkBaseException awsException = copyOutcome.getAwsException(); - if (awsException != null) { - changeTracker.processException(awsException, "copy"); - throw awsException; - } - CopyResult result = copyOutcome.getCopyResult(); + + Copy copy = transferManagerV2.copy( + CopyRequest.builder().copyObjectRequest(copyObjectRequestBuilder.build()).build()); + + CompletedCopy completedCopy = copy.completionFuture().join(); + + // TODO: Check what should happen for these exceptions. +// InterruptedException interruptedException = +// copyOutcome.getInterruptedException(); +// if (interruptedException != null) { +// // copy interrupted: convert to an IOException. +// throw (IOException)new InterruptedIOException( +// "Interrupted copying " + srcKey +// + " to " + dstKey + ", cancelling") +// .initCause(interruptedException); +// } +// SdkBaseException awsException = copyOutcome.getAwsException(); +// if (awsException != null) { +// changeTracker.processException(awsException, "copy"); +// throw awsException; +// } + CopyObjectResponse result = completedCopy.response(); changeTracker.processResponse(result); incrementWriteOperations(); instrumentation.filesCopied(1, size); @@ -4422,10 +4463,19 @@ private void createFakeDirectory(final String objectName, @Retries.RetryTranslated private void createEmptyObject(final String objectName, PutObjectOptions putOptions) throws IOException { - invoker.retry("PUT 0-byte object ", objectName, - true, () -> - putObjectDirect(getRequestFactory() - .newDirectoryMarkerRequest(objectName), putOptions)); + + final InputStream im = new InputStream() { + @Override + public int read() throws IOException { + return -1; + } + }; + + S3ADataBlocks.BlockUploadData uploadData = new S3ADataBlocks.BlockUploadData(im); + + invoker.retry("PUT 0-byte object ", objectName, true, + () -> putObjectDirect(getRequestFactory().newDirectoryMarkerRequest(objectName).build(), + putOptions, uploadData, false)); incrementPutProgressStatistics(objectName, 0); instrumentation.directoryCreated(); } @@ -4682,10 +4732,10 @@ public EtagChecksum getFileChecksum(Path f, final long length) ETAG_CHECKSUM_ENABLED_DEFAULT)) { return trackDurationAndSpan(INVOCATION_GET_FILE_CHECKSUM, path, () -> { LOG.debug("getFileChecksum({})", path); - ObjectMetadata headers = getObjectMetadata(path, null, + HeadObjectResponse headers = getObjectMetadata(path, null, invoker, "getFileChecksum are"); - String eTag = headers.getETag(); + String eTag = headers.eTag(); return eTag != null ? new EtagChecksum(eTag) : null; }); } else { @@ -4767,7 +4817,7 @@ protected final class HeaderProcessingCallbacksImpl implements HeaderProcessing.HeaderProcessingCallbacks { @Override - public ObjectMetadata getObjectMetadata(final String key) + public HeadObjectResponse getObjectMetadata(final String key) throws IOException { return once("getObjectMetadata", key, () -> S3AFileSystem.this.getObjectMetadata(key)); 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 88d69e9c3d5f7..b68fd747abdf6 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 @@ -55,6 +55,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.services.s3.model.S3Object; import javax.annotation.Nullable; @@ -87,6 +90,7 @@ import static org.apache.commons.lang3.StringUtils.isEmpty; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket; +import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucketV2; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.translateDeleteException; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; @@ -174,6 +178,51 @@ public static IOException translateException(String operation, return translateException(operation, path.toString(), exception); } + // TODO: This is a temporary and incomplete implementation of error translations, done only as + // it is required by GetObjectMetadata so it can throw a FNFE. Will be done properly as part + // of error translation work. + public static IOException translateExceptionV2(@Nullable String operation, + String path, + SdkException exception) { + String message = String.format("%s%s: %s", + operation, + StringUtils.isNotEmpty(path)? (" on " + path) : "", + exception); + + AwsServiceException ase = (AwsServiceException) exception; + + int status = ase.statusCode(); + IOException ioe; + message = message + ":" + ase.awsErrorDetails().errorCode(); + switch (status) { + + case 403: + ioe = new AccessDeniedException(path, null, message); + ioe.initCause(ase); + break; + + case 404: + if (isUnknownBucketV2(ase)) { + // this is a missing bucket + ioe = new UnknownStoreException(path, message, ase); + } else { + // a normal unknown object + ioe = new FileNotFoundException(message); + ioe.initCause(ase); + } + break; + + default: + // no specific exit code. Choose an IOE subclass based on the class + // of the caught exception + ioe = new IOException(); + break; + } + + return ioe; + } + + /** * Translate an exception raised in an operation into an IOException. * The specific type of IOException depends on the class of diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java index 238cd97ed8f48..ee3a7c8a7e557 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/UploadInfo.java @@ -18,22 +18,22 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.services.s3.transfer.Upload; +import software.amazon.awssdk.transfer.s3.FileUpload; /** * Simple struct that contains information about a S3 upload. */ public class UploadInfo { - private final Upload upload; + private final FileUpload fileUpload; private final long length; - public UploadInfo(Upload upload, long length) { - this.upload = upload; + public UploadInfo(FileUpload upload, long length) { + this.fileUpload = upload; this.length = length; } - public Upload getUpload() { - return upload; + public FileUpload getFileUpload() { + return fileUpload; } public long getLength() { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java index 14ffeed4a55bb..232f0a7f552aa 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 @@ -31,10 +31,7 @@ import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.MultipartUpload; -import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.PutObjectResult; import com.amazonaws.services.s3.model.SelectObjectContentRequest; import com.amazonaws.services.s3.model.SelectObjectContentResult; import com.amazonaws.services.s3.model.UploadPartRequest; @@ -42,6 +39,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -238,51 +238,30 @@ private void deactivateAuditSpan() { /** * Create a {@link PutObjectRequest} request against the specific key. * @param destKey destination key - * @param inputStream source data. * @param length size, if known. Use -1 for not known * @param options options for the request + * @param isFile is data to be uploaded a file * @return the request */ @Retries.OnceRaw public PutObjectRequest createPutObjectRequest(String destKey, - InputStream inputStream, long length, - final PutObjectOptions options) { - activateAuditSpan(); - ObjectMetadata objectMetadata = newObjectMetadata(length); - return getRequestFactory().newPutObjectRequest( - destKey, - objectMetadata, - options, - inputStream); - } + final PutObjectOptions options, + boolean isFile) { + + if(isFile) { + Preconditions.checkState(length < Integer.MAX_VALUE, + "File length is too big for a single PUT upload"); + } - /** - * Create a {@link PutObjectRequest} request to upload a file. - * @param dest key to PUT to. - * @param sourceFile source file - * @param options options for the request - * @return the request - */ - @Retries.OnceRaw - public PutObjectRequest createPutObjectRequest( - String dest, - File sourceFile, - final PutObjectOptions options) { - Preconditions.checkState(sourceFile.length() < Integer.MAX_VALUE, - "File length is too big for a single PUT upload"); activateAuditSpan(); - final ObjectMetadata objectMetadata = - newObjectMetadata((int) sourceFile.length()); - PutObjectRequest putObjectRequest = getRequestFactory(). - newPutObjectRequest(dest, - objectMetadata, - options, - sourceFile); - return putObjectRequest; + return getRequestFactory() + .newPutObjectRequestBuilder(destKey, options, length, false) + .build(); } + /** * Callback on a successful write. * @param length length of the write @@ -298,17 +277,6 @@ public void writeFailed(Exception ex) { LOG.debug("Write to {} failed", this, ex); } - /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * @param length size, if known. Use -1 for not known - * @return a new metadata instance - */ - public ObjectMetadata newObjectMetadata(long length) { - return getRequestFactory().newObjectMetadata(length); - } - /** * {@inheritDoc} */ @@ -566,17 +534,20 @@ public String toString() { * file, from the content length of the header. * @param putObjectRequest the request * @param putOptions put object options + * @param uploadData data to be uploaded + * @param isFile is data to be uploaded a file + * * @return the upload initiated * @throws IOException on problems */ @Retries.RetryTranslated - public PutObjectResult putObject(PutObjectRequest putObjectRequest, - PutObjectOptions putOptions) + public PutObjectResponse putObject(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions, S3ADataBlocks.BlockUploadData uploadData, boolean isFile) throws IOException { return retry("Writing Object", - putObjectRequest.getKey(), true, + putObjectRequest.key(), true, withinAuditSpan(getAuditSpan(), () -> - owner.putObjectDirect(putObjectRequest, putOptions))); + owner.putObjectDirect(putObjectRequest, putOptions, uploadData, isFile))); } /** @@ -584,18 +555,19 @@ public PutObjectResult putObject(PutObjectRequest putObjectRequest, * * @param putObjectRequest the request * @param putOptions put object options + * @param uploadData data to be uploaded + * @param isFile is data to be uploaded a file * * @throws IOException on problems */ @Retries.RetryTranslated public void uploadObject(PutObjectRequest putObjectRequest, - PutObjectOptions putOptions) + PutObjectOptions putOptions, S3ADataBlocks.BlockUploadData uploadData, boolean isFile) throws IOException { - retry("Writing Object", - putObjectRequest.getKey(), true, + retry("Writing Object", putObjectRequest.key(), true, withinAuditSpan(getAuditSpan(), () -> - owner.putObjectDirect(putObjectRequest, putOptions))); + owner.putObjectDirect(putObjectRequest, putOptions, uploadData, isFile))); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java index 321390446f705..0ba7eb9068f31 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java @@ -29,15 +29,15 @@ import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; import com.amazonaws.services.s3.model.MultipartUpload; -import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.PutObjectResult; import com.amazonaws.services.s3.model.SelectObjectContentRequest; import com.amazonaws.services.s3.model.SelectObjectContentResult; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; @@ -76,27 +76,15 @@ T retry(String action, /** * Create a {@link PutObjectRequest} request against the specific key. * @param destKey destination key - * @param inputStream source data. * @param length size, if known. Use -1 for not known * @param options options for the request + * @param isFile is data to be uploaded a file * @return the request */ PutObjectRequest createPutObjectRequest(String destKey, - InputStream inputStream, long length, - @Nullable PutObjectOptions options); - - /** - * Create a {@link PutObjectRequest} request to upload a file. - * @param dest key to PUT to. - * @param sourceFile source file - * @param options options for the request - * @return the request - */ - PutObjectRequest createPutObjectRequest( - String dest, - File sourceFile, - @Nullable PutObjectOptions options); + @Nullable PutObjectOptions options, + boolean isFile); /** * Callback on a successful write. @@ -110,15 +98,6 @@ PutObjectRequest createPutObjectRequest( */ void writeFailed(Exception ex); - /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * @param length size, if known. Use -1 for not known - * @return a new metadata instance - */ - ObjectMetadata newObjectMetadata(long length); - /** * Start the multipart upload process. * Retry policy: retrying, translated. @@ -244,12 +223,14 @@ UploadPartRequest newUploadPartRequest( * file, from the content length of the header. * @param putObjectRequest the request * @param putOptions put object options + * @param uploadData data to be uploaded + * @param isFile is data to be uploaded a file * @return the upload initiated * @throws IOException on problems */ @Retries.RetryTranslated - PutObjectResult putObject(PutObjectRequest putObjectRequest, - PutObjectOptions putOptions) + PutObjectResponse putObject(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions, S3ADataBlocks.BlockUploadData uploadData, boolean isFile) throws IOException; /** @@ -257,12 +238,14 @@ PutObjectResult putObject(PutObjectRequest putObjectRequest, * * @param putObjectRequest the request * @param putOptions put object options + * @param uploadData data to be uploaded + * @param isFile is data to be uploaded a file * * @throws IOException on problems */ @Retries.RetryTranslated void uploadObject(PutObjectRequest putObjectRequest, - PutObjectOptions putOptions) + PutObjectOptions putOptions, S3ADataBlocks.BlockUploadData uploadData, boolean isFile) throws IOException; /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1ToV2AwsCredentialProviderAdapter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1ToV2AwsCredentialProviderAdapter.java index efea6540ef9ac..db82267044404 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1ToV2AwsCredentialProviderAdapter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/adapter/V1ToV2AwsCredentialProviderAdapter.java @@ -21,6 +21,9 @@ import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.AWSSessionCredentials; +import com.amazonaws.auth.AnonymousAWSCredentials; + +import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; @@ -45,6 +48,8 @@ public AwsCredentials resolveCredentials() { return AwsSessionCredentials.create(toAdapt.getAWSAccessKeyId(), toAdapt.getAWSSecretKey(), ((AWSSessionCredentials) toAdapt).getSessionToken()); + } else if (toAdapt instanceof AnonymousAWSCredentials) { + return AnonymousCredentialsProvider.create().resolveCredentials(); } else { return AwsBasicCredentials.create(toAdapt.getAWSAccessKeyId(), toAdapt.getAWSSecretKey()); } 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 c0c691741b280..4666432d3153a 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 @@ -27,25 +27,26 @@ import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; import com.amazonaws.services.s3.model.CannedAccessControlList; import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; -import com.amazonaws.services.s3.model.CopyObjectRequest; import com.amazonaws.services.s3.model.DeleteObjectRequest; import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; + +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import software.amazon.awssdk.services.s3.model.ListObjectsRequest; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import com.amazonaws.services.s3.model.ObjectListing; -import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; import com.amazonaws.services.s3.model.SSECustomerKey; import com.amazonaws.services.s3.model.SelectObjectContentRequest; import com.amazonaws.services.s3.model.StorageClass; import com.amazonaws.services.s3.model.UploadPartRequest; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; @@ -115,63 +116,42 @@ public interface RequestFactory { StorageClass getStorageClass(); /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * - * @param length length of data to set in header; Ignored if negative - * @return a new metadata instance - */ - ObjectMetadata newObjectMetadata(long length); - - /** - * Create a copy request. + * Create a copy request builder. * This includes the work of copying the relevant parts * of the metadata from the source * @param srcKey source * @param dstKey destination * @param srcom source object metadata. - * @return the request + * @return the request builder */ - CopyObjectRequest newCopyObjectRequest(String srcKey, + CopyObjectRequest.Builder newCopyObjectRequestBuilder(String srcKey, String dstKey, - ObjectMetadata srcom); + HeadObjectResponse srcom); - /** - * Create a putObject request. - * Adds the ACL and metadata - * @param key key of object - * @param metadata metadata header - * @param options options for the request - * @param srcfile source file - * @return the request - */ - PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, PutObjectOptions options, File srcfile); /** - * Create a {@link PutObjectRequest} request. + * Create a {@link PutObjectRequest} request builder. * The metadata is assumed to have been configured with the size of the * operation. * @param key key of object - * @param metadata metadata header * @param options options for the request - * @param inputStream source data. - * @return the request + * @param length length of object to be uploaded + * @param isDirectoryMarker true if object to be uploaded is a directory marker + * @return the request builder */ - PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, + PutObjectRequest.Builder newPutObjectRequestBuilder(String key, PutObjectOptions options, - InputStream inputStream); + long length, + boolean isDirectoryMarker); /** * Create a {@link PutObjectRequest} request for creating * an empty directory. * * @param directory destination directory. - * @return request for a zero byte upload. + * @return request builder for a zero byte upload. */ - PutObjectRequest newDirectoryMarkerRequest(String directory); + PutObjectRequest.Builder newDirectoryMarkerRequest(String directory); /** * List all multipart uploads under a prefix. @@ -214,11 +194,12 @@ CompleteMultipartUploadRequest newCompleteMultipartUploadRequest( List partETags); /** - * Create a HEAD request. + * Create a HEAD request builder. * @param key key, may have trailing / - * @return the request. + * @return the request builder. */ - GetObjectMetadataRequest newGetObjectMetadataRequest(String key); + HeadObjectRequest.Builder newHeadObjectRequestBuilder(String key); + /** * Create a GET request. @@ -262,13 +243,13 @@ UploadPartRequest newUploadPartRequest( SelectObjectContentRequest newSelectRequest(String key); /** - * Create the (legacy) V1 list request. + * Create the (legacy) V1 list request builder. * @param key key to list under * @param delimiter delimiter for keys * @param maxKeys maximum number in a list page. - * @return the request + * @return the request builder. */ - ListObjectsRequest newListObjectsV1Request(String key, + ListObjectsRequest.Builder newListObjectsV1RequestBuilder(String key, String delimiter, int maxKeys); @@ -283,14 +264,14 @@ ListNextBatchOfObjectsRequest newListNextBatchOfObjectsRequest( ObjectListing prev); /** - * Create a V2 list request. + * Create a V2 list request builder. * This will be recycled for any subsequent requests. * @param key key to list under * @param delimiter delimiter for keys * @param maxKeys maximum number in a list page. - * @return the request + * @return the request builder. */ - ListObjectsV2Request newListObjectsV2Request(String key, + ListObjectsV2Request.Builder newListObjectsV2RequestBuilder(String key, String delimiter, int maxKeys); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java index 6526f9a947815..44c3252186033 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java @@ -50,6 +50,35 @@ public static Optional createSSECustomerKey( } } + /*** + * Gets the SSE-C client side key if present. + * + * @param secrets source of the encryption secrets. + * @return an optional key to attach to a request. + */ + public static Optional getSSECustomerKey(final EncryptionSecrets secrets) { + if (secrets.hasEncryptionKey() && secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_C) { + return Optional.of(secrets.getEncryptionKey()); + } else { + return Optional.empty(); + } + } + + /** + * Gets the SSE-KMS key if present, else let S3 use AWS managed key. + * + * @param secrets source of the encryption secrets. + * @return an optional key to attach to a request. + */ + public static Optional getSSEAwsKMSKey(final EncryptionSecrets secrets) { + if (secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_KMS + && secrets.hasEncryptionKey()) { + return Optional.of(secrets.getEncryptionKey()); + } else { + return Optional.empty(); + } + } + /** * Create SSE-KMS options for a request, iff the encryption is SSE-KMS. * @return an optional SSE-KMS param to attach to a request. 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 bfb7e6966457b..34a3a8f1ef1f4 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 @@ -228,7 +228,7 @@ protected void serviceStop() throws Exception { try { super.serviceStop(); } finally { - ServiceOperations.stopQuietly(LOG, tokenBinding); + // ServiceOperations.stopQuietly(LOG, tokenBinding); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java index c85571a1949a1..697eaff993fd8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java @@ -20,19 +20,22 @@ import java.io.ByteArrayInputStream; import java.io.IOException; +import java.io.InputStream; import java.util.HashMap; import java.util.List; import java.util.Map; import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.PutObjectRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; + import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.S3ADataBlocks; import org.apache.hadoop.fs.s3a.WriteOperationHelper; import org.apache.hadoop.fs.s3a.commit.PutTracker; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; @@ -144,10 +147,9 @@ public boolean aboutToComplete(String uploadId, headers.put(X_HEADER_MAGIC_MARKER, Long.toString(bytesWritten)); PutObjectRequest originalDestPut = writer.createPutObjectRequest( originalDestKey, - new ByteArrayInputStream(EMPTY), 0, - new PutObjectOptions(true, null, headers)); - upload(originalDestPut); + new PutObjectOptions(true, null, headers), false); + upload(originalDestPut, new ByteArrayInputStream(EMPTY)); // build the commit summary SinglePendingCommit commitData = new SinglePendingCommit(); @@ -170,22 +172,22 @@ public boolean aboutToComplete(String uploadId, path, pendingPartKey, commitData); PutObjectRequest put = writer.createPutObjectRequest( pendingPartKey, - new ByteArrayInputStream(bytes), - bytes.length, null); - upload(put); + bytes.length, null, false); + upload(put, new ByteArrayInputStream(bytes)); return false; } /** * PUT an object. * @param request the request + * @param inputStream input stream of data to be uploaded * @throws IOException on problems */ @Retries.RetryTranslated - private void upload(PutObjectRequest request) throws IOException { - trackDurationOfInvocation(trackerStatistics, - COMMITTER_MAGIC_MARKER_PUT.getSymbol(), () -> - writer.uploadObject(request, PutObjectOptions.keepingDirs())); + private void upload(PutObjectRequest request, InputStream inputStream) throws IOException { + trackDurationOfInvocation(trackerStatistics, COMMITTER_MAGIC_MARKER_PUT.getSymbol(), + () -> writer.uploadObject(request, PutObjectOptions.keepingDirs(), + new S3ADataBlocks.BlockUploadData(inputStream), false)); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java index dcb538dc668de..c90b6f391dd33 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java @@ -20,15 +20,16 @@ import java.util.Locale; -import com.amazonaws.services.s3.model.CopyObjectRequest; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.transfer.model.CopyResult; import org.apache.hadoop.classification.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.CopyObjectResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; + import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -211,11 +212,11 @@ public String toString() { * null if the attribute is unavailable (such as when the policy says to use * versionId but object versioning is not enabled for the bucket). */ - public abstract String getRevisionId(ObjectMetadata objectMetadata, + public abstract String getRevisionId(HeadObjectResponse objectMetadata, String uri); /** - * Like {{@link #getRevisionId(ObjectMetadata, String)}}, but retrieves the + * Like {{@link #getRevisionId(HeadObjectResponse, String)}}, but retrieves the * revision identifier from {@link S3ObjectAttributes}. * * @param s3Attributes the object attributes @@ -226,18 +227,18 @@ public abstract String getRevisionId(ObjectMetadata objectMetadata, public abstract String getRevisionId(S3ObjectAttributes s3Attributes); /** - * Like {{@link #getRevisionId(ObjectMetadata, String)}}, but retrieves the - * revision identifier from {@link CopyResult}. + * Like {{@link #getRevisionId(HeadObjectResponse, String)}}, but retrieves the + * revision identifier from {@link CopyObjectResponse}. * - * @param copyResult the copy result + * @param copyObjectResponse the copy response * @return the revisionId string as interpreted by this policy, or potentially * null if the attribute is unavailable (such as when the policy says to use * versionId but object versioning is not enabled for the bucket). */ - public abstract String getRevisionId(CopyResult copyResult); + public abstract String getRevisionId(CopyObjectResponse copyObjectResponse); /** - * Applies the given {@link #getRevisionId(ObjectMetadata, String) revisionId} + * Applies the given {@link #getRevisionId(HeadObjectResponse, String) revisionId} * as a server-side qualification on the {@code GetObjectRequest}. * * @param request the request @@ -247,23 +248,23 @@ public abstract void applyRevisionConstraint(GetObjectRequest request, String revisionId); /** - * Applies the given {@link #getRevisionId(ObjectMetadata, String) revisionId} + * Applies the given {@link #getRevisionId(HeadObjectResponse, String) revisionId} * as a server-side qualification on the {@code CopyObjectRequest}. * - * @param request the request + * @param requestBuilder the copy object request builder * @param revisionId the revision id */ - public abstract void applyRevisionConstraint(CopyObjectRequest request, + public abstract void applyRevisionConstraint(CopyObjectRequest.Builder requestBuilder, String revisionId); /** - * Applies the given {@link #getRevisionId(ObjectMetadata, String) revisionId} + * Applies the given {@link #getRevisionId(HeadObjectResponse, String) revisionId} * as a server-side qualification on the {@code GetObjectMetadataRequest}. * - * @param request the request + * @param requestBuilder the head object request builder * @param revisionId the revision id */ - public abstract void applyRevisionConstraint(GetObjectMetadataRequest request, + public abstract void applyRevisionConstraint(HeadObjectRequest.Builder requestBuilder, String revisionId); /** @@ -323,7 +324,7 @@ public ImmutablePair onChangeDetected( } /** - * Change detection policy based on {@link ObjectMetadata#getETag() eTag}. + * Change detection policy based on {@link HeadObjectResponse#eTag()} () eTag}. */ static class ETagChangeDetectionPolicy extends ChangeDetectionPolicy { @@ -332,8 +333,8 @@ static class ETagChangeDetectionPolicy extends ChangeDetectionPolicy { } @Override - public String getRevisionId(ObjectMetadata objectMetadata, String uri) { - return objectMetadata.getETag(); + public String getRevisionId(HeadObjectResponse objectMetadata, String uri) { + return objectMetadata.eTag(); } @Override @@ -342,8 +343,8 @@ public String getRevisionId(S3ObjectAttributes s3Attributes) { } @Override - public String getRevisionId(CopyResult copyResult) { - return copyResult.getETag(); + public String getRevisionId(CopyObjectResponse copyObjectResponse) { + return copyObjectResponse.copyObjectResult().eTag(); } @Override @@ -358,18 +359,18 @@ public void applyRevisionConstraint(GetObjectRequest request, } @Override - public void applyRevisionConstraint(CopyObjectRequest request, + public void applyRevisionConstraint(CopyObjectRequest.Builder requestBuilder, String revisionId) { if (revisionId != null) { LOG.debug("Restricting copy request to etag {}", revisionId); - request.withMatchingETagConstraint(revisionId); + requestBuilder.copySourceIfMatch(revisionId); } else { LOG.debug("No etag revision ID to use as a constraint"); } } @Override - public void applyRevisionConstraint(GetObjectMetadataRequest request, + public void applyRevisionConstraint(HeadObjectRequest.Builder requestBuilder, String revisionId) { LOG.debug("Unable to restrict HEAD request to etag; will check later"); } @@ -388,7 +389,7 @@ public String toString() { /** * Change detection policy based on - * {@link ObjectMetadata#getVersionId() versionId}. + * {@link HeadObjectResponse#versionId()} () versionId}. */ static class VersionIdChangeDetectionPolicy extends ChangeDetectionPolicy { @@ -398,8 +399,8 @@ static class VersionIdChangeDetectionPolicy extends } @Override - public String getRevisionId(ObjectMetadata objectMetadata, String uri) { - String versionId = objectMetadata.getVersionId(); + public String getRevisionId(HeadObjectResponse objectMetadata, String uri) { + String versionId = objectMetadata.versionId(); if (versionId == null) { // this policy doesn't work if the bucket doesn't have object versioning // enabled (which isn't by default) @@ -419,8 +420,8 @@ public String getRevisionId(S3ObjectAttributes s3Attributes) { } @Override - public String getRevisionId(CopyResult copyResult) { - return copyResult.getVersionId(); + public String getRevisionId(CopyObjectResponse copyObjectResponse) { + return copyObjectResponse.versionId(); } @Override @@ -435,22 +436,22 @@ public void applyRevisionConstraint(GetObjectRequest request, } @Override - public void applyRevisionConstraint(CopyObjectRequest request, + public void applyRevisionConstraint(CopyObjectRequest.Builder requestBuilder, String revisionId) { if (revisionId != null) { LOG.debug("Restricting copy request to version {}", revisionId); - request.withSourceVersionId(revisionId); + requestBuilder.sourceVersionId(revisionId); } else { LOG.debug("No version ID to use as a constraint"); } } @Override - public void applyRevisionConstraint(GetObjectMetadataRequest request, + public void applyRevisionConstraint(HeadObjectRequest.Builder requestBuilder, String revisionId) { if (revisionId != null) { LOG.debug("Restricting metadata request to version {}", revisionId); - request.withVersionId(revisionId); + requestBuilder.versionId(revisionId).build(); } else { LOG.debug("No version ID to use as a constraint"); } @@ -482,7 +483,7 @@ public Source getSource() { } @Override - public String getRevisionId(final ObjectMetadata objectMetadata, + public String getRevisionId(final HeadObjectResponse objectMetadata, final String uri) { return null; } @@ -493,7 +494,7 @@ public String getRevisionId(final S3ObjectAttributes s3ObjectAttributes) { } @Override - public String getRevisionId(CopyResult copyResult) { + public String getRevisionId(CopyObjectResponse copyObjectResponse) { return null; } @@ -504,13 +505,13 @@ public void applyRevisionConstraint(final GetObjectRequest request, } @Override - public void applyRevisionConstraint(CopyObjectRequest request, + public void applyRevisionConstraint(CopyObjectRequest.Builder requestBuilder, String revisionId) { } @Override - public void applyRevisionConstraint(GetObjectMetadataRequest request, + public void applyRevisionConstraint(HeadObjectRequest.Builder requestBuilder, String revisionId) { } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java index 8b8cd0a9588c9..f42d9f3845b1b 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 @@ -20,16 +20,17 @@ import com.amazonaws.AmazonServiceException; import com.amazonaws.SdkBaseException; -import com.amazonaws.services.s3.model.CopyObjectRequest; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.transfer.model.CopyResult; import org.apache.hadoop.classification.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.CopyObjectResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; + import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -134,26 +135,26 @@ public boolean maybeApplyConstraint( /** * Apply any revision control set by the policy if it is to be * enforced on the server. - * @param request request to modify + * @param requestBuilder copy object request builder to modify * @return true iff a constraint was added. */ public boolean maybeApplyConstraint( - final CopyObjectRequest request) { + final CopyObjectRequest.Builder requestBuilder) { if (policy.getMode() == ChangeDetectionPolicy.Mode.Server && revisionId != null) { - policy.applyRevisionConstraint(request, revisionId); + policy.applyRevisionConstraint(requestBuilder, revisionId); return true; } return false; } public boolean maybeApplyConstraint( - final GetObjectMetadataRequest request) { + final HeadObjectRequest.Builder requestBuilder) { if (policy.getMode() == ChangeDetectionPolicy.Mode.Server && revisionId != null) { - policy.applyRevisionConstraint(request, revisionId); + policy.applyRevisionConstraint(requestBuilder, revisionId); return true; } return false; @@ -191,30 +192,33 @@ public void processResponse(final S3Object object, } } - processMetadata(object.getObjectMetadata(), operation); + // TODO: will be done with GetObject update + // processMetadata(object.getObjectMetadata(), operation); } /** * Process the response from the server for validation against the * change policy. - * @param copyResult result of a copy operation + * @param copyObjectResponse response of a copy operation * @throws PathIOException raised on failure * @throws RemoteFileChangedException if the remote file has changed. */ - public void processResponse(final CopyResult copyResult) + public void processResponse(final CopyObjectResponse copyObjectResponse) throws PathIOException { // ETag (sometimes, depending on encryption and/or multipart) is not the // same on the copied object as the original. Version Id seems to never // be the same on the copy. As such, there isn't really anything that // can be verified on the response, except that a revision ID is present // if required. - String newRevisionId = policy.getRevisionId(copyResult); - 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())); - } + // 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())); +// } } /** @@ -254,7 +258,7 @@ public void processException(SdkBaseException e, String operation) throws * @throws PathIOException raised on failure * @throws RemoteFileChangedException if the remote file has changed. */ - public void processMetadata(final ObjectMetadata metadata, + public void processMetadata(final HeadObjectResponse metadata, final String operation) throws PathIOException { final String newRevisionId = policy.getRevisionId(metadata, uri); processNewRevision(newRevisionId, operation, -1); @@ -281,10 +285,7 @@ private void processNewRevision(final String newRevisionId, LOG.debug("Setting revision ID for object at {}: {}", uri, newRevisionId); revisionId = newRevisionId; - //TODO: Remove this. This is a temporary fix to prevent tests from failing. Needed because - // SDKV2 returns etag with quotation marks, and V1 does not use quotations so this equality - // fails. Regex removes quotation marks. - } else if (!revisionId.replaceAll("^\"|\"$", "").equals(newRevisionId)) { + } else if (!revisionId.equals(newRevisionId)) { LOG.debug("Revision ID changed from {} to {}", revisionId, newRevisionId); ImmutablePair pair = diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java index f7e06413a3761..37f1e05c7aa38 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a.impl; import com.amazonaws.AmazonServiceException; +import software.amazon.awssdk.awscore.exception.AwsServiceException; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404; @@ -54,6 +55,19 @@ public static boolean isUnknownBucket(AmazonServiceException e) { && AwsErrorCodes.E_NO_SUCH_BUCKET.equals(e.getErrorCode()); } + // TODO: This method will be replace isUnkownBucket() during error translation work. + /** + * Does this exception indicate that the AWS Bucket was unknown. + * @param e exception. + * @return true if the status code and error code mean that the + * remote bucket is unknown. + */ + public static boolean isUnknownBucketV2(AwsServiceException e) { + return e.statusCode() == SC_404 + && AwsErrorCodes.E_NO_SUCH_BUCKET.equals(e.awsErrorDetails().errorCode()); + } + + /** * Does this exception indicate that a reference to an object * returned a 404. Unknown bucket errors do not match this @@ -62,8 +76,8 @@ public static boolean isUnknownBucket(AmazonServiceException e) { * @return true if the status code and error code mean that the * HEAD request returned 404 but the bucket was there. */ - public static boolean isObjectNotFound(AmazonServiceException e) { - return e.getStatusCode() == SC_404 && !isUnknownBucket(e); + public static boolean isObjectNotFound(AwsServiceException e) { + return e.statusCode() == SC_404 && !isUnknownBucketV2(e); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java index f75066e049d3e..a38f94b110487 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Date; import java.util.List; import java.util.Map; import java.util.Optional; @@ -32,6 +33,8 @@ import com.amazonaws.services.s3.model.ObjectMetadata; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.Path; @@ -275,7 +278,7 @@ private Map retrieveHeaders( final Statistic statistic) throws IOException { StoreContext context = getStoreContext(); String objectKey = context.pathToKey(path); - ObjectMetadata md; + HeadObjectResponse md; String symbol = statistic.getSymbol(); S3AStatisticsContext instrumentation = context.getInstrumentation(); try { @@ -287,59 +290,67 @@ private Map retrieveHeaders( callbacks.getObjectMetadata(objectKey + "/")); } // all user metadata - Map rawHeaders = md.getUserMetadata(); + Map rawHeaders = md.metadata(); Map headers = new TreeMap<>(); rawHeaders.forEach((key, value) -> headers.put(XA_HEADER_PREFIX + key, encodeBytes(value))); // and add the usual content length &c, if set maybeSetHeader(headers, XA_CACHE_CONTROL, - md.getCacheControl()); + md.cacheControl()); maybeSetHeader(headers, XA_CONTENT_DISPOSITION, - md.getContentDisposition()); + md.contentDisposition()); maybeSetHeader(headers, XA_CONTENT_ENCODING, - md.getContentEncoding()); + md.contentEncoding()); maybeSetHeader(headers, XA_CONTENT_LANGUAGE, - md.getContentLanguage()); + md.contentLanguage()); // If CSE is enabled, use the unencrypted content length. - if (md.getUserMetaDataOf(Headers.CRYPTO_CEK_ALGORITHM) != null - && md.getUserMetaDataOf(Headers.UNENCRYPTED_CONTENT_LENGTH) != null) { - maybeSetHeader(headers, XA_CONTENT_LENGTH, - md.getUserMetaDataOf(Headers.UNENCRYPTED_CONTENT_LENGTH)); - } else { - maybeSetHeader(headers, XA_CONTENT_LENGTH, - md.getContentLength()); + // TODO: CSE is not supported yet, add these headers in during CSE work. +// if (md.getUserMetaDataOf(Headers.CRYPTO_CEK_ALGORITHM) != null +// && md.getUserMetaDataOf(Headers.UNENCRYPTED_CONTENT_LENGTH) != null) { +// maybeSetHeader(headers, XA_CONTENT_LENGTH, +// md.getUserMetaDataOf(Headers.UNENCRYPTED_CONTENT_LENGTH)); +// } else { +// maybeSetHeader(headers, XA_CONTENT_LENGTH, +// md.contentLength()); +// } +// maybeSetHeader(headers, XA_CONTENT_MD5, +// md.getContentMD5()); + // TODO: Add back in else block during CSE work. + maybeSetHeader(headers, XA_CONTENT_LENGTH, + md.contentLength()); + if (md.sdkHttpResponse() != null && md.sdkHttpResponse().headers() != null + && md.sdkHttpResponse().headers().get("Content-Range") != null) { + maybeSetHeader(headers, XA_CONTENT_RANGE, + md.sdkHttpResponse().headers().get("Content-Range").get(0)); } - maybeSetHeader(headers, XA_CONTENT_MD5, - md.getContentMD5()); - maybeSetHeader(headers, XA_CONTENT_RANGE, - md.getContentRange()); maybeSetHeader(headers, XA_CONTENT_TYPE, - md.getContentType()); + md.contentType()); maybeSetHeader(headers, XA_ETAG, - md.getETag()); + md.eTag()); maybeSetHeader(headers, XA_LAST_MODIFIED, - md.getLastModified()); + Date.from(md.lastModified())); // AWS custom headers maybeSetHeader(headers, XA_ARCHIVE_STATUS, - md.getArchiveStatus()); + md.archiveStatus()); maybeSetHeader(headers, XA_OBJECT_LOCK_LEGAL_HOLD_STATUS, - md.getObjectLockLegalHoldStatus()); + md.objectLockLegalHoldStatus()); maybeSetHeader(headers, XA_OBJECT_LOCK_MODE, - md.getObjectLockMode()); + md.objectLockMode()); maybeSetHeader(headers, XA_OBJECT_LOCK_RETAIN_UNTIL_DATE, - md.getObjectLockRetainUntilDate()); + md.objectLockRetainUntilDate()); maybeSetHeader(headers, XA_OBJECT_REPLICATION_STATUS, - md.getReplicationStatus()); + md.replicationStatus()); maybeSetHeader(headers, XA_S3_VERSION_ID, - md.getVersionId()); + md.versionId()); maybeSetHeader(headers, XA_SERVER_SIDE_ENCRYPTION, - md.getSSEAlgorithm()); + md.serverSideEncryptionAsString()); maybeSetHeader(headers, XA_STORAGE_CLASS, - md.getStorageClass()); - maybeSetHeader(headers, XA_STORAGE_CLASS, - md.getReplicationStatus()); + md.storageClassAsString()); + // TODO: check this, looks wrong. + // maybeSetHeader(headers, XA_STORAGE_CLASS, +// md.getReplicationStatus()); return headers; } @@ -458,70 +469,51 @@ public static Optional extractXAttrLongValue(byte[] data) { } /** - * Creates a copy of the passed {@link ObjectMetadata}. - * Does so without using the {@link ObjectMetadata#clone()} method, - * to avoid copying unnecessary headers. + * Creates a copy of the passed metadata. * This operation does not copy the {@code X_HEADER_MAGIC_MARKER} * header to avoid confusion. If a marker file is renamed, * it loses information about any remapped file. * If new fields are added to ObjectMetadata which are not * present in the user metadata headers, they will not be picked * up or cloned unless this operation is updated. - * @param source the {@link ObjectMetadata} to copy + * @param source the source metadata to copy * @param dest the metadata to update; this is the return value. + * @param copyObjectRequestBuilder CopyObjectRequest builder */ - public static void cloneObjectMetadata(ObjectMetadata source, - ObjectMetadata dest) { + public static void cloneObjectMetadata(HeadObjectResponse source, + Map dest, CopyObjectRequest.Builder copyObjectRequestBuilder) { // Possibly null attributes // Allowing nulls to pass breaks it during later use - if (source.getCacheControl() != null) { - dest.setCacheControl(source.getCacheControl()); - } - if (source.getContentDisposition() != null) { - dest.setContentDisposition(source.getContentDisposition()); - } - if (source.getContentEncoding() != null) { - dest.setContentEncoding(source.getContentEncoding()); - } - if (source.getContentMD5() != null) { - dest.setContentMD5(source.getContentMD5()); - } - if (source.getContentType() != null) { - dest.setContentType(source.getContentType()); - } - if (source.getExpirationTime() != null) { - dest.setExpirationTime(source.getExpirationTime()); + if (source.cacheControl() != null) { + copyObjectRequestBuilder.cacheControl(source.cacheControl()); } - if (source.getExpirationTimeRuleId() != null) { - dest.setExpirationTimeRuleId(source.getExpirationTimeRuleId()); + if (source.contentDisposition() != null) { + copyObjectRequestBuilder.contentDisposition(source.contentDisposition()); } - if (source.getHttpExpiresDate() != null) { - dest.setHttpExpiresDate(source.getHttpExpiresDate()); + if (source.contentEncoding() != null) { + copyObjectRequestBuilder.contentEncoding(source.contentEncoding()); } - if (source.getLastModified() != null) { - dest.setLastModified(source.getLastModified()); - } - if (source.getOngoingRestore() != null) { - dest.setOngoingRestore(source.getOngoingRestore()); - } - if (source.getRestoreExpirationTime() != null) { - dest.setRestoreExpirationTime(source.getRestoreExpirationTime()); + + if (source.contentType() != null) { + copyObjectRequestBuilder.contentType(source.contentType()); } - if (source.getSSEAlgorithm() != null) { - dest.setSSEAlgorithm(source.getSSEAlgorithm()); + + if (source.serverSideEncryption() != null) { + copyObjectRequestBuilder.serverSideEncryption(source.serverSideEncryption()); } - if (source.getSSECustomerAlgorithm() != null) { - dest.setSSECustomerAlgorithm(source.getSSECustomerAlgorithm()); + + if (source.sseCustomerAlgorithm() != null) { + copyObjectRequestBuilder.copySourceSSECustomerAlgorithm(source.sseCustomerAlgorithm()); } - if (source.getSSECustomerKeyMd5() != null) { - dest.setSSECustomerKeyMd5(source.getSSECustomerKeyMd5()); + if (source.sseCustomerKeyMD5() != null) { + copyObjectRequestBuilder.copySourceSSECustomerKeyMD5(source.sseCustomerKeyMD5()); } // copy user metadata except the magic marker header. - source.getUserMetadata().entrySet().stream() + source.metadata().entrySet().stream() .filter(e -> !e.getKey().equals(X_HEADER_MAGIC_MARKER)) - .forEach(e -> dest.addUserMetadata(e.getKey(), e.getValue())); + .forEach(e -> dest.put(e.getKey(), e.getValue())); } public interface HeaderProcessingCallbacks { @@ -534,6 +526,6 @@ public interface HeaderProcessingCallbacks { * @throws IOException IO and object access problems. */ @Retries.RetryTranslated - ObjectMetadata getObjectMetadata(String key) throws IOException; + HeadObjectResponse getObjectMetadata(String key) throws IOException; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java index ecfe2c0ba0a24..0b0636fa5a5f7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java @@ -26,6 +26,7 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import com.amazonaws.services.s3.transfer.model.CopyResult; +import software.amazon.awssdk.services.s3.model.CopyObjectResponse; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.InvalidRequestException; @@ -126,7 +127,7 @@ RemoteIterator listFilesAndDirectoryMarkers( * @throws IOException Other IO problems */ @Retries.RetryTranslated - CopyResult copyFile(String srcKey, + CopyObjectResponse copyFile(String srcKey, String destKey, S3ObjectAttributes srcAttributes, S3AReadOpContext readContext) 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 fba75bb4bcc82..eeb9359a79dde 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 @@ -22,6 +22,8 @@ import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; +import java.util.Base64; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -31,7 +33,6 @@ import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; import com.amazonaws.services.s3.model.CannedAccessControlList; import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; -import com.amazonaws.services.s3.model.CopyObjectRequest; import com.amazonaws.services.s3.model.DeleteObjectRequest; import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.GetObjectMetadataRequest; @@ -40,9 +41,7 @@ import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; import com.amazonaws.services.s3.model.ObjectListing; -import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; import com.amazonaws.services.s3.model.SSECustomerKey; import com.amazonaws.services.s3.model.SelectObjectContentRequest; @@ -51,9 +50,17 @@ import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import software.amazon.awssdk.awscore.AwsRequest; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import software.amazon.awssdk.services.s3.model.ListObjectsRequest; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.MetadataDirective; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.ServerSideEncryption; +import software.amazon.awssdk.utils.Md5Utils; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.Retries; @@ -102,6 +109,8 @@ public class RequestFactoryImpl implements RequestFactory { /** * ACL For new objects. */ + // TODO: Set this to V2's ObjectCannedACL, and replace cannedACL.toString(). + // To be done during MPU update work. private final CannedAccessControlList cannedACL; /** @@ -159,7 +168,7 @@ private T prepareRequest(T t) { */ // TODO: Currently this is a NOOP, will be completed separately as part of auditor work. @Retries.OnceRaw - private T prepareV2Request(T t) { + private T prepareV2Request(T t) { return t; } @@ -185,6 +194,8 @@ protected String getBucket() { * if the encryption secrets contain the information/settings for this. * @return an optional set of KMS Key settings */ + // TODO: This method can be removed during getObject work, as the key now comes directly from + // EncryptionSecretOperations.getSSEAwsKMSKey. @Override public Optional generateSSEAwsKeyParams() { return EncryptionSecretOperations.createSSEAwsKeyManagementParams( @@ -197,6 +208,8 @@ public Optional generateSSEAwsKeyParams() { * This will contain a secret extracted from the bucket/configuration. * @return an optional customer key. */ + // TODO: This method can be removed during getObject work, as the key now comes directly from + // EncryptionSecretOperations.getSSECustomerKey. @Override public Optional generateSSECustomerKey() { return EncryptionSecretOperations.createSSECustomerKey( @@ -240,16 +253,6 @@ protected void setOptionalUploadPartRequestParameters( generateSSECustomerKey().ifPresent(request::setSSECustomerKey); } - /** - * Sets server side encryption parameters to the GET reuquest. - * request when encryption is enabled. - * @param request upload part request - */ - protected void setOptionalGetObjectMetadataParameters( - GetObjectMetadataRequest request) { - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); - } - /** * Set the optional parameters when initiating the request (encryption, * headers, storage, etc). @@ -261,192 +264,167 @@ protected void setOptionalMultipartUploadRequestParameters( generateSSECustomerKey().ifPresent(request::setSSECustomerKey); } - /** - * Set the optional parameters for a PUT request. - * @param request request to patch. - */ - protected void setOptionalPutRequestParameters(PutObjectRequest request) { - generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams); - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); - } + private CopyObjectRequest.Builder buildCopyObjectRequest() { - /** - * Set the optional metadata for an object being created or copied. - * @param metadata to update. - * @param isDirectoryMarker is this for a directory marker? - */ - protected void setOptionalObjectMetadata(ObjectMetadata metadata, - boolean isDirectoryMarker) { - final S3AEncryptionMethods algorithm - = getServerSideEncryptionAlgorithm(); - if (S3AEncryptionMethods.SSE_S3 == algorithm) { - metadata.setSSEAlgorithm(algorithm.getMethod()); - } - if (contentEncoding != null && !isDirectoryMarker) { - metadata.setContentEncoding(contentEncoding); + CopyObjectRequest.Builder copyObjectRequestBuilder = CopyObjectRequest.builder(); + + if (contentEncoding != null) { + copyObjectRequestBuilder.contentEncoding(contentEncoding); } + + return copyObjectRequestBuilder; } - /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * - * @param length length of data to set in header; Ignored if negative - * @return a new metadata instance - */ @Override - public ObjectMetadata newObjectMetadata(long length) { - return createObjectMetadata(length, false); - } + public CopyObjectRequest.Builder newCopyObjectRequestBuilder(String srcKey, + String dstKey, + HeadObjectResponse srcom) { - /** - * Create a new object metadata instance. - * Any standard metadata headers are added here, for example: - * encryption. - * - * @param length length of data to set in header; Ignored if negative - * @param isDirectoryMarker is this for a directory marker? - * @return a new metadata instance - */ - private ObjectMetadata createObjectMetadata(long length, boolean isDirectoryMarker) { - final ObjectMetadata om = new ObjectMetadata(); - setOptionalObjectMetadata(om, isDirectoryMarker); - if (length >= 0) { - om.setContentLength(length); + CopyObjectRequest.Builder copyObjectRequestBuilder = buildCopyObjectRequest(); + + Map dstom = new HashMap<>(); + HeaderProcessing.cloneObjectMetadata(srcom, dstom, copyObjectRequestBuilder); + copyEncryptionParameters(copyObjectRequestBuilder); + + // TODO: CannedACL will be converted to V2's ObjectCannedACL during MPU work. + if (cannedACL != null) { + copyObjectRequestBuilder.acl(cannedACL.toString()); } - return om; - } - @Override - public CopyObjectRequest newCopyObjectRequest(String srcKey, - String dstKey, - ObjectMetadata srcom) { - CopyObjectRequest copyObjectRequest = - new CopyObjectRequest(getBucket(), srcKey, getBucket(), dstKey); - ObjectMetadata dstom = newObjectMetadata(srcom.getContentLength()); - HeaderProcessing.cloneObjectMetadata(srcom, dstom); - setOptionalObjectMetadata(dstom, false); - copyEncryptionParameters(srcom, copyObjectRequest); - copyObjectRequest.setCannedAccessControlList(cannedACL); - copyObjectRequest.setNewObjectMetadata(dstom); - Optional.ofNullable(srcom.getStorageClass()) - .ifPresent(copyObjectRequest::setStorageClass); - return prepareRequest(copyObjectRequest); + copyObjectRequestBuilder + .metadata(dstom) + .metadataDirective(MetadataDirective.REPLACE); + if (srcom.storageClass() != null) { + copyObjectRequestBuilder.storageClass(srcom.storageClass()); + } + + copyObjectRequestBuilder.destinationBucket(getBucket()) + .destinationKey(dstKey).sourceBucket(getBucket()).sourceKey(srcKey); + + return prepareV2Request(copyObjectRequestBuilder); } /** * Propagate encryption parameters from source file if set else use the * current filesystem encryption settings. - * @param srcom source object metadata. - * @param copyObjectRequest copy object request body. + * @param copyObjectRequestBuilder copy object request builder. */ - protected void copyEncryptionParameters( - ObjectMetadata srcom, - CopyObjectRequest copyObjectRequest) { - String sourceKMSId = srcom.getSSEAwsKmsKeyId(); - if (isNotEmpty(sourceKMSId)) { - // source KMS ID is propagated - LOG.debug("Propagating SSE-KMS settings from source {}", - sourceKMSId); - copyObjectRequest.setSSEAwsKeyManagementParams( - new SSEAwsKeyManagementParams(sourceKMSId)); - } - switch (getServerSideEncryptionAlgorithm()) { - case SSE_S3: - /* no-op; this is set in destination object metadata */ - break; - - case SSE_C: - generateSSECustomerKey().ifPresent(customerKey -> { - copyObjectRequest.setSourceSSECustomerKey(customerKey); - copyObjectRequest.setDestinationSSECustomerKey(customerKey); - }); - break; + protected void copyEncryptionParameters(CopyObjectRequest.Builder copyObjectRequestBuilder) { - case SSE_KMS: - generateSSEAwsKeyParams().ifPresent( - copyObjectRequest::setSSEAwsKeyManagementParams); - break; - default: + final S3AEncryptionMethods algorithm + = getServerSideEncryptionAlgorithm(); + + if (S3AEncryptionMethods.SSE_S3 == algorithm) { + copyObjectRequestBuilder.serverSideEncryption(algorithm.getMethod()); + } else if (S3AEncryptionMethods.SSE_KMS == algorithm) { + copyObjectRequestBuilder.serverSideEncryption(ServerSideEncryption.AWS_KMS); + // Set the KMS key if present, else S3 uses AWS managed key. + EncryptionSecretOperations.getSSEAwsKMSKey(encryptionSecrets) + .ifPresent(kmsKey -> copyObjectRequestBuilder.ssekmsKeyId(kmsKey)); + } else if (S3AEncryptionMethods.SSE_C == algorithm) { + EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets).ifPresent(base64customerKey -> { + copyObjectRequestBuilder.copySourceSSECustomerAlgorithm(ServerSideEncryption.AES256.name()) + .copySourceSSECustomerKey(base64customerKey).copySourceSSECustomerKeyMD5( + Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey))) + .sseCustomerAlgorithm(ServerSideEncryption.AES256.name()) + .sseCustomerKey(base64customerKey) + .sseCustomerKeyMD5(Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey))); + }); } } /** * Create a putObject request. * Adds the ACL, storage class and metadata * @param key key of object - * @param metadata metadata header * @param options options for the request, including headers - * @param srcfile source file - * @return the request + * @param length length of object to be uploaded + * @param isDirectoryMarker true if object to be uploaded is a directory marker + * @return the request builder */ @Override - public PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, + public PutObjectRequest.Builder newPutObjectRequestBuilder(String key, final PutObjectOptions options, - File srcfile) { - Preconditions.checkNotNull(srcfile); - PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, - srcfile); - maybeSetMetadata(options, metadata); - setOptionalPutRequestParameters(putObjectRequest); - putObjectRequest.setCannedAcl(cannedACL); + long length, + boolean isDirectoryMarker) { + + Preconditions.checkArgument(isNotEmpty(key), "Null/empty key"); + + PutObjectRequest.Builder putObjectRequestBuilder = + buildPutObjectRequest(length, isDirectoryMarker); + putObjectRequestBuilder.bucket(getBucket()).key(key); + + if (options != null) { + putObjectRequestBuilder.metadata(options.getHeaders()); + } + + putEncryptionParameters(putObjectRequestBuilder); + + // TODO: CannedACL will be converted to V2's ObjectCannedACL during MPU work. + if (cannedACL != null) { + putObjectRequestBuilder.acl(cannedACL.toString()); + } + if (storageClass != null) { - putObjectRequest.setStorageClass(storageClass); + putObjectRequestBuilder.storageClass(storageClass.toString()); } - putObjectRequest.setMetadata(metadata); - return prepareRequest(putObjectRequest); + + return prepareV2Request(putObjectRequestBuilder); } - /** - * Create a {@link PutObjectRequest} request. - * The metadata is assumed to have been configured with the size of the - * operation. - * @param key key of object - * @param metadata metadata header - * @param options options for the request - * @param inputStream source data. - * @return the request - */ - @Override - public PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, - @Nullable final PutObjectOptions options, - InputStream inputStream) { - Preconditions.checkNotNull(inputStream); - Preconditions.checkArgument(isNotEmpty(key), "Null/empty key"); - maybeSetMetadata(options, metadata); - PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, - inputStream, metadata); - setOptionalPutRequestParameters(putObjectRequest); - putObjectRequest.setCannedAcl(cannedACL); - if (storageClass != null) { - putObjectRequest.setStorageClass(storageClass); + private PutObjectRequest.Builder buildPutObjectRequest(long length, boolean isDirectoryMarker) { + + PutObjectRequest.Builder putObjectRequestBuilder = PutObjectRequest.builder(); + + if (length >= 0) { + putObjectRequestBuilder.contentLength(length); + } + + if (contentEncoding != null && !isDirectoryMarker) { + putObjectRequestBuilder.contentEncoding(contentEncoding); + } + + return putObjectRequestBuilder; + } + + private void putEncryptionParameters(PutObjectRequest.Builder putObjectRequestBuilder) { + final S3AEncryptionMethods algorithm + = getServerSideEncryptionAlgorithm(); + + if (S3AEncryptionMethods.SSE_S3 == algorithm) { + putObjectRequestBuilder.serverSideEncryption(algorithm.getMethod()); + } else if (S3AEncryptionMethods.SSE_KMS == algorithm) { + putObjectRequestBuilder.serverSideEncryption(ServerSideEncryption.AWS_KMS); + // Set the KMS key if present, else S3 uses AWS managed key. + EncryptionSecretOperations.getSSEAwsKMSKey(encryptionSecrets) + .ifPresent(kmsKey -> putObjectRequestBuilder.ssekmsKeyId(kmsKey)); + } else if (S3AEncryptionMethods.SSE_C == algorithm) { + EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets) + .ifPresent(base64customerKey -> { + putObjectRequestBuilder.sseCustomerAlgorithm(ServerSideEncryption.AES256.name()) + .sseCustomerKey(base64customerKey).sseCustomerKeyMD5( + Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey))); + }); } - return prepareRequest(putObjectRequest); } @Override - public PutObjectRequest newDirectoryMarkerRequest(String directory) { + public PutObjectRequest.Builder newDirectoryMarkerRequest(String directory) { String key = directory.endsWith("/") ? directory : (directory + "/"); - // an input stream which is always empty - final InputStream inputStream = new InputStream() { - @Override - public int read() throws IOException { - return -1; - } - }; + // preparation happens in here - final ObjectMetadata metadata = createObjectMetadata(0L, true); - metadata.setContentType(HeaderProcessing.CONTENT_TYPE_X_DIRECTORY); + PutObjectRequest.Builder putObjectRequestBuilder = buildPutObjectRequest(0L, true); + + putObjectRequestBuilder.bucket(getBucket()).key(key) + .contentType(HeaderProcessing.CONTENT_TYPE_X_DIRECTORY); - PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, - inputStream, metadata); - setOptionalPutRequestParameters(putObjectRequest); - putObjectRequest.setCannedAcl(cannedACL); - return prepareRequest(putObjectRequest); + putEncryptionParameters(putObjectRequestBuilder); + if(cannedACL != null) { + putObjectRequestBuilder.acl(cannedACL.toString()); + } + + return prepareV2Request(putObjectRequestBuilder); } @Override @@ -473,12 +451,13 @@ public AbortMultipartUploadRequest newAbortMultipartUploadRequest( public InitiateMultipartUploadRequest newMultipartUploadRequest( final String destKey, @Nullable final PutObjectOptions options) { - final ObjectMetadata objectMetadata = newObjectMetadata(-1); - maybeSetMetadata(options, objectMetadata); + // TODO: Temporarily removing metadata, + // will be added back in when this operation is updated. + // final ObjectMetadata objectMetadata = newObjectMetadata(-1); + // maybeSetMetadata(options, objectMetadata); final InitiateMultipartUploadRequest initiateMPURequest = new InitiateMultipartUploadRequest(getBucket(), - destKey, - objectMetadata); + destKey); initiateMPURequest.setCannedACL(getCannedACL()); if (getStorageClass() != null) { initiateMPURequest.withStorageClass(getStorageClass()); @@ -500,12 +479,19 @@ public CompleteMultipartUploadRequest newCompleteMultipartUploadRequest( } @Override - public GetObjectMetadataRequest newGetObjectMetadataRequest(String key) { - GetObjectMetadataRequest request = - new GetObjectMetadataRequest(getBucket(), key); - //SSE-C requires to be filled in if enabled for object metadata - setOptionalGetObjectMetadataParameters(request); - return prepareRequest(request); + public HeadObjectRequest.Builder newHeadObjectRequestBuilder(String key) { + + HeadObjectRequest.Builder headObjectRequestBuilder = + HeadObjectRequest.builder().bucket(getBucket()).key(key); + + // need to set key to get metadata for objects encrypted with SSE_C + EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets).ifPresent(base64customerKey -> { + headObjectRequestBuilder.sseCustomerAlgorithm(ServerSideEncryption.AES256.name()) + .sseCustomerKey(base64customerKey) + .sseCustomerKeyMD5(Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey))); + }); + + return prepareV2Request(headObjectRequestBuilder); } @Override @@ -577,7 +563,7 @@ public SelectObjectContentRequest newSelectRequest(String key) { } @Override - public ListObjectsRequest newListObjectsV1Request( + public ListObjectsRequest.Builder newListObjectsV1RequestBuilder( final String key, final String delimiter, final int maxKeys) { @@ -589,7 +575,7 @@ public ListObjectsRequest newListObjectsV1Request( requestBuilder.delimiter(delimiter); } - return prepareV2Request(requestBuilder.build()); + return prepareV2Request(requestBuilder); } @Override @@ -599,7 +585,7 @@ public ListNextBatchOfObjectsRequest newListNextBatchOfObjectsRequest( } @Override - public ListObjectsV2Request newListObjectsV2Request( + public ListObjectsV2Request.Builder newListObjectsV2RequestBuilder( final String key, final String delimiter, final int maxKeys) { @@ -613,7 +599,7 @@ public ListObjectsV2Request newListObjectsV2Request( requestBuilder.delimiter(delimiter); } - return prepareV2Request(requestBuilder.build()); + return prepareV2Request(requestBuilder); } @Override @@ -635,23 +621,6 @@ public void setEncryptionSecrets(final EncryptionSecrets secrets) { encryptionSecrets = secrets; } - /** - * Set the metadata from the options if the options are not - * null and the metadata contains headers. - * @param options options for the request - * @param objectMetadata metadata to patch - */ - private void maybeSetMetadata( - @Nullable PutObjectOptions options, - final ObjectMetadata objectMetadata) { - if (options != null) { - Map headers = options.getHeaders(); - if (headers != null) { - objectMetadata.setUserMetadata(headers); - } - } - } - /** * Create a builder. * @return new builder. 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 d2fbd4def4151..9dac21dc22bf6 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 @@ -51,7 +51,7 @@ public abstract class AbstractS3AMockTest { protected S3AFileSystem fs; protected AmazonS3 s3; - private S3Client s3V2; + protected S3Client s3V2; @Before public void setup() throws Exception { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/EncryptionTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/EncryptionTestUtils.java index 4013e9db29a3e..244dea387244e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/EncryptionTestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/EncryptionTestUtils.java @@ -21,6 +21,7 @@ import java.io.IOException; import com.amazonaws.services.s3.model.ObjectMetadata; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.net.util.Base64; @@ -69,33 +70,33 @@ public static void assertEncrypted(S3AFileSystem fs, final S3AEncryptionMethods algorithm, final String kmsKeyArn) throws IOException { - ObjectMetadata md = fs.getObjectMetadata(path); + HeadObjectResponse md = fs.getObjectMetadata(path); String details = String.format( "file %s with encryption algorithm %s and key %s", path, - md.getSSEAlgorithm(), - md.getSSEAwsKmsKeyId()); + md.serverSideEncryptionAsString(), + md.ssekmsKeyId()); switch(algorithm) { case SSE_C: assertNull("Metadata algorithm should have been null in " + details, - md.getSSEAlgorithm()); + md.serverSideEncryptionAsString()); assertEquals("Wrong SSE-C algorithm in " + details, - SSE_C_ALGORITHM, md.getSSECustomerAlgorithm()); + SSE_C_ALGORITHM, md.sseCustomerAlgorithm()); String md5Key = convertKeyToMd5(fs); assertEquals("getSSECustomerKeyMd5() wrong in " + details, - md5Key, md.getSSECustomerKeyMd5()); + md5Key, md.sseCustomerKeyMD5()); break; case SSE_KMS: assertEquals("Wrong algorithm in " + details, - AWS_KMS_SSE_ALGORITHM, md.getSSEAlgorithm()); + AWS_KMS_SSE_ALGORITHM, md.serverSideEncryptionAsString()); assertEquals("Wrong KMS key in " + details, kmsKeyArn, - md.getSSEAwsKmsKeyId()); + md.ssekmsKeyId()); break; default: - assertEquals("AES256", md.getSSEAlgorithm()); + assertEquals("AES256", md.serverSideEncryptionAsString()); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSDefaultKey.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSDefaultKey.java index 68ab5bd9e8c19..ab06303640856 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSDefaultKey.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSDefaultKey.java @@ -21,6 +21,7 @@ import java.io.IOException; import com.amazonaws.services.s3.model.ObjectMetadata; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -51,9 +52,9 @@ protected S3AEncryptionMethods getSSEAlgorithm() { @Override protected void assertEncrypted(Path path) throws IOException { - ObjectMetadata md = getFileSystem().getObjectMetadata(path); + HeadObjectResponse md = getFileSystem().getObjectMetadata(path); assertEquals("SSE Algorithm", EncryptionTestUtils.AWS_KMS_SSE_ALGORITHM, - md.getSSEAlgorithm()); - assertThat(md.getSSEAwsKmsKeyId(), containsString("arn:aws:kms:")); + md.serverSideEncryptionAsString()); + assertThat(md.ssekmsKeyId(), containsString("arn:aws:kms:")); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionWithDefaultS3Settings.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionWithDefaultS3Settings.java index 56ce9300dc4e7..95ceae608e17b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionWithDefaultS3Settings.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionWithDefaultS3Settings.java @@ -118,7 +118,7 @@ public void testEncryptionOverRename() throws Throwable { S3AFileSystem fs = getFileSystem(); Path path = path(getMethodName() + "find-encryption-algo"); ContractTestUtils.touch(fs, path); - String sseAlgorithm = fs.getObjectMetadata(path).getSSEAlgorithm(); + String sseAlgorithm = fs.getObjectMetadata(path).serverSideEncryptionAsString(); if(StringUtils.isBlank(sseAlgorithm) || !sseAlgorithm.equals(AWS_KMS_SSE_ALGORITHM)) { skip("Test bucket is not configured with " + AWS_KMS_SSE_ALGORITHM); 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 2d29282ad0195..6621be2b8a913 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 @@ -27,17 +27,19 @@ import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.GetBucketEncryptionResult; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.PutObjectRequest; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.Test; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; +import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.store.EtagChecksum; import org.apache.hadoop.test.LambdaTestUtils; @@ -106,15 +108,15 @@ public void testCreateNonRecursiveSuccess() throws IOException { public void testPutObjectDirect() throws Throwable { final S3AFileSystem fs = getFileSystem(); try (AuditSpan span = span()) { - ObjectMetadata metadata = fs.newObjectMetadata(-1); - metadata.setContentLength(-1); + RequestFactory factory = RequestFactoryImpl.builder().withBucket(fs.getBucket()).build(); Path path = path("putDirect"); - final PutObjectRequest put = new PutObjectRequest(fs.getBucket(), - path.toUri().getPath(), - new ByteArrayInputStream("PUT".getBytes()), - metadata); + PutObjectRequest.Builder putObjectRequestBuilder = + factory.newPutObjectRequestBuilder(path.toUri().getPath(), null, -1, false); + putObjectRequestBuilder.contentLength(-1L); LambdaTestUtils.intercept(IllegalStateException.class, - () -> fs.putObjectDirect(put, PutObjectOptions.keepingDirs())); + () -> fs.putObjectDirect(putObjectRequestBuilder.build(), PutObjectOptions.keepingDirs(), + new S3ADataBlocks.BlockUploadData(new ByteArrayInputStream("PUT".getBytes())), + false)); assertPathDoesNotExist("put object was created", path); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index d3925d35a99d3..5b6ea46cd8a0a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -107,7 +107,7 @@ public void testRequesterPaysDisabledFails() throws Throwable { try (FileSystem fs = requesterPaysPath.getFileSystem(conf)) { intercept( AccessDeniedException.class, - "403 Forbidden", + "403", "Expected requester pays bucket to fail without header set", () -> fs.open(requesterPaysPath).close() ); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java index 62a99d7209263..4d0db3bc2034f 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 @@ -36,6 +36,9 @@ import org.junit.Test; import org.mockito.ArgumentMatcher; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; /** * deleteOnExit test for S3A. @@ -74,25 +77,25 @@ public void testDeleteOnExit() throws Exception { // unset S3CSE property from config to avoid pathIOE. conf.unset(Constants.S3_ENCRYPTION_ALGORITHM); testFs.initialize(uri, conf); - AmazonS3 testS3 = testFs.getAmazonS3ClientForTesting("mocking"); + S3Client testS3 = testFs.getAmazonS3V2ClientForTesting("mocking"); Path path = new Path("/file"); String key = path.toUri().getPath().substring(1); - ObjectMetadata meta = new ObjectMetadata(); - meta.setContentLength(1L); - meta.setLastModified(new Date(2L)); - when(testS3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) - .thenReturn(meta); + HeadObjectResponse objectMetadata = + HeadObjectResponse.builder().contentLength(1L).lastModified(new Date(2L).toInstant()) + .build(); + when(testS3.headObject(argThat(correctGetMetadataRequest(BUCKET, key)))) + .thenReturn(objectMetadata); testFs.deleteOnExit(path); testFs.close(); assertEquals(0, testFs.getDeleteOnDnExitCount()); } - private ArgumentMatcher correctGetMetadataRequest( + private ArgumentMatcher correctGetMetadataRequest( String bucket, String key) { return request -> request != null - && request.getBucketName().equals(bucket) - && request.getKey().equals(key); + && request.bucket().equals(bucket) + && request.key().equals(key); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java index cc61ef1fe2ab7..70ddd6d20ff77 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java @@ -41,6 +41,8 @@ import org.mockito.ArgumentMatcher; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.CommonPrefix; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import software.amazon.awssdk.services.s3.model.ListObjectsRequest; import software.amazon.awssdk.services.s3.model.ListObjectsResponse; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; @@ -56,17 +58,17 @@ public class TestS3AGetFileStatus extends AbstractS3AMockTest { public void testFile() throws Exception { Path path = new Path("/file"); String key = path.toUri().getPath().substring(1); - ObjectMetadata meta = new ObjectMetadata(); - meta.setContentLength(1L); - meta.setLastModified(new Date(2L)); - when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) - .thenReturn(meta); + HeadObjectResponse objectMetadata = + HeadObjectResponse.builder().contentLength(1L).lastModified(new Date(2L).toInstant()) + .build(); + when(s3V2.headObject(argThat(correctGetMetadataRequest(BUCKET, key)))) + .thenReturn(objectMetadata); FileStatus stat = fs.getFileStatus(path); assertNotNull(stat); assertEquals(fs.makeQualified(path), stat.getPath()); assertTrue(stat.isFile()); - assertEquals(meta.getContentLength(), stat.getLen()); - assertEquals(meta.getLastModified().getTime(), stat.getModificationTime()); + assertEquals(objectMetadata.contentLength().longValue(), stat.getLen()); + assertEquals(Date.from(objectMetadata.lastModified()).getTime(), stat.getModificationTime()); ContractTestUtils.assertNotErasureCoded(fs, path); assertTrue(path + " should have erasure coding unset in " + "FileStatus#toString(): " + stat, @@ -77,8 +79,7 @@ public void testFile() throws Exception { public void testFakeDirectory() throws Exception { Path path = new Path("/dir"); String key = path.toUri().getPath().substring(1); - S3Client s3V2 = getS3Client(); - when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) + when(s3V2.headObject(argThat(correctGetMetadataRequest(BUCKET, key)))) .thenThrow(NOT_FOUND); String keyDir = key + "/"; List s3Objects = new ArrayList<>(1); @@ -98,9 +99,9 @@ public void testFakeDirectory() throws Exception { public void testImplicitDirectory() throws Exception { Path path = new Path("/dir"); String key = path.toUri().getPath().substring(1); - when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) + when(s3V2.headObject(argThat(correctGetMetadataRequest(BUCKET, key)))) .thenThrow(NOT_FOUND); - when(s3.getObjectMetadata(argThat( + when(s3V2.headObject(argThat( correctGetMetadataRequest(BUCKET, key + "/")) )).thenThrow(NOT_FOUND); setupListMocks(Collections.singletonList(CommonPrefix.builder().prefix("dir/").build()), @@ -119,9 +120,9 @@ public void testImplicitDirectory() throws Exception { public void testRoot() throws Exception { Path path = new Path("/"); String key = path.toUri().getPath().substring(1); - when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) + when(s3V2.headObject(argThat(correctGetMetadataRequest(BUCKET, key)))) .thenThrow(NOT_FOUND); - when(s3.getObjectMetadata(argThat( + when(s3V2.headObject(argThat( correctGetMetadataRequest(BUCKET, key + "/") ))).thenThrow(NOT_FOUND); setupListMocks(Collections.emptyList(), Collections.emptyList()); @@ -136,9 +137,9 @@ public void testRoot() throws Exception { public void testNotFound() throws Exception { Path path = new Path("/dir"); String key = path.toUri().getPath().substring(1); - when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) + when(s3V2.headObject(argThat(correctGetMetadataRequest(BUCKET, key)))) .thenThrow(NOT_FOUND); - when(s3.getObjectMetadata(argThat( + when(s3V2.headObject(argThat( correctGetMetadataRequest(BUCKET, key + "/") ))).thenThrow(NOT_FOUND); setupListMocks(Collections.emptyList(), Collections.emptyList()); @@ -148,7 +149,6 @@ public void testNotFound() throws Exception { private void setupListMocks(List prefixes, List s3Objects) { - S3Client s3V2 = getS3Client(); // V1 list API mock ListObjectsResponse v1Response = mock(ListObjectsResponse.class); when(v1Response.commonPrefixes()).thenReturn(prefixes); @@ -164,11 +164,11 @@ private void setupListMocks(List prefixes, v2Result); } - private ArgumentMatcher correctGetMetadataRequest( + private ArgumentMatcher correctGetMetadataRequest( String bucket, String key) { return request -> request != null - && request.getBucketName().equals(bucket) - && request.getKey().equals(key); + && request.bucket().equals(bucket) + && request.key().equals(key); } private ArgumentMatcher matchListV2Request( diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java index 0e105c25c3a45..83c89b99260c1 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java @@ -26,9 +26,12 @@ import org.apache.hadoop.fs.Path; import org.junit.Test; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import java.io.IOException; import java.util.Date; +import java.util.function.Consumer; import static org.junit.Assert.assertEquals; @@ -51,11 +54,11 @@ public class TestS3AUnbuffer extends AbstractS3AMockTest { public void testUnbuffer() throws IOException { // Create mock ObjectMetadata for getFileStatus() Path path = new Path("/file"); - ObjectMetadata meta = mock(ObjectMetadata.class); - when(meta.getContentLength()).thenReturn(1L); - when(meta.getLastModified()).thenReturn(new Date(2L)); - when(meta.getETag()).thenReturn("mock-etag"); - when(s3.getObjectMetadata(any())).thenReturn(meta); + HeadObjectResponse objectMetadata = mock(HeadObjectResponse.class); + when(objectMetadata.contentLength()).thenReturn(1L); + when(objectMetadata.lastModified()).thenReturn(new Date(2L).toInstant()); + when(objectMetadata.eTag()).thenReturn("mock-etag"); + when(s3V2.headObject((HeadObjectRequest) any())).thenReturn(objectMetadata); // Create mock S3ObjectInputStream and S3Object for open() S3ObjectInputStream objectStream = mock(S3ObjectInputStream.class); @@ -63,6 +66,14 @@ public void testUnbuffer() throws IOException { when(objectStream.read(any(byte[].class))).thenReturn(-1); when(objectStream.read(any(byte[].class), anyInt(), anyInt())).thenReturn(-1); + // TODO: Remove during getObject work, required currently for the + // s3Object.getObjectMetadata() call. + ObjectMetadata meta = mock(ObjectMetadata.class); + when(meta.getContentLength()).thenReturn(1L); + when(meta.getLastModified()).thenReturn(new Date(2L)); + when(meta.getETag()).thenReturn("mock-etag"); + when(s3.getObjectMetadata(any())).thenReturn(meta); + S3Object s3Object = mock(S3Object.class); when(s3Object.getObjectContent()).thenReturn(objectStream); when(s3Object.getObjectMetadata()).thenReturn(meta); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java index 42de7cdffc80e..7512b0495189b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java @@ -21,15 +21,18 @@ import com.amazonaws.AmazonServiceException; import com.amazonaws.SdkBaseException; import com.amazonaws.services.s3.Headers; -import com.amazonaws.services.s3.model.CopyObjectRequest; import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.transfer.model.CopyResult; + import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.CopyObjectResponse; +import software.amazon.awssdk.services.s3.model.CopyObjectResult; + import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; @@ -266,9 +269,9 @@ protected void assertConstraintApplied(final ChangeTracker tracker, } protected void assertConstraintApplied(final ChangeTracker tracker, - final CopyObjectRequest request) throws PathIOException { + final CopyObjectRequest.Builder requestBuilder) throws PathIOException { assertTrue("Tracker should have applied contraints " + tracker, - tracker.maybeApplyConstraint(request)); + tracker.maybeApplyConstraint(requestBuilder)); } protected RemoteFileChangedException expectChangeException( @@ -298,7 +301,7 @@ protected PathIOException expectNoVersionAttributeException( protected PathIOException expectNoVersionAttributeException( final ChangeTracker tracker, - final CopyResult response, + final CopyObjectResponse response, final String message) throws Exception { return expectException(tracker, response, message, NoVersionAttributeException.class); @@ -320,7 +323,7 @@ protected T expectException( protected T expectException( final ChangeTracker tracker, - final CopyResult response, + final CopyObjectResponse response, final String message, final Class clazz) throws Exception { return intercept( @@ -398,19 +401,16 @@ private GetObjectRequest newGetObjectRequest() { return new GetObjectRequest(BUCKET, OBJECT); } - private CopyObjectRequest newCopyObjectRequest() { - return new CopyObjectRequest(BUCKET, OBJECT, BUCKET, DEST_OBJECT); + private CopyObjectRequest.Builder newCopyObjectRequest() { + return CopyObjectRequest.builder().sourceBucket(BUCKET).sourceKey(OBJECT) + .destinationBucket(BUCKET).destinationKey(DEST_OBJECT); } - private CopyResult newCopyResult(String eTag, String versionId) { - CopyResult copyResult = new CopyResult(); - copyResult.setSourceBucketName(BUCKET); - copyResult.setSourceKey(OBJECT); - copyResult.setDestinationBucketName(BUCKET); - copyResult.setDestinationKey(DEST_OBJECT); - copyResult.setETag(eTag); - copyResult.setVersionId(versionId); - return copyResult; + private CopyObjectResponse newCopyResult(String eTag, String versionId) { + CopyObjectResponse.Builder copyObjectResponseBuilder = CopyObjectResponse.builder(); + + return copyObjectResponseBuilder.versionId(versionId) + .copyObjectResult(CopyObjectResult.builder().eTag(eTag).build()).build(); } private S3Object newResponse(String etag, String versionId) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java index c76e3fa968f92..4295709296888 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/AbstractAuditingTest.java @@ -133,9 +133,11 @@ protected AuditSpanS3A activeSpan() { * callback. * @return a processed request. */ + // TODO: Temporary change as auditor still expects V1 request, will be updated during auditor work. protected GetObjectMetadataRequest head() { - return manager.beforeExecution( - requestFactory.newGetObjectMetadataRequest("/")); +// return manager.beforeExecution( +// requestFactory.newGetObjectMetadataRequest("/")); + return manager.beforeExecution(new GetObjectMetadataRequest("test", "/")); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java index bd552b91aadcb..6652c3b6f1647 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditIntegration.java @@ -138,32 +138,34 @@ public void testAuditManagerLifecycle() throws Throwable { assertServiceStateStopped(auditor); } - @Test - public void testSingleRequestHandler() throws Throwable { - AuditManagerS3A manager = AuditIntegration.createAndStartAuditManager( - noopAuditConfig(), - ioStatistics); - List handlers - = manager.createRequestHandlers(); - assertThat(handlers) - .hasSize(1); - RequestHandler2 handler = handlers.get(0); - RequestFactory requestFactory = RequestFactoryImpl.builder() - .withBucket("bucket") - .build(); - // test the basic pre-request sequence while avoiding - // the complexity of recreating the full sequence - // (and probably getting it wrong) - GetObjectMetadataRequest r - = requestFactory.newGetObjectMetadataRequest("/"); - DefaultRequest dr = new DefaultRequest(r, "S3"); - assertThat(handler.beforeMarshalling(r)) - .isNotNull(); - assertThat(handler.beforeExecution(r)) - .isNotNull(); - handler.beforeRequest(dr); - - } + // TODO: Temporarily commenting out as auditor still expects V1 request, + // will be updated during auditor work. +// @Test +// public void testSingleRequestHandler() throws Throwable { +// AuditManagerS3A manager = AuditIntegration.createAndStartAuditManager( +// noopAuditConfig(), +// ioStatistics); +// List handlers +// = manager.createRequestHandlers(); +// assertThat(handlers) +// .hasSize(1); +// RequestHandler2 handler = handlers.get(0); +// RequestFactory requestFactory = RequestFactoryImpl.builder() +// .withBucket("bucket") +// .build(); +// // test the basic pre-request sequence while avoiding +// // the complexity of recreating the full sequence +// // (and probably getting it wrong) +// GetObjectMetadataRequest r +// = requestFactory.newGetObjectMetadataRequest("/"); +// DefaultRequest dr = new DefaultRequest(r, "S3"); +// assertThat(handler.beforeMarshalling(r)) +// .isNotNull(); +// assertThat(handler.beforeExecution(r)) +// .isNotNull(); +// handler.beforeRequest(dr); +// +// } /** * Register a second handler, verify it makes it to the list. 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 9598ef084fa49..71e14d39797cf 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 @@ -330,7 +330,10 @@ 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 - fs.getObjectMetadata(new Path("/")); + // 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("/")); readLandsatMetadata(fs); URI uri = fs.getUri(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java index b193cca03db00..94251431ad0a4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java @@ -718,7 +718,7 @@ private void validateContent(Path dir, private void validateStorageClass(Path dir, String expectedStorageClass) throws Exception { Path expectedFile = getPart0000(dir); S3AFileSystem fs = getFileSystem(); - String actualStorageClass = fs.getObjectMetadata(expectedFile).getStorageClass(); + String actualStorageClass = fs.getObjectMetadata(expectedFile).storageClassAsString(); Assertions.assertThat(actualStorageClass) .describedAs("Storage class of object %s", expectedFile) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestXAttrCost.java index 3a390e34ecad2..f69870afe50c9 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,6 +64,9 @@ 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 82592b1d01950..3e7515fc2ca8a 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 @@ -31,6 +31,8 @@ import org.assertj.core.util.Lists; import org.junit.Before; import org.junit.Test; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.MockS3AFileSystem; @@ -206,20 +208,20 @@ public void testMetadataCopySkipsMagicAttribute() throws Throwable { final String owner = "x-header-owner"; final String root = "root"; CONTEXT_ACCESSORS.userHeaders.put(owner, root); - final ObjectMetadata source = CONTEXT_ACCESSORS + final HeadObjectResponse source = CONTEXT_ACCESSORS .getObjectMetadata(MAGIC_KEY); - final Map sourceUserMD = source.getUserMetadata(); + final Map sourceUserMD = source.metadata(); Assertions.assertThat(sourceUserMD.get(owner)) .describedAs("owner header in copied MD") .isEqualTo(root); - ObjectMetadata dest = new ObjectMetadata(); - headerProcessing.cloneObjectMetadata(source, dest); + Map destUserMetadata = new HashMap<>(); + headerProcessing.cloneObjectMetadata(source, destUserMetadata, CopyObjectRequest.builder()); - Assertions.assertThat(dest.getUserMetadata().get(X_HEADER_MAGIC_MARKER)) + Assertions.assertThat(destUserMetadata.get(X_HEADER_MAGIC_MARKER)) .describedAs("Magic marker header in copied MD") .isNull(); - Assertions.assertThat(dest.getUserMetadata().get(owner)) + Assertions.assertThat(destUserMetadata.get(owner)) .describedAs("owner header in copied MD") .isEqualTo(root); } @@ -307,14 +309,13 @@ public RequestFactory getRequestFactory() { } @Override - public ObjectMetadata getObjectMetadata(final String key) + public HeadObjectResponse getObjectMetadata(final String key) throws IOException { if (MAGIC_KEY.equals(key)) { - ObjectMetadata omd = new ObjectMetadata(); - omd.setUserMetadata(userHeaders); - omd.setContentLength(len); - omd.setLastModified(date); - return omd; + return HeadObjectResponse.builder() + .metadata(userHeaders) + .contentLength(len) + .lastModified(date.toInstant()).build(); } else { throw new FileNotFoundException(key); } 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 03f03c46f97b8..49a5f86939926 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 @@ -26,12 +26,14 @@ import com.amazonaws.AmazonWebServiceRequest; import com.amazonaws.services.s3.model.CannedAccessControlList; import com.amazonaws.services.s3.model.ObjectListing; -import com.amazonaws.services.s3.model.ObjectMetadata; import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; + import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.audit.AWSRequestAnalyzer; @@ -87,17 +89,19 @@ public void testRequestFactoryWithCannedACL() throws Throwable { .build(); String path = "path"; String path2 = "path2"; - ObjectMetadata md = factory.newObjectMetadata(128); - Assertions.assertThat( - factory.newPutObjectRequest(path, md, - null, new ByteArrayInputStream(new byte[0])) - .getCannedAcl()) + HeadObjectResponse md = HeadObjectResponse.builder().contentLength(128L).build(); + + // TODO: When updating MPU, remove toString() from ACLs, instead use V2's ObjectCannedACL + Assertions.assertThat(factory.newPutObjectRequestBuilder(path, null, 128, false) + .build() + .aclAsString()) .describedAs("ACL of PUT") - .isEqualTo(acl); - Assertions.assertThat(factory.newCopyObjectRequest(path, path2, md) - .getCannedAccessControlList()) + .isEqualTo(acl.toString()); + Assertions.assertThat(factory.newCopyObjectRequestBuilder(path, path2, md) + .build() + .aclAsString()) .describedAs("ACL of COPY") - .isEqualTo(acl); + .isEqualTo(acl.toString()); Assertions.assertThat(factory.newMultipartUploadRequest(path, null) .getCannedACL()) @@ -159,16 +163,16 @@ private void createFactoryObjects(RequestFactory factory) { String path = "path"; String path2 = "path2"; String id = "1"; - ObjectMetadata md = factory.newObjectMetadata(128); + // ObjectMetadata md = factory.newObjectMetadata(128); a(factory.newAbortMultipartUploadRequest(path, id)); a(factory.newCompleteMultipartUploadRequest(path, id, new ArrayList<>())); - a(factory.newCopyObjectRequest(path, path2, md)); + // a(factory.newCopyObjectRequest(path, path2, md)); a(factory.newDeleteObjectRequest(path)); a(factory.newBulkDeleteRequest(new ArrayList<>())); - a(factory.newDirectoryMarkerRequest(path)); + // a(factory.newDirectoryMarkerRequest(path)); a(factory.newGetObjectRequest(path)); - a(factory.newGetObjectMetadataRequest(path)); + // a(factory.newGetObjectMetadataRequest(path)); a(factory.newListMultipartUploadsRequest(path)); //TODO: Commenting out for now, new request extends AwsRequest, this can be updated once all // request factory operations are updated. @@ -177,10 +181,10 @@ private void createFactoryObjects(RequestFactory factory) { // a(factory.newListObjectsV2Request(path, "/", 1)); a(factory.newMultipartUploadRequest(path, null)); File srcfile = new File("/tmp/a"); - a(factory.newPutObjectRequest(path, - factory.newObjectMetadata(-1), null, srcfile)); +// a(factory.newPutObjectRequest(path, +// factory.newObjectMetadata(-1), null, srcfile)); ByteArrayInputStream stream = new ByteArrayInputStream(new byte[0]); - a(factory.newPutObjectRequest(path, md, null, stream)); + // a(factory.newPutObjectRequest(path, md, null, stream)); a(factory.newSelectRequest(path)); } 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 91ea0c8e62fb3..0a02cdac2d550 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.fs.s3a.S3ADataBlocks; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; @@ -41,6 +42,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; + import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; @@ -49,10 +53,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.amazonaws.services.s3.model.PutObjectResult; - import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; import static org.apache.hadoop.fs.s3a.Statistic.*; @@ -249,18 +249,19 @@ public void testMultiPagesListingPerformanceAndCorrectness() = fs.getWriteOperationHelper(); final RequestFactory requestFactory = writeOperationHelper.getRequestFactory(); - List> futures = + List> futures = new ArrayList<>(numOfPutRequests); for (int i=0; i - writeOperationHelper.putObject(put, PutObjectOptions.keepingDirs()))); + PutObjectRequest.Builder putObjectRequestBuilder = requestFactory + .newPutObjectRequestBuilder(fs.pathToKey(file), + null, 128, false); + futures.add(submit(executorService, + () -> writeOperationHelper.putObject(putObjectRequestBuilder.build(), + PutObjectOptions.keepingDirs(), + new S3ADataBlocks.BlockUploadData(new FailingInputStream()), false))); } LOG.info("Waiting for PUTs to complete"); waitForCompletion(futures); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java index 99407467df56d..006c989604fd7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java @@ -126,7 +126,7 @@ private void skipQuietly(String text) { protected void assertStorageClass(Path hugeFile) throws IOException { S3AFileSystem fs = getFileSystem(); - String actual = fs.getObjectMetadata(hugeFile).getStorageClass(); + String actual = fs.getObjectMetadata(hugeFile).storageClassAsString(); assertTrue( "Storage class of object is " + actual + ", expected " + STORAGE_CLASS_REDUCED_REDUNDANCY, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalOperationCallbacks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalOperationCallbacks.java index fa1ad2db62af7..e8f8ecd16230e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalOperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalOperationCallbacks.java @@ -25,6 +25,7 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import com.amazonaws.services.s3.transfer.model.CopyResult; +import software.amazon.awssdk.services.s3.model.CopyObjectResponse; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -88,7 +89,7 @@ public RemoteIterator listFilesAndDirectoryMarkers( } @Override - public CopyResult copyFile( + public CopyObjectResponse copyFile( String srcKey, String destKey, S3ObjectAttributes srcAttributes, From 6a03f912229ea90b86462c1fab45e22fc3f3e174 Mon Sep 17 00:00:00 2001 From: Alessandro Passaro Date: Fri, 30 Sep 2022 13:27:42 +0100 Subject: [PATCH 3/3] Upgrade GetObject to use SDK v2. Key change: `getObject` now returns a `ResponseInputStream` rather than a `S3Object`. This makes it simpler to handle the input stream lifetime in various classes such as `S3AInputStream`, `S3ARemoteObject`, or `SDKStreamDrainer`. --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 13 +- .../apache/hadoop/fs/s3a/S3AInputStream.java | 106 ++++++-------- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 11 ++ .../hadoop/fs/s3a/api/RequestFactory.java | 22 +-- .../fs/s3a/impl/ChangeDetectionPolicy.java | 54 ++++++-- .../hadoop/fs/s3a/impl/ChangeTracker.java | 29 ++-- .../fs/s3a/impl/RequestFactoryImpl.java | 20 ++- .../hadoop/fs/s3a/impl/SDKStreamDrainer.java | 43 ++---- .../fs/s3a/prefetch/S3ARemoteObject.java | 54 +++----- .../s3a/prefetch/S3ARemoteObjectReader.java | 7 +- .../fs/s3a/TestS3AInputStreamRetry.java | 130 ++++++++---------- .../apache/hadoop/fs/s3a/TestS3AUnbuffer.java | 44 +++--- .../fs/s3a/TestStreamChangeTracker.java | 55 +++----- .../fs/s3a/impl/TestRequestFactory.java | 2 +- .../fs/s3a/impl/TestSDKStreamDrainer.java | 16 +-- .../fs/s3a/prefetch/MockS3ARemoteObject.java | 29 ++-- .../fs/s3a/prefetch/S3APrefetchFakes.java | 49 +++---- 17 files changed, 321 insertions(+), 363 deletions(-) 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 59a974d71fd09..a500d5abe7250 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 @@ -59,13 +59,11 @@ import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.DeleteObjectsResult; -import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadResult; import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import com.amazonaws.services.s3.model.MultipartUpload; -import com.amazonaws.services.s3.model.S3Object; import com.amazonaws.services.s3.model.SelectObjectContentRequest; import com.amazonaws.services.s3.model.SelectObjectContentResult; import com.amazonaws.services.s3.model.StorageClass; @@ -78,7 +76,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.ResponseInputStream; import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; import software.amazon.awssdk.services.s3.model.ListObjectsRequest; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.awscore.exception.AwsServiceException; @@ -1643,18 +1644,18 @@ public void close() { } @Override - public GetObjectRequest newGetRequest(final String key) { + public GetObjectRequest.Builder newGetRequestBuilder(final String key) { // active the audit span used for the operation try (AuditSpan span = auditSpan.activate()) { - return getRequestFactory().newGetObjectRequest(key); + return getRequestFactory().newGetObjectRequestBuilder(key); } } @Override - public S3Object getObject(GetObjectRequest request) { + public ResponseInputStream getObject(GetObjectRequest request) { // active the audit span used for the operation try (AuditSpan span = auditSpan.activate()) { - return s3.getObject(request); + return s3V2.getObject(request); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java index b6ac8669a6734..ebd77f2b57925 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java @@ -22,6 +22,7 @@ import java.io.Closeable; import java.io.EOFException; import java.io.IOException; +import java.io.InputStream; import java.io.InterruptedIOException; import java.net.SocketTimeoutException; import java.nio.ByteBuffer; @@ -31,9 +32,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.IntFunction; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.S3ObjectInputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,7 +44,6 @@ import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.FileRange; -import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.impl.CombinedFileRange; import org.apache.hadoop.fs.VectoredReadUtils; @@ -61,6 +58,10 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.functional.CallableRaisingIOE; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; + import static java.util.Objects.requireNonNull; import static org.apache.commons.lang3.StringUtils.isNotEmpty; import static org.apache.hadoop.fs.VectoredReadUtils.isOrderedDisjoint; @@ -125,14 +126,9 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, */ private volatile boolean closed; /** - * wrappedStream is associated with an object (instance of S3Object). When - * the object is garbage collected, the associated wrappedStream will be - * closed. Keep a reference to this object to prevent the wrapperStream - * still in use from being closed unexpectedly due to garbage collection. - * See HADOOP-17338 for details. + * Input stream returned by a getObject call. */ - private S3Object object; - private S3ObjectInputStream wrappedStream; + private ResponseInputStream wrappedStream; private final S3AReadOpContext context; private final InputStreamCallbacks client; @@ -271,28 +267,22 @@ private synchronized void reopen(String reason, long targetPos, long length, uri, reason, targetPos, contentRangeFinish, length, pos, nextReadPos, inputPolicy); + GetObjectRequest request = client.newGetRequestBuilder(key) + .range(S3AUtils.formatRange(targetPos, contentRangeFinish - 1)) + .applyMutation(changeTracker::maybeApplyConstraint) + .build(); long opencount = streamStatistics.streamOpened(); - GetObjectRequest request = client.newGetRequest(key) - .withRange(targetPos, contentRangeFinish - 1); String operation = opencount == 0 ? OPERATION_OPEN : OPERATION_REOPEN; String text = String.format("%s %s at %d", operation, uri, targetPos); - changeTracker.maybeApplyConstraint(request); - - object = onceTrackingDuration(text, uri, + wrappedStream = onceTrackingDuration(text, uri, streamStatistics.initiateGetRequest(), () -> client.getObject(request)); - - changeTracker.processResponse(object, operation, + changeTracker.processResponse(wrappedStream.response(), operation, targetPos); - wrappedStream = object.getObjectContent(); - contentRangeStart = targetPos; - if (wrappedStream == null) { - throw new PathIOException(uri, - "Null IO stream from " + operation + " of (" + reason + ") "); - } + contentRangeStart = targetPos; this.pos = targetPos; } @@ -505,14 +495,15 @@ public synchronized int read() throws IOException { */ @Retries.OnceTranslated private void onReadFailure(IOException ioe, boolean forceAbort) { + GetObjectResponse objectResponse = wrappedStream == null ? null : wrappedStream.response(); if (LOG.isDebugEnabled()) { LOG.debug("Got exception while trying to read from stream {}, " + "client: {} object: {}, trying to recover: ", - uri, client, object, ioe); + uri, client, objectResponse, ioe); } else { LOG.info("Got exception while trying to read from stream {}, " + "client: {} object: {}, trying to recover: " + ioe, - uri, client, object); + uri, client, objectResponse); } streamStatistics.readException(); closeStream("failure recovery", forceAbort, false); @@ -672,7 +663,6 @@ private CompletableFuture closeStream( CompletableFuture operation; SDKStreamDrainer drainer = new SDKStreamDrainer( uri, - object, wrappedStream, shouldAbort, (int) remaining, @@ -694,7 +684,6 @@ private CompletableFuture closeStream( // either the stream is closed in the blocking call or the async call is // submitted with its own copy of the references wrappedStream = null; - object = null; return operation; } @@ -910,29 +899,21 @@ public void readVectored(List ranges, private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRange, IntFunction allocate) { LOG.debug("Start reading combined range {} from path {} ", combinedFileRange, pathStr); - // This reference is must be kept till all buffers are populated as this is a - // finalizable object which closes the internal stream when gc triggers. - S3Object objectRange = null; - S3ObjectInputStream objectContent = null; + ResponseInputStream rangeContent = null; try { checkIfVectoredIOStopped(); final String operationName = "readCombinedFileRange"; - objectRange = getS3Object(operationName, + rangeContent = getS3Object(operationName, combinedFileRange.getOffset(), combinedFileRange.getLength()); - objectContent = objectRange.getObjectContent(); - if (objectContent == null) { - throw new PathIOException(uri, - "Null IO stream received during " + operationName); - } - populateChildBuffers(combinedFileRange, objectContent, allocate); + populateChildBuffers(combinedFileRange, rangeContent, allocate); } catch (Exception ex) { LOG.debug("Exception while reading a range {} from path {} ", combinedFileRange, pathStr, ex); for(FileRange child : combinedFileRange.getUnderlying()) { child.getData().completeExceptionally(ex); } } finally { - IOUtils.cleanupWithLogger(LOG, objectRange, objectContent); + IOUtils.cleanupWithLogger(LOG, rangeContent); } LOG.debug("Finished reading range {} from path {} ", combinedFileRange, pathStr); } @@ -945,7 +926,7 @@ private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRa * @throws IOException any IOE. */ private void populateChildBuffers(CombinedFileRange combinedFileRange, - S3ObjectInputStream objectContent, + InputStream objectContent, IntFunction allocate) throws IOException { // If the combined file range just contains a single child // range, we only have to fill that one child buffer else @@ -977,7 +958,7 @@ private void populateChildBuffers(CombinedFileRange combinedFileRange, * @param drainQuantity how many bytes to drain. * @throws IOException any IOE. */ - private void drainUnnecessaryData(S3ObjectInputStream objectContent, long drainQuantity) + private void drainUnnecessaryData(InputStream objectContent, long drainQuantity) throws IOException { int drainBytes = 0; int readCount; @@ -1019,26 +1000,20 @@ private void validateRangeRequest(FileRange range) throws EOFException { */ private void readSingleRange(FileRange range, ByteBuffer buffer) { LOG.debug("Start reading range {} from path {} ", range, pathStr); - S3Object objectRange = null; - S3ObjectInputStream objectContent = null; + ResponseInputStream objectRange = null; try { checkIfVectoredIOStopped(); long position = range.getOffset(); int length = range.getLength(); final String operationName = "readRange"; objectRange = getS3Object(operationName, position, length); - objectContent = objectRange.getObjectContent(); - if (objectContent == null) { - throw new PathIOException(uri, - "Null IO stream received during " + operationName); - } - populateBuffer(length, buffer, objectContent); + populateBuffer(length, buffer, objectRange); range.getData().complete(buffer); } catch (Exception ex) { LOG.warn("Exception while reading a range {} from path {} ", range, pathStr, ex); range.getData().completeExceptionally(ex); } finally { - IOUtils.cleanupWithLogger(LOG, objectRange, objectContent); + IOUtils.cleanupWithLogger(LOG, objectRange); } LOG.debug("Finished reading range {} from path {} ", range, pathStr); } @@ -1053,7 +1028,7 @@ private void readSingleRange(FileRange range, ByteBuffer buffer) { */ private void populateBuffer(int length, ByteBuffer buffer, - S3ObjectInputStream objectContent) throws IOException { + InputStream objectContent) throws IOException { if (buffer.isDirect()) { VectoredReadUtils.readInDirectBuffer(length, buffer, @@ -1078,7 +1053,7 @@ private void populateBuffer(int length, * @param length number of bytes to fill in dest. * @throws IOException any IOE. */ - private void readByteArray(S3ObjectInputStream objectContent, + private void readByteArray(InputStream objectContent, byte[] dest, int offset, int length) throws IOException { @@ -1105,13 +1080,16 @@ private void readByteArray(S3ObjectInputStream objectContent, * @return S3Object result s3 object. * @throws IOException exception if any. */ - private S3Object getS3Object(String operationName, long position, - int length) throws IOException { - final GetObjectRequest request = client.newGetRequest(key) - .withRange(position, position + length - 1); - changeTracker.maybeApplyConstraint(request); + private ResponseInputStream getS3Object(String operationName, + long position, + int length) + throws IOException { + final GetObjectRequest request = client.newGetRequestBuilder(key) + .range(S3AUtils.formatRange(position, position + length - 1)) + .applyMutation(changeTracker::maybeApplyConstraint) + .build(); DurationTracker tracker = streamStatistics.initiateGetRequest(); - S3Object objectRange; + ResponseInputStream objectRange; Invoker invoker = context.getReadInvoker(); try { objectRange = invoker.retry(operationName, pathStr, true, @@ -1126,7 +1104,7 @@ private S3Object getS3Object(String operationName, long position, } finally { tracker.close(); } - changeTracker.processResponse(objectRange, operationName, + changeTracker.processResponse(objectRange.response(), operationName, position); return objectRange; } @@ -1279,11 +1257,11 @@ public IOStatistics getIOStatistics() { public interface InputStreamCallbacks extends Closeable { /** - * Create a GET request. + * Create a GET request builder. * @param key object key - * @return the request + * @return the request builder */ - GetObjectRequest newGetRequest(String key); + GetObjectRequest.Builder newGetRequestBuilder(String key); /** * Execute the request. @@ -1291,7 +1269,7 @@ public interface InputStreamCallbacks extends Closeable { * @return the response */ @Retries.OnceRaw - S3Object getObject(GetObjectRequest request); + ResponseInputStream getObject(GetObjectRequest request); /** * Submit some asynchronous work, for example, draining a stream. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java index b68fd747abdf6..85a4606edb3ee 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 @@ -1932,4 +1932,15 @@ public String toString() { } }; + /** + * Format a byte range for a request header. + * See https://www.rfc-editor.org/rfc/rfc9110.html#section-14.1.2 + * + * @param rangeStart the start byte offset + * @param rangeEnd the end byte offset (inclusive) + * @return a formatted byte range + */ + public static String formatRange(long rangeStart, long rangeEnd) { + return String.format("bytes=%d-%d", rangeStart, rangeEnd); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java index 4666432d3153a..6e6d327e7f172 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,16 +29,9 @@ import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; import com.amazonaws.services.s3.model.DeleteObjectRequest; import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; - -import software.amazon.awssdk.services.s3.model.CopyObjectRequest; -import software.amazon.awssdk.services.s3.model.HeadObjectRequest; -import software.amazon.awssdk.services.s3.model.HeadObjectResponse; -import software.amazon.awssdk.services.s3.model.ListObjectsRequest; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import com.amazonaws.services.s3.model.ObjectListing; import com.amazonaws.services.s3.model.PartETag; import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; @@ -46,13 +39,20 @@ import com.amazonaws.services.s3.model.SelectObjectContentRequest; import com.amazonaws.services.s3.model.StorageClass; import com.amazonaws.services.s3.model.UploadPartRequest; -import software.amazon.awssdk.services.s3.model.PutObjectRequest; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; +import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; + /** * Factory for S3 objects. * @@ -202,11 +202,11 @@ CompleteMultipartUploadRequest newCompleteMultipartUploadRequest( /** - * Create a GET request. + * Create a GET request builder. * @param key object key - * @return the request. + * @return the request builder. */ - GetObjectRequest newGetObjectRequest(String key); + GetObjectRequest.Builder newGetObjectRequestBuilder(String key); /** * Create and initialize a part request of a multipart upload. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java index c90b6f391dd33..b72d30cbf3013 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java @@ -20,13 +20,14 @@ import java.util.Locale; -import com.amazonaws.services.s3.model.GetObjectRequest; import org.apache.hadoop.classification.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.s3.model.CopyObjectRequest; import software.amazon.awssdk.services.s3.model.CopyObjectResponse; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; import software.amazon.awssdk.services.s3.model.HeadObjectRequest; import software.amazon.awssdk.services.s3.model.HeadObjectResponse; @@ -215,6 +216,19 @@ public String toString() { public abstract String getRevisionId(HeadObjectResponse objectMetadata, String uri); + /** + * Like {{@link #getRevisionId(HeadObjectResponse, String)}}, but retrieves the + * * revision identifier from {@link GetObjectResponse}. + * + * @param getObjectResponse the response instance + * @param uri the URI of the object + * @return the revisionId string as interpreted by this policy, or potentially + * null if the attribute is unavailable (such as when the policy says to use + * versionId but object versioning is not enabled for the bucket). + */ + public abstract String getRevisionId(GetObjectResponse getObjectResponse, + String uri); + /** * Like {{@link #getRevisionId(HeadObjectResponse, String)}}, but retrieves the * revision identifier from {@link S3ObjectAttributes}. @@ -239,12 +253,12 @@ public abstract String getRevisionId(HeadObjectResponse objectMetadata, /** * Applies the given {@link #getRevisionId(HeadObjectResponse, String) revisionId} - * as a server-side qualification on the {@code GetObjectRequest}. + * as a server-side qualification on the {@code GetObjectRequest.Builder}. * * @param request the request * @param revisionId the revision id */ - public abstract void applyRevisionConstraint(GetObjectRequest request, + public abstract void applyRevisionConstraint(GetObjectRequest.Builder request, String revisionId); /** @@ -332,6 +346,11 @@ static class ETagChangeDetectionPolicy extends ChangeDetectionPolicy { super(mode, requireVersion); } + @Override + public String getRevisionId(GetObjectResponse objectMetadata, String uri) { + return objectMetadata.eTag(); + } + @Override public String getRevisionId(HeadObjectResponse objectMetadata, String uri) { return objectMetadata.eTag(); @@ -348,11 +367,11 @@ public String getRevisionId(CopyObjectResponse copyObjectResponse) { } @Override - public void applyRevisionConstraint(GetObjectRequest request, + public void applyRevisionConstraint(GetObjectRequest.Builder builder, String revisionId) { if (revisionId != null) { LOG.debug("Restricting get request to etag {}", revisionId); - request.withMatchingETagConstraint(revisionId); + builder.ifMatch(revisionId); } else { LOG.debug("No etag revision ID to use as a constraint"); } @@ -400,7 +419,15 @@ static class VersionIdChangeDetectionPolicy extends @Override public String getRevisionId(HeadObjectResponse objectMetadata, String uri) { - String versionId = objectMetadata.versionId(); + return logIfNull(objectMetadata.versionId(), uri); + } + + @Override + public String getRevisionId(GetObjectResponse getObjectResponse, String uri) { + return logIfNull(getObjectResponse.versionId(), uri); + } + + private String logIfNull(String versionId, String uri) { if (versionId == null) { // this policy doesn't work if the bucket doesn't have object versioning // enabled (which isn't by default) @@ -408,8 +435,7 @@ public String getRevisionId(HeadObjectResponse objectMetadata, String uri) { CHANGE_DETECT_MODE + " set to " + Source.VersionId + " but no versionId available while reading {}. " + "Ensure your bucket has object versioning enabled. " - + "You may see inconsistent reads.", - uri); + + "You may see inconsistent reads.", uri); } return versionId; } @@ -425,11 +451,11 @@ public String getRevisionId(CopyObjectResponse copyObjectResponse) { } @Override - public void applyRevisionConstraint(GetObjectRequest request, + public void applyRevisionConstraint(GetObjectRequest.Builder builder, String revisionId) { if (revisionId != null) { LOG.debug("Restricting get request to version {}", revisionId); - request.withVersionId(revisionId); + builder.versionId(revisionId); } else { LOG.debug("No version ID to use as a constraint"); } @@ -482,6 +508,12 @@ public Source getSource() { return Source.None; } + @Override + public String getRevisionId(final GetObjectResponse objectMetadata, + final String uri) { + return null; + } + @Override public String getRevisionId(final HeadObjectResponse objectMetadata, final String uri) { @@ -499,7 +531,7 @@ public String getRevisionId(CopyObjectResponse copyObjectResponse) { } @Override - public void applyRevisionConstraint(final GetObjectRequest request, + public void applyRevisionConstraint(final GetObjectRequest.Builder builder, final String revisionId) { } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java index f42d9f3845b1b..aadbd31fa22db 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 @@ -20,14 +20,14 @@ import com.amazonaws.AmazonServiceException; import com.amazonaws.SdkBaseException; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.S3Object; import org.apache.hadoop.classification.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.s3.model.CopyObjectRequest; import software.amazon.awssdk.services.s3.model.CopyObjectResponse; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; import software.amazon.awssdk.services.s3.model.HeadObjectRequest; import software.amazon.awssdk.services.s3.model.HeadObjectResponse; @@ -118,15 +118,15 @@ public long getVersionMismatches() { /** * Apply any revision control set by the policy if it is to be * enforced on the server. - * @param request request to modify + * @param builder request builder to modify * @return true iff a constraint was added. */ public boolean maybeApplyConstraint( - final GetObjectRequest request) { + final GetObjectRequest.Builder builder) { if (policy.getMode() == ChangeDetectionPolicy.Mode.Server && revisionId != null) { - policy.applyRevisionConstraint(request, revisionId); + policy.applyRevisionConstraint(builder, revisionId); return true; } return false; @@ -169,7 +169,7 @@ public boolean maybeApplyConstraint( * @throws PathIOException raised on failure * @throws RemoteFileChangedException if the remote file has changed. */ - public void processResponse(final S3Object object, + public void processResponse(final GetObjectResponse object, final String operation, final long pos) throws PathIOException { if (object == null) { @@ -192,8 +192,7 @@ public void processResponse(final S3Object object, } } - // TODO: will be done with GetObject update - // processMetadata(object.getObjectMetadata(), operation); + processMetadata(object, operation); } /** @@ -264,6 +263,20 @@ public void processMetadata(final HeadObjectResponse metadata, processNewRevision(newRevisionId, operation, -1); } + /** + * Process the response from server for validation against the change + * policy. + * @param getObjectResponse response returned from server + * @param operation operation in progress + * @throws PathIOException raised on failure + * @throws RemoteFileChangedException if the remote file has changed. + */ + public void processMetadata(final GetObjectResponse getObjectResponse, + final String operation) throws PathIOException { + final String newRevisionId = policy.getRevisionId(getObjectResponse, uri); + processNewRevision(newRevisionId, operation, -1); + } + /** * Validate a revision from the server against our expectations. * @param newRevisionId new revision. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java index eeb9359a79dde..1ebec5b1e8b36 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 @@ -19,7 +19,6 @@ package org.apache.hadoop.fs.s3a.impl; import java.io.File; -import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; import java.util.Base64; @@ -35,8 +34,6 @@ import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; import com.amazonaws.services.s3.model.DeleteObjectRequest; import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.GetObjectMetadataRequest; -import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.ListMultipartUploadsRequest; import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest; @@ -53,6 +50,7 @@ import software.amazon.awssdk.awscore.AwsRequest; import software.amazon.awssdk.services.s3.model.CopyObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; import software.amazon.awssdk.services.s3.model.HeadObjectRequest; import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import software.amazon.awssdk.services.s3.model.ListObjectsRequest; @@ -495,11 +493,19 @@ public HeadObjectRequest.Builder newHeadObjectRequestBuilder(String key) { } @Override - public GetObjectRequest newGetObjectRequest(String key) { - GetObjectRequest request = new GetObjectRequest(bucket, key); - generateSSECustomerKey().ifPresent(request::setSSECustomerKey); + public GetObjectRequest.Builder newGetObjectRequestBuilder(String key) { + GetObjectRequest.Builder builder = GetObjectRequest.builder() + .bucket(bucket) + .key(key); - return prepareRequest(request); + // need to set key to get objects encrypted with SSE_C + EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets).ifPresent(base64customerKey -> { + builder.sseCustomerAlgorithm(ServerSideEncryption.AES256.name()) + .sseCustomerKey(base64customerKey) + .sseCustomerKeyMD5(Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey))); + }); + + return prepareV2Request(builder); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/SDKStreamDrainer.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/SDKStreamDrainer.java index b566f9ad42765..206d74e549d88 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/SDKStreamDrainer.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/SDKStreamDrainer.java @@ -18,12 +18,9 @@ package org.apache.hadoop.fs.s3a.impl; -import java.io.Closeable; +import java.io.InputStream; import java.util.concurrent.atomic.AtomicBoolean; -import javax.annotation.Nullable; - -import com.amazonaws.internal.SdkFilterInputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,23 +28,19 @@ import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import org.apache.hadoop.util.functional.CallableRaisingIOE; +import software.amazon.awssdk.http.Abortable; + import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DRAIN_BUFFER_SIZE; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration; -import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; /** * Drains/aborts s3 or other AWS SDK streams. * It is callable so can be passed directly to a submitter * for async invocation. - * A request object may be passed in; it will be implicitly - * cached until this object is GCd. - * This is because in some versions of the AWS SDK, the S3Object - * has a finalize() method which releases the http connection, - * even when the stream is still open. - * See HADOOP-17338 for details. */ -public class SDKStreamDrainer implements CallableRaisingIOE { +public class SDKStreamDrainer + implements CallableRaisingIOE { private static final Logger LOG = LoggerFactory.getLogger( SDKStreamDrainer.class); @@ -58,17 +51,9 @@ public class SDKStreamDrainer implements CallableRaisingIOE { private final String uri; /** - * Request object; usually S3Object - * Never used, but needed to keep the http connection - * open long enough for draining to take place. + * Stream from the getObject response for draining and closing. */ - @Nullable - private final Closeable requestObject; - - /** - * Stream from the {@link #requestObject} for draining and closing. - */ - private final SdkFilterInputStream sdkStream; + private final TStream sdkStream; /** * Should the request be aborted? @@ -118,7 +103,6 @@ public class SDKStreamDrainer implements CallableRaisingIOE { /** * Prepare to drain the stream. * @param uri URI for messages - * @param requestObject http request object; needed to avoid GC issues. * @param sdkStream stream to close. * @param shouldAbort force an abort; used if explicitly requested. * @param streamStatistics stats to update @@ -126,14 +110,12 @@ public class SDKStreamDrainer implements CallableRaisingIOE { * @param remaining remaining bytes */ public SDKStreamDrainer(final String uri, - @Nullable final Closeable requestObject, - final SdkFilterInputStream sdkStream, + final TStream sdkStream, final boolean shouldAbort, final int remaining, final S3AInputStreamStatistics streamStatistics, final String reason) { this.uri = uri; - this.requestObject = requestObject; this.sdkStream = requireNonNull(sdkStream); this.shouldAbort = shouldAbort; this.remaining = remaining; @@ -233,7 +215,6 @@ private boolean drainOrAbortHttpStream() { LOG.debug("Closing stream"); sdkStream.close(); - cleanupWithLogger(LOG, requestObject); // this MUST come after the close, so that if the IO operations fail // and an abort is triggered, the initial attempt's statistics // aren't collected. @@ -255,8 +236,6 @@ private boolean drainOrAbortHttpStream() { LOG.warn("When aborting {} stream after failing to close it for {}", uri, reason, e); thrown = e; - } finally { - cleanupWithLogger(LOG, requestObject); } streamStatistics.streamClose(true, remaining); @@ -269,11 +248,7 @@ public String getUri() { return uri; } - public Object getRequestObject() { - return requestObject; - } - - public SdkFilterInputStream getSdkStream() { + public TStream getSdkStream() { return sdkStream; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java index 3ab0022bb082e..65b5a4235133b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java @@ -19,15 +19,8 @@ package org.apache.hadoop.fs.s3a.prefetch; - import java.io.IOException; -import java.io.InputStream; -import java.util.IdentityHashMap; -import java.util.Map; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.S3ObjectInputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,12 +28,17 @@ import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.S3AReadOpContext; +import org.apache.hadoop.fs.s3a.S3AUtils; import org.apache.hadoop.fs.s3a.S3ObjectAttributes; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; import org.apache.hadoop.fs.s3a.impl.SDKStreamDrainer; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; import org.apache.hadoop.fs.statistics.DurationTracker; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; + /** * Encapsulates low level interactions with S3 object on AWS. */ @@ -74,12 +72,6 @@ public class S3ARemoteObject { */ private final ChangeTracker changeTracker; - /** - * Maps a stream returned by openForRead() to the associated S3 object. - * That allows us to close the object when closing the stream. - */ - private final Map s3Objects; - /** * uri of the object being read. */ @@ -123,7 +115,6 @@ public S3ARemoteObject( this.client = client; this.streamStatistics = streamStatistics; this.changeTracker = changeTracker; - this.s3Objects = new IdentityHashMap<>(); this.uri = this.getPath(); } @@ -187,21 +178,23 @@ public long size() { * @throws IllegalArgumentException if offset is greater than or equal to file size. * @throws IllegalArgumentException if size is greater than the remaining bytes. */ - public InputStream openForRead(long offset, int size) throws IOException { + public ResponseInputStream openForRead(long offset, int size) + throws IOException { Validate.checkNotNegative(offset, "offset"); Validate.checkLessOrEqual(offset, "offset", size(), "size()"); Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset"); streamStatistics.streamOpened(); - final GetObjectRequest request = - client.newGetRequest(s3Attributes.getKey()) - .withRange(offset, offset + size - 1); - changeTracker.maybeApplyConstraint(request); + final GetObjectRequest request = client + .newGetRequestBuilder(s3Attributes.getKey()) + .range(S3AUtils.formatRange(offset, offset + size - 1)) + .applyMutation(changeTracker::maybeApplyConstraint) + .build(); String operation = String.format( "%s %s at %d", S3AInputStream.OPERATION_OPEN, uri, offset); DurationTracker tracker = streamStatistics.initiateGetRequest(); - S3Object object = null; + ResponseInputStream object = null; try { object = Invoker.once(operation, uri, () -> client.getObject(request)); @@ -212,27 +205,14 @@ public InputStream openForRead(long offset, int size) throws IOException { tracker.close(); } - changeTracker.processResponse(object, operation, offset); - InputStream stream = object.getObjectContent(); - synchronized (s3Objects) { - s3Objects.put(stream, object); - } - - return stream; + changeTracker.processResponse(object.response(), operation, offset); + return object; } - void close(InputStream inputStream, int numRemainingBytes) { - S3Object obj; - synchronized (s3Objects) { - obj = s3Objects.remove(inputStream); - if (obj == null) { - throw new IllegalArgumentException("inputStream not found"); - } - } + void close(ResponseInputStream inputStream, int numRemainingBytes) { SDKStreamDrainer drainer = new SDKStreamDrainer( uri, - obj, - (S3ObjectInputStream)inputStream, + inputStream, false, numRemainingBytes, streamStatistics, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java index 89ea77d6d0ebb..b49b2699f916b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java @@ -22,7 +22,6 @@ import java.io.Closeable; import java.io.EOFException; import java.io.IOException; -import java.io.InputStream; import java.net.SocketTimeoutException; import java.nio.ByteBuffer; @@ -33,6 +32,9 @@ import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; + import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_REMOTE_BLOCK_READ; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation; @@ -144,7 +146,8 @@ private void readOneBlock(ByteBuffer buffer, long offset, int size) return; } - InputStream inputStream = remoteObject.openForRead(offset, readSize); + ResponseInputStream inputStream = + remoteObject.openForRead(offset, readSize); int numRemainingBytes = readSize; byte[] bytes = new byte[READ_BUFFER_SIZE]; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java index c62bf5daca3a4..58f045828c77c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java @@ -19,16 +19,12 @@ package org.apache.hadoop.fs.s3a; import javax.net.ssl.SSLException; +import java.io.FilterInputStream; import java.io.IOException; import java.net.SocketException; import java.nio.charset.StandardCharsets; import java.util.concurrent.CompletableFuture; -import com.amazonaws.SdkClientException; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.S3ObjectInputStream; import org.junit.Test; import org.apache.commons.io.IOUtils; @@ -37,6 +33,13 @@ import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.util.functional.CallableRaisingIOE; +import software.amazon.awssdk.awscore.exception.AwsErrorDetails; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.http.AbortableInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; + import static java.lang.Math.min; import static org.apache.hadoop.util.functional.FutureIO.eval; import static org.junit.Assert.assertArrayEquals; @@ -121,13 +124,22 @@ private S3AInputStream getMockedS3AInputStream() { * @return mocked object. */ private S3AInputStream.InputStreamCallbacks getMockedInputStreamCallback() { - return new S3AInputStream.InputStreamCallbacks() { + GetObjectResponse objectResponse = GetObjectResponse.builder() + .eTag("test-etag") + .build(); + + ResponseInputStream[] responseInputStreams = + new ResponseInputStream[] { + getMockedInputStream(objectResponse, true), + getMockedInputStream(objectResponse, true), + getMockedInputStream(objectResponse, false) + }; - private final S3Object mockedS3Object = getMockedS3Object(); + return new S3AInputStream.InputStreamCallbacks() { private Integer mockedS3ObjectIndex = 0; @Override - public S3Object getObject(GetObjectRequest request) { + public ResponseInputStream getObject(GetObjectRequest request) { // Set s3 client to return mocked s3object with defined read behavior. mockedS3ObjectIndex++; // open() -> lazySeek() -> reopen() @@ -144,14 +156,17 @@ public S3Object getObject(GetObjectRequest request) { // -> getObjectContent(objectInputStreamGood)-> objectInputStreamGood // -> wrappedStream.read if (mockedS3ObjectIndex == 3) { - throw new SdkClientException("Failed to get S3Object"); + throw AwsServiceException.builder() + .message("Failed to get S3Object") + .awsErrorDetails(AwsErrorDetails.builder().errorCode("test-code").build()) + .build(); } - return mockedS3Object; + return responseInputStreams[min(mockedS3ObjectIndex, responseInputStreams.length) - 1]; } @Override - public GetObjectRequest newGetRequest(String key) { - return new GetObjectRequest(fs.getBucket(), key); + public GetObjectRequest.Builder newGetRequestBuilder(String key) { + return GetObjectRequest.builder().bucket(fs.getBucket()).key(key); } @Override @@ -166,70 +181,41 @@ public void close() { } /** - * Get mocked S3Object that returns bad input stream on the initial of - * getObjectContent calls. - * - * @return mocked object. - */ - private S3Object getMockedS3Object() { - S3ObjectInputStream objectInputStreamBad1 = getMockedInputStream(true); - S3ObjectInputStream objectInputStreamBad2 = getMockedInputStream(true); - S3ObjectInputStream objectInputStreamGood = getMockedInputStream(false); - - return new S3Object() { - private final S3ObjectInputStream[] inputStreams = - {objectInputStreamBad1, objectInputStreamBad2, objectInputStreamGood}; - - private Integer inputStreamIndex = 0; - - @Override - public S3ObjectInputStream getObjectContent() { - // Set getObjectContent behavior: - // Returns bad stream twice, and good stream afterwards. - inputStreamIndex++; - return inputStreams[min(inputStreamIndex, inputStreams.length) - 1]; - } - - @Override - public ObjectMetadata getObjectMetadata() { - // Set getObjectMetadata behavior: returns dummy metadata - ObjectMetadata metadata = new ObjectMetadata(); - metadata.setHeader("ETag", "test-etag"); - return metadata; - } - }; - } - - /** - * Get mocked S3ObjectInputStream where we can trigger IOException to + * Get mocked ResponseInputStream where we can trigger IOException to * simulate the read failure. * * @param triggerFailure true when a failure injection is enabled. * @return mocked object. */ - private S3ObjectInputStream getMockedInputStream(boolean triggerFailure) { - return new S3ObjectInputStream(IOUtils.toInputStream(INPUT, StandardCharsets.UTF_8), null) { - - private final IOException exception = - new SSLException(new SocketException("Connection reset")); - - @Override - public int read() throws IOException { - int result = super.read(); - if (triggerFailure) { - throw exception; - } - return result; - } - - @Override - public int read(byte[] b, int off, int len) throws IOException { - int result = super.read(b, off, len); - if (triggerFailure) { - throw exception; - } - return result; - } - }; + private ResponseInputStream getMockedInputStream( + GetObjectResponse objectResponse, boolean triggerFailure) { + + FilterInputStream inputStream = + new FilterInputStream(IOUtils.toInputStream(INPUT, StandardCharsets.UTF_8)) { + + private final IOException exception = + new SSLException(new SocketException("Connection reset")); + + @Override + public int read() throws IOException { + int result = super.read(); + if (triggerFailure) { + throw exception; + } + return result; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + int result = super.read(b, off, len); + if (triggerFailure) { + throw exception; + } + return result; + } + }; + + return new ResponseInputStream(objectResponse, + AbortableInputStream.create(inputStream, () -> {})); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java index 83c89b99260c1..0f36ad55fa903 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AUnbuffer.java @@ -18,20 +18,19 @@ package org.apache.hadoop.fs.s3a; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.S3ObjectInputStream; - import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; import org.junit.Test; + +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; import software.amazon.awssdk.services.s3.model.HeadObjectRequest; import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import java.io.IOException; -import java.util.Date; -import java.util.function.Consumer; +import java.time.Instant; import static org.junit.Assert.assertEquals; @@ -43,10 +42,10 @@ import static org.mockito.Mockito.when; /** - * Uses mocks to check that the {@link S3ObjectInputStream} is closed when - * {@link org.apache.hadoop.fs.CanUnbuffer#unbuffer} is called. Unlike the - * other unbuffer tests, this specifically tests that the underlying S3 object - * stream is closed. + * Uses mocks to check that the {@link ResponseInputStream} is + * closed when {@link org.apache.hadoop.fs.CanUnbuffer#unbuffer} is called. + * Unlike the other unbuffer tests, this specifically tests that the underlying + * S3 object stream is closed. */ public class TestS3AUnbuffer extends AbstractS3AMockTest { @@ -56,28 +55,21 @@ public void testUnbuffer() throws IOException { Path path = new Path("/file"); HeadObjectResponse objectMetadata = mock(HeadObjectResponse.class); when(objectMetadata.contentLength()).thenReturn(1L); - when(objectMetadata.lastModified()).thenReturn(new Date(2L).toInstant()); + when(objectMetadata.lastModified()).thenReturn(Instant.ofEpochMilli(2L)); when(objectMetadata.eTag()).thenReturn("mock-etag"); when(s3V2.headObject((HeadObjectRequest) any())).thenReturn(objectMetadata); - // Create mock S3ObjectInputStream and S3Object for open() - S3ObjectInputStream objectStream = mock(S3ObjectInputStream.class); + // Create mock ResponseInputStream and GetObjectResponse for open() + GetObjectResponse objectResponse = mock(GetObjectResponse.class); + when(objectResponse.contentLength()).thenReturn(1L); + when(objectResponse.lastModified()).thenReturn(Instant.ofEpochMilli(2L)); + when(objectResponse.eTag()).thenReturn("mock-etag"); + ResponseInputStream objectStream = mock(ResponseInputStream.class); + when(objectStream.response()).thenReturn(objectResponse); when(objectStream.read()).thenReturn(-1); when(objectStream.read(any(byte[].class))).thenReturn(-1); when(objectStream.read(any(byte[].class), anyInt(), anyInt())).thenReturn(-1); - - // TODO: Remove during getObject work, required currently for the - // s3Object.getObjectMetadata() call. - ObjectMetadata meta = mock(ObjectMetadata.class); - when(meta.getContentLength()).thenReturn(1L); - when(meta.getLastModified()).thenReturn(new Date(2L)); - when(meta.getETag()).thenReturn("mock-etag"); - when(s3.getObjectMetadata(any())).thenReturn(meta); - - S3Object s3Object = mock(S3Object.class); - when(s3Object.getObjectContent()).thenReturn(objectStream); - when(s3Object.getObjectMetadata()).thenReturn(meta); - when(s3.getObject(any())).thenReturn(s3Object); + when(s3V2.getObject((GetObjectRequest) any())).thenReturn(objectStream); // Call read and then unbuffer FSDataInputStream stream = fs.open(path); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java index 7512b0495189b..fa7f55b5e521a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestStreamChangeTracker.java @@ -20,10 +20,6 @@ import com.amazonaws.AmazonServiceException; import com.amazonaws.SdkBaseException; -import com.amazonaws.services.s3.Headers; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.S3Object; import org.junit.Test; import org.slf4j.Logger; @@ -32,6 +28,8 @@ import software.amazon.awssdk.services.s3.model.CopyObjectRequest; import software.amazon.awssdk.services.s3.model.CopyObjectResponse; import software.amazon.awssdk.services.s3.model.CopyObjectResult; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; @@ -71,7 +69,7 @@ public void testVersionCheckingHandlingNoVersions() throws Throwable { ChangeDetectionPolicy.Source.VersionId, false); assertFalse("Tracker should not have applied contraints " + tracker, - tracker.maybeApplyConstraint(newGetObjectRequest())); + tracker.maybeApplyConstraint(newGetObjectRequestBuilder())); tracker.processResponse( newResponse(null, null), "", 0); @@ -99,7 +97,7 @@ public void testEtagCheckingWarn() throws Throwable { ChangeDetectionPolicy.Source.ETag, false); assertFalse("Tracker should not have applied constraints " + tracker, - tracker.maybeApplyConstraint(newGetObjectRequest())); + tracker.maybeApplyConstraint(newGetObjectRequestBuilder())); tracker.processResponse( newResponse("e1", null), "", 0); @@ -125,13 +123,13 @@ public void testVersionCheckingOnClient() throws Throwable { ChangeDetectionPolicy.Source.VersionId, false); assertFalse("Tracker should not have applied constraints " + tracker, - tracker.maybeApplyConstraint(newGetObjectRequest())); + tracker.maybeApplyConstraint(newGetObjectRequestBuilder())); tracker.processResponse( newResponse(null, "rev1"), "", 0); assertTrackerMismatchCount(tracker, 0); assertRevisionId(tracker, "rev1"); - GetObjectRequest request = newGetObjectRequest(); + GetObjectRequest request = newGetObjectRequestBuilder().build(); expectChangeException(tracker, newResponse(null, "rev2"), "change detected"); // mismatch was noted (so gets to FS stats) @@ -152,14 +150,14 @@ public void testVersionCheckingOnServer() throws Throwable { ChangeDetectionPolicy.Source.VersionId, false); assertFalse("Tracker should not have applied contraints " + tracker, - tracker.maybeApplyConstraint(newGetObjectRequest())); + tracker.maybeApplyConstraint(newGetObjectRequestBuilder())); tracker.processResponse( newResponse(null, "rev1"), "", 0); assertTrackerMismatchCount(tracker, 0); assertRevisionId(tracker, "rev1"); - GetObjectRequest request = newGetObjectRequest(); - assertConstraintApplied(tracker, request); + GetObjectRequest.Builder builder = newGetObjectRequestBuilder(); + assertConstraintApplied(tracker, builder); // now, the tracker expects a null response expectChangeException(tracker, null, CHANGE_REPORTED_BY_S3); assertTrackerMismatchCount(tracker, 1); @@ -263,9 +261,9 @@ public void testCopyVersionMismatch() throws Throwable { } protected void assertConstraintApplied(final ChangeTracker tracker, - final GetObjectRequest request) { + final GetObjectRequest.Builder builder) { assertTrue("Tracker should have applied contraints " + tracker, - tracker.maybeApplyConstraint(request)); + tracker.maybeApplyConstraint(builder)); } protected void assertConstraintApplied(final ChangeTracker tracker, @@ -276,7 +274,7 @@ protected void assertConstraintApplied(final ChangeTracker tracker, protected RemoteFileChangedException expectChangeException( final ChangeTracker tracker, - final S3Object response, + final GetObjectResponse response, final String message) throws Exception { return expectException(tracker, response, message, RemoteFileChangedException.class); @@ -293,7 +291,7 @@ protected RemoteFileChangedException expectChangeException( protected PathIOException expectNoVersionAttributeException( final ChangeTracker tracker, - final S3Object response, + final GetObjectResponse response, final String message) throws Exception { return expectException(tracker, response, message, NoVersionAttributeException.class); @@ -309,7 +307,7 @@ protected PathIOException expectNoVersionAttributeException( protected T expectException( final ChangeTracker tracker, - final S3Object response, + final GetObjectResponse response, final String message, final Class clazz) throws Exception { return intercept( @@ -392,13 +390,13 @@ protected ChangeTracker newTracker(final ChangeDetectionPolicy.Mode mode, if (objectAttributes.getVersionId() == null && objectAttributes.getETag() == null) { assertFalse("Tracker should not have applied constraints " + tracker, - tracker.maybeApplyConstraint(newGetObjectRequest())); + tracker.maybeApplyConstraint(newGetObjectRequestBuilder())); } return tracker; } - private GetObjectRequest newGetObjectRequest() { - return new GetObjectRequest(BUCKET, OBJECT); + private GetObjectRequest.Builder newGetObjectRequestBuilder() { + return GetObjectRequest.builder().bucket(BUCKET).key(OBJECT); } private CopyObjectRequest.Builder newCopyObjectRequest() { @@ -413,24 +411,15 @@ private CopyObjectResponse newCopyResult(String eTag, String versionId) { .copyObjectResult(CopyObjectResult.builder().eTag(eTag).build()).build(); } - private S3Object newResponse(String etag, String versionId) { - ObjectMetadata md = new ObjectMetadata(); + private GetObjectResponse newResponse(String etag, String versionId) { + GetObjectResponse.Builder builder = GetObjectResponse.builder(); if (etag != null) { - md.setHeader(Headers.ETAG, etag); + builder.eTag(etag); } if (versionId != null) { - md.setHeader(Headers.S3_VERSION_ID, versionId); + builder.versionId(versionId); } - S3Object response = emptyResponse(); - response.setObjectMetadata(md); - return response; - } - - private S3Object emptyResponse() { - S3Object response = new S3Object(); - response.setBucketName(BUCKET); - response.setKey(OBJECT); - return response; + return builder.build(); } private S3ObjectAttributes objectAttributes( diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java index 49a5f86939926..638ccc3724eda 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 @@ -171,7 +171,7 @@ private void createFactoryObjects(RequestFactory factory) { a(factory.newDeleteObjectRequest(path)); a(factory.newBulkDeleteRequest(new ArrayList<>())); // a(factory.newDirectoryMarkerRequest(path)); - a(factory.newGetObjectRequest(path)); + // a(factory.newGetObjectRequest(path)); // a(factory.newGetObjectMetadataRequest(path)); a(factory.newListMultipartUploadsRequest(path)); //TODO: Commenting out for now, new request extends AwsRequest, this can be updated once all diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestSDKStreamDrainer.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestSDKStreamDrainer.java index 33a44a9ad78f7..85970a65887f4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestSDKStreamDrainer.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestSDKStreamDrainer.java @@ -19,13 +19,15 @@ package org.apache.hadoop.fs.s3a.impl; import java.io.IOException; +import java.io.InputStream; -import com.amazonaws.internal.SdkFilterInputStream; import org.assertj.core.api.Assertions; import org.junit.Test; import org.apache.hadoop.test.HadoopTestBase; +import software.amazon.awssdk.http.Abortable; + import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DRAIN_BUFFER_SIZE; import static org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext.EMPTY_INPUT_STREAM_STATISTICS; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -127,7 +129,6 @@ public void testStreamUnderflow() throws Throwable { public void testReadFailure() throws Throwable { int threshold = 50; SDKStreamDrainer drainer = new SDKStreamDrainer("s3://example/", - null, new FakeSDKInputStream(BYTES, threshold), false, BYTES, @@ -145,7 +146,6 @@ public void testReadFailure() throws Throwable { public void testReadFailureDoesNotSurfaceInAbort() throws Throwable { int threshold = 50; SDKStreamDrainer drainer = new SDKStreamDrainer("s3://example/", - null, new FakeSDKInputStream(BYTES, threshold), true, BYTES, @@ -183,7 +183,6 @@ private SDKStreamDrainer drainer(int remaining, boolean shouldAbort, FakeSDKInputStream in) throws Throwable { SDKStreamDrainer drainer = new SDKStreamDrainer("s3://example/", - null, in, shouldAbort, remaining, @@ -246,7 +245,8 @@ private static SDKStreamDrainer assertBytesRead(final SDKStreamDrainer drainer, * Fake stream; generates data dynamically. * Only overrides the methods used in stream draining. */ - private static final class FakeSDKInputStream extends SdkFilterInputStream { + private static final class FakeSDKInputStream extends InputStream + implements Abortable { private final int capacity; @@ -264,7 +264,6 @@ private static final class FakeSDKInputStream extends SdkFilterInputStream { * @param readToRaiseIOE position to raise an IOE, or -1 */ private FakeSDKInputStream(final int capacity, final int readToRaiseIOE) { - super(null); this.capacity = capacity; this.readToRaiseIOE = readToRaiseIOE; } @@ -282,11 +281,6 @@ public void abort() { aborted = true; } - @Override - protected boolean isAborted() { - return aborted; - } - @Override public int read() throws IOException { if (bytesRead >= capacity) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java index 6e2f547a22ec1..0c1d402305c0e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java @@ -21,17 +21,18 @@ import java.io.ByteArrayInputStream; import java.io.IOException; -import java.io.InputStream; import java.util.concurrent.CompletableFuture; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.S3Object; - import org.apache.hadoop.fs.impl.prefetch.Validate; import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; import org.apache.hadoop.util.functional.CallableRaisingIOE; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.http.AbortableInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; + /** * A mock s3 file with some fault injection. */ @@ -55,7 +56,7 @@ class MockS3ARemoteObject extends S3ARemoteObject { super( S3APrefetchFakes.createReadContext(null, KEY, size, 1, 1), S3APrefetchFakes.createObjectAttributes(BUCKET, KEY, size), - S3APrefetchFakes.createInputStreamCallbacks(BUCKET, KEY), + S3APrefetchFakes.createInputStreamCallbacks(BUCKET), EmptyS3AStatisticsContext.EMPTY_INPUT_STREAM_STATISTICS, S3APrefetchFakes.createChangeTracker(BUCKET, KEY, size) ); @@ -68,7 +69,8 @@ class MockS3ARemoteObject extends S3ARemoteObject { } @Override - public InputStream openForRead(long offset, int size) throws IOException { + public ResponseInputStream openForRead(long offset, int size) + throws IOException { Validate.checkLessOrEqual(offset, "offset", size(), "size()"); Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset"); @@ -77,11 +79,15 @@ public InputStream openForRead(long offset, int size) throws IOException { throw new IOException("Throwing because throwExceptionOnOpen is true "); } int bufSize = (int) Math.min(size, size() - offset); - return new ByteArrayInputStream(contents, (int) offset, bufSize); + GetObjectResponse objectResponse = GetObjectResponse.builder().build(); + return new ResponseInputStream(objectResponse, + AbortableInputStream.create(new ByteArrayInputStream(contents, + (int) offset, bufSize), () -> {})); } @Override - public void close(InputStream inputStream, int numRemainingBytes) { + public void close(ResponseInputStream inputStream, + int numRemainingBytes) { // do nothing since we do not use a real S3 stream. } @@ -92,7 +98,8 @@ public static byte byteAtOffset(int offset) { public static S3AInputStream.InputStreamCallbacks createClient(String bucketName) { return new S3AInputStream.InputStreamCallbacks() { @Override - public S3Object getObject(GetObjectRequest request) { + public ResponseInputStream getObject( + GetObjectRequest request) { return null; } @@ -102,8 +109,8 @@ public CompletableFuture submit(CallableRaisingIOE operation) { } @Override - public GetObjectRequest newGetRequest(String key) { - return new GetObjectRequest(bucketName, key); + public GetObjectRequest.Builder newGetRequestBuilder(String key) { + return GetObjectRequest.builder().bucket(bucketName).key(key); } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java index bab07f4f9ec83..c6793e09ad880 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java @@ -31,11 +31,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.S3ObjectInputStream; - import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.impl.prefetch.BlockCache; import org.apache.hadoop.fs.impl.prefetch.BlockData; @@ -60,6 +55,11 @@ import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.util.functional.CallableRaisingIOE; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.http.AbortableInputStream; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; + import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatisticsStore; /** @@ -169,32 +169,26 @@ public static ChangeTracker createChangeTracker( createObjectAttributes(bucket, key, fileSize)); } - public static S3ObjectInputStream createS3ObjectInputStream(byte[] buffer) { - return new S3ObjectInputStream(new ByteArrayInputStream(buffer), null); + public static ResponseInputStream createS3ObjectInputStream( + GetObjectResponse objectResponse, byte[] buffer) { + return new ResponseInputStream(objectResponse, + AbortableInputStream.create(new ByteArrayInputStream(buffer), () -> {})); } public static S3AInputStream.InputStreamCallbacks createInputStreamCallbacks( - String bucket, - String key) { + String bucket) { - S3Object object = new S3Object() { - @Override - public S3ObjectInputStream getObjectContent() { - return createS3ObjectInputStream(new byte[8]); - } + GetObjectResponse objectResponse = GetObjectResponse.builder() + .eTag(E_TAG) + .build(); - @Override - public ObjectMetadata getObjectMetadata() { - ObjectMetadata metadata = new ObjectMetadata(); - metadata.setHeader("ETag", E_TAG); - return metadata; - } - }; + ResponseInputStream responseInputStream = + createS3ObjectInputStream(objectResponse, new byte[8]); return new S3AInputStream.InputStreamCallbacks() { @Override - public S3Object getObject(GetObjectRequest request) { - return object; + public ResponseInputStream getObject(GetObjectRequest request) { + return responseInputStream; } @Override @@ -203,8 +197,8 @@ public CompletableFuture submit(CallableRaisingIOE operation) { } @Override - public GetObjectRequest newGetRequest(String key) { - return new GetObjectRequest(bucket, key); + public GetObjectRequest.Builder newGetRequestBuilder(String key) { + return GetObjectRequest.builder().bucket(bucket).key(key); } @Override @@ -223,9 +217,6 @@ public static S3ARemoteInputStream createInputStream( int prefetchBlockSize, int prefetchBlockCount) { - org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key); - - S3AFileStatus fileStatus = createFileStatus(key, fileSize); S3ObjectAttributes s3ObjectAttributes = createObjectAttributes(bucket, key, fileSize); S3AReadOpContext s3AReadOpContext = createReadContext( @@ -236,7 +227,7 @@ public static S3ARemoteInputStream createInputStream( prefetchBlockCount); S3AInputStream.InputStreamCallbacks callbacks = - createInputStreamCallbacks(bucket, key); + createInputStreamCallbacks(bucket); S3AInputStreamStatistics stats = s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics();