HADOOP-18073. Upgrade AWS SDK to v2 in S3A [work in progress]
See aws_sdk_v2_changelog.md for details. Co-authored-by: Ahmar Suhail <ahmarsu@amazon.co.uk> Co-authored-by: Alessandro Passaro <alexpax@amazon.co.uk>
This commit is contained in:
parent
a90c722143
commit
f2bf08cae1
|
@ -185,6 +185,7 @@
|
|||
<surefire.fork.timeout>900</surefire.fork.timeout>
|
||||
<aws-java-sdk.version>1.12.316</aws-java-sdk.version>
|
||||
<hsqldb.version>2.7.1</hsqldb.version>
|
||||
<aws-java-sdk-v2.version>2.18.19</aws-java-sdk-v2.version>
|
||||
<frontend-maven-plugin.version>1.11.2</frontend-maven-plugin.version>
|
||||
<jasmine-maven-plugin.version>2.1</jasmine-maven-plugin.version>
|
||||
<phantomjs-maven-plugin.version>0.7</phantomjs-maven-plugin.version>
|
||||
|
@ -1138,6 +1139,28 @@
|
|||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>software.amazon.awssdk</groupId>
|
||||
<artifactId>bundle</artifactId>
|
||||
<version>${aws-java-sdk-v2.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>software.amazon.awssdk</groupId>
|
||||
<artifactId>s3-transfer-manager</artifactId>
|
||||
<version>${aws-java-sdk-v2.version}-PREVIEW</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>*</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.mina</groupId>
|
||||
<artifactId>mina-core</artifactId>
|
||||
|
|
|
@ -499,6 +499,16 @@
|
|||
<artifactId>aws-java-sdk-bundle</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>software.amazon.awssdk</groupId>
|
||||
<artifactId>bundle</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>software.amazon.awssdk</groupId>
|
||||
<artifactId>s3-transfer-manager</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.assertj</groupId>
|
||||
<artifactId>assertj-core</artifactId>
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.AmazonServiceException;
|
||||
import software.amazon.awssdk.awscore.exception.AwsServiceException;
|
||||
|
||||
/**
|
||||
* A 400 "Bad Request" exception was received.
|
||||
|
@ -36,7 +36,7 @@ public class AWSBadRequestException extends AWSServiceIOException {
|
|||
* @param cause the underlying cause
|
||||
*/
|
||||
public AWSBadRequestException(String operation,
|
||||
AmazonServiceException cause) {
|
||||
AwsServiceException cause) {
|
||||
super(operation, cause);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* Enum to map AWS SDK V1 Acl values to SDK V2.
|
||||
*/
|
||||
public enum AWSCannedACL {
|
||||
Private("PRIVATE"),
|
||||
PublicRead("PUBLIC_READ"),
|
||||
PublicReadWrite("PUBLIC_READ_WRITE"),
|
||||
AuthenticatedRead("AUTHENTICATED_READ"),
|
||||
AwsExecRead("AWS_EXEC_READ"),
|
||||
BucketOwnerRead("BUCKET_OWNER_READ"),
|
||||
BucketOwnerFullControl("BUCKET_OWNER_FULL_CONTROL");
|
||||
|
||||
private final String value;
|
||||
|
||||
AWSCannedACL(String value){
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
public String toString() { return this.value; }
|
||||
}
|
|
@ -0,0 +1,340 @@
|
|||
/*
|
||||
* 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 software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient;
|
||||
import software.amazon.awssdk.thirdparty.org.apache.http.client.utils.URIBuilder;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.util.VersionInfo;
|
||||
|
||||
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 async http client.
|
||||
*
|
||||
* @param conf The Hadoop configuration
|
||||
* @return Http client builder
|
||||
*/
|
||||
public static NettyNioAsyncHttpClient.Builder createAsyncHttpClientBuilder(Configuration conf) {
|
||||
NettyNioAsyncHttpClient.Builder httpClientBuilder =
|
||||
NettyNioAsyncHttpClient.builder();
|
||||
|
||||
httpClientBuilder.maxConcurrency(S3AUtils.intOption(conf, MAXIMUM_CONNECTIONS,
|
||||
DEFAULT_MAXIMUM_CONNECTIONS, 1));
|
||||
|
||||
int connectionEstablishTimeout =
|
||||
S3AUtils.intOption(conf, ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT, 0);
|
||||
int socketTimeout = S3AUtils.intOption(conf, SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT, 0);
|
||||
|
||||
httpClientBuilder.connectionTimeout(Duration.ofSeconds(connectionEstablishTimeout));
|
||||
httpClientBuilder.readTimeout(Duration.ofSeconds(socketTimeout));
|
||||
httpClientBuilder.writeTimeout(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
|
||||
* @throws IOException on any IO problem
|
||||
*/
|
||||
public static ProxyConfiguration createProxyConfiguration(Configuration conf,
|
||||
String bucket) throws IOException {
|
||||
|
||||
ProxyConfiguration.Builder proxyConfigBuilder = ProxyConfiguration.builder();
|
||||
|
||||
String proxyHost = conf.getTrimmed(PROXY_HOST, "");
|
||||
int proxyPort = conf.getInt(PROXY_PORT, -1);
|
||||
|
||||
if (!proxyHost.isEmpty()) {
|
||||
if (proxyPort >= 0) {
|
||||
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.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Configures the proxy for the async http client.
|
||||
*
|
||||
* @param conf The Hadoop configuration
|
||||
* @param bucket Optional bucket to use to look up per-bucket proxy secrets
|
||||
* @return Proxy configuration
|
||||
* @throws IOException on any IO problem
|
||||
*/
|
||||
public static software.amazon.awssdk.http.nio.netty.ProxyConfiguration
|
||||
createAsyncProxyConfiguration(Configuration conf,
|
||||
String bucket) throws IOException {
|
||||
|
||||
software.amazon.awssdk.http.nio.netty.ProxyConfiguration.Builder proxyConfigBuilder =
|
||||
software.amazon.awssdk.http.nio.netty.ProxyConfiguration.builder();
|
||||
|
||||
String proxyHost = conf.getTrimmed(PROXY_HOST, "");
|
||||
int proxyPort = conf.getInt(PROXY_PORT, -1);
|
||||
|
||||
if (!proxyHost.isEmpty()) {
|
||||
if (proxyPort >= 0) {
|
||||
proxyConfigBuilder.host(proxyHost);
|
||||
proxyConfigBuilder.port(proxyPort);
|
||||
} else {
|
||||
if (conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS)) {
|
||||
LOG.warn("Proxy host set without port. Using HTTPS default 443");
|
||||
proxyConfigBuilder.host(proxyHost);
|
||||
proxyConfigBuilder.port(443);
|
||||
} else {
|
||||
LOG.warn("Proxy host set without port. Using HTTP default 80");
|
||||
proxyConfigBuilder.host(proxyHost);
|
||||
proxyConfigBuilder.port(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);
|
||||
// TODO: check NTLM support
|
||||
// proxyConfigBuilder.ntlmDomain(conf.getTrimmed(PROXY_DOMAIN));
|
||||
// proxyConfigBuilder.ntlmWorkstation(conf.getTrimmed(PROXY_WORKSTATION));
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Using proxy server {}:{} as user {} with password {} on "
|
||||
+ "domain {} as workstation {}", proxyHost, proxyPort, proxyUsername, proxyPassword,
|
||||
PROXY_DOMAIN, PROXY_WORKSTATION);
|
||||
}
|
||||
} else if (proxyPort >= 0) {
|
||||
String msg =
|
||||
"Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST;
|
||||
LOG.error(msg);
|
||||
throw new IllegalArgumentException(msg);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
||||
return proxyConfigBuilder.build();
|
||||
}
|
||||
|
||||
/***
|
||||
* Builds a URI, throws an IllegalArgumentException in case of errors.
|
||||
*
|
||||
* @param host proxy host
|
||||
* @param port proxy port
|
||||
* @return uri with host and port
|
||||
*/
|
||||
private static URI buildURI(String 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));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -18,34 +18,33 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.AmazonClientException;
|
||||
import com.amazonaws.SdkBaseException;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import software.amazon.awssdk.core.exception.SdkException;
|
||||
|
||||
/**
|
||||
* IOException equivalent of an {@link AmazonClientException}.
|
||||
* IOException equivalent of an {@link SdkException}.
|
||||
*/
|
||||
public class AWSClientIOException extends IOException {
|
||||
|
||||
private final String operation;
|
||||
|
||||
public AWSClientIOException(String operation,
|
||||
SdkBaseException cause) {
|
||||
SdkException cause) {
|
||||
super(cause);
|
||||
Preconditions.checkArgument(operation != null, "Null 'operation' argument");
|
||||
Preconditions.checkArgument(cause != null, "Null 'cause' argument");
|
||||
this.operation = operation;
|
||||
}
|
||||
|
||||
public AmazonClientException getCause() {
|
||||
return (AmazonClientException) super.getCause();
|
||||
public SdkException getCause() {
|
||||
return (SdkException) super.getCause();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getMessage() {
|
||||
return operation + ": " + getCause().getMessage();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -27,12 +27,14 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import com.amazonaws.AmazonClientException;
|
||||
import com.amazonaws.auth.AWSCredentials;
|
||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||
import com.amazonaws.auth.AnonymousAWSCredentials;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.fs.s3a.adapter.V1V2AwsCredentialProviderAdapter;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
|
||||
import com.amazonaws.auth.BasicAWSCredentials;
|
||||
import com.amazonaws.auth.BasicSessionCredentials;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -43,6 +45,12 @@ import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
|
|||
import org.apache.hadoop.fs.s3a.auth.NoAwsCredentialsException;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
|
||||
import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
|
||||
import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
|
||||
import software.amazon.awssdk.core.exception.SdkException;
|
||||
|
||||
/**
|
||||
* A list of providers.
|
||||
*
|
||||
|
@ -51,17 +59,17 @@ import org.apache.hadoop.io.IOUtils;
|
|||
* <ol>
|
||||
* <li>Allows extra providers to be added dynamically.</li>
|
||||
* <li>If any provider in the chain throws an exception other than
|
||||
* an {@link AmazonClientException}, that is rethrown, rather than
|
||||
* an {@link SdkException}, that is rethrown, rather than
|
||||
* swallowed.</li>
|
||||
* <li>Has some more diagnostics.</li>
|
||||
* <li>On failure, the last "relevant" AmazonClientException raised is
|
||||
* <li>On failure, the last "relevant" {@link SdkException} raised is
|
||||
* rethrown; exceptions other than 'no credentials' have priority.</li>
|
||||
* <li>Special handling of {@link AnonymousAWSCredentials}.</li>
|
||||
* <li>Special handling of {@link AnonymousCredentialsProvider}.</li>
|
||||
* </ol>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public final class AWSCredentialProviderList implements AWSCredentialsProvider,
|
||||
public final class AWSCredentialProviderList implements AwsCredentialsProvider,
|
||||
AutoCloseable {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(
|
||||
|
@ -73,9 +81,9 @@ public final class AWSCredentialProviderList implements AWSCredentialsProvider,
|
|||
CREDENTIALS_REQUESTED_WHEN_CLOSED
|
||||
= "Credentials requested after provider list was closed";
|
||||
|
||||
private final List<AWSCredentialsProvider> providers = new ArrayList<>(1);
|
||||
private final List<AwsCredentialsProvider> providers = new ArrayList<>(1);
|
||||
private boolean reuseLastProvider = true;
|
||||
private AWSCredentialsProvider lastProvider;
|
||||
private AwsCredentialsProvider lastProvider;
|
||||
|
||||
private final AtomicInteger refCount = new AtomicInteger(1);
|
||||
|
||||
|
@ -99,7 +107,9 @@ public final class AWSCredentialProviderList implements AWSCredentialsProvider,
|
|||
*/
|
||||
public AWSCredentialProviderList(
|
||||
Collection<AWSCredentialsProvider> providers) {
|
||||
this.providers.addAll(providers);
|
||||
for (AWSCredentialsProvider provider: providers) {
|
||||
this.providers.add(V1V2AwsCredentialProviderAdapter.adapt(provider));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -110,6 +120,19 @@ public final class AWSCredentialProviderList implements AWSCredentialsProvider,
|
|||
public AWSCredentialProviderList(final String name,
|
||||
final AWSCredentialsProvider... providerArgs) {
|
||||
setName(name);
|
||||
for (AWSCredentialsProvider provider: providerArgs) {
|
||||
this.providers.add(V1V2AwsCredentialProviderAdapter.adapt(provider));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create with an initial list of SDK V2 credential providers.
|
||||
* @param name name for error messages, may be ""
|
||||
* @param providerArgs provider list.
|
||||
*/
|
||||
public AWSCredentialProviderList(final String name,
|
||||
final AwsCredentialsProvider... providerArgs) {
|
||||
setName(name);
|
||||
Collections.addAll(providers, providerArgs);
|
||||
}
|
||||
|
||||
|
@ -127,12 +150,21 @@ public final class AWSCredentialProviderList implements AWSCredentialsProvider,
|
|||
|
||||
/**
|
||||
* Add a new provider.
|
||||
* @param p provider
|
||||
* @param provider provider
|
||||
*/
|
||||
public void add(AWSCredentialsProvider p) {
|
||||
providers.add(p);
|
||||
public void add(AWSCredentialsProvider provider) {
|
||||
providers.add(V1V2AwsCredentialProviderAdapter.adapt(provider));
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new SDK V2 provider.
|
||||
* @param provider provider
|
||||
*/
|
||||
public void add(AwsCredentialsProvider provider) {
|
||||
providers.add(provider);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Add all providers from another list to this one.
|
||||
* @param other the other list.
|
||||
|
@ -142,15 +174,18 @@ public final class AWSCredentialProviderList implements AWSCredentialsProvider,
|
|||
}
|
||||
|
||||
/**
|
||||
* Refresh all child entries.
|
||||
* This method will get credentials using SDK V2's resolveCredentials and then convert it into
|
||||
* V1 credentials. This required by delegation token binding classes.
|
||||
* @return SDK V1 credentials
|
||||
*/
|
||||
@Override
|
||||
public void refresh() {
|
||||
if (isClosed()) {
|
||||
return;
|
||||
}
|
||||
for (AWSCredentialsProvider provider : providers) {
|
||||
provider.refresh();
|
||||
public AWSCredentials getCredentials() {
|
||||
AwsCredentials credentials = resolveCredentials();
|
||||
if (credentials instanceof AwsSessionCredentials) {
|
||||
return new BasicSessionCredentials(credentials.accessKeyId(),
|
||||
credentials.secretAccessKey(),
|
||||
((AwsSessionCredentials) credentials).sessionToken());
|
||||
} else {
|
||||
return new BasicAWSCredentials(credentials.accessKeyId(), credentials.secretAccessKey());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -160,7 +195,7 @@ public final class AWSCredentialProviderList implements AWSCredentialsProvider,
|
|||
* @return a set of credentials (possibly anonymous), for authenticating.
|
||||
*/
|
||||
@Override
|
||||
public AWSCredentials getCredentials() {
|
||||
public AwsCredentials resolveCredentials() {
|
||||
if (isClosed()) {
|
||||
LOG.warn(CREDENTIALS_REQUESTED_WHEN_CLOSED);
|
||||
throw new NoAuthWithAWSException(name +
|
||||
|
@ -168,18 +203,18 @@ public final class AWSCredentialProviderList implements AWSCredentialsProvider,
|
|||
}
|
||||
checkNotEmpty();
|
||||
if (reuseLastProvider && lastProvider != null) {
|
||||
return lastProvider.getCredentials();
|
||||
return lastProvider.resolveCredentials();
|
||||
}
|
||||
|
||||
AmazonClientException lastException = null;
|
||||
for (AWSCredentialsProvider provider : providers) {
|
||||
SdkException lastException = null;
|
||||
for (AwsCredentialsProvider provider : providers) {
|
||||
try {
|
||||
AWSCredentials credentials = provider.getCredentials();
|
||||
AwsCredentials credentials = provider.resolveCredentials();
|
||||
Preconditions.checkNotNull(credentials,
|
||||
"Null credentials returned by %s", provider);
|
||||
if ((credentials.getAWSAccessKeyId() != null &&
|
||||
credentials.getAWSSecretKey() != null)
|
||||
|| (credentials instanceof AnonymousAWSCredentials)) {
|
||||
if ((credentials.accessKeyId() != null && credentials.secretAccessKey() != null) || (
|
||||
provider instanceof AnonymousCredentialsProvider
|
||||
|| provider instanceof AnonymousAWSCredentialsProvider)) {
|
||||
lastProvider = provider;
|
||||
LOG.debug("Using credentials from {}", provider);
|
||||
return credentials;
|
||||
|
@ -196,7 +231,7 @@ public final class AWSCredentialProviderList implements AWSCredentialsProvider,
|
|||
}
|
||||
LOG.debug("No credentials from {}: {}",
|
||||
provider, e.toString());
|
||||
} catch (AmazonClientException e) {
|
||||
} catch (SdkException e) {
|
||||
lastException = e;
|
||||
LOG.debug("No credentials provided by {}: {}",
|
||||
provider, e.toString(), e);
|
||||
|
@ -223,13 +258,13 @@ public final class AWSCredentialProviderList implements AWSCredentialsProvider,
|
|||
* @return providers
|
||||
*/
|
||||
@VisibleForTesting
|
||||
List<AWSCredentialsProvider> getProviders() {
|
||||
List<AwsCredentialsProvider> getProviders() {
|
||||
return providers;
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify that the provider list is not empty.
|
||||
* @throws AmazonClientException if there are no providers.
|
||||
* @throws SdkException if there are no providers.
|
||||
*/
|
||||
public void checkNotEmpty() {
|
||||
if (providers.isEmpty()) {
|
||||
|
@ -317,7 +352,7 @@ public final class AWSCredentialProviderList implements AWSCredentialsProvider,
|
|||
}
|
||||
|
||||
// do this outside the synchronized block.
|
||||
for (AWSCredentialsProvider p : providers) {
|
||||
for (AwsCredentialsProvider p : providers) {
|
||||
if (p instanceof Closeable) {
|
||||
IOUtils.closeStream((Closeable) p);
|
||||
} else if (p instanceof AutoCloseable) {
|
||||
|
|
|
@ -18,14 +18,14 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.AmazonServiceException;
|
||||
import software.amazon.awssdk.awscore.exception.AwsServiceException;
|
||||
|
||||
/**
|
||||
* Status code 443, no response from server. This is considered idempotent.
|
||||
*/
|
||||
public class AWSNoResponseException extends AWSServiceIOException {
|
||||
public AWSNoResponseException(String operation,
|
||||
AmazonServiceException cause) {
|
||||
AwsServiceException cause) {
|
||||
super(operation, cause);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.AmazonServiceException;
|
||||
import software.amazon.awssdk.awscore.exception.AwsServiceException;
|
||||
|
||||
/**
|
||||
* Request is redirected.
|
||||
|
@ -32,7 +32,7 @@ public class AWSRedirectException extends AWSServiceIOException {
|
|||
* @param cause the underlying cause
|
||||
*/
|
||||
public AWSRedirectException(String operation,
|
||||
AmazonServiceException cause) {
|
||||
AwsServiceException cause) {
|
||||
super(operation, cause);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,14 +18,13 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.services.s3.model.AmazonS3Exception;
|
||||
import software.amazon.awssdk.services.s3.model.S3Exception;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Wrap a {@link AmazonS3Exception} as an IOE, relaying all
|
||||
* Wrap a {@link S3Exception} as an IOE, relaying all
|
||||
* getters.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
|
@ -38,24 +37,12 @@ public class AWSS3IOException extends AWSServiceIOException {
|
|||
* @param cause the underlying cause
|
||||
*/
|
||||
public AWSS3IOException(String operation,
|
||||
AmazonS3Exception cause) {
|
||||
S3Exception cause) {
|
||||
super(operation, cause);
|
||||
}
|
||||
|
||||
public AmazonS3Exception getCause() {
|
||||
return (AmazonS3Exception) super.getCause();
|
||||
}
|
||||
|
||||
public String getErrorResponseXml() {
|
||||
return getCause().getErrorResponseXml();
|
||||
}
|
||||
|
||||
public Map<String, String> getAdditionalDetails() {
|
||||
return getCause().getAdditionalDetails();
|
||||
}
|
||||
|
||||
public String getExtendedRequestId() {
|
||||
return getCause().getExtendedRequestId();
|
||||
public S3Exception getCause() {
|
||||
return (S3Exception) super.getCause();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -18,13 +18,15 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.AmazonServiceException;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
import software.amazon.awssdk.awscore.exception.AwsErrorDetails;
|
||||
import software.amazon.awssdk.awscore.exception.AwsServiceException;
|
||||
|
||||
/**
|
||||
* A specific exception from AWS operations.
|
||||
* The exception must always be created with an {@link AmazonServiceException}.
|
||||
* The exception must always be created with an {@link AwsServiceException}.
|
||||
* The attributes of this exception can all be directly accessed.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
|
@ -37,36 +39,31 @@ public class AWSServiceIOException extends AWSClientIOException {
|
|||
* @param cause the underlying cause
|
||||
*/
|
||||
public AWSServiceIOException(String operation,
|
||||
AmazonServiceException cause) {
|
||||
AwsServiceException cause) {
|
||||
super(operation, cause);
|
||||
}
|
||||
|
||||
public AmazonServiceException getCause() {
|
||||
return (AmazonServiceException) super.getCause();
|
||||
public AwsServiceException getCause() {
|
||||
return (AwsServiceException) super.getCause();
|
||||
}
|
||||
|
||||
public String getRequestId() {
|
||||
return getCause().getRequestId();
|
||||
public boolean retryable() {
|
||||
return getCause().retryable();
|
||||
}
|
||||
|
||||
public String getServiceName() {
|
||||
return getCause().getServiceName();
|
||||
public String requestId() {
|
||||
return getCause().requestId();
|
||||
}
|
||||
|
||||
public String getErrorCode() {
|
||||
return getCause().getErrorCode();
|
||||
public AwsErrorDetails awsErrorDetails() {
|
||||
return getCause().awsErrorDetails();
|
||||
}
|
||||
|
||||
public int getStatusCode() {
|
||||
return getCause().getStatusCode();
|
||||
public int statusCode() {
|
||||
return getCause().statusCode();
|
||||
}
|
||||
|
||||
public String getRawResponseContent() {
|
||||
return getCause().getRawResponseContent();
|
||||
public String extendedRequestId() {
|
||||
return getCause().extendedRequestId();
|
||||
}
|
||||
|
||||
public boolean isRetryable() {
|
||||
return getCause().isRetryable();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.AmazonServiceException;
|
||||
import software.amazon.awssdk.awscore.exception.AwsServiceException;
|
||||
|
||||
/**
|
||||
* Exception raised when a service was throttled.
|
||||
|
@ -36,7 +36,7 @@ public class AWSServiceThrottledException extends AWSServiceIOException {
|
|||
* @param cause the underlying cause
|
||||
*/
|
||||
public AWSServiceThrottledException(String operation,
|
||||
AmazonServiceException cause) {
|
||||
AwsServiceException cause) {
|
||||
super(operation, cause);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.AmazonServiceException;
|
||||
import software.amazon.awssdk.awscore.exception.AwsServiceException;
|
||||
|
||||
/**
|
||||
* A 500 response came back from a service.
|
||||
|
@ -31,7 +31,7 @@ import com.amazonaws.AmazonServiceException;
|
|||
*/
|
||||
public class AWSStatus500Exception extends AWSServiceIOException {
|
||||
public AWSStatus500Exception(String operation,
|
||||
AmazonServiceException cause) {
|
||||
AwsServiceException cause) {
|
||||
super(operation, cause);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,9 +18,10 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||
import com.amazonaws.auth.AnonymousAWSCredentials;
|
||||
import com.amazonaws.auth.AWSCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
|
@ -35,23 +36,18 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
* property fs.s3a.aws.credentials.provider. Therefore, changing the class name
|
||||
* would be a backward-incompatible change.
|
||||
*
|
||||
* @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider
|
||||
* as part of upgrading S3A to SDK V2. See HADOOP-18073.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Stable
|
||||
@Deprecated
|
||||
public class AnonymousAWSCredentialsProvider implements AWSCredentialsProvider {
|
||||
public class AnonymousAWSCredentialsProvider implements AwsCredentialsProvider {
|
||||
|
||||
public static final String NAME
|
||||
= "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider";
|
||||
|
||||
public AWSCredentials getCredentials() {
|
||||
return new AnonymousAWSCredentials();
|
||||
public AwsCredentials resolveCredentials() {
|
||||
return AnonymousCredentialsProvider.create().resolveCredentials();
|
||||
}
|
||||
|
||||
public void refresh() {}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getClass().getSimpleName();
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.hadoop.fs.s3a;
|
|||
|
||||
import javax.annotation.Nonnull;
|
||||
|
||||
import com.amazonaws.arn.Arn;
|
||||
import software.amazon.awssdk.arns.Arn;
|
||||
|
||||
/**
|
||||
* Represents an Arn Resource, this can be an accesspoint or bucket.
|
||||
|
@ -120,14 +120,14 @@ public final class ArnResource {
|
|||
public static ArnResource accessPointFromArn(String arn) throws IllegalArgumentException {
|
||||
Arn parsed = Arn.fromString(arn);
|
||||
|
||||
if (parsed.getRegion().isEmpty() || parsed.getAccountId().isEmpty() ||
|
||||
parsed.getResourceAsString().isEmpty()) {
|
||||
if (!parsed.region().isPresent() || !parsed.accountId().isPresent() ||
|
||||
parsed.resourceAsString().isEmpty()) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format("Access Point Arn %s has an invalid format or missing properties", arn));
|
||||
}
|
||||
|
||||
String resourceName = parsed.getResource().getResource();
|
||||
return new ArnResource(resourceName, parsed.getAccountId(), parsed.getRegion(),
|
||||
parsed.getPartition(), arn);
|
||||
String resourceName = parsed.resource().resource();
|
||||
return new ArnResource(resourceName, parsed.accountId().get(), parsed.region().get(),
|
||||
parsed.partition(), arn);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -140,7 +140,6 @@ public final class Constants {
|
|||
public static final String ASSUMED_ROLE_POLICY =
|
||||
"fs.s3a.assumed.role.policy";
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
public static final String ASSUMED_ROLE_CREDENTIALS_DEFAULT =
|
||||
SimpleAWSCredentialsProvider.NAME;
|
||||
|
||||
|
@ -735,7 +734,6 @@ public final class Constants {
|
|||
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
@SuppressWarnings("deprecation")
|
||||
public static final Class<? extends S3ClientFactory>
|
||||
DEFAULT_S3_CLIENT_FACTORY_IMPL =
|
||||
DefaultS3ClientFactory.class;
|
||||
|
@ -1276,4 +1274,8 @@ public final class Constants {
|
|||
public static final String STORE_CAPABILITY_DIRECTORY_MARKER_MULTIPART_UPLOAD_ENABLED =
|
||||
"fs.s3a.capability.multipart.uploads.enabled";
|
||||
|
||||
/**
|
||||
* The bucket region header.
|
||||
*/
|
||||
public static final String BUCKET_REGION_HEADER = "x-amz-bucket-region";
|
||||
}
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.AmazonClientException;
|
||||
import software.amazon.awssdk.core.exception.SdkException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
@ -26,8 +26,8 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
/**
|
||||
* Exception which Hadoop's AWSCredentialsProvider implementations should
|
||||
* throw when there is a problem with the credential setup. This
|
||||
* is a subclass of {@link AmazonClientException} which sets
|
||||
* {@link #isRetryable()} to false, so as to fail fast.
|
||||
* is a subclass of {@link SdkException} which sets
|
||||
* {@link #retryable()} to false, so as to fail fast.
|
||||
* This is used in credential providers and elsewhere.
|
||||
* When passed through {@code S3AUtils.translateException()} it
|
||||
* is mapped to an AccessDeniedException. As a result, the Invoker
|
||||
|
@ -35,13 +35,14 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public class CredentialInitializationException extends AmazonClientException {
|
||||
public class CredentialInitializationException extends SdkException {
|
||||
|
||||
public CredentialInitializationException(String message, Throwable t) {
|
||||
super(message, t);
|
||||
super(builder().message(message).cause(t));
|
||||
}
|
||||
|
||||
public CredentialInitializationException(String message) {
|
||||
super(message);
|
||||
super(builder().message(message));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -49,7 +50,5 @@ public class CredentialInitializationException extends AmazonClientException {
|
|||
* @return false, always.
|
||||
*/
|
||||
@Override
|
||||
public boolean isRetryable() {
|
||||
return false;
|
||||
}
|
||||
public boolean retryable() { return false; }
|
||||
}
|
||||
|
|
|
@ -20,11 +20,12 @@ package org.apache.hadoop.fs.s3a;
|
|||
|
||||
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.client.builder.AwsClientBuilder;
|
||||
import com.amazonaws.handlers.RequestHandler2;
|
||||
import com.amazonaws.regions.RegionUtils;
|
||||
import com.amazonaws.services.s3.AmazonS3;
|
||||
import com.amazonaws.services.s3.AmazonS3Builder;
|
||||
|
@ -41,11 +42,28 @@ import com.amazonaws.services.s3.model.EncryptionMaterialsProvider;
|
|||
import com.amazonaws.services.s3.model.KMSEncryptionMaterialsProvider;
|
||||
import com.amazonaws.util.AwsHostNameUtils;
|
||||
import com.amazonaws.util.RuntimeHttpUtils;
|
||||
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
|
||||
import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration;
|
||||
import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption;
|
||||
import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
|
||||
import software.amazon.awssdk.core.retry.RetryPolicy;
|
||||
import software.amazon.awssdk.http.apache.ApacheHttpClient;
|
||||
import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient;
|
||||
import software.amazon.awssdk.regions.Region;
|
||||
import software.amazon.awssdk.services.s3.S3AsyncClient;
|
||||
import software.amazon.awssdk.services.s3.S3BaseClientBuilder;
|
||||
import software.amazon.awssdk.services.s3.S3Client;
|
||||
import software.amazon.awssdk.services.s3.S3Configuration;
|
||||
import software.amazon.awssdk.services.s3.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,12 +75,16 @@ import org.apache.hadoop.fs.store.LogExactlyOnce;
|
|||
import static com.amazonaws.services.s3.Headers.REQUESTER_PAYS_HEADER;
|
||||
import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION;
|
||||
import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_CENTRAL_REGION;
|
||||
import static org.apache.hadoop.fs.s3a.Constants.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.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;
|
||||
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_301_MOVED_PERMANENTLY;
|
||||
|
||||
/**
|
||||
* The default {@link S3ClientFactory} implementation.
|
||||
|
@ -71,7 +93,6 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.translateException;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
@SuppressWarnings("deprecation")
|
||||
public class DefaultS3ClientFactory extends Configured
|
||||
implements S3ClientFactory {
|
||||
|
||||
|
@ -108,6 +129,7 @@ public class DefaultS3ClientFactory extends Configured
|
|||
* Create the client by preparing the AwsConf configuration
|
||||
* and then invoking {@code buildAmazonS3Client()}.
|
||||
*/
|
||||
// TODO: Remove this and all code that configures the V1 S3 client.
|
||||
@Override
|
||||
public AmazonS3 createS3Client(
|
||||
final URI uri,
|
||||
|
@ -157,10 +179,123 @@ public class DefaultS3ClientFactory extends Configured
|
|||
}
|
||||
} catch (SdkClientException e) {
|
||||
// SDK refused to build.
|
||||
throw translateException("creating AWS S3 client", uri.toString(), e);
|
||||
// TODO: remove?
|
||||
//throw translateException("creating AWS S3 client", uri.toString(), e);
|
||||
throw new IOException("creating AWS S3 client: "+ uri.toString(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public S3Client createS3ClientV2(
|
||||
final URI uri,
|
||||
final S3ClientCreationParameters parameters) throws IOException {
|
||||
|
||||
Configuration conf = getConf();
|
||||
bucket = uri.getHost();
|
||||
|
||||
ApacheHttpClient.Builder httpClientBuilder = AWSClientConfig
|
||||
.createHttpClientBuilder(conf)
|
||||
.proxyConfiguration(AWSClientConfig.createProxyConfiguration(conf, bucket));
|
||||
return configureClientBuilder(S3Client.builder(), parameters, conf, bucket)
|
||||
.httpClientBuilder(httpClientBuilder)
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public S3AsyncClient createS3AsyncClient(
|
||||
final URI uri,
|
||||
final S3ClientCreationParameters parameters) throws IOException {
|
||||
|
||||
Configuration conf = getConf();
|
||||
bucket = uri.getHost();
|
||||
NettyNioAsyncHttpClient.Builder httpClientBuilder = AWSClientConfig
|
||||
.createAsyncHttpClientBuilder(conf)
|
||||
.proxyConfiguration(AWSClientConfig.createAsyncProxyConfiguration(conf, bucket));
|
||||
return configureClientBuilder(S3AsyncClient.builder(), parameters, conf, bucket)
|
||||
.httpClientBuilder(httpClientBuilder)
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Configure a sync or async S3 client builder.
|
||||
* This method handles all shared configuration.
|
||||
* @param builder S3 client builder
|
||||
* @param parameters parameter object
|
||||
* @param conf configuration object
|
||||
* @param bucket bucket name
|
||||
* @return the builder object
|
||||
* @param <BuilderT> S3 client builder type
|
||||
* @param <ClientT> S3 client type
|
||||
*/
|
||||
private <BuilderT extends S3BaseClientBuilder<BuilderT, ClientT>, ClientT>
|
||||
BuilderT configureClientBuilder(
|
||||
BuilderT builder,
|
||||
S3ClientCreationParameters parameters,
|
||||
Configuration conf,
|
||||
String bucket) {
|
||||
|
||||
URI endpoint = getS3Endpoint(parameters.getEndpoint(), conf);
|
||||
Region region = getS3Region(conf.getTrimmed(AWS_REGION), bucket,
|
||||
parameters.getCredentialSet());
|
||||
LOG.debug("Using endpoint {}; and region {}", endpoint, region);
|
||||
|
||||
// TODO: Some configuration done in configureBasicParams is not done yet.
|
||||
S3Configuration serviceConfiguration = S3Configuration.builder()
|
||||
.pathStyleAccessEnabled(parameters.isPathStyleAccess())
|
||||
// TODO: Review. Currently required to pass access point tests in ITestS3ABucketExistence,
|
||||
// but resolving the region from the ap may be the correct solution.
|
||||
.useArnRegionEnabled(true)
|
||||
.build();
|
||||
|
||||
return builder
|
||||
.overrideConfiguration(createClientOverrideConfiguration(parameters, conf))
|
||||
.credentialsProvider(parameters.getCredentialSet())
|
||||
.endpointOverride(endpoint)
|
||||
.region(region)
|
||||
.serviceConfiguration(serviceConfiguration);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create an override configuration for an S3 client.
|
||||
* @param parameters parameter object
|
||||
* @param conf configuration object
|
||||
* @return the override configuration
|
||||
*/
|
||||
protected ClientOverrideConfiguration createClientOverrideConfiguration(
|
||||
S3ClientCreationParameters parameters, Configuration conf) {
|
||||
final ClientOverrideConfiguration.Builder clientOverrideConfigBuilder =
|
||||
AWSClientConfig.createClientConfigBuilder(conf);
|
||||
|
||||
// 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());
|
||||
}
|
||||
|
||||
if (parameters.getExecutionInterceptors() != null) {
|
||||
for (ExecutionInterceptor interceptor : parameters.getExecutionInterceptors()) {
|
||||
clientOverrideConfigBuilder.addExecutionInterceptor(interceptor);
|
||||
}
|
||||
}
|
||||
|
||||
if (parameters.getMetrics() != null) {
|
||||
clientOverrideConfigBuilder.addMetricPublisher(
|
||||
new AwsStatisticsCollector(parameters.getMetrics()));
|
||||
}
|
||||
|
||||
final RetryPolicy.Builder retryPolicyBuilder = AWSClientConfig.createRetryPolicyBuilder(conf);
|
||||
clientOverrideConfigBuilder.retryPolicy(retryPolicyBuilder.build());
|
||||
|
||||
return clientOverrideConfigBuilder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create an {@link AmazonS3} client of type
|
||||
* {@link AmazonS3EncryptionV2} if CSE is enabled.
|
||||
|
@ -252,18 +387,11 @@ public class DefaultS3ClientFactory extends Configured
|
|||
*/
|
||||
private void configureBasicParams(AmazonS3Builder builder,
|
||||
ClientConfiguration awsConf, S3ClientCreationParameters parameters) {
|
||||
builder.withCredentials(parameters.getCredentialSet());
|
||||
// TODO: This whole block will be removed when we remove the V1 client.
|
||||
// builder.withCredentials(parameters.getCredentialSet());
|
||||
builder.withClientConfiguration(awsConf);
|
||||
builder.withPathStyleAccessEnabled(parameters.isPathStyleAccess());
|
||||
|
||||
if (parameters.getMetrics() != null) {
|
||||
builder.withMetricsCollector(
|
||||
new AwsStatisticsCollector(parameters.getMetrics()));
|
||||
}
|
||||
if (parameters.getRequestHandlers() != null) {
|
||||
builder.withRequestHandlers(
|
||||
parameters.getRequestHandlers().toArray(new RequestHandler2[0]));
|
||||
}
|
||||
if (parameters.getMonitoringListener() != null) {
|
||||
builder.withMonitoringListener(parameters.getMonitoringListener());
|
||||
}
|
||||
|
@ -391,4 +519,74 @@ public class DefaultS3ClientFactory extends Configured
|
|||
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 bucket Bucket name.
|
||||
* @param credentialsProvider Credentials provider to be used with the default s3 client.
|
||||
* @return region of the bucket.
|
||||
*/
|
||||
private static Region getS3Region(String region, String bucket,
|
||||
AwsCredentialsProvider credentialsProvider) {
|
||||
|
||||
if (!StringUtils.isBlank(region)) {
|
||||
return Region.of(region);
|
||||
}
|
||||
|
||||
try {
|
||||
// build a s3 client with region eu-west-1 that can be used to get the region of the bucket.
|
||||
// Using eu-west-1, as headBucket() doesn't work with us-east-1. This is because
|
||||
// us-east-1 uses the endpoint s3.amazonaws.com, which resolves bucket.s3.amazonaws.com to
|
||||
// the actual region the bucket is in. As the request is signed with us-east-1 and not the
|
||||
// bucket's region, it fails.
|
||||
S3Client s3Client = S3Client.builder().region(Region.EU_WEST_1)
|
||||
.credentialsProvider(credentialsProvider)
|
||||
.build();
|
||||
|
||||
HeadBucketResponse headBucketResponse =
|
||||
s3Client.headBucket(HeadBucketRequest.builder().bucket(bucket).build());
|
||||
return Region.of(
|
||||
headBucketResponse.sdkHttpResponse().headers().get(BUCKET_REGION_HEADER).get(0));
|
||||
} catch (S3Exception exception) {
|
||||
if (exception.statusCode() == SC_301_MOVED_PERMANENTLY) {
|
||||
List<String> bucketRegion =
|
||||
exception.awsErrorDetails().sdkHttpResponse().headers().get(BUCKET_REGION_HEADER);
|
||||
return Region.of(bucketRegion.get(0));
|
||||
}
|
||||
}
|
||||
|
||||
return Region.US_EAST_1;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -36,7 +36,7 @@ public class FailureInjectionPolicy {
|
|||
public static final String DEFAULT_DELAY_KEY_SUBSTRING = "DELAY_LISTING_ME";
|
||||
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(InconsistentAmazonS3Client.class);
|
||||
LoggerFactory.getLogger(FailureInjectionPolicy.class);
|
||||
|
||||
/**
|
||||
* Probability of throttling a request.
|
||||
|
|
|
@ -1,345 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import com.amazonaws.AmazonClientException;
|
||||
import com.amazonaws.AmazonServiceException;
|
||||
import com.amazonaws.ClientConfiguration;
|
||||
import com.amazonaws.SdkClientException;
|
||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||
import com.amazonaws.services.s3.AmazonS3;
|
||||
import com.amazonaws.services.s3.AmazonS3Client;
|
||||
import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectRequest;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsResult;
|
||||
import com.amazonaws.services.s3.model.GetObjectRequest;
|
||||
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
|
||||
import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
|
||||
import com.amazonaws.services.s3.model.ListObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.ListObjectsV2Request;
|
||||
import com.amazonaws.services.s3.model.ListObjectsV2Result;
|
||||
import com.amazonaws.services.s3.model.MultipartUploadListing;
|
||||
import com.amazonaws.services.s3.model.ObjectListing;
|
||||
import com.amazonaws.services.s3.model.PutObjectRequest;
|
||||
import com.amazonaws.services.s3.model.PutObjectResult;
|
||||
import com.amazonaws.services.s3.model.S3Object;
|
||||
import com.amazonaws.services.s3.model.S3ObjectSummary;
|
||||
import com.amazonaws.services.s3.model.UploadPartRequest;
|
||||
import com.amazonaws.services.s3.model.UploadPartResult;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
/**
|
||||
* A wrapper around {@link com.amazonaws.services.s3.AmazonS3} that injects
|
||||
* failures.
|
||||
* It used to also inject inconsistency, but this was removed with S3Guard;
|
||||
* what is retained is the ability to throttle AWS operations and for the
|
||||
* input stream to be inconsistent.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
public class InconsistentAmazonS3Client extends AmazonS3Client {
|
||||
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(InconsistentAmazonS3Client.class);
|
||||
|
||||
private FailureInjectionPolicy policy;
|
||||
|
||||
/**
|
||||
* Counter of failures since last reset.
|
||||
*/
|
||||
private final AtomicLong failureCounter = new AtomicLong(0);
|
||||
|
||||
|
||||
/**
|
||||
* Instantiate.
|
||||
* This subclasses a deprecated constructor of the parent
|
||||
* {@code AmazonS3Client} class; we can't use the builder API because,
|
||||
* that only creates the consistent client.
|
||||
* @param credentials credentials to auth.
|
||||
* @param clientConfiguration connection settings
|
||||
* @param conf hadoop configuration.
|
||||
*/
|
||||
@SuppressWarnings("deprecation")
|
||||
public InconsistentAmazonS3Client(AWSCredentialsProvider credentials,
|
||||
ClientConfiguration clientConfiguration, Configuration conf) {
|
||||
super(credentials, clientConfiguration);
|
||||
policy = new FailureInjectionPolicy(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* A way for tests to patch in a different fault injection policy at runtime.
|
||||
* @param fs filesystem under test
|
||||
* @param policy failure injection settings to set
|
||||
* @throws Exception on failure
|
||||
*/
|
||||
public static void setFailureInjectionPolicy(S3AFileSystem fs,
|
||||
FailureInjectionPolicy policy) throws Exception {
|
||||
AmazonS3 s3 = fs.getAmazonS3ClientForTesting("s3guard");
|
||||
InconsistentAmazonS3Client ic = InconsistentAmazonS3Client.castFrom(s3);
|
||||
ic.replacePolicy(policy);
|
||||
}
|
||||
|
||||
private void replacePolicy(FailureInjectionPolicy pol) {
|
||||
this.policy = pol;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("Inconsistent S3 Client: %s; failure count %d",
|
||||
policy, failureCounter.get());
|
||||
}
|
||||
|
||||
/**
|
||||
* Convenience function for test code to cast from supertype.
|
||||
* @param c supertype to cast from
|
||||
* @return subtype, not null
|
||||
* @throws Exception on error
|
||||
*/
|
||||
public static InconsistentAmazonS3Client castFrom(AmazonS3 c) throws
|
||||
Exception {
|
||||
InconsistentAmazonS3Client ic = null;
|
||||
if (c instanceof InconsistentAmazonS3Client) {
|
||||
ic = (InconsistentAmazonS3Client) c;
|
||||
}
|
||||
Preconditions.checkNotNull(ic, "Not an instance of " +
|
||||
"InconsistentAmazonS3Client");
|
||||
return ic;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DeleteObjectsResult deleteObjects(DeleteObjectsRequest
|
||||
deleteObjectsRequest)
|
||||
throws AmazonClientException, AmazonServiceException {
|
||||
maybeFail();
|
||||
return super.deleteObjects(deleteObjectsRequest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteObject(DeleteObjectRequest deleteObjectRequest)
|
||||
throws AmazonClientException, AmazonServiceException {
|
||||
String key = deleteObjectRequest.getKey();
|
||||
LOG.debug("key {}", key);
|
||||
maybeFail();
|
||||
super.deleteObject(deleteObjectRequest);
|
||||
}
|
||||
|
||||
/* We should only need to override this version of putObject() */
|
||||
@Override
|
||||
public PutObjectResult putObject(PutObjectRequest putObjectRequest)
|
||||
throws AmazonClientException, AmazonServiceException {
|
||||
LOG.debug("key {}", putObjectRequest.getKey());
|
||||
maybeFail();
|
||||
return super.putObject(putObjectRequest);
|
||||
}
|
||||
|
||||
/* We should only need to override these versions of listObjects() */
|
||||
@Override
|
||||
public ObjectListing listObjects(ListObjectsRequest listObjectsRequest)
|
||||
throws AmazonClientException, AmazonServiceException {
|
||||
maybeFail();
|
||||
return super.listObjects(listObjectsRequest);
|
||||
}
|
||||
|
||||
/* consistent listing with possibility of failing. */
|
||||
@Override
|
||||
public ListObjectsV2Result listObjectsV2(ListObjectsV2Request request)
|
||||
throws AmazonClientException, AmazonServiceException {
|
||||
maybeFail();
|
||||
return super.listObjectsV2(request);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public CompleteMultipartUploadResult completeMultipartUpload(
|
||||
CompleteMultipartUploadRequest completeMultipartUploadRequest)
|
||||
throws SdkClientException, AmazonServiceException {
|
||||
maybeFail();
|
||||
return super.completeMultipartUpload(completeMultipartUploadRequest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public UploadPartResult uploadPart(UploadPartRequest uploadPartRequest)
|
||||
throws SdkClientException, AmazonServiceException {
|
||||
maybeFail();
|
||||
return super.uploadPart(uploadPartRequest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InitiateMultipartUploadResult initiateMultipartUpload(
|
||||
InitiateMultipartUploadRequest initiateMultipartUploadRequest)
|
||||
throws SdkClientException, AmazonServiceException {
|
||||
maybeFail();
|
||||
return super.initiateMultipartUpload(initiateMultipartUploadRequest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MultipartUploadListing listMultipartUploads(
|
||||
ListMultipartUploadsRequest listMultipartUploadsRequest)
|
||||
throws SdkClientException, AmazonServiceException {
|
||||
maybeFail();
|
||||
return super.listMultipartUploads(listMultipartUploadsRequest);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the probability of throttling a request.
|
||||
* @param throttleProbability the probability of a request being throttled.
|
||||
*/
|
||||
public void setThrottleProbability(float throttleProbability) {
|
||||
policy.setThrottleProbability(throttleProbability);
|
||||
}
|
||||
|
||||
/**
|
||||
* Conditionally fail the operation.
|
||||
* @param errorMsg description of failure
|
||||
* @param statusCode http status code for error
|
||||
* @throws AmazonClientException if the client chooses to fail
|
||||
* the request.
|
||||
*/
|
||||
private void maybeFail(String errorMsg, int statusCode)
|
||||
throws AmazonClientException {
|
||||
// code structure here is to line up for more failures later
|
||||
AmazonServiceException ex = null;
|
||||
if (FailureInjectionPolicy.trueWithProbability(policy.getThrottleProbability())) {
|
||||
// throttle the request
|
||||
ex = new AmazonServiceException(errorMsg
|
||||
+ " count = " + (failureCounter.get() + 1), null);
|
||||
ex.setStatusCode(statusCode);
|
||||
}
|
||||
|
||||
int failureLimit = policy.getFailureLimit();
|
||||
if (ex != null) {
|
||||
long count = failureCounter.incrementAndGet();
|
||||
if (failureLimit == 0
|
||||
|| (failureLimit > 0 && count < failureLimit)) {
|
||||
throw ex;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void maybeFail() {
|
||||
maybeFail("throttled", 503);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the limit on failures before all operations pass through.
|
||||
* This resets the failure count.
|
||||
* @param limit limit; "0" means "no limit"
|
||||
*/
|
||||
public void setFailureLimit(int limit) {
|
||||
policy.setFailureLimit(limit);
|
||||
failureCounter.set(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public S3Object getObject(GetObjectRequest var1) throws SdkClientException,
|
||||
AmazonServiceException {
|
||||
maybeFail();
|
||||
return super.getObject(var1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public S3Object getObject(String bucketName, String key)
|
||||
throws SdkClientException, AmazonServiceException {
|
||||
maybeFail();
|
||||
return super.getObject(bucketName, key);
|
||||
|
||||
}
|
||||
|
||||
/** Since ObjectListing is immutable, we just override it with wrapper. */
|
||||
@SuppressWarnings("serial")
|
||||
private static class CustomObjectListing extends ObjectListing {
|
||||
|
||||
private final List<S3ObjectSummary> customListing;
|
||||
private final List<String> customPrefixes;
|
||||
|
||||
CustomObjectListing(ObjectListing rawListing,
|
||||
List<S3ObjectSummary> customListing,
|
||||
List<String> customPrefixes) {
|
||||
super();
|
||||
this.customListing = customListing;
|
||||
this.customPrefixes = customPrefixes;
|
||||
|
||||
this.setBucketName(rawListing.getBucketName());
|
||||
this.setCommonPrefixes(rawListing.getCommonPrefixes());
|
||||
this.setDelimiter(rawListing.getDelimiter());
|
||||
this.setEncodingType(rawListing.getEncodingType());
|
||||
this.setMarker(rawListing.getMarker());
|
||||
this.setMaxKeys(rawListing.getMaxKeys());
|
||||
this.setNextMarker(rawListing.getNextMarker());
|
||||
this.setPrefix(rawListing.getPrefix());
|
||||
this.setTruncated(rawListing.isTruncated());
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<S3ObjectSummary> getObjectSummaries() {
|
||||
return customListing;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getCommonPrefixes() {
|
||||
return customPrefixes;
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("serial")
|
||||
private static class CustomListObjectsV2Result extends ListObjectsV2Result {
|
||||
|
||||
private final List<S3ObjectSummary> customListing;
|
||||
private final List<String> customPrefixes;
|
||||
|
||||
CustomListObjectsV2Result(ListObjectsV2Result raw,
|
||||
List<S3ObjectSummary> customListing, List<String> customPrefixes) {
|
||||
super();
|
||||
this.customListing = customListing;
|
||||
this.customPrefixes = customPrefixes;
|
||||
|
||||
this.setBucketName(raw.getBucketName());
|
||||
this.setCommonPrefixes(raw.getCommonPrefixes());
|
||||
this.setDelimiter(raw.getDelimiter());
|
||||
this.setEncodingType(raw.getEncodingType());
|
||||
this.setStartAfter(raw.getStartAfter());
|
||||
this.setMaxKeys(raw.getMaxKeys());
|
||||
this.setContinuationToken(raw.getContinuationToken());
|
||||
this.setPrefix(raw.getPrefix());
|
||||
this.setTruncated(raw.isTruncated());
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<S3ObjectSummary> getObjectSummaries() {
|
||||
return customListing;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getCommonPrefixes() {
|
||||
return customPrefixes;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -18,11 +18,18 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.ClientConfiguration;
|
||||
import com.amazonaws.services.s3.AmazonS3;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import software.amazon.awssdk.awscore.exception.AwsServiceException;
|
||||
import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration;
|
||||
import software.amazon.awssdk.core.exception.SdkException;
|
||||
import software.amazon.awssdk.core.interceptor.Context;
|
||||
import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
|
||||
import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
/**
|
||||
* S3 Client factory used for testing with eventual consistency fault injection.
|
||||
|
@ -30,25 +37,74 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
* {@code hadoop-aws} module to enable integration tests to use this
|
||||
* just by editing the Hadoop configuration used to bring up the client.
|
||||
*
|
||||
* The factory uses the older constructor-based instantiation/configuration
|
||||
* of the client, so does not wire up metrics, handlers etc.
|
||||
* The factory injects an {@link ExecutionInterceptor} to inject failures.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
public class InconsistentS3ClientFactory extends DefaultS3ClientFactory {
|
||||
|
||||
@Override
|
||||
protected AmazonS3 buildAmazonS3Client(
|
||||
final ClientConfiguration awsConf,
|
||||
final S3ClientCreationParameters parameters) {
|
||||
protected ClientOverrideConfiguration createClientOverrideConfiguration(
|
||||
S3ClientCreationParameters parameters, Configuration conf) {
|
||||
LOG.warn("** FAILURE INJECTION ENABLED. Do not run in production! **");
|
||||
LOG.warn("List inconsistency is no longer emulated; only throttling and read errors");
|
||||
InconsistentAmazonS3Client s3
|
||||
= new InconsistentAmazonS3Client(
|
||||
parameters.getCredentialSet(), awsConf, getConf());
|
||||
configureAmazonS3Client(s3,
|
||||
parameters.getEndpoint(),
|
||||
parameters.isPathStyleAccess());
|
||||
return s3;
|
||||
return super.createClientOverrideConfiguration(parameters, conf)
|
||||
.toBuilder()
|
||||
.addExecutionInterceptor(new FailureInjectionInterceptor(
|
||||
new FailureInjectionPolicy(conf)))
|
||||
.build();
|
||||
}
|
||||
|
||||
private static class FailureInjectionInterceptor implements ExecutionInterceptor {
|
||||
|
||||
private final FailureInjectionPolicy policy;
|
||||
|
||||
/**
|
||||
* Counter of failures since last reset.
|
||||
*/
|
||||
private final AtomicLong failureCounter = new AtomicLong(0);
|
||||
|
||||
public FailureInjectionInterceptor(FailureInjectionPolicy policy) {
|
||||
this.policy = policy;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeExecution(Context.BeforeExecution context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
maybeFail();
|
||||
}
|
||||
|
||||
private void maybeFail() {
|
||||
maybeFail("throttled", 503);
|
||||
}
|
||||
|
||||
/**
|
||||
* Conditionally fail the operation.
|
||||
* @param errorMsg description of failure
|
||||
* @param statusCode http status code for error
|
||||
* @throws SdkException if the client chooses to fail
|
||||
* the request.
|
||||
*/
|
||||
private void maybeFail(String errorMsg, int statusCode)
|
||||
throws SdkException {
|
||||
// code structure here is to line up for more failures later
|
||||
AwsServiceException ex = null;
|
||||
if (FailureInjectionPolicy.trueWithProbability(policy.getThrottleProbability())) {
|
||||
// throttle the request
|
||||
ex = AwsServiceException.builder()
|
||||
.message(errorMsg + " count = " + (failureCounter.get() + 1))
|
||||
.statusCode(statusCode)
|
||||
.build();
|
||||
}
|
||||
|
||||
int failureLimit = policy.getFailureLimit();
|
||||
if (ex != null) {
|
||||
long count = failureCounter.incrementAndGet();
|
||||
if (failureLimit == 0
|
||||
|| (failureLimit > 0 && count < failureLimit)) {
|
||||
throw ex;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,8 +24,6 @@ import java.util.Optional;
|
|||
import java.util.concurrent.Future;
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import com.amazonaws.AmazonClientException;
|
||||
import com.amazonaws.SdkBaseException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -39,6 +37,8 @@ import org.apache.hadoop.util.functional.FutureIO;
|
|||
import org.apache.hadoop.util.functional.InvocationRaisingIOE;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
|
||||
import software.amazon.awssdk.core.exception.SdkException;
|
||||
|
||||
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration;
|
||||
|
||||
/**
|
||||
|
@ -120,7 +120,7 @@ public class Invoker {
|
|||
throws IOException {
|
||||
try (DurationInfo ignored = new DurationInfo(LOG, false, "%s", action)) {
|
||||
return operation.apply();
|
||||
} catch (AmazonClientException e) {
|
||||
} catch (SdkException e) {
|
||||
throw S3AUtils.translateException(action, path, e);
|
||||
}
|
||||
}
|
||||
|
@ -145,7 +145,7 @@ public class Invoker {
|
|||
throws IOException {
|
||||
try {
|
||||
return invokeTrackingDuration(tracker, operation);
|
||||
} catch (AmazonClientException e) {
|
||||
} catch (SdkException e) {
|
||||
throw S3AUtils.translateException(action, path, e);
|
||||
}
|
||||
}
|
||||
|
@ -170,7 +170,7 @@ public class Invoker {
|
|||
|
||||
/**
|
||||
*
|
||||
* Wait for a future, translating AmazonClientException into an IOException.
|
||||
* Wait for a future, translating SdkException into an IOException.
|
||||
* @param action action to execute (used in error messages)
|
||||
* @param path path of work (used in error messages)
|
||||
* @param future future to await for
|
||||
|
@ -186,7 +186,7 @@ public class Invoker {
|
|||
throws IOException {
|
||||
try (DurationInfo ignored = new DurationInfo(LOG, false, "%s", action)) {
|
||||
return FutureIO.awaitFuture(future);
|
||||
} catch (AmazonClientException e) {
|
||||
} catch (SdkException e) {
|
||||
throw S3AUtils.translateException(action, path, e);
|
||||
}
|
||||
}
|
||||
|
@ -466,7 +466,7 @@ public class Invoker {
|
|||
}
|
||||
// execute the operation, returning if successful
|
||||
return operation.apply();
|
||||
} catch (IOException | SdkBaseException e) {
|
||||
} catch (IOException | SdkException e) {
|
||||
caught = e;
|
||||
}
|
||||
// you only get here if the operation didn't complete
|
||||
|
@ -478,7 +478,7 @@ public class Invoker {
|
|||
translated = (IOException) caught;
|
||||
} else {
|
||||
translated = S3AUtils.translateException(text, "",
|
||||
(SdkBaseException)caught);
|
||||
(SdkException) caught);
|
||||
}
|
||||
|
||||
try {
|
||||
|
@ -517,11 +517,10 @@ public class Invoker {
|
|||
if (caught instanceof IOException) {
|
||||
throw (IOException) caught;
|
||||
} else {
|
||||
throw (SdkBaseException) caught;
|
||||
throw (SdkException) caught;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Execute an operation; any exception raised is simply caught and
|
||||
* logged at debug.
|
||||
|
|
|
@ -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.fs.store.audit.AuditSpan;
|
|||
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 class Listing extends AbstractStoreOperation {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 @@ public class Listing extends AbstractStoreOperation {
|
|||
int added = 0, ignored = 0;
|
||||
// list to fill in with results. Initial size will be list maximum.
|
||||
List<S3AFileStatus> 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 @@ public class Listing extends AbstractStoreOperation {
|
|||
}
|
||||
|
||||
// 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 class Listing extends AbstractStoreOperation {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 class Listing extends AbstractStoreOperation {
|
|||
*/
|
||||
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 class Listing extends AbstractStoreOperation {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -16,32 +16,37 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.s3a.impl;
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.AccessDeniedException;
|
||||
import java.util.List;
|
||||
|
||||
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.fs.s3a.AWSS3IOException;
|
||||
import software.amazon.awssdk.services.s3.model.S3Error;
|
||||
import software.amazon.awssdk.services.s3.model.S3Exception;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_200_OK;
|
||||
|
||||
/**
|
||||
* Support for Multi Object Deletion.
|
||||
* This is used to be a complex piece of code as it was required to
|
||||
* update s3guard.
|
||||
* Now all that is left is the exception extraction for better
|
||||
* reporting,
|
||||
* Exception raised in {@link S3AFileSystem#deleteObjects} when
|
||||
* one or more of the keys could not be deleted.
|
||||
*
|
||||
* Used to reproduce the behaviour of SDK v1 for partial failures
|
||||
* on DeleteObjects. In SDK v2, the errors are returned as part of
|
||||
* the response objects.
|
||||
*/
|
||||
public final class MultiObjectDeleteSupport {
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Unstable
|
||||
public class MultiObjectDeleteException extends S3Exception {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(
|
||||
MultiObjectDeleteSupport.class);
|
||||
|
||||
private MultiObjectDeleteSupport() {
|
||||
}
|
||||
MultiObjectDeleteException.class);
|
||||
|
||||
/**
|
||||
* This is the exception exit code if access was denied on a delete.
|
||||
|
@ -49,6 +54,15 @@ public final class MultiObjectDeleteSupport {
|
|||
*/
|
||||
public static final String ACCESS_DENIED = "AccessDenied";
|
||||
|
||||
private final List<S3Error> errors;
|
||||
|
||||
public MultiObjectDeleteException(List<S3Error> errors) {
|
||||
super(builder().message(errors.toString()).statusCode(SC_200_OK));
|
||||
this.errors = errors;
|
||||
}
|
||||
|
||||
public List<S3Error> errors() { return errors; }
|
||||
|
||||
/**
|
||||
* A {@code MultiObjectDeleteException} is raised if one or more
|
||||
* paths listed in a bulk DELETE operation failed.
|
||||
|
@ -58,29 +72,23 @@ public final class MultiObjectDeleteSupport {
|
|||
* the causes, otherwise grabs the status code and uses it in the
|
||||
* returned exception.
|
||||
* @param message text for the exception
|
||||
* @param deleteException the delete exception. to translate
|
||||
* @return an IOE with more detail.
|
||||
*/
|
||||
public static IOException translateDeleteException(
|
||||
final String message,
|
||||
final MultiObjectDeleteException deleteException) {
|
||||
List<MultiObjectDeleteException.DeleteError> errors
|
||||
= deleteException.getErrors();
|
||||
public IOException translateException(final String message) {
|
||||
LOG.info("Bulk delete operation failed to delete all objects;"
|
||||
+ " failure count = {}",
|
||||
errors.size());
|
||||
errors().size());
|
||||
final StringBuilder result = new StringBuilder(
|
||||
errors.size() * 256);
|
||||
errors().size() * 256);
|
||||
result.append(message).append(": ");
|
||||
String exitCode = "";
|
||||
for (MultiObjectDeleteException.DeleteError error :
|
||||
deleteException.getErrors()) {
|
||||
String code = error.getCode();
|
||||
String item = String.format("%s: %s%s: %s%n", code, error.getKey(),
|
||||
(error.getVersionId() != null
|
||||
? (" (" + error.getVersionId() + ")")
|
||||
for (S3Error error : errors()) {
|
||||
String code = error.code();
|
||||
String item = String.format("%s: %s%s: %s%n", code, error.key(),
|
||||
(error.versionId() != null
|
||||
? (" (" + error.versionId() + ")")
|
||||
: ""),
|
||||
error.getMessage());
|
||||
error.message());
|
||||
LOG.info(item);
|
||||
result.append(item);
|
||||
if (exitCode == null || exitCode.isEmpty() || ACCESS_DENIED.equals(code)) {
|
||||
|
@ -89,9 +97,9 @@ public final class MultiObjectDeleteSupport {
|
|||
}
|
||||
if (ACCESS_DENIED.equals(exitCode)) {
|
||||
return (IOException) new AccessDeniedException(result.toString())
|
||||
.initCause(deleteException);
|
||||
.initCause(this);
|
||||
} else {
|
||||
return new AWSS3IOException(result.toString(), deleteException);
|
||||
return new AWSS3IOException(result.toString(), this);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -23,13 +23,14 @@ import java.util.ListIterator;
|
|||
import java.util.NoSuchElementException;
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import com.amazonaws.services.s3.AmazonS3;
|
||||
import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
|
||||
import com.amazonaws.services.s3.model.MultipartUpload;
|
||||
import com.amazonaws.services.s3.model.MultipartUploadListing;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import software.amazon.awssdk.services.s3.S3Client;
|
||||
import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest;
|
||||
import software.amazon.awssdk.services.s3.model.ListMultipartUploadsResponse;
|
||||
import software.amazon.awssdk.services.s3.model.MultipartUpload;
|
||||
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.s3a.api.RequestFactory;
|
||||
import org.apache.hadoop.fs.s3a.impl.StoreContext;
|
||||
|
@ -43,7 +44,7 @@ import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDura
|
|||
* MultipartUtils upload-specific functions for use by S3AFileSystem and Hadoop
|
||||
* CLI.
|
||||
* The Audit span active when
|
||||
* {@link #listMultipartUploads(StoreContext, AmazonS3, String, int)}
|
||||
* {@link #listMultipartUploads(StoreContext, S3Client, String, int)}
|
||||
* was invoked is retained for all subsequent operations.
|
||||
*/
|
||||
public final class MultipartUtils {
|
||||
|
@ -67,7 +68,7 @@ public final class MultipartUtils {
|
|||
*/
|
||||
static MultipartUtils.UploadIterator listMultipartUploads(
|
||||
final StoreContext storeContext,
|
||||
AmazonS3 s3,
|
||||
S3Client s3,
|
||||
@Nullable String prefix,
|
||||
int maxKeys)
|
||||
throws IOException {
|
||||
|
@ -84,14 +85,14 @@ public final class MultipartUtils {
|
|||
* at the time the iterator was constructed.
|
||||
*/
|
||||
static class ListingIterator implements
|
||||
RemoteIterator<MultipartUploadListing> {
|
||||
RemoteIterator<ListMultipartUploadsResponse> {
|
||||
|
||||
private final String prefix;
|
||||
|
||||
private final RequestFactory requestFactory;
|
||||
|
||||
private final int maxKeys;
|
||||
private final AmazonS3 s3;
|
||||
private final S3Client s3;
|
||||
private final Invoker invoker;
|
||||
|
||||
private final AuditSpan auditSpan;
|
||||
|
@ -101,7 +102,7 @@ public final class MultipartUtils {
|
|||
/**
|
||||
* Most recent listing results.
|
||||
*/
|
||||
private MultipartUploadListing listing;
|
||||
private ListMultipartUploadsResponse listing;
|
||||
|
||||
/**
|
||||
* Indicator that this is the first listing.
|
||||
|
@ -114,7 +115,7 @@ public final class MultipartUtils {
|
|||
private int listCount = 0;
|
||||
|
||||
ListingIterator(final StoreContext storeContext,
|
||||
AmazonS3 s3,
|
||||
S3Client s3,
|
||||
@Nullable String prefix,
|
||||
int maxKeys) throws IOException {
|
||||
this.storeContext = storeContext;
|
||||
|
@ -153,7 +154,7 @@ public final class MultipartUtils {
|
|||
*/
|
||||
@Override
|
||||
@Retries.RetryTranslated
|
||||
public MultipartUploadListing next() throws IOException {
|
||||
public ListMultipartUploadsResponse next() throws IOException {
|
||||
if (firstListing) {
|
||||
firstListing = false;
|
||||
} else {
|
||||
|
@ -171,32 +172,34 @@ public final class MultipartUtils {
|
|||
public String toString() {
|
||||
return "Upload iterator: prefix " + prefix
|
||||
+ "; list count " + listCount
|
||||
+ "; upload count " + listing.getMultipartUploads().size()
|
||||
+ "; upload count " + listing.uploads().size()
|
||||
+ "; isTruncated=" + listing.isTruncated();
|
||||
}
|
||||
|
||||
@Retries.RetryTranslated
|
||||
private void requestNextBatch() throws IOException {
|
||||
try (AuditSpan span = auditSpan.activate()) {
|
||||
ListMultipartUploadsRequest req = requestFactory
|
||||
.newListMultipartUploadsRequest(prefix);
|
||||
ListMultipartUploadsRequest.Builder requestBuilder = requestFactory
|
||||
.newListMultipartUploadsRequestBuilder(prefix);
|
||||
if (!firstListing) {
|
||||
req.setKeyMarker(listing.getNextKeyMarker());
|
||||
req.setUploadIdMarker(listing.getNextUploadIdMarker());
|
||||
requestBuilder.keyMarker(listing.nextKeyMarker());
|
||||
requestBuilder.uploadIdMarker(listing.nextUploadIdMarker());
|
||||
}
|
||||
req.setMaxUploads(maxKeys);
|
||||
requestBuilder.maxUploads(maxKeys);
|
||||
|
||||
ListMultipartUploadsRequest request = requestBuilder.build();
|
||||
|
||||
LOG.debug("[{}], Requesting next {} uploads prefix {}, " +
|
||||
"next key {}, next upload id {}", listCount, maxKeys, prefix,
|
||||
req.getKeyMarker(), req.getUploadIdMarker());
|
||||
request.keyMarker(), request.uploadIdMarker());
|
||||
listCount++;
|
||||
|
||||
listing = invoker.retry("listMultipartUploads", prefix, true,
|
||||
trackDurationOfOperation(storeContext.getInstrumentation(),
|
||||
MULTIPART_UPLOAD_LIST.getSymbol(),
|
||||
() -> s3.listMultipartUploads(req)));
|
||||
() -> s3.listMultipartUploads(requestBuilder.build())));
|
||||
LOG.debug("Listing found {} upload(s)",
|
||||
listing.getMultipartUploads().size());
|
||||
listing.uploads().size());
|
||||
LOG.debug("New listing state: {}", this);
|
||||
}
|
||||
}
|
||||
|
@ -216,14 +219,14 @@ public final class MultipartUtils {
|
|||
*/
|
||||
private ListingIterator lister;
|
||||
/** Current listing: the last upload listing we fetched. */
|
||||
private MultipartUploadListing listing;
|
||||
private ListMultipartUploadsResponse listing;
|
||||
/** Iterator over the current listing. */
|
||||
private ListIterator<MultipartUpload> batchIterator;
|
||||
|
||||
@Retries.RetryTranslated
|
||||
public UploadIterator(
|
||||
final StoreContext storeContext,
|
||||
AmazonS3 s3,
|
||||
S3Client s3,
|
||||
int maxKeys,
|
||||
@Nullable String prefix)
|
||||
throws IOException {
|
||||
|
@ -249,7 +252,7 @@ public final class MultipartUtils {
|
|||
private boolean requestNextBatch() throws IOException {
|
||||
if (lister.hasNext()) {
|
||||
listing = lister.next();
|
||||
batchIterator = listing.getMultipartUploads().listIterator();
|
||||
batchIterator = listing.uploads().listIterator();
|
||||
return batchIterator.hasNext();
|
||||
}
|
||||
return false;
|
||||
|
|
|
@ -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) {
|
||||
public void transferInitiated(TransferListener.Context.TransferInitiated context) {
|
||||
fs.incrementWriteOperations();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void transferComplete(TransferListener.Context.TransferComplete context) {
|
||||
fs.incrementWriteOperations();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void bytesTransferred(TransferListener.Context.BytesTransferred context) {
|
||||
|
||||
if(progress != null) {
|
||||
progress.progress();
|
||||
}
|
||||
|
||||
// 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();
|
||||
}
|
||||
|
||||
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 class ProgressableProgressListener implements ProgressListener {
|
|||
* 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);
|
||||
|
|
|
@ -31,14 +31,9 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import com.amazonaws.SdkBaseException;
|
||||
import com.amazonaws.event.ProgressEvent;
|
||||
import com.amazonaws.event.ProgressEventType;
|
||||
import com.amazonaws.event.ProgressListener;
|
||||
import com.amazonaws.services.s3.model.PartETag;
|
||||
import com.amazonaws.services.s3.model.PutObjectRequest;
|
||||
import com.amazonaws.services.s3.model.PutObjectResult;
|
||||
import com.amazonaws.services.s3.model.UploadPartRequest;
|
||||
|
||||
import org.apache.hadoop.fs.s3a.impl.PutObjectOptions;
|
||||
import org.apache.hadoop.fs.statistics.IOStatisticsAggregator;
|
||||
|
@ -50,6 +45,14 @@ import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecut
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import software.amazon.awssdk.core.exception.SdkException;
|
||||
import software.amazon.awssdk.core.sync.RequestBody;
|
||||
import software.amazon.awssdk.services.s3.model.CompletedPart;
|
||||
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.PutObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.Abortable;
|
||||
|
@ -441,7 +444,7 @@ class S3ABlockOutputStream extends OutputStream implements
|
|||
uploadCurrentBlock(true);
|
||||
}
|
||||
// wait for the partial uploads to finish
|
||||
final List<PartETag> partETags =
|
||||
final List<CompletedPart> partETags =
|
||||
multiPartUpload.waitForAllPartUploads();
|
||||
bytes = bytesSubmitted;
|
||||
|
||||
|
@ -599,27 +602,30 @@ class S3ABlockOutputStream extends OutputStream implements
|
|||
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> putObjectResult =
|
||||
ListenableFuture<PutObjectResponse> putObjectResult =
|
||||
executorService.submit(() -> {
|
||||
try {
|
||||
// the putObject call automatically closes the input
|
||||
// stream afterwards.
|
||||
return writeOperationHelper.putObject(
|
||||
putObjectRequest,
|
||||
builder.putOptions,
|
||||
statistics);
|
||||
return writeOperationHelper.putObject(putObjectRequest, builder.putOptions, uploadData,
|
||||
uploadData.hasFile(), statistics);
|
||||
} finally {
|
||||
cleanupWithLogger(LOG, uploadData, block);
|
||||
}
|
||||
|
@ -770,7 +776,7 @@ class S3ABlockOutputStream extends OutputStream implements
|
|||
*/
|
||||
private class MultiPartUpload {
|
||||
private final String uploadId;
|
||||
private final List<ListenableFuture<PartETag>> partETagsFutures;
|
||||
private final List<ListenableFuture<CompletedPart>> partETagsFutures;
|
||||
private int partsSubmitted;
|
||||
private int partsUploaded;
|
||||
private long bytesSubmitted;
|
||||
|
@ -875,18 +881,19 @@ class S3ABlockOutputStream extends OutputStream implements
|
|||
final int currentPartNumber = partETagsFutures.size() + 1;
|
||||
final UploadPartRequest request;
|
||||
final S3ADataBlocks.BlockUploadData uploadData;
|
||||
final RequestBody requestBody;
|
||||
try {
|
||||
uploadData = block.startUpload();
|
||||
request = writeOperationHelper.newUploadPartRequest(
|
||||
requestBody = uploadData.hasFile()
|
||||
? RequestBody.fromFile(uploadData.getFile())
|
||||
: RequestBody.fromInputStream(uploadData.getUploadStream(), size);
|
||||
|
||||
request = writeOperationHelper.newUploadPartRequestBuilder(
|
||||
key,
|
||||
uploadId,
|
||||
currentPartNumber,
|
||||
size,
|
||||
uploadData.getUploadStream(),
|
||||
uploadData.getFile(),
|
||||
0L);
|
||||
request.setLastPart(isLast);
|
||||
} catch (SdkBaseException aws) {
|
||||
size).build();
|
||||
} catch (SdkException aws) {
|
||||
// catch and translate
|
||||
IOException e = translateException("upload", key, aws);
|
||||
// failure to start the upload.
|
||||
|
@ -897,25 +904,32 @@ class S3ABlockOutputStream extends OutputStream implements
|
|||
noteUploadFailure(e);
|
||||
throw e;
|
||||
}
|
||||
BlockUploadProgress callback =
|
||||
new BlockUploadProgress(
|
||||
block, progressListener, now());
|
||||
request.setGeneralProgressListener(callback);
|
||||
|
||||
// TODO: You cannot currently add progress listeners to requests not via the TM.
|
||||
// See also putObject
|
||||
// BlockUploadProgress callback =
|
||||
// new BlockUploadProgress(
|
||||
// block, progressListener, now());
|
||||
// request.setGeneralProgressListener(callback);
|
||||
|
||||
statistics.blockUploadQueued(block.dataSize());
|
||||
ListenableFuture<PartETag> partETagFuture =
|
||||
ListenableFuture<CompletedPart> partETagFuture =
|
||||
executorService.submit(() -> {
|
||||
// this is the queued upload operation
|
||||
// do the upload
|
||||
try {
|
||||
LOG.debug("Uploading part {} for id '{}'",
|
||||
currentPartNumber, uploadId);
|
||||
PartETag partETag = writeOperationHelper.uploadPart(request, statistics)
|
||||
.getPartETag();
|
||||
UploadPartResponse response = writeOperationHelper
|
||||
.uploadPart(request, requestBody, statistics);
|
||||
LOG.debug("Completed upload of {} to part {}",
|
||||
block, partETag.getETag());
|
||||
block, response.eTag());
|
||||
LOG.debug("Stream statistics of {}", statistics);
|
||||
partsUploaded++;
|
||||
return partETag;
|
||||
return CompletedPart.builder()
|
||||
.eTag(response.eTag())
|
||||
.partNumber(currentPartNumber)
|
||||
.build();
|
||||
} catch (IOException e) {
|
||||
// save immediately.
|
||||
noteUploadFailure(e);
|
||||
|
@ -933,7 +947,7 @@ class S3ABlockOutputStream extends OutputStream implements
|
|||
* @return list of results
|
||||
* @throws IOException IO Problems
|
||||
*/
|
||||
private List<PartETag> waitForAllPartUploads() throws IOException {
|
||||
private List<CompletedPart> waitForAllPartUploads() throws IOException {
|
||||
LOG.debug("Waiting for {} uploads to complete", partETagsFutures.size());
|
||||
try {
|
||||
return Futures.allAsList(partETagsFutures).get();
|
||||
|
@ -957,7 +971,7 @@ class S3ABlockOutputStream extends OutputStream implements
|
|||
*/
|
||||
private void cancelAllActiveFutures() {
|
||||
LOG.debug("Cancelling futures");
|
||||
for (ListenableFuture<PartETag> future : partETagsFutures) {
|
||||
for (ListenableFuture<CompletedPart> future : partETagsFutures) {
|
||||
future.cancel(true);
|
||||
}
|
||||
}
|
||||
|
@ -969,7 +983,7 @@ class S3ABlockOutputStream extends OutputStream implements
|
|||
* @param partETags list of partial uploads
|
||||
* @throws IOException on any problem
|
||||
*/
|
||||
private void complete(List<PartETag> partETags)
|
||||
private void complete(List<CompletedPart> partETags)
|
||||
throws IOException {
|
||||
maybeRethrowUploadFailure();
|
||||
AtomicInteger errorCount = new AtomicInteger(0);
|
||||
|
|
|
@ -47,7 +47,7 @@ import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
|||
* 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 @@ final class S3ADataBlocks {
|
|||
* 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 @@ final class S3ADataBlocks {
|
|||
* 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 @@ final class S3ADataBlocks {
|
|||
* 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;
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -22,6 +22,7 @@ import javax.annotation.Nullable;
|
|||
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.ExecutorService;
|
|||
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.CanUnbuffer;
|
|||
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.fs.statistics.IOStatisticsSource;
|
|||
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<GetObjectResponse> wrappedStream;
|
||||
private final S3AReadOpContext context;
|
||||
private final InputStreamCallbacks client;
|
||||
|
||||
|
@ -271,28 +267,22 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
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 class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
*/
|
||||
@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 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
CompletableFuture<Boolean> operation;
|
||||
SDKStreamDrainer drainer = new SDKStreamDrainer(
|
||||
uri,
|
||||
object,
|
||||
wrappedStream,
|
||||
shouldAbort,
|
||||
(int) remaining,
|
||||
|
@ -694,7 +684,6 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
// either the stream is closed in the blocking call or the async call is
|
||||
// submitted with its own copy of the references
|
||||
wrappedStream = null;
|
||||
object = null;
|
||||
return operation;
|
||||
}
|
||||
|
||||
|
@ -910,23 +899,19 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRange,
|
||||
IntFunction<ByteBuffer> allocate) {
|
||||
LOG.debug("Start reading combined range {} from path {} ", combinedFileRange, pathStr);
|
||||
// This reference must be kept till all buffers are populated as this is a
|
||||
// finalizable object which closes the internal stream when gc triggers.
|
||||
S3Object objectRange = null;
|
||||
S3ObjectInputStream objectContent = null;
|
||||
ResponseInputStream<GetObjectResponse> rangeContent = null;
|
||||
try {
|
||||
objectRange = getS3ObjectAndValidateNotNull("readCombinedFileRange",
|
||||
rangeContent = getS3ObjectInputStream("readCombinedFileRange",
|
||||
combinedFileRange.getOffset(),
|
||||
combinedFileRange.getLength());
|
||||
objectContent = objectRange.getObjectContent();
|
||||
populateChildBuffers(combinedFileRange, objectContent, allocate);
|
||||
populateChildBuffers(combinedFileRange, rangeContent, allocate);
|
||||
} catch (Exception ex) {
|
||||
LOG.debug("Exception while reading a range {} from path {} ", combinedFileRange, pathStr, ex);
|
||||
for(FileRange child : combinedFileRange.getUnderlying()) {
|
||||
child.getData().completeExceptionally(ex);
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanupWithLogger(LOG, objectRange, objectContent);
|
||||
IOUtils.cleanupWithLogger(LOG, rangeContent);
|
||||
}
|
||||
LOG.debug("Finished reading range {} from path {} ", combinedFileRange, pathStr);
|
||||
}
|
||||
|
@ -939,7 +924,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
* @throws IOException any IOE.
|
||||
*/
|
||||
private void populateChildBuffers(CombinedFileRange combinedFileRange,
|
||||
S3ObjectInputStream objectContent,
|
||||
InputStream objectContent,
|
||||
IntFunction<ByteBuffer> allocate) throws IOException {
|
||||
// If the combined file range just contains a single child
|
||||
// range, we only have to fill that one child buffer else
|
||||
|
@ -971,7 +956,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
* @param drainQuantity how many bytes to drain.
|
||||
* @throws IOException any IOE.
|
||||
*/
|
||||
private void drainUnnecessaryData(S3ObjectInputStream objectContent, long drainQuantity)
|
||||
private void drainUnnecessaryData(InputStream objectContent, long drainQuantity)
|
||||
throws IOException {
|
||||
int drainBytes = 0;
|
||||
int readCount;
|
||||
|
@ -1013,28 +998,24 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
*/
|
||||
private void readSingleRange(FileRange range, ByteBuffer buffer) {
|
||||
LOG.debug("Start reading range {} from path {} ", range, pathStr);
|
||||
// This reference must be kept till all buffers are populated as this is a
|
||||
// finalizable object which closes the internal stream when gc triggers.
|
||||
S3Object objectRange = null;
|
||||
S3ObjectInputStream objectContent = null;
|
||||
ResponseInputStream<GetObjectResponse> objectRange = null;
|
||||
try {
|
||||
long position = range.getOffset();
|
||||
int length = range.getLength();
|
||||
objectRange = getS3ObjectAndValidateNotNull("readSingleRange", position, length);
|
||||
objectContent = objectRange.getObjectContent();
|
||||
populateBuffer(length, buffer, objectContent);
|
||||
objectRange = getS3ObjectInputStream("readSingleRange", position, length);
|
||||
populateBuffer(length, buffer, objectRange);
|
||||
range.getData().complete(buffer);
|
||||
} catch (Exception ex) {
|
||||
LOG.warn("Exception while reading a range {} from path {} ", range, pathStr, ex);
|
||||
range.getData().completeExceptionally(ex);
|
||||
} finally {
|
||||
IOUtils.cleanupWithLogger(LOG, objectRange, objectContent);
|
||||
IOUtils.cleanupWithLogger(LOG, objectRange);
|
||||
}
|
||||
LOG.debug("Finished reading range {} from path {} ", range, pathStr);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the s3 object for S3 server for a specified range.
|
||||
* Get the s3 object input stream for S3 server for a specified range.
|
||||
* Also checks if the vectored io operation has been stopped before and after
|
||||
* the http get request such that we don't waste time populating the buffers.
|
||||
* @param operationName name of the operation for which get object on S3 is called.
|
||||
|
@ -1043,15 +1024,11 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
* @return result s3 object.
|
||||
* @throws IOException exception if any.
|
||||
*/
|
||||
private S3Object getS3ObjectAndValidateNotNull(final String operationName,
|
||||
final long position,
|
||||
final int length) throws IOException {
|
||||
private ResponseInputStream<GetObjectResponse> getS3ObjectInputStream(
|
||||
final String operationName, final long position, final int length) throws IOException {
|
||||
checkIfVectoredIOStopped();
|
||||
S3Object objectRange = getS3Object(operationName, position, length);
|
||||
if (objectRange.getObjectContent() == null) {
|
||||
throw new PathIOException(uri,
|
||||
"Null IO stream received during " + operationName);
|
||||
}
|
||||
ResponseInputStream<GetObjectResponse> objectRange =
|
||||
getS3Object(operationName, position, length);
|
||||
checkIfVectoredIOStopped();
|
||||
return objectRange;
|
||||
}
|
||||
|
@ -1066,7 +1043,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
*/
|
||||
private void populateBuffer(int length,
|
||||
ByteBuffer buffer,
|
||||
S3ObjectInputStream objectContent) throws IOException {
|
||||
InputStream objectContent) throws IOException {
|
||||
|
||||
if (buffer.isDirect()) {
|
||||
VectoredReadUtils.readInDirectBuffer(length, buffer,
|
||||
|
@ -1091,7 +1068,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
* @param length number of bytes to fill in dest.
|
||||
* @throws IOException any IOE.
|
||||
*/
|
||||
private void readByteArray(S3ObjectInputStream objectContent,
|
||||
private void readByteArray(InputStream objectContent,
|
||||
byte[] dest,
|
||||
int offset,
|
||||
int length) throws IOException {
|
||||
|
@ -1118,13 +1095,16 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
* @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<GetObjectResponse> 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<GetObjectResponse> objectRange;
|
||||
Invoker invoker = context.getReadInvoker();
|
||||
try {
|
||||
objectRange = invoker.retry(operationName, pathStr, true,
|
||||
|
@ -1139,7 +1119,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
} finally {
|
||||
tracker.close();
|
||||
}
|
||||
changeTracker.processResponse(objectRange, operationName,
|
||||
changeTracker.processResponse(objectRange.response(), operationName,
|
||||
position);
|
||||
return objectRange;
|
||||
}
|
||||
|
@ -1293,11 +1273,11 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
public interface InputStreamCallbacks extends Closeable {
|
||||
|
||||
/**
|
||||
* Create a GET request.
|
||||
* Create a GET request builder.
|
||||
* @param key object key
|
||||
* @return the request
|
||||
* @return the request builder
|
||||
*/
|
||||
GetObjectRequest newGetRequest(String key);
|
||||
GetObjectRequest.Builder newGetRequestBuilder(String key);
|
||||
|
||||
/**
|
||||
* Execute the request.
|
||||
|
@ -1305,7 +1285,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
* @return the response
|
||||
*/
|
||||
@Retries.OnceRaw
|
||||
S3Object getObject(GetObjectRequest request);
|
||||
ResponseInputStream<GetObjectResponse> getObject(GetObjectRequest request);
|
||||
|
||||
/**
|
||||
* Submit some asynchronous work, for example, draining a stream.
|
||||
|
|
|
@ -30,7 +30,6 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import com.amazonaws.AmazonClientException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -44,6 +43,8 @@ import org.apache.hadoop.io.retry.RetryPolicy;
|
|||
import org.apache.hadoop.net.ConnectTimeoutException;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
|
||||
import software.amazon.awssdk.core.exception.SdkException;
|
||||
|
||||
import static org.apache.hadoop.io.retry.RetryPolicies.*;
|
||||
|
||||
import static org.apache.hadoop.fs.s3a.Constants.*;
|
||||
|
@ -68,9 +69,9 @@ import static org.apache.hadoop.fs.s3a.Constants.*;
|
|||
*
|
||||
* The retry policy is all built around that of the normal IO exceptions,
|
||||
* particularly those extracted from
|
||||
* {@link S3AUtils#translateException(String, Path, AmazonClientException)}.
|
||||
* {@link S3AUtils#translateException(String, Path, SdkException)}.
|
||||
* Because the {@link #shouldRetry(Exception, int, int, boolean)} method
|
||||
* does this translation if an {@code AmazonClientException} is processed,
|
||||
* does this translation if an {@code SdkException} is processed,
|
||||
* the policy defined for the IOEs also applies to the original exceptions.
|
||||
*
|
||||
* Put differently: this retry policy aims to work for handlers of the
|
||||
|
@ -242,11 +243,10 @@ public class S3ARetryPolicy implements RetryPolicy {
|
|||
boolean idempotent) throws Exception {
|
||||
Preconditions.checkArgument(exception != null, "Null exception");
|
||||
Exception ex = exception;
|
||||
if (exception instanceof AmazonClientException) {
|
||||
// uprate the amazon client exception for the purpose of exception
|
||||
if (exception instanceof SdkException) {
|
||||
// update the sdk exception for the purpose of exception
|
||||
// processing.
|
||||
ex = S3AUtils.translateException("", "",
|
||||
(AmazonClientException) exception);
|
||||
ex = S3AUtils.translateException("", "", (SdkException) exception);
|
||||
}
|
||||
return retryPolicy.shouldRetry(ex, retries, failovers, idempotent);
|
||||
}
|
||||
|
|
|
@ -18,22 +18,12 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.AbortedException;
|
||||
import com.amazonaws.AmazonClientException;
|
||||
import com.amazonaws.AmazonServiceException;
|
||||
import com.amazonaws.ClientConfiguration;
|
||||
import com.amazonaws.Protocol;
|
||||
import com.amazonaws.SdkBaseException;
|
||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||
import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
|
||||
import com.amazonaws.retry.RetryUtils;
|
||||
import com.amazonaws.services.s3.model.AmazonS3Exception;
|
||||
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
|
||||
import com.amazonaws.services.s3.model.S3ObjectSummary;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -49,16 +39,26 @@ import org.apache.hadoop.fs.s3a.audit.AuditIntegration;
|
|||
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
|
||||
import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider;
|
||||
import org.apache.hadoop.fs.s3a.impl.NetworkBinding;
|
||||
import org.apache.hadoop.fs.s3a.impl.V2Migration;
|
||||
import org.apache.hadoop.fs.s3native.S3xLoginHelper;
|
||||
import org.apache.hadoop.net.ConnectTimeoutException;
|
||||
import org.apache.hadoop.security.ProviderUtils;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
import org.apache.hadoop.util.VersionInfo;
|
||||
|
||||
import org.apache.hadoop.util.Lists;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
|
||||
import software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider;
|
||||
import software.amazon.awssdk.awscore.exception.AwsServiceException;
|
||||
import software.amazon.awssdk.core.exception.AbortedException;
|
||||
import software.amazon.awssdk.core.exception.SdkException;
|
||||
import software.amazon.awssdk.core.retry.RetryUtils;
|
||||
import software.amazon.awssdk.services.s3.model.S3Exception;
|
||||
import software.amazon.awssdk.services.s3.model.S3Object;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
import java.io.EOFException;
|
||||
|
@ -78,19 +78,20 @@ import java.util.Arrays;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CompletionException;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.apache.commons.lang3.StringUtils.isEmpty;
|
||||
import static org.apache.hadoop.fs.s3a.Constants.*;
|
||||
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket;
|
||||
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH;
|
||||
import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.translateDeleteException;
|
||||
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.*;
|
||||
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
||||
import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator;
|
||||
|
||||
|
@ -109,6 +110,8 @@ public final class S3AUtils {
|
|||
= "instantiation exception";
|
||||
static final String NOT_AWS_PROVIDER =
|
||||
"does not implement AWSCredentialsProvider";
|
||||
static final String NOT_AWS_V2_PROVIDER =
|
||||
"does not implement AwsCredentialsProvider";
|
||||
static final String ABSTRACT_PROVIDER =
|
||||
"is abstract and therefore cannot be created";
|
||||
static final String ENDPOINT_KEY = "Endpoint";
|
||||
|
@ -159,7 +162,7 @@ public final class S3AUtils {
|
|||
/**
|
||||
* Translate an exception raised in an operation into an IOException.
|
||||
* The specific type of IOException depends on the class of
|
||||
* {@link AmazonClientException} passed in, and any status codes included
|
||||
* {@link SdkException} passed in, and any status codes included
|
||||
* in the operation. That is: HTTP error codes are examined and can be
|
||||
* used to build a more specific response.
|
||||
*
|
||||
|
@ -172,14 +175,14 @@ public final class S3AUtils {
|
|||
*/
|
||||
public static IOException translateException(String operation,
|
||||
Path path,
|
||||
AmazonClientException exception) {
|
||||
SdkException exception) {
|
||||
return translateException(operation, path.toString(), exception);
|
||||
}
|
||||
|
||||
/**
|
||||
* Translate an exception raised in an operation into an IOException.
|
||||
* The specific type of IOException depends on the class of
|
||||
* {@link AmazonClientException} passed in, and any status codes included
|
||||
* {@link SdkException} passed in, and any status codes included
|
||||
* in the operation. That is: HTTP error codes are examined and can be
|
||||
* used to build a more specific response.
|
||||
* @param operation operation
|
||||
|
@ -190,12 +193,12 @@ public final class S3AUtils {
|
|||
@SuppressWarnings("ThrowableInstanceNeverThrown")
|
||||
public static IOException translateException(@Nullable String operation,
|
||||
String path,
|
||||
SdkBaseException exception) {
|
||||
SdkException exception) {
|
||||
String message = String.format("%s%s: %s",
|
||||
operation,
|
||||
StringUtils.isNotEmpty(path)? (" on " + path) : "",
|
||||
exception);
|
||||
if (!(exception instanceof AmazonServiceException)) {
|
||||
if (!(exception instanceof AwsServiceException)) {
|
||||
Exception innerCause = containsInterruptedException(exception);
|
||||
if (innerCause != null) {
|
||||
// interrupted IO, or a socket exception underneath that class
|
||||
|
@ -219,45 +222,49 @@ public final class S3AUtils {
|
|||
return new AWSClientIOException(message, exception);
|
||||
} else {
|
||||
IOException ioe;
|
||||
AmazonServiceException ase = (AmazonServiceException) exception;
|
||||
AwsServiceException ase = (AwsServiceException) exception;
|
||||
// this exception is non-null if the service exception is an s3 one
|
||||
AmazonS3Exception s3Exception = ase instanceof AmazonS3Exception
|
||||
? (AmazonS3Exception) ase
|
||||
S3Exception s3Exception = ase instanceof S3Exception
|
||||
? (S3Exception) ase
|
||||
: null;
|
||||
int status = ase.getStatusCode();
|
||||
message = message + ":" + ase.getErrorCode();
|
||||
int status = ase.statusCode();
|
||||
if (ase.awsErrorDetails() != null) {
|
||||
message = message + ":" + ase.awsErrorDetails().errorCode();
|
||||
}
|
||||
switch (status) {
|
||||
|
||||
case 301:
|
||||
case 307:
|
||||
case SC_301_MOVED_PERMANENTLY:
|
||||
case SC_307_TEMPORARY_REDIRECT:
|
||||
if (s3Exception != null) {
|
||||
if (s3Exception.getAdditionalDetails() != null &&
|
||||
s3Exception.getAdditionalDetails().containsKey(ENDPOINT_KEY)) {
|
||||
message = String.format("Received permanent redirect response to "
|
||||
+ "endpoint %s. This likely indicates that the S3 endpoint "
|
||||
+ "configured in %s does not match the AWS region containing "
|
||||
+ "the bucket.",
|
||||
s3Exception.getAdditionalDetails().get(ENDPOINT_KEY), ENDPOINT);
|
||||
}
|
||||
// TODO: Can we get the endpoint in v2?
|
||||
// Maybe not: https://github.com/aws/aws-sdk-java-v2/issues/3048
|
||||
// if (s3Exception.getAdditionalDetails() != null &&
|
||||
// s3Exception.getAdditionalDetails().containsKey(ENDPOINT_KEY)) {
|
||||
// message = String.format("Received permanent redirect response to "
|
||||
// + "endpoint %s. This likely indicates that the S3 endpoint "
|
||||
// + "configured in %s does not match the AWS region containing "
|
||||
// + "the bucket.",
|
||||
// s3Exception.getAdditionalDetails().get(ENDPOINT_KEY), ENDPOINT);
|
||||
// }
|
||||
ioe = new AWSRedirectException(message, s3Exception);
|
||||
} else {
|
||||
ioe = new AWSRedirectException(message, ase);
|
||||
}
|
||||
break;
|
||||
|
||||
case 400:
|
||||
case SC_400_BAD_REQUEST:
|
||||
ioe = new AWSBadRequestException(message, ase);
|
||||
break;
|
||||
|
||||
// permissions
|
||||
case 401:
|
||||
case 403:
|
||||
case SC_401_UNAUTHORIZED:
|
||||
case SC_403_FORBIDDEN:
|
||||
ioe = new AccessDeniedException(path, null, message);
|
||||
ioe.initCause(ase);
|
||||
break;
|
||||
|
||||
// the object isn't there
|
||||
case 404:
|
||||
case SC_404_NOT_FOUND:
|
||||
if (isUnknownBucket(ase)) {
|
||||
// this is a missing bucket
|
||||
ioe = new UnknownStoreException(path, message, ase);
|
||||
|
@ -270,20 +277,20 @@ public final class S3AUtils {
|
|||
|
||||
// this also surfaces sometimes and is considered to
|
||||
// be ~ a not found exception.
|
||||
case 410:
|
||||
case SC_410_GONE:
|
||||
ioe = new FileNotFoundException(message);
|
||||
ioe.initCause(ase);
|
||||
break;
|
||||
|
||||
// method not allowed; seen on S3 Select.
|
||||
// treated as a bad request
|
||||
case 405:
|
||||
case SC_405_METHOD_NOT_ALLOWED:
|
||||
ioe = new AWSBadRequestException(message, s3Exception);
|
||||
break;
|
||||
|
||||
// out of range. This may happen if an object is overwritten with
|
||||
// a shorter one while it is being read.
|
||||
case 416:
|
||||
case SC_416_RANGE_NOT_SATISFIABLE:
|
||||
ioe = new EOFException(message);
|
||||
ioe.initCause(ase);
|
||||
break;
|
||||
|
@ -291,26 +298,26 @@ public final class S3AUtils {
|
|||
// this has surfaced as a "no response from server" message.
|
||||
// so rare we haven't replicated it.
|
||||
// Treating as an idempotent proxy error.
|
||||
case 443:
|
||||
case 444:
|
||||
case SC_443_NO_RESPONSE:
|
||||
case SC_444_NO_RESPONSE:
|
||||
ioe = new AWSNoResponseException(message, ase);
|
||||
break;
|
||||
|
||||
// throttling
|
||||
case 503:
|
||||
case SC_503_SERVICE_UNAVAILABLE:
|
||||
ioe = new AWSServiceThrottledException(message, ase);
|
||||
break;
|
||||
|
||||
// internal error
|
||||
case 500:
|
||||
case SC_500_INTERNAL_SERVER_ERROR:
|
||||
ioe = new AWSStatus500Exception(message, ase);
|
||||
break;
|
||||
|
||||
case 200:
|
||||
case SC_200_OK:
|
||||
if (exception instanceof MultiObjectDeleteException) {
|
||||
// failure during a bulk delete
|
||||
return translateDeleteException(message,
|
||||
(MultiObjectDeleteException) exception);
|
||||
return ((MultiObjectDeleteException) exception)
|
||||
.translateException(message);
|
||||
}
|
||||
// other 200: FALL THROUGH
|
||||
|
||||
|
@ -336,10 +343,35 @@ public final class S3AUtils {
|
|||
public static IOException extractException(String operation,
|
||||
String path,
|
||||
ExecutionException ee) {
|
||||
return convertExceptionCause(operation, path, ee.getCause());
|
||||
}
|
||||
|
||||
/**
|
||||
* Extract an exception from a failed future, and convert to an IOE.
|
||||
* @param operation operation which failed
|
||||
* @param path path operated on (may be null)
|
||||
* @param ce completion exception
|
||||
* @return an IOE which can be thrown
|
||||
*/
|
||||
public static IOException extractException(String operation,
|
||||
String path,
|
||||
CompletionException ce) {
|
||||
return convertExceptionCause(operation, path, ce.getCause());
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert the cause of a concurrent exception to an IOE.
|
||||
* @param operation operation which failed
|
||||
* @param path path operated on (may be null)
|
||||
* @param cause cause of a concurrent exception
|
||||
* @return an IOE which can be thrown
|
||||
*/
|
||||
private static IOException convertExceptionCause(String operation,
|
||||
String path,
|
||||
Throwable cause) {
|
||||
IOException ioe;
|
||||
Throwable cause = ee.getCause();
|
||||
if (cause instanceof AmazonClientException) {
|
||||
ioe = translateException(operation, path, (AmazonClientException) cause);
|
||||
if (cause instanceof SdkException) {
|
||||
ioe = translateException(operation, path, (SdkException) cause);
|
||||
} else if (cause instanceof IOException) {
|
||||
ioe = (IOException) cause;
|
||||
} else {
|
||||
|
@ -377,7 +409,7 @@ public final class S3AUtils {
|
|||
* @return an IOE which can be rethrown
|
||||
*/
|
||||
private static InterruptedIOException translateInterruptedException(
|
||||
SdkBaseException exception,
|
||||
SdkException exception,
|
||||
final Exception innerCause,
|
||||
String message) {
|
||||
InterruptedIOException ioe;
|
||||
|
@ -388,6 +420,7 @@ public final class S3AUtils {
|
|||
if (name.endsWith(".ConnectTimeoutException")
|
||||
|| name.endsWith(".ConnectionPoolTimeoutException")
|
||||
|| name.endsWith("$ConnectTimeoutException")) {
|
||||
// TODO: review in v2
|
||||
// TCP connection http timeout from the shaded or unshaded filenames
|
||||
// com.amazonaws.thirdparty.apache.http.conn.ConnectTimeoutException
|
||||
ioe = new ConnectTimeoutException(message);
|
||||
|
@ -411,10 +444,10 @@ public final class S3AUtils {
|
|||
*/
|
||||
public static boolean isThrottleException(Exception ex) {
|
||||
return ex instanceof AWSServiceThrottledException
|
||||
|| (ex instanceof AmazonServiceException
|
||||
&& 503 == ((AmazonServiceException)ex).getStatusCode())
|
||||
|| (ex instanceof SdkBaseException
|
||||
&& RetryUtils.isThrottlingException((SdkBaseException) ex));
|
||||
|| (ex instanceof AwsServiceException
|
||||
&& 503 == ((AwsServiceException)ex).statusCode())
|
||||
|| (ex instanceof SdkException
|
||||
&& RetryUtils.isThrottlingException((SdkException) ex));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -424,7 +457,8 @@ public final class S3AUtils {
|
|||
* @param ex exception
|
||||
* @return true if this is believed to be a sign the connection was broken.
|
||||
*/
|
||||
public static boolean isMessageTranslatableToEOF(SdkBaseException ex) {
|
||||
public static boolean isMessageTranslatableToEOF(SdkException ex) {
|
||||
// TODO: review in v2
|
||||
return ex.toString().contains(EOF_MESSAGE_IN_XML_PARSER) ||
|
||||
ex.toString().contains(EOF_READ_DIFFERENT_LENGTH);
|
||||
}
|
||||
|
@ -434,47 +468,26 @@ public final class S3AUtils {
|
|||
* @param e exception
|
||||
* @return string details
|
||||
*/
|
||||
public static String stringify(AmazonServiceException e) {
|
||||
public static String stringify(AwsServiceException e) {
|
||||
StringBuilder builder = new StringBuilder(
|
||||
String.format("%s: %s error %d: %s; %s%s%n",
|
||||
e.getErrorType(),
|
||||
e.getServiceName(),
|
||||
e.getStatusCode(),
|
||||
e.getErrorCode(),
|
||||
e.getErrorMessage(),
|
||||
(e.isRetryable() ? " (retryable)": "")
|
||||
String.format("%s error %d: %s; %s%s%n",
|
||||
e.awsErrorDetails().serviceName(),
|
||||
e.statusCode(),
|
||||
e.awsErrorDetails().errorCode(),
|
||||
e.awsErrorDetails().errorMessage(),
|
||||
(e.retryable() ? " (retryable)": "")
|
||||
));
|
||||
String rawResponseContent = e.getRawResponseContent();
|
||||
String rawResponseContent = e.awsErrorDetails().rawResponse().asUtf8String();
|
||||
if (rawResponseContent != null) {
|
||||
builder.append(rawResponseContent);
|
||||
}
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get low level details of an amazon exception for logging; multi-line.
|
||||
* @param e exception
|
||||
* @return string details
|
||||
*/
|
||||
public static String stringify(AmazonS3Exception e) {
|
||||
// get the low level details of an exception,
|
||||
StringBuilder builder = new StringBuilder(
|
||||
stringify((AmazonServiceException) e));
|
||||
Map<String, String> details = e.getAdditionalDetails();
|
||||
if (details != null) {
|
||||
builder.append('\n');
|
||||
for (Map.Entry<String, String> d : details.entrySet()) {
|
||||
builder.append(d.getKey()).append('=')
|
||||
.append(d.getValue()).append('\n');
|
||||
}
|
||||
}
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a files status instance from a listing.
|
||||
* @param keyPath path to entry
|
||||
* @param summary summary from AWS
|
||||
* @param s3Object s3Object entry
|
||||
* @param blockSize block size to declare.
|
||||
* @param owner owner of the file
|
||||
* @param eTag S3 object eTag or null if unavailable
|
||||
|
@ -483,20 +496,20 @@ public final class S3AUtils {
|
|||
* @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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -560,7 +573,6 @@ public final class S3AUtils {
|
|||
/**
|
||||
* The standard AWS provider list for AWS connections.
|
||||
*/
|
||||
@SuppressWarnings("deprecation")
|
||||
public static final List<Class<?>>
|
||||
STANDARD_AWS_PROVIDERS = Collections.unmodifiableList(
|
||||
Arrays.asList(
|
||||
|
@ -614,6 +626,22 @@ public final class S3AUtils {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Maps V1 credential providers to either their equivalent SDK V2 class or hadoop provider.
|
||||
*/
|
||||
private static Map<String, Class> initCredentialProvidersMap() {
|
||||
Map<String, Class> v1v2CredentialProviderMap = new HashMap<>();
|
||||
|
||||
v1v2CredentialProviderMap.put("EnvironmentVariableCredentialsProvider",
|
||||
EnvironmentVariableCredentialsProvider.class);
|
||||
v1v2CredentialProviderMap.put("EC2ContainerCredentialsProviderWrapper",
|
||||
IAMInstanceCredentialsProvider.class);
|
||||
v1v2CredentialProviderMap.put("InstanceProfileCredentialsProvider",
|
||||
IAMInstanceCredentialsProvider.class);
|
||||
|
||||
return v1v2CredentialProviderMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Load list of AWS credential provider/credential provider factory classes;
|
||||
* support a forbidden list to prevent loops, mandate full secrets, etc.
|
||||
|
@ -636,6 +664,8 @@ public final class S3AUtils {
|
|||
List<Class<?>> awsClasses = loadAWSProviderClasses(conf,
|
||||
key,
|
||||
defaultValues.toArray(new Class[defaultValues.size()]));
|
||||
|
||||
Map<String, Class> v1v2CredentialProviderMap = initCredentialProvidersMap();
|
||||
// and if the list is empty, switch back to the defaults.
|
||||
// this is to address the issue that configuration.getClasses()
|
||||
// doesn't return the default if the config value is just whitespace.
|
||||
|
@ -647,19 +677,22 @@ public final class S3AUtils {
|
|||
AWSCredentialProviderList providers = new AWSCredentialProviderList();
|
||||
for (Class<?> aClass : awsClasses) {
|
||||
|
||||
// List of V1 credential providers that will be migrated with V2 upgrade
|
||||
if (!Arrays.asList("EnvironmentVariableCredentialsProvider",
|
||||
"EC2ContainerCredentialsProviderWrapper", "InstanceProfileCredentialsProvider")
|
||||
.contains(aClass.getSimpleName()) && aClass.getName().contains(AWS_AUTH_CLASS_PREFIX)) {
|
||||
V2Migration.v1ProviderReferenced(aClass.getName());
|
||||
}
|
||||
|
||||
if (forbidden.contains(aClass)) {
|
||||
throw new IOException(E_FORBIDDEN_AWS_PROVIDER
|
||||
+ " in option " + key + ": " + aClass);
|
||||
}
|
||||
providers.add(createAWSCredentialProvider(conf,
|
||||
aClass, binding));
|
||||
|
||||
if (v1v2CredentialProviderMap.containsKey(aClass.getSimpleName()) &&
|
||||
aClass.getName().contains(AWS_AUTH_CLASS_PREFIX)){
|
||||
providers.add(createAWSV2CredentialProvider(conf,
|
||||
v1v2CredentialProviderMap.get(aClass.getSimpleName()), binding));
|
||||
} else if (AWSCredentialsProvider.class.isAssignableFrom(aClass)) {
|
||||
providers.add(createAWSV1CredentialProvider(conf,
|
||||
aClass, binding));
|
||||
} else {
|
||||
providers.add(createAWSV2CredentialProvider(conf, aClass, binding));
|
||||
}
|
||||
|
||||
}
|
||||
return providers;
|
||||
}
|
||||
|
@ -686,7 +719,7 @@ public final class S3AUtils {
|
|||
* @return the instantiated class
|
||||
* @throws IOException on any instantiation failure.
|
||||
*/
|
||||
private static AWSCredentialsProvider createAWSCredentialProvider(
|
||||
private static AWSCredentialsProvider createAWSV1CredentialProvider(
|
||||
Configuration conf,
|
||||
Class<?> credClass,
|
||||
@Nullable URI uri) throws IOException {
|
||||
|
@ -743,9 +776,9 @@ public final class S3AUtils {
|
|||
}
|
||||
if (targetException instanceof IOException) {
|
||||
throw (IOException) targetException;
|
||||
} else if (targetException instanceof SdkBaseException) {
|
||||
} else if (targetException instanceof SdkException) {
|
||||
throw translateException("Instantiate " + className, "",
|
||||
(SdkBaseException) targetException);
|
||||
(SdkException) targetException);
|
||||
} else {
|
||||
// supported constructor or factory method found, but the call failed
|
||||
throw new IOException(className + " " + INSTANTIATION_EXCEPTION
|
||||
|
@ -760,6 +793,105 @@ public final class S3AUtils {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create an AWS credential provider from its class by using reflection. The
|
||||
* class must implement one of the following means of construction, which are
|
||||
* attempted in order:
|
||||
*
|
||||
* <ol>
|
||||
* <li>a public constructor accepting java.net.URI and
|
||||
* org.apache.hadoop.conf.Configuration</li>
|
||||
* <li>a public constructor accepting
|
||||
* org.apache.hadoop.conf.Configuration</li>
|
||||
* <li>a public static method named getInstance that accepts no
|
||||
* arguments and returns an instance of
|
||||
* software.amazon.awssdk.auth.credentials.AwsCredentialsProvider, or</li>
|
||||
* <li>a public default constructor.</li>
|
||||
* </ol>
|
||||
*
|
||||
* @param conf configuration
|
||||
* @param credClass credential class
|
||||
* @param uri URI of the FS
|
||||
* @return the instantiated class
|
||||
* @throws IOException on any instantiation failure.
|
||||
*/
|
||||
private static AwsCredentialsProvider createAWSV2CredentialProvider(
|
||||
Configuration conf,
|
||||
Class<?> credClass,
|
||||
@Nullable URI uri) throws IOException {
|
||||
AwsCredentialsProvider credentials = null;
|
||||
String className = credClass.getName();
|
||||
if (!AwsCredentialsProvider.class.isAssignableFrom(credClass)) {
|
||||
throw new IOException("Class " + credClass + " " + NOT_AWS_V2_PROVIDER);
|
||||
}
|
||||
if (Modifier.isAbstract(credClass.getModifiers())) {
|
||||
throw new IOException("Class " + credClass + " " + ABSTRACT_PROVIDER);
|
||||
}
|
||||
LOG.debug("Credential provider class is {}", className);
|
||||
|
||||
try {
|
||||
// new X(uri, conf)
|
||||
Constructor cons = getConstructor(credClass, URI.class,
|
||||
Configuration.class);
|
||||
if (cons != null) {
|
||||
credentials = (AwsCredentialsProvider)cons.newInstance(uri, conf);
|
||||
return credentials;
|
||||
}
|
||||
// new X(conf)
|
||||
cons = getConstructor(credClass, Configuration.class);
|
||||
if (cons != null) {
|
||||
credentials = (AwsCredentialsProvider)cons.newInstance(conf);
|
||||
return credentials;
|
||||
}
|
||||
|
||||
// X.getInstance()
|
||||
Method factory = getFactoryMethod(credClass, AwsCredentialsProvider.class,
|
||||
"create");
|
||||
if (factory != null) {
|
||||
credentials = (AwsCredentialsProvider)factory.invoke(null);
|
||||
return credentials;
|
||||
}
|
||||
|
||||
// new X()
|
||||
cons = getConstructor(credClass);
|
||||
if (cons != null) {
|
||||
credentials = (AwsCredentialsProvider)cons.newInstance();
|
||||
return credentials;
|
||||
}
|
||||
|
||||
// no supported constructor or factory method found
|
||||
throw new IOException(String.format("%s " + CONSTRUCTOR_EXCEPTION
|
||||
+ ". A class specified in %s must provide a public constructor "
|
||||
+ "of a supported signature, or a public factory method named "
|
||||
+ "create that accepts no arguments.",
|
||||
className, AWS_CREDENTIALS_PROVIDER));
|
||||
} catch (InvocationTargetException e) {
|
||||
// TODO: Can probably be moved to a common method, but before doing this, check if we still
|
||||
// want to extend V2 providers the same way v1 providers are.
|
||||
Throwable targetException = e.getTargetException();
|
||||
if (targetException == null) {
|
||||
targetException = e;
|
||||
}
|
||||
if (targetException instanceof IOException) {
|
||||
throw (IOException) targetException;
|
||||
} else if (targetException instanceof SdkException) {
|
||||
throw translateException("Instantiate " + className, "",
|
||||
(SdkException) targetException);
|
||||
} else {
|
||||
// supported constructor or factory method found, but the call failed
|
||||
throw new IOException(className + " " + INSTANTIATION_EXCEPTION
|
||||
+ ": " + targetException,
|
||||
targetException);
|
||||
}
|
||||
} catch (ReflectiveOperationException | IllegalArgumentException e) {
|
||||
// supported constructor or factory method found, but the call failed
|
||||
throw new IOException(className + " " + INSTANTIATION_EXCEPTION
|
||||
+ ": " + e,
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Set a key if the value is non-empty.
|
||||
* @param config config to patch
|
||||
|
@ -946,13 +1078,13 @@ public final class S3AUtils {
|
|||
|
||||
/**
|
||||
* 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();
|
||||
}
|
||||
|
||||
|
@ -1928,4 +2060,15 @@ public final class S3AUtils {
|
|||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,8 +25,6 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||
import com.amazonaws.handlers.RequestHandler2;
|
||||
import com.amazonaws.monitoring.MonitoringListener;
|
||||
import com.amazonaws.services.s3.AmazonS3;
|
||||
|
||||
|
@ -34,6 +32,11 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
|
||||
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
|
||||
import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
|
||||
import software.amazon.awssdk.services.s3.S3AsyncClient;
|
||||
import software.amazon.awssdk.services.s3.S3Client;
|
||||
|
||||
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ENDPOINT;
|
||||
|
||||
/**
|
||||
|
@ -49,7 +52,6 @@ import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ENDPOINT;
|
|||
*/
|
||||
@InterfaceAudience.LimitedPrivate("HBoss")
|
||||
@InterfaceStability.Evolving
|
||||
@Deprecated
|
||||
public interface S3ClientFactory {
|
||||
|
||||
/**
|
||||
|
@ -63,6 +65,34 @@ public interface S3ClientFactory {
|
|||
AmazonS3 createS3Client(URI uri,
|
||||
S3ClientCreationParameters parameters) throws IOException;
|
||||
|
||||
/**
|
||||
* Creates a new {@link S3Client}.
|
||||
* The client returned supports synchronous operations. For
|
||||
* asynchronous operations, use
|
||||
* {@link #createS3AsyncClient(URI, S3ClientCreationParameters)}.
|
||||
*
|
||||
* @param uri S3A file system URI
|
||||
* @param parameters parameter object
|
||||
* @return S3 client
|
||||
* @throws IOException on any IO problem
|
||||
*/
|
||||
S3Client createS3ClientV2(URI uri,
|
||||
S3ClientCreationParameters parameters) throws IOException;
|
||||
|
||||
/**
|
||||
* Creates a new {@link S3AsyncClient}.
|
||||
* The client returned supports asynchronous operations. For
|
||||
* synchronous operations, use
|
||||
* {@link #createS3ClientV2(URI, S3ClientCreationParameters)}.
|
||||
*
|
||||
* @param uri S3A file system URI
|
||||
* @param parameters parameter object
|
||||
* @return Async S3 client
|
||||
* @throws IOException on any IO problem
|
||||
*/
|
||||
S3AsyncClient createS3AsyncClient(URI uri,
|
||||
S3ClientCreationParameters parameters) throws IOException;
|
||||
|
||||
/**
|
||||
* Settings for the S3 Client.
|
||||
* Implemented as a class to pass in so that adding
|
||||
|
@ -74,7 +104,7 @@ public interface S3ClientFactory {
|
|||
/**
|
||||
* Credentials.
|
||||
*/
|
||||
private AWSCredentialsProvider credentialSet;
|
||||
private AwsCredentialsProvider credentialSet;
|
||||
|
||||
/**
|
||||
* Endpoint.
|
||||
|
@ -109,9 +139,9 @@ public interface S3ClientFactory {
|
|||
private boolean requesterPays;
|
||||
|
||||
/**
|
||||
* Request handlers; used for auditing, X-Ray etc.
|
||||
*/
|
||||
private List<RequestHandler2> requestHandlers;
|
||||
* Execution interceptors; used for auditing, X-Ray etc.
|
||||
* */
|
||||
private List<ExecutionInterceptor> executionInterceptors;
|
||||
|
||||
/**
|
||||
* Suffix to UA.
|
||||
|
@ -125,22 +155,22 @@ public interface S3ClientFactory {
|
|||
private URI pathUri;
|
||||
|
||||
/**
|
||||
* List of request handlers to include in the chain
|
||||
* of request execution in the SDK.
|
||||
* @return the handler list
|
||||
* List of execution interceptors to include in the chain
|
||||
* of interceptors in the SDK.
|
||||
* @return the interceptors list
|
||||
*/
|
||||
public List<RequestHandler2> getRequestHandlers() {
|
||||
return requestHandlers;
|
||||
public List<ExecutionInterceptor> getExecutionInterceptors() {
|
||||
return executionInterceptors;
|
||||
}
|
||||
|
||||
/**
|
||||
* List of request handlers.
|
||||
* @param handlers handler list.
|
||||
* List of execution interceptors.
|
||||
* @param interceptors interceptors list.
|
||||
* @return this object
|
||||
*/
|
||||
public S3ClientCreationParameters withRequestHandlers(
|
||||
@Nullable final List<RequestHandler2> handlers) {
|
||||
requestHandlers = handlers;
|
||||
public S3ClientCreationParameters withExecutionInterceptors(
|
||||
@Nullable final List<ExecutionInterceptor> interceptors) {
|
||||
executionInterceptors = interceptors;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -191,7 +221,7 @@ public interface S3ClientFactory {
|
|||
return requesterPays;
|
||||
}
|
||||
|
||||
public AWSCredentialsProvider getCredentialSet() {
|
||||
public AwsCredentialsProvider getCredentialSet() {
|
||||
return credentialSet;
|
||||
}
|
||||
|
||||
|
@ -202,7 +232,7 @@ public interface S3ClientFactory {
|
|||
*/
|
||||
|
||||
public S3ClientCreationParameters withCredentialSet(
|
||||
final AWSCredentialsProvider value) {
|
||||
final AwsCredentialsProvider value) {
|
||||
credentialSet = value;
|
||||
return this;
|
||||
}
|
||||
|
|
|
@ -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 final class S3ListRequest {
|
|||
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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,19 +22,21 @@ import java.util.Collection;
|
|||
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 @@ public class S3ListResult {
|
|||
* @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 class S3ListResult {
|
|||
* @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 class S3ListResult {
|
|||
return v1Result != null;
|
||||
}
|
||||
|
||||
public ObjectListing getV1() {
|
||||
public ListObjectsResponse getV1() {
|
||||
return v1Result;
|
||||
}
|
||||
|
||||
public ListObjectsV2Result getV2() {
|
||||
public ListObjectsV2Response getV2() {
|
||||
return v2Result;
|
||||
}
|
||||
|
||||
public List<S3ObjectSummary> getObjectSummaries() {
|
||||
public List<S3Object> getS3Objects() {
|
||||
if (isV1()) {
|
||||
return v1Result.getObjectSummaries();
|
||||
return v1Result.contents();
|
||||
} else {
|
||||
return v2Result.getObjectSummaries();
|
||||
return v2Result.contents();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -89,21 +91,21 @@ public class S3ListResult {
|
|||
}
|
||||
}
|
||||
|
||||
public List<String> getCommonPrefixes() {
|
||||
public List<CommonPrefix> 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<String> objectSummaryKeys() {
|
||||
return getObjectSummaries().stream()
|
||||
.map(S3ObjectSummary::getKey)
|
||||
private List<String> objectKeys() {
|
||||
return getS3Objects().stream()
|
||||
.map(S3Object::key)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
|
@ -112,9 +114,8 @@ public class S3ListResult {
|
|||
* @return true if the result is non-empty
|
||||
*/
|
||||
public boolean hasPrefixesOrObjects() {
|
||||
|
||||
return !(getCommonPrefixes()).isEmpty()
|
||||
|| !getObjectSummaries().isEmpty();
|
||||
|| !getS3Objects().isEmpty();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -128,7 +129,7 @@ public class S3ListResult {
|
|||
// no children.
|
||||
// So the listing must contain the marker entry only as an object,
|
||||
// and prefixes is null
|
||||
List<String> keys = objectSummaryKeys();
|
||||
List<String> keys = objectKeys();
|
||||
return keys.size() == 1 && keys.contains(dirKey)
|
||||
&& getCommonPrefixes().isEmpty();
|
||||
}
|
||||
|
@ -138,15 +139,15 @@ public class S3ListResult {
|
|||
* @param log log to use
|
||||
*/
|
||||
public void logAtDebug(Logger log) {
|
||||
Collection<String> prefixes = getCommonPrefixes();
|
||||
Collection<S3ObjectSummary> summaries = getObjectSummaries();
|
||||
Collection<CommonPrefix> prefixes = getCommonPrefixes();
|
||||
Collection<S3Object> 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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.services.s3.transfer.model.CopyResult;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -74,31 +72,6 @@ public class S3ObjectAttributes {
|
|||
this.len = len;
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct from the result of a copy and those parameters
|
||||
* which aren't included in an AWS SDK response.
|
||||
* @param path path
|
||||
* @param copyResult copy result.
|
||||
* @param serverSideEncryptionAlgorithm current encryption algorithm
|
||||
* @param serverSideEncryptionKey any server side encryption key?
|
||||
* @param len object length
|
||||
*/
|
||||
public S3ObjectAttributes(
|
||||
final Path path,
|
||||
final CopyResult copyResult,
|
||||
final S3AEncryptionMethods serverSideEncryptionAlgorithm,
|
||||
final String serverSideEncryptionKey,
|
||||
final long len) {
|
||||
this.bucket = copyResult.getDestinationBucketName();
|
||||
this.key = copyResult.getDestinationKey();
|
||||
this.path = path;
|
||||
this.serverSideEncryptionAlgorithm = serverSideEncryptionAlgorithm;
|
||||
this.serverSideEncryptionKey = serverSideEncryptionKey;
|
||||
this.eTag = copyResult.getETag();
|
||||
this.versionId = copyResult.getVersionId();
|
||||
this.len = len;
|
||||
}
|
||||
|
||||
public String getBucket() {
|
||||
return bucket;
|
||||
}
|
||||
|
|
|
@ -39,6 +39,5 @@ import org.apache.hadoop.fs.s3a.auth.NoAwsCredentialsException;
|
|||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
@SuppressWarnings("deprecation")
|
||||
public final class SharedInstanceCredentialProvider extends IAMInstanceCredentialsProvider {
|
||||
}
|
||||
|
|
|
@ -18,9 +18,10 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import com.amazonaws.auth.AWSCredentials;
|
||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||
import com.amazonaws.auth.BasicAWSCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
|
||||
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
@ -42,13 +43,10 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.getAWSAccessKeys;
|
|||
* property fs.s3a.aws.credentials.provider. Therefore, changing the class name
|
||||
* would be a backward-incompatible change.
|
||||
*
|
||||
* @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider
|
||||
* as part of upgrading S3A to SDK V2. See HADOOP-18073.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
@Deprecated
|
||||
public class SimpleAWSCredentialsProvider implements AWSCredentialsProvider {
|
||||
public class SimpleAWSCredentialsProvider implements AwsCredentialsProvider {
|
||||
|
||||
public static final String NAME
|
||||
= "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider";
|
||||
|
@ -80,17 +78,14 @@ public class SimpleAWSCredentialsProvider implements AWSCredentialsProvider {
|
|||
}
|
||||
|
||||
@Override
|
||||
public AWSCredentials getCredentials() {
|
||||
public AwsCredentials resolveCredentials() {
|
||||
if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) {
|
||||
return new BasicAWSCredentials(accessKey, secretKey);
|
||||
return AwsBasicCredentials.create(accessKey, secretKey);
|
||||
}
|
||||
throw new NoAwsCredentialsException("SimpleAWSCredentialsProvider",
|
||||
"No AWS credentials in the Hadoop configuration");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void refresh() {}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getClass().getSimpleName();
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.hadoop.fs.s3a;
|
|||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
|
||||
import com.amazonaws.auth.AWSCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentials;
|
||||
|
||||
import java.net.URI;
|
||||
|
||||
|
@ -44,12 +44,9 @@ import org.apache.hadoop.fs.s3a.auth.NoAwsCredentialsException;
|
|||
* This credential provider must not fail in creation because that will
|
||||
* break a chain of credential providers.
|
||||
*
|
||||
* @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider
|
||||
* as part of upgrading S3A to SDK V2. See HADOOP-18073.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
@Deprecated
|
||||
public class TemporaryAWSCredentialsProvider extends AbstractSessionCredentialsProvider {
|
||||
|
||||
public static final String NAME
|
||||
|
@ -92,7 +89,7 @@ public class TemporaryAWSCredentialsProvider extends AbstractSessionCredentialsP
|
|||
* @throws NoAwsCredentialsException the credentials are actually empty.
|
||||
*/
|
||||
@Override
|
||||
protected AWSCredentials createCredentials(Configuration config)
|
||||
protected AwsCredentials createCredentials(Configuration config)
|
||||
throws IOException {
|
||||
MarshalledCredentials creds = MarshalledCredentialBinding.fromFileSystem(
|
||||
getUri(), config);
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -19,29 +19,28 @@
|
|||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import com.amazonaws.services.s3.model.AmazonS3Exception;
|
||||
import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
|
||||
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.MultipartUpload;
|
||||
import com.amazonaws.services.s3.model.ObjectMetadata;
|
||||
import com.amazonaws.services.s3.model.PartETag;
|
||||
import com.amazonaws.services.s3.model.PutObjectRequest;
|
||||
import com.amazonaws.services.s3.model.PutObjectResult;
|
||||
import com.amazonaws.services.s3.model.SelectObjectContentRequest;
|
||||
import com.amazonaws.services.s3.model.SelectObjectContentResult;
|
||||
import com.amazonaws.services.s3.model.UploadPartRequest;
|
||||
import com.amazonaws.services.s3.model.UploadPartResult;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import software.amazon.awssdk.core.sync.RequestBody;
|
||||
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
|
||||
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
|
||||
import software.amazon.awssdk.services.s3.model.CompletedPart;
|
||||
import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest;
|
||||
import software.amazon.awssdk.services.s3.model.MultipartUpload;
|
||||
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.PutObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest;
|
||||
import software.amazon.awssdk.services.s3.model.SelectObjectContentResponseHandler;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -50,6 +49,8 @@ import org.apache.hadoop.fs.PathIOException;
|
|||
import org.apache.hadoop.fs.s3a.api.RequestFactory;
|
||||
import org.apache.hadoop.fs.s3a.impl.PutObjectOptions;
|
||||
import org.apache.hadoop.fs.s3a.impl.StoreContext;
|
||||
import org.apache.hadoop.fs.s3a.select.SelectEventStreamPublisher;
|
||||
import org.apache.hadoop.fs.s3a.select.SelectObjectContentHelper;
|
||||
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
|
||||
import org.apache.hadoop.fs.s3a.select.SelectBinding;
|
||||
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
|
||||
|
@ -240,48 +241,24 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
/**
|
||||
* 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) {
|
||||
final PutObjectOptions options,
|
||||
boolean isFile) {
|
||||
|
||||
activateAuditSpan();
|
||||
ObjectMetadata objectMetadata = newObjectMetadata(length);
|
||||
return getRequestFactory().newPutObjectRequest(
|
||||
destKey,
|
||||
objectMetadata,
|
||||
options,
|
||||
inputStream);
|
||||
|
||||
return getRequestFactory()
|
||||
.newPutObjectRequestBuilder(destKey, options, length, false)
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a {@link PutObjectRequest} request to upload a file.
|
||||
* @param dest key to PUT to.
|
||||
* @param sourceFile source file
|
||||
* @param options options for the request
|
||||
* @return the request
|
||||
*/
|
||||
@Retries.OnceRaw
|
||||
public PutObjectRequest createPutObjectRequest(
|
||||
String dest,
|
||||
File sourceFile,
|
||||
final PutObjectOptions options) {
|
||||
activateAuditSpan();
|
||||
final ObjectMetadata objectMetadata =
|
||||
newObjectMetadata((int) sourceFile.length());
|
||||
|
||||
PutObjectRequest putObjectRequest = getRequestFactory().
|
||||
newPutObjectRequest(dest,
|
||||
objectMetadata,
|
||||
options,
|
||||
sourceFile);
|
||||
return putObjectRequest;
|
||||
}
|
||||
|
||||
/**
|
||||
* Callback on a successful write.
|
||||
|
@ -298,17 +275,6 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
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}
|
||||
*/
|
||||
|
@ -321,11 +287,11 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
try (AuditSpan span = activateAuditSpan()) {
|
||||
return retry("initiate MultiPartUpload", destKey, true,
|
||||
() -> {
|
||||
final InitiateMultipartUploadRequest initiateMPURequest =
|
||||
getRequestFactory().newMultipartUploadRequest(
|
||||
final CreateMultipartUploadRequest.Builder initiateMPURequestBuilder =
|
||||
getRequestFactory().newMultipartUploadRequestBuilder(
|
||||
destKey, options);
|
||||
return owner.initiateMultipartUpload(initiateMPURequest)
|
||||
.getUploadId();
|
||||
return owner.initiateMultipartUpload(initiateMPURequestBuilder.build())
|
||||
.uploadId();
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -346,10 +312,10 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
* @throws IOException on problems.
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
private CompleteMultipartUploadResult finalizeMultipartUpload(
|
||||
private CompleteMultipartUploadResponse finalizeMultipartUpload(
|
||||
String destKey,
|
||||
String uploadId,
|
||||
List<PartETag> partETags,
|
||||
List<CompletedPart> partETags,
|
||||
long length,
|
||||
PutObjectOptions putOptions,
|
||||
Retried retrying) throws IOException {
|
||||
|
@ -358,18 +324,18 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
"No upload parts in multipart upload");
|
||||
}
|
||||
try (AuditSpan span = activateAuditSpan()) {
|
||||
CompleteMultipartUploadResult uploadResult;
|
||||
CompleteMultipartUploadResponse uploadResult;
|
||||
uploadResult = invoker.retry("Completing multipart upload", destKey,
|
||||
true,
|
||||
retrying,
|
||||
() -> {
|
||||
final CompleteMultipartUploadRequest request =
|
||||
getRequestFactory().newCompleteMultipartUploadRequest(
|
||||
final CompleteMultipartUploadRequest.Builder requestBuilder =
|
||||
getRequestFactory().newCompleteMultipartUploadRequestBuilder(
|
||||
destKey, uploadId, partETags);
|
||||
return writeOperationHelperCallbacks.completeMultipartUpload(request);
|
||||
return writeOperationHelperCallbacks.completeMultipartUpload(requestBuilder.build());
|
||||
});
|
||||
owner.finishedWrite(destKey, length, uploadResult.getETag(),
|
||||
uploadResult.getVersionId(),
|
||||
owner.finishedWrite(destKey, length, uploadResult.eTag(),
|
||||
uploadResult.versionId(),
|
||||
putOptions);
|
||||
return uploadResult;
|
||||
}
|
||||
|
@ -392,10 +358,10 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
* the retry count was exceeded
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
public CompleteMultipartUploadResult completeMPUwithRetries(
|
||||
public CompleteMultipartUploadResponse completeMPUwithRetries(
|
||||
String destKey,
|
||||
String uploadId,
|
||||
List<PartETag> partETags,
|
||||
List<CompletedPart> partETags,
|
||||
long length,
|
||||
AtomicInteger errorCount,
|
||||
PutObjectOptions putOptions)
|
||||
|
@ -453,7 +419,7 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
@Retries.RetryTranslated
|
||||
public void abortMultipartUpload(MultipartUpload upload)
|
||||
throws IOException {
|
||||
invoker.retry("Aborting multipart commit", upload.getKey(), true,
|
||||
invoker.retry("Aborting multipart commit", upload.key(), true,
|
||||
withinAuditSpan(getAuditSpan(),
|
||||
() -> owner.abortMultipartUpload(upload)));
|
||||
}
|
||||
|
@ -478,7 +444,7 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
abortMultipartUpload(upload);
|
||||
count++;
|
||||
} catch (FileNotFoundException e) {
|
||||
LOG.debug("Already aborted: {}", upload.getKey(), e);
|
||||
LOG.debug("Already aborted: {}", upload.key(), e);
|
||||
}
|
||||
}
|
||||
return count;
|
||||
|
@ -507,45 +473,31 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
}
|
||||
|
||||
/**
|
||||
* Create and initialize a part request of a multipart upload.
|
||||
* Exactly one of: {@code uploadStream} or {@code sourceFile}
|
||||
* must be specified.
|
||||
* A subset of the file may be posted, by providing the starting point
|
||||
* in {@code offset} and a length of block in {@code size} equal to
|
||||
* or less than the remaining bytes.
|
||||
* Create and initialize a part request builder of a multipart upload.
|
||||
* The part number must be less than 10000.
|
||||
* Retry policy is once-translated; to much effort
|
||||
* @param destKey destination key of ongoing operation
|
||||
* @param uploadId ID of ongoing upload
|
||||
* @param partNumber current part number of the upload
|
||||
* @param size amount of data
|
||||
* @param uploadStream source of data to upload
|
||||
* @param sourceFile optional source file.
|
||||
* @param offset offset in file to start reading.
|
||||
* @return the request.
|
||||
* @return the request builder.
|
||||
* @throws IllegalArgumentException if the parameters are invalid.
|
||||
* @throws PathIOException if the part number is out of range.
|
||||
*/
|
||||
@Override
|
||||
@Retries.OnceTranslated
|
||||
public UploadPartRequest newUploadPartRequest(
|
||||
public UploadPartRequest.Builder newUploadPartRequestBuilder(
|
||||
String destKey,
|
||||
String uploadId,
|
||||
int partNumber,
|
||||
long size,
|
||||
InputStream uploadStream,
|
||||
File sourceFile,
|
||||
Long offset) throws IOException {
|
||||
long size) throws IOException {
|
||||
return once("upload part request", destKey,
|
||||
withinAuditSpan(getAuditSpan(), () ->
|
||||
getRequestFactory().newUploadPartRequest(
|
||||
getRequestFactory().newUploadPartRequestBuilder(
|
||||
destKey,
|
||||
uploadId,
|
||||
partNumber,
|
||||
size,
|
||||
uploadStream,
|
||||
sourceFile,
|
||||
offset)));
|
||||
size)));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -567,18 +519,20 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
* @param putObjectRequest the request
|
||||
* @param putOptions put object options
|
||||
* @param durationTrackerFactory factory for duration tracking
|
||||
* @param uploadData data to be uploaded
|
||||
* @param isFile is data to be uploaded a file
|
||||
*
|
||||
* @return the upload initiated
|
||||
* @throws IOException on problems
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
public PutObjectResult putObject(PutObjectRequest putObjectRequest,
|
||||
PutObjectOptions putOptions,
|
||||
public PutObjectResponse putObject(PutObjectRequest putObjectRequest,
|
||||
PutObjectOptions putOptions, S3ADataBlocks.BlockUploadData uploadData, boolean isFile,
|
||||
DurationTrackerFactory durationTrackerFactory)
|
||||
throws IOException {
|
||||
return retry("Writing Object",
|
||||
putObjectRequest.getKey(), true,
|
||||
withinAuditSpan(getAuditSpan(), () ->
|
||||
owner.putObjectDirect(putObjectRequest, putOptions, durationTrackerFactory)));
|
||||
return retry("Writing Object", putObjectRequest.key(), true, withinAuditSpan(getAuditSpan(),
|
||||
() -> owner.putObjectDirect(putObjectRequest, putOptions, uploadData, isFile,
|
||||
durationTrackerFactory)));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -614,10 +568,10 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
* the retry count was exceeded
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
public CompleteMultipartUploadResult commitUpload(
|
||||
public CompleteMultipartUploadResponse commitUpload(
|
||||
String destKey,
|
||||
String uploadId,
|
||||
List<PartETag> partETags,
|
||||
List<CompletedPart> partETags,
|
||||
long length)
|
||||
throws IOException {
|
||||
checkNotNull(uploadId);
|
||||
|
@ -636,19 +590,21 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
* Upload part of a multi-partition file.
|
||||
* @param request request
|
||||
* @param durationTrackerFactory duration tracker factory for operation
|
||||
* @param request the upload part request.
|
||||
* @param body the request body.
|
||||
* @return the result of the operation.
|
||||
* @throws IOException on problems
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
public UploadPartResult uploadPart(UploadPartRequest request,
|
||||
public UploadPartResponse uploadPart(UploadPartRequest request, RequestBody body,
|
||||
final DurationTrackerFactory durationTrackerFactory)
|
||||
throws IOException {
|
||||
return retry("upload part #" + request.getPartNumber()
|
||||
+ " upload ID " + request.getUploadId(),
|
||||
request.getKey(),
|
||||
return retry("upload part #" + request.partNumber()
|
||||
+ " upload ID " + request.uploadId(),
|
||||
request.key(),
|
||||
true,
|
||||
withinAuditSpan(getAuditSpan(),
|
||||
() -> owner.uploadPart(request, durationTrackerFactory)));
|
||||
() -> owner.uploadPart(request, body, durationTrackerFactory)));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -660,15 +616,9 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
return conf;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a S3 Select request for the destination path.
|
||||
* This does not build the query.
|
||||
* @param path pre-qualified path for query
|
||||
* @return the request
|
||||
*/
|
||||
public SelectObjectContentRequest newSelectRequest(Path path) {
|
||||
public SelectObjectContentRequest.Builder newSelectRequestBuilder(Path path) {
|
||||
try (AuditSpan span = getAuditSpan()) {
|
||||
return getRequestFactory().newSelectRequest(
|
||||
return getRequestFactory().newSelectRequestBuilder(
|
||||
storeContext.pathToKey(path));
|
||||
}
|
||||
}
|
||||
|
@ -677,26 +627,27 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
* Execute an S3 Select operation.
|
||||
* On a failure, the request is only logged at debug to avoid the
|
||||
* select exception being printed.
|
||||
* @param source source for selection
|
||||
*
|
||||
* @param source source for selection
|
||||
* @param request Select request to issue.
|
||||
* @param action the action for use in exception creation
|
||||
* @param action the action for use in exception creation
|
||||
* @return response
|
||||
* @throws IOException failure
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
public SelectObjectContentResult select(
|
||||
public SelectEventStreamPublisher select(
|
||||
final Path source,
|
||||
final SelectObjectContentRequest request,
|
||||
final String action)
|
||||
throws IOException {
|
||||
// no setting of span here as the select binding is (statically) created
|
||||
// without any span.
|
||||
String bucketName = request.getBucketName();
|
||||
String bucketName = request.bucket();
|
||||
Preconditions.checkArgument(bucket.equals(bucketName),
|
||||
"wrong bucket: %s", bucketName);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Initiating select call {} {}",
|
||||
source, request.getExpression());
|
||||
source, request.expression());
|
||||
LOG.debug(SelectBinding.toString(request));
|
||||
}
|
||||
return invoker.retry(
|
||||
|
@ -707,8 +658,9 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
try (DurationInfo ignored =
|
||||
new DurationInfo(LOG, "S3 Select operation")) {
|
||||
try {
|
||||
return writeOperationHelperCallbacks.selectObjectContent(request);
|
||||
} catch (AmazonS3Exception e) {
|
||||
return SelectObjectContentHelper.select(
|
||||
writeOperationHelperCallbacks, source, request, action);
|
||||
} catch (Throwable e) {
|
||||
LOG.error("Failure of S3 Select request against {}",
|
||||
source);
|
||||
LOG.debug("S3 Select request against {}:\n{}",
|
||||
|
@ -759,14 +711,14 @@ public class WriteOperationHelper implements WriteOperations {
|
|||
* @param request selectObjectContent request
|
||||
* @return selectObjectContentResult
|
||||
*/
|
||||
SelectObjectContentResult selectObjectContent(SelectObjectContentRequest request);
|
||||
CompletableFuture<Void> selectObjectContent(SelectObjectContentRequest request, SelectObjectContentResponseHandler t);
|
||||
|
||||
/**
|
||||
* Initiates a complete multi-part upload request.
|
||||
* @param request Complete multi-part upload request
|
||||
* @return completeMultipartUploadResult
|
||||
*/
|
||||
CompleteMultipartUploadResult completeMultipartUpload(CompleteMultipartUploadRequest request);
|
||||
CompleteMultipartUploadResponse completeMultipartUpload(CompleteMultipartUploadRequest request);
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -20,23 +20,20 @@ package org.apache.hadoop.fs.s3a;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
|
||||
import com.amazonaws.services.s3.model.MultipartUpload;
|
||||
import com.amazonaws.services.s3.model.ObjectMetadata;
|
||||
import com.amazonaws.services.s3.model.PartETag;
|
||||
import com.amazonaws.services.s3.model.PutObjectRequest;
|
||||
import com.amazonaws.services.s3.model.PutObjectResult;
|
||||
import com.amazonaws.services.s3.model.SelectObjectContentRequest;
|
||||
import com.amazonaws.services.s3.model.SelectObjectContentResult;
|
||||
import com.amazonaws.services.s3.model.UploadPartRequest;
|
||||
import com.amazonaws.services.s3.model.UploadPartResult;
|
||||
import software.amazon.awssdk.core.sync.RequestBody;
|
||||
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
|
||||
import software.amazon.awssdk.services.s3.model.CompletedPart;
|
||||
import software.amazon.awssdk.services.s3.model.MultipartUpload;
|
||||
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.PutObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -44,6 +41,7 @@ import org.apache.hadoop.fs.PathIOException;
|
|||
import org.apache.hadoop.fs.s3a.impl.PutObjectOptions;
|
||||
import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
|
||||
import org.apache.hadoop.fs.store.audit.AuditSpan;
|
||||
import org.apache.hadoop.fs.s3a.select.SelectEventStreamPublisher;
|
||||
import org.apache.hadoop.fs.store.audit.AuditSpanSource;
|
||||
import org.apache.hadoop.util.functional.CallableRaisingIOE;
|
||||
|
||||
|
@ -78,27 +76,15 @@ public interface WriteOperations extends AuditSpanSource, Closeable {
|
|||
/**
|
||||
* 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.
|
||||
|
@ -112,15 +98,6 @@ public interface WriteOperations extends AuditSpanSource, Closeable {
|
|||
*/
|
||||
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.
|
||||
|
@ -149,10 +126,10 @@ public interface WriteOperations extends AuditSpanSource, Closeable {
|
|||
* the retry count was exceeded
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
CompleteMultipartUploadResult completeMPUwithRetries(
|
||||
CompleteMultipartUploadResponse completeMPUwithRetries(
|
||||
String destKey,
|
||||
String uploadId,
|
||||
List<PartETag> partETags,
|
||||
List<CompletedPart> partETags,
|
||||
long length,
|
||||
AtomicInteger errorCount,
|
||||
PutObjectOptions putOptions)
|
||||
|
@ -214,31 +191,20 @@ public interface WriteOperations extends AuditSpanSource, Closeable {
|
|||
throws IOException;
|
||||
|
||||
/**
|
||||
* Create and initialize a part request of a multipart upload.
|
||||
* Exactly one of: {@code uploadStream} or {@code sourceFile}
|
||||
* must be specified.
|
||||
* A subset of the file may be posted, by providing the starting point
|
||||
* in {@code offset} and a length of block in {@code size} equal to
|
||||
* or less than the remaining bytes.
|
||||
* Create and initialize a part request builder of a multipart upload.
|
||||
* @param destKey destination key of ongoing operation
|
||||
* @param uploadId ID of ongoing upload
|
||||
* @param partNumber current part number of the upload
|
||||
* @param size amount of data
|
||||
* @param uploadStream source of data to upload
|
||||
* @param sourceFile optional source file.
|
||||
* @param offset offset in file to start reading.
|
||||
* @return the request.
|
||||
* @return the request builder.
|
||||
* @throws IllegalArgumentException if the parameters are invalid
|
||||
* @throws PathIOException if the part number is out of range.
|
||||
*/
|
||||
UploadPartRequest newUploadPartRequest(
|
||||
UploadPartRequest.Builder newUploadPartRequestBuilder(
|
||||
String destKey,
|
||||
String uploadId,
|
||||
int partNumber,
|
||||
long size,
|
||||
InputStream uploadStream,
|
||||
File sourceFile,
|
||||
Long offset) throws IOException;
|
||||
long size) throws IOException;
|
||||
|
||||
/**
|
||||
* PUT an object directly (i.e. not via the transfer manager).
|
||||
|
@ -247,12 +213,14 @@ public interface WriteOperations extends AuditSpanSource, Closeable {
|
|||
* @param putObjectRequest the request
|
||||
* @param putOptions put object options
|
||||
* @param durationTrackerFactory factory for duration tracking
|
||||
* @param uploadData data to be uploaded
|
||||
* @param isFile is data to be uploaded a file
|
||||
* @return the upload initiated
|
||||
* @throws IOException on problems
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
PutObjectResult putObject(PutObjectRequest putObjectRequest,
|
||||
PutObjectOptions putOptions,
|
||||
PutObjectResponse putObject(PutObjectRequest putObjectRequest,
|
||||
PutObjectOptions putOptions, S3ADataBlocks.BlockUploadData uploadData, boolean isFile,
|
||||
DurationTrackerFactory durationTrackerFactory)
|
||||
throws IOException;
|
||||
|
||||
|
@ -280,22 +248,23 @@ public interface WriteOperations extends AuditSpanSource, Closeable {
|
|||
* the retry count was exceeded
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
CompleteMultipartUploadResult commitUpload(
|
||||
CompleteMultipartUploadResponse commitUpload(
|
||||
String destKey,
|
||||
String uploadId,
|
||||
List<PartETag> partETags,
|
||||
List<CompletedPart> partETags,
|
||||
long length)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Upload part of a multi-partition file.
|
||||
* @param request request
|
||||
* @param request the upload part request.
|
||||
* @param body the request body.
|
||||
* @param durationTrackerFactory factory for duration tracking
|
||||
* @return the result of the operation.
|
||||
* @throws IOException on problems
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
UploadPartResult uploadPart(UploadPartRequest request,
|
||||
UploadPartResponse uploadPart(UploadPartRequest request, RequestBody body,
|
||||
DurationTrackerFactory durationTrackerFactory)
|
||||
throws IOException;
|
||||
|
||||
|
@ -313,25 +282,26 @@ public interface WriteOperations extends AuditSpanSource, Closeable {
|
|||
AuditSpan getAuditSpan();
|
||||
|
||||
/**
|
||||
* Create a S3 Select request for the destination path.
|
||||
* Create a S3 Select request builder for the destination path.
|
||||
* This does not build the query.
|
||||
* @param path pre-qualified path for query
|
||||
* @return the request
|
||||
* @return the request builder
|
||||
*/
|
||||
SelectObjectContentRequest newSelectRequest(Path path);
|
||||
SelectObjectContentRequest.Builder newSelectRequestBuilder(Path path);
|
||||
|
||||
/**
|
||||
* Execute an S3 Select operation.
|
||||
* On a failure, the request is only logged at debug to avoid the
|
||||
* select exception being printed.
|
||||
* @param source source for selection
|
||||
*
|
||||
* @param source source for selection
|
||||
* @param request Select request to issue.
|
||||
* @param action the action for use in exception creation
|
||||
* @param action the action for use in exception creation
|
||||
* @return response
|
||||
* @throws IOException failure
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
SelectObjectContentResult select(
|
||||
SelectEventStreamPublisher select(
|
||||
Path source,
|
||||
SelectObjectContentRequest request,
|
||||
String action)
|
||||
|
|
|
@ -0,0 +1,75 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.s3a.adapter;
|
||||
|
||||
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;
|
||||
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 if (toAdapt instanceof AnonymousAWSCredentials) {
|
||||
return AnonymousCredentialsProvider.create().resolveCredentials();
|
||||
} 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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
|
@ -19,39 +19,33 @@
|
|||
package org.apache.hadoop.fs.s3a.api;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.InputStream;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.CannedAccessControlList;
|
||||
import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.CopyObjectRequest;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectRequest;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
|
||||
import com.amazonaws.services.s3.model.GetObjectRequest;
|
||||
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
|
||||
import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.ListObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.ListObjectsV2Request;
|
||||
import com.amazonaws.services.s3.model.ObjectListing;
|
||||
import com.amazonaws.services.s3.model.ObjectMetadata;
|
||||
import com.amazonaws.services.s3.model.PartETag;
|
||||
import com.amazonaws.services.s3.model.PutObjectRequest;
|
||||
import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams;
|
||||
import com.amazonaws.services.s3.model.SSECustomerKey;
|
||||
import com.amazonaws.services.s3.model.SelectObjectContentRequest;
|
||||
import com.amazonaws.services.s3.model.StorageClass;
|
||||
import com.amazonaws.services.s3.model.UploadPartRequest;
|
||||
|
||||
import 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.AbortMultipartUploadRequest;
|
||||
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
|
||||
import software.amazon.awssdk.services.s3.model.CompletedPart;
|
||||
import software.amazon.awssdk.services.s3.model.CopyObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest;
|
||||
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
|
||||
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest;
|
||||
import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
|
||||
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
|
||||
import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
|
||||
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
|
||||
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest;
|
||||
import software.amazon.awssdk.services.s3.model.StorageClass;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
|
||||
|
||||
/**
|
||||
* Factory for S3 objects.
|
||||
*
|
||||
|
@ -79,22 +73,7 @@ public interface RequestFactory {
|
|||
* Get the canned ACL of this FS.
|
||||
* @return an ACL, if any
|
||||
*/
|
||||
CannedAccessControlList getCannedACL();
|
||||
|
||||
/**
|
||||
* Create the AWS SDK structure used to configure SSE,
|
||||
* if the encryption secrets contain the information/settings for this.
|
||||
* @return an optional set of KMS Key settings
|
||||
*/
|
||||
Optional<SSEAwsKeyManagementParams> generateSSEAwsKeyParams();
|
||||
|
||||
/**
|
||||
* Create the SSE-C structure for the AWS SDK, if the encryption secrets
|
||||
* contain the information/settings for this.
|
||||
* This will contain a secret extracted from the bucket/configuration.
|
||||
* @return an optional customer key.
|
||||
*/
|
||||
Optional<SSECustomerKey> generateSSECustomerKey();
|
||||
ObjectCannedACL getCannedACL();
|
||||
|
||||
/**
|
||||
* Get the encryption algorithm of this endpoint.
|
||||
|
@ -115,79 +94,58 @@ public interface RequestFactory {
|
|||
StorageClass getStorageClass();
|
||||
|
||||
/**
|
||||
* Create a new object metadata instance.
|
||||
* Any standard metadata headers are added here, for example:
|
||||
* encryption.
|
||||
*
|
||||
* @param length length of data to set in header; Ignored if negative
|
||||
* @return a new metadata instance
|
||||
*/
|
||||
ObjectMetadata newObjectMetadata(long length);
|
||||
|
||||
/**
|
||||
* Create a copy request.
|
||||
* Create a copy request builder.
|
||||
* This includes the work of copying the relevant parts
|
||||
* of the metadata from the source
|
||||
* @param srcKey source
|
||||
* @param dstKey destination
|
||||
* @param srcom source object metadata.
|
||||
* @return the request
|
||||
* @return the request builder
|
||||
*/
|
||||
CopyObjectRequest newCopyObjectRequest(String srcKey,
|
||||
CopyObjectRequest.Builder newCopyObjectRequestBuilder(String srcKey,
|
||||
String dstKey,
|
||||
ObjectMetadata srcom);
|
||||
HeadObjectResponse srcom);
|
||||
|
||||
|
||||
/**
|
||||
* Create a putObject request.
|
||||
* Adds the ACL and metadata
|
||||
* @param key key of object
|
||||
* @param metadata metadata header
|
||||
* @param options options for the request
|
||||
* @param srcfile source file
|
||||
* @return the request
|
||||
*/
|
||||
PutObjectRequest newPutObjectRequest(String key,
|
||||
ObjectMetadata metadata, PutObjectOptions options, File srcfile);
|
||||
|
||||
/**
|
||||
* Create a {@link PutObjectRequest} request.
|
||||
* Create a {@link PutObjectRequest} request builder.
|
||||
* The metadata is assumed to have been configured with the size of the
|
||||
* operation.
|
||||
* @param key key of object
|
||||
* @param metadata metadata header
|
||||
* @param options options for the request
|
||||
* @param inputStream source data.
|
||||
* @return the request
|
||||
* @param length length of object to be uploaded
|
||||
* @param isDirectoryMarker true if object to be uploaded is a directory marker
|
||||
* @return the request builder
|
||||
*/
|
||||
PutObjectRequest newPutObjectRequest(String key,
|
||||
ObjectMetadata metadata,
|
||||
PutObjectRequest.Builder newPutObjectRequestBuilder(String key,
|
||||
PutObjectOptions options,
|
||||
InputStream inputStream);
|
||||
long length,
|
||||
boolean isDirectoryMarker);
|
||||
|
||||
/**
|
||||
* Create a {@link PutObjectRequest} request for creating
|
||||
* an empty directory.
|
||||
*
|
||||
* @param directory destination directory.
|
||||
* @return request for a zero byte upload.
|
||||
* @return request builder for a zero byte upload.
|
||||
*/
|
||||
PutObjectRequest newDirectoryMarkerRequest(String directory);
|
||||
PutObjectRequest.Builder newDirectoryMarkerRequest(String directory);
|
||||
|
||||
/**
|
||||
* List all multipart uploads under a prefix.
|
||||
* @param prefix prefix to list under
|
||||
* @return the request.
|
||||
* @return the request builder.
|
||||
*/
|
||||
ListMultipartUploadsRequest newListMultipartUploadsRequest(
|
||||
ListMultipartUploadsRequest.Builder newListMultipartUploadsRequestBuilder(
|
||||
@Nullable String prefix);
|
||||
|
||||
/**
|
||||
* Abort a multipart upload.
|
||||
* @param destKey destination object key
|
||||
* @param uploadId ID of initiated upload
|
||||
* @return the request.
|
||||
* @return the request builder.
|
||||
*/
|
||||
AbortMultipartUploadRequest newAbortMultipartUploadRequest(
|
||||
AbortMultipartUploadRequest.Builder newAbortMultipartUploadRequestBuilder(
|
||||
String destKey,
|
||||
String uploadId);
|
||||
|
||||
|
@ -195,10 +153,10 @@ public interface RequestFactory {
|
|||
* Start a multipart upload.
|
||||
* @param destKey destination object key
|
||||
* @param options options for the request
|
||||
* @return the request.
|
||||
* @return the request builder.
|
||||
* @throws PathIOException if multipart uploads are disabled
|
||||
*/
|
||||
InitiateMultipartUploadRequest newMultipartUploadRequest(
|
||||
CreateMultipartUploadRequest.Builder newMultipartUploadRequestBuilder(
|
||||
String destKey,
|
||||
@Nullable PutObjectOptions options) throws PathIOException;
|
||||
|
||||
|
@ -207,107 +165,88 @@ public interface RequestFactory {
|
|||
* @param destKey destination object key
|
||||
* @param uploadId ID of initiated upload
|
||||
* @param partETags ordered list of etags
|
||||
* @return the request.
|
||||
* @return the request builder.
|
||||
*/
|
||||
CompleteMultipartUploadRequest newCompleteMultipartUploadRequest(
|
||||
CompleteMultipartUploadRequest.Builder newCompleteMultipartUploadRequestBuilder(
|
||||
String destKey,
|
||||
String uploadId,
|
||||
List<PartETag> partETags);
|
||||
List<CompletedPart> 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.
|
||||
* Create a GET request builder.
|
||||
* @param key object key
|
||||
* @return the request.
|
||||
* @return the request builder.
|
||||
*/
|
||||
GetObjectRequest newGetObjectRequest(String key);
|
||||
GetObjectRequest.Builder newGetObjectRequestBuilder(String key);
|
||||
|
||||
/**
|
||||
* Create and initialize a part request of a multipart upload.
|
||||
* Exactly one of: {@code uploadStream} or {@code sourceFile}
|
||||
* must be specified.
|
||||
* A subset of the file may be posted, by providing the starting point
|
||||
* in {@code offset} and a length of block in {@code size} equal to
|
||||
* or less than the remaining bytes.
|
||||
* @param destKey destination key of ongoing operation
|
||||
* @param uploadId ID of ongoing upload
|
||||
* @param partNumber current part number of the upload
|
||||
* @param size amount of data
|
||||
* @param uploadStream source of data to upload
|
||||
* @param sourceFile optional source file.
|
||||
* @param offset offset in file to start reading.
|
||||
* @return the request.
|
||||
* Create and initialize a part request builder of a multipart upload.
|
||||
*
|
||||
* @param destKey destination key of ongoing operation
|
||||
* @param uploadId ID of ongoing upload
|
||||
* @param partNumber current part number of the upload
|
||||
* @param size amount of data
|
||||
* @return the request builder.
|
||||
* @throws PathIOException if the part number is out of range.
|
||||
*/
|
||||
UploadPartRequest newUploadPartRequest(
|
||||
UploadPartRequest.Builder newUploadPartRequestBuilder(
|
||||
String destKey,
|
||||
String uploadId,
|
||||
int partNumber,
|
||||
long size,
|
||||
InputStream uploadStream,
|
||||
File sourceFile,
|
||||
long offset) throws PathIOException;
|
||||
long size) throws PathIOException;
|
||||
|
||||
/**
|
||||
* Create a S3 Select request for the destination object.
|
||||
* Create a S3 Select request builder for the destination object.
|
||||
* This does not build the query.
|
||||
* @param key object key
|
||||
* @return the request
|
||||
* @return the request builder
|
||||
*/
|
||||
SelectObjectContentRequest newSelectRequest(String key);
|
||||
SelectObjectContentRequest.Builder newSelectRequestBuilder(String key);
|
||||
|
||||
/**
|
||||
* Create the (legacy) V1 list request.
|
||||
* Create the (legacy) V1 list request builder.
|
||||
* @param key key to list under
|
||||
* @param delimiter delimiter for keys
|
||||
* @param maxKeys maximum number in a list page.
|
||||
* @return the request
|
||||
* @return the request builder.
|
||||
*/
|
||||
ListObjectsRequest newListObjectsV1Request(String key,
|
||||
ListObjectsRequest.Builder newListObjectsV1RequestBuilder(String key,
|
||||
String delimiter,
|
||||
int maxKeys);
|
||||
|
||||
/**
|
||||
* Create the next V1 page list request, following
|
||||
* on from the previous response.
|
||||
* @param prev previous response
|
||||
* @return the request
|
||||
*/
|
||||
|
||||
ListNextBatchOfObjectsRequest newListNextBatchOfObjectsRequest(
|
||||
ObjectListing prev);
|
||||
|
||||
/**
|
||||
* Create a V2 list request.
|
||||
* Create a V2 list request builder.
|
||||
* This will be recycled for any subsequent requests.
|
||||
* @param key key to list under
|
||||
* @param delimiter delimiter for keys
|
||||
* @param maxKeys maximum number in a list page.
|
||||
* @return the request
|
||||
* @return the request builder.
|
||||
*/
|
||||
ListObjectsV2Request newListObjectsV2Request(String key,
|
||||
ListObjectsV2Request.Builder newListObjectsV2RequestBuilder(String key,
|
||||
String delimiter,
|
||||
int maxKeys);
|
||||
|
||||
/**
|
||||
* Create a request to delete a single object.
|
||||
* Create a request builder to delete a single object.
|
||||
* @param key object to delete
|
||||
* @return the request
|
||||
* @return the request builder.
|
||||
*/
|
||||
DeleteObjectRequest newDeleteObjectRequest(String key);
|
||||
DeleteObjectRequest.Builder newDeleteObjectRequestBuilder(String key);
|
||||
|
||||
/**
|
||||
* Bulk delete request.
|
||||
* Create a request builder to delete objects in bulk.
|
||||
* @param keysToDelete list of keys to delete.
|
||||
* @return the request
|
||||
* @return the request builder.
|
||||
*/
|
||||
DeleteObjectsRequest newBulkDeleteRequest(
|
||||
List<DeleteObjectsRequest.KeyVersion> keysToDelete);
|
||||
DeleteObjectsRequest.Builder newBulkDeleteRequestBuilder(
|
||||
List<ObjectIdentifier> keysToDelete);
|
||||
|
||||
}
|
||||
|
|
|
@ -18,15 +18,9 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a.audit;
|
||||
|
||||
import com.amazonaws.AmazonWebServiceRequest;
|
||||
import com.amazonaws.Request;
|
||||
import com.amazonaws.Response;
|
||||
import com.amazonaws.SdkBaseException;
|
||||
import com.amazonaws.handlers.HandlerAfterAttemptContext;
|
||||
import com.amazonaws.handlers.HandlerBeforeAttemptContext;
|
||||
import com.amazonaws.http.HttpResponse;
|
||||
import software.amazon.awssdk.core.SdkRequest;
|
||||
import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
|
||||
|
||||
import org.apache.hadoop.fs.s3a.Retries;
|
||||
|
||||
/**
|
||||
* Callbacks for audit spans. This is implemented
|
||||
|
@ -37,10 +31,10 @@ import org.apache.hadoop.fs.s3a.Retries;
|
|||
* detect this and raise an exception.
|
||||
*
|
||||
* Look at the documentation for
|
||||
* {@code com.amazonaws.handlers.IRequestHandler2} for details
|
||||
* {@code ExecutionInterceptor} for details
|
||||
* on the callbacks.
|
||||
*/
|
||||
public interface AWSAuditEventCallbacks {
|
||||
public interface AWSAuditEventCallbacks extends ExecutionInterceptor {
|
||||
|
||||
/**
|
||||
* Return a span ID which must be unique for all spans within
|
||||
|
@ -66,95 +60,8 @@ public interface AWSAuditEventCallbacks {
|
|||
* It is not invoked on any AWS requests created in the SDK.
|
||||
* Avoid raising exceptions or talking to any remote service;
|
||||
* this callback is for annotation rather than validation.
|
||||
* @param request request request.
|
||||
* @param <T> type of request
|
||||
* @return the request, possibly modified.
|
||||
* @param builder the request builder.
|
||||
*/
|
||||
default <T extends AmazonWebServiceRequest> T requestCreated(T request) {
|
||||
return request;
|
||||
}
|
||||
default void requestCreated(SdkRequest.Builder builder) {}
|
||||
|
||||
/**
|
||||
* Preflight preparation of AWS request.
|
||||
* @param request request
|
||||
* @param <T> type of request
|
||||
* @return an updated request.
|
||||
* @throws AuditFailureException for generic audit failures
|
||||
* @throws SdkBaseException for other reasons.
|
||||
*/
|
||||
@Retries.OnceRaw
|
||||
default <T extends AmazonWebServiceRequest> T beforeExecution(T request)
|
||||
throws AuditFailureException, SdkBaseException {
|
||||
return request;
|
||||
}
|
||||
|
||||
/**
|
||||
* Callback after S3 responded to a request.
|
||||
* @param request request
|
||||
* @param response response.
|
||||
* @throws AuditFailureException for generic audit failures
|
||||
* @throws SdkBaseException for other reasons.
|
||||
*/
|
||||
default void afterResponse(Request<?> request,
|
||||
Response<?> response)
|
||||
throws AuditFailureException, SdkBaseException {
|
||||
}
|
||||
|
||||
/**
|
||||
* Callback after a request resulted in an error.
|
||||
* @param request request
|
||||
* @param response response.
|
||||
* @param exception exception raised.
|
||||
* @throws AuditFailureException for generic audit failures
|
||||
* @throws SdkBaseException for other reasons.
|
||||
*/
|
||||
default void afterError(Request<?> request,
|
||||
Response<?> response,
|
||||
Exception exception)
|
||||
throws AuditFailureException, SdkBaseException {
|
||||
}
|
||||
|
||||
/**
|
||||
* Request before marshalling.
|
||||
* @param request request
|
||||
* @return possibly modified request.
|
||||
*/
|
||||
default AmazonWebServiceRequest beforeMarshalling(
|
||||
AmazonWebServiceRequest request) {
|
||||
return request;
|
||||
}
|
||||
|
||||
/**
|
||||
* Request before marshalling.
|
||||
* @param request request
|
||||
*/
|
||||
default void beforeRequest(Request<?> request) {
|
||||
}
|
||||
|
||||
/**
|
||||
* Before any attempt is made.
|
||||
* @param context full context, including the request.
|
||||
*/
|
||||
default void beforeAttempt(HandlerBeforeAttemptContext context) {
|
||||
}
|
||||
|
||||
/**
|
||||
* After any attempt is made.
|
||||
* @param context full context, including the request.
|
||||
*/
|
||||
default void afterAttempt(
|
||||
HandlerAfterAttemptContext context) {
|
||||
}
|
||||
|
||||
/**
|
||||
* Before unmarshalling the response.
|
||||
* @param request request made.
|
||||
* @param httpResponse response received
|
||||
* @return updated response.
|
||||
*/
|
||||
default HttpResponse beforeUnmarshalling(
|
||||
final Request<?> request,
|
||||
final HttpResponse httpResponse) {
|
||||
return httpResponse;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,24 +20,23 @@ package org.apache.hadoop.fs.s3a.audit;
|
|||
|
||||
import java.util.List;
|
||||
|
||||
import com.amazonaws.AmazonWebServiceRequest;
|
||||
import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.CopyPartRequest;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectRequest;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.GetBucketLocationRequest;
|
||||
import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
|
||||
import com.amazonaws.services.s3.model.GetObjectRequest;
|
||||
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
|
||||
import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.ListObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.ListObjectsV2Request;
|
||||
import com.amazonaws.services.s3.model.ObjectListing;
|
||||
import com.amazonaws.services.s3.model.PutObjectRequest;
|
||||
import com.amazonaws.services.s3.model.SelectObjectContentRequest;
|
||||
import com.amazonaws.services.s3.model.UploadPartRequest;
|
||||
import software.amazon.awssdk.core.SdkRequest;
|
||||
import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest;
|
||||
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
|
||||
import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest;
|
||||
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
|
||||
import software.amazon.awssdk.services.s3.model.GetBucketLocationRequest;
|
||||
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest;
|
||||
import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
|
||||
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
|
||||
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
|
||||
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartCopyRequest;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
|
||||
|
||||
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_GET_REQUEST;
|
||||
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_HTTP_HEAD_REQUEST;
|
||||
|
@ -64,102 +63,85 @@ public class AWSRequestAnalyzer {
|
|||
* read/write and path.
|
||||
* @param request request.
|
||||
* @return information about the request.
|
||||
* @param <T> type of request.
|
||||
*/
|
||||
public <T extends AmazonWebServiceRequest> RequestInfo analyze(T request) {
|
||||
public RequestInfo analyze(SdkRequest request) {
|
||||
|
||||
// this is where Scala's case statement would massively
|
||||
// simplify life.
|
||||
// Please Keep in Alphabetical Order.
|
||||
if (request instanceof AbortMultipartUploadRequest) {
|
||||
return writing(MULTIPART_UPLOAD_ABORTED,
|
||||
((AbortMultipartUploadRequest) request).getKey(),
|
||||
((AbortMultipartUploadRequest) request).key(),
|
||||
0);
|
||||
} else if (request instanceof CompleteMultipartUploadRequest) {
|
||||
CompleteMultipartUploadRequest r
|
||||
= (CompleteMultipartUploadRequest) request;
|
||||
return writing(MULTIPART_UPLOAD_COMPLETED,
|
||||
r.getKey(),
|
||||
r.getPartETags().size());
|
||||
r.key(),
|
||||
r.multipartUpload().parts().size());
|
||||
} else if (request instanceof CreateMultipartUploadRequest) {
|
||||
return writing(MULTIPART_UPLOAD_STARTED,
|
||||
((CreateMultipartUploadRequest) request).key(),
|
||||
0);
|
||||
} else if (request instanceof DeleteObjectRequest) {
|
||||
// DeleteObject: single object
|
||||
return writing(OBJECT_DELETE_REQUEST,
|
||||
((DeleteObjectRequest) request).getKey(),
|
||||
((DeleteObjectRequest) request).key(),
|
||||
1);
|
||||
} else if (request instanceof DeleteObjectsRequest) {
|
||||
// DeleteObjects: bulk delete
|
||||
// use first key as the path
|
||||
DeleteObjectsRequest r = (DeleteObjectsRequest) request;
|
||||
List<DeleteObjectsRequest.KeyVersion> keys
|
||||
= r.getKeys();
|
||||
List<ObjectIdentifier> objectIdentifiers
|
||||
= r.delete().objects();
|
||||
return writing(OBJECT_BULK_DELETE_REQUEST,
|
||||
keys.isEmpty() ? null : keys.get(0).getKey(),
|
||||
keys.size());
|
||||
objectIdentifiers.isEmpty() ? null : objectIdentifiers.get(0).key(),
|
||||
objectIdentifiers.size());
|
||||
} else if (request instanceof GetBucketLocationRequest) {
|
||||
GetBucketLocationRequest r = (GetBucketLocationRequest) request;
|
||||
return reading(STORE_EXISTS_PROBE,
|
||||
r.getBucketName(),
|
||||
r.bucket(),
|
||||
0);
|
||||
} else if (request instanceof GetObjectMetadataRequest) {
|
||||
return reading(ACTION_HTTP_HEAD_REQUEST,
|
||||
((GetObjectMetadataRequest) request).getKey(), 0);
|
||||
} else if (request instanceof GetObjectRequest) {
|
||||
GetObjectRequest r = (GetObjectRequest) request;
|
||||
long[] range = r.getRange();
|
||||
long size = range == null
|
||||
? -1
|
||||
: range[1] - range[0];
|
||||
return reading(ACTION_HTTP_GET_REQUEST,
|
||||
r.getKey(),
|
||||
size);
|
||||
} else if (request instanceof InitiateMultipartUploadRequest) {
|
||||
return writing(MULTIPART_UPLOAD_STARTED,
|
||||
((InitiateMultipartUploadRequest) request).getKey(),
|
||||
0);
|
||||
r.key(),
|
||||
sizeFromRangeHeader(r.range()));
|
||||
} else if (request instanceof HeadObjectRequest) {
|
||||
return reading(ACTION_HTTP_HEAD_REQUEST,
|
||||
((HeadObjectRequest) request).key(), 0);
|
||||
} else if (request instanceof ListMultipartUploadsRequest) {
|
||||
ListMultipartUploadsRequest r
|
||||
= (ListMultipartUploadsRequest) request;
|
||||
return reading(MULTIPART_UPLOAD_LIST,
|
||||
r.getPrefix(),
|
||||
r.getMaxUploads());
|
||||
r.prefix(),
|
||||
r.maxUploads());
|
||||
} else if (request instanceof ListObjectsRequest) {
|
||||
ListObjectsRequest r = (ListObjectsRequest) request;
|
||||
return reading(OBJECT_LIST_REQUEST,
|
||||
r.getPrefix(),
|
||||
r.getMaxKeys());
|
||||
} else if (request instanceof ListNextBatchOfObjectsRequest) {
|
||||
ListNextBatchOfObjectsRequest r = (ListNextBatchOfObjectsRequest) request;
|
||||
ObjectListing l = r.getPreviousObjectListing();
|
||||
String prefix = "";
|
||||
int size = 0;
|
||||
if (l != null) {
|
||||
prefix = l.getPrefix();
|
||||
size = l.getMaxKeys();
|
||||
}
|
||||
return reading(OBJECT_LIST_REQUEST,
|
||||
prefix,
|
||||
size);
|
||||
r.prefix(),
|
||||
r.maxKeys());
|
||||
} else if (request instanceof ListObjectsV2Request) {
|
||||
ListObjectsV2Request r = (ListObjectsV2Request) request;
|
||||
return reading(OBJECT_LIST_REQUEST,
|
||||
r.getPrefix(),
|
||||
r.getMaxKeys());
|
||||
r.prefix(),
|
||||
r.maxKeys());
|
||||
} else if (request instanceof PutObjectRequest) {
|
||||
PutObjectRequest r = (PutObjectRequest) request;
|
||||
return writing(OBJECT_PUT_REQUEST,
|
||||
r.getKey(),
|
||||
r.key(),
|
||||
0);
|
||||
} else if (request instanceof SelectObjectContentRequest) {
|
||||
SelectObjectContentRequest r =
|
||||
(SelectObjectContentRequest) request;
|
||||
return reading(OBJECT_SELECT_REQUESTS,
|
||||
r.getKey(),
|
||||
r.key(),
|
||||
1);
|
||||
} else if (request instanceof UploadPartRequest) {
|
||||
UploadPartRequest r = (UploadPartRequest) request;
|
||||
return writing(MULTIPART_UPLOAD_PART_PUT,
|
||||
r.getKey(),
|
||||
r.getPartSize());
|
||||
r.key(),
|
||||
r.contentLength());
|
||||
}
|
||||
// no explicit support, return classname
|
||||
return writing(request.getClass().getName(), null, 0);
|
||||
|
@ -212,7 +194,7 @@ public class AWSRequestAnalyzer {
|
|||
*/
|
||||
public static boolean
|
||||
isRequestNotAlwaysInSpan(final Object request) {
|
||||
return request instanceof CopyPartRequest
|
||||
return request instanceof UploadPartCopyRequest
|
||||
|| request instanceof CompleteMultipartUploadRequest
|
||||
|| request instanceof GetBucketLocationRequest;
|
||||
}
|
||||
|
@ -225,9 +207,9 @@ public class AWSRequestAnalyzer {
|
|||
* @return true if the transfer manager creates them.
|
||||
*/
|
||||
public static boolean isRequestMultipartIO(final Object request) {
|
||||
return request instanceof CopyPartRequest
|
||||
return request instanceof UploadPartCopyRequest
|
||||
|| request instanceof CompleteMultipartUploadRequest
|
||||
|| request instanceof InitiateMultipartUploadRequest
|
||||
|| request instanceof CreateMultipartUploadRequest
|
||||
|| request instanceof UploadPartRequest;
|
||||
}
|
||||
|
||||
|
@ -307,4 +289,23 @@ public class AWSRequestAnalyzer {
|
|||
private static long toSafeLong(final Number size) {
|
||||
return size != null ? size.longValue() : 0;
|
||||
}
|
||||
|
||||
private static final String BYTES_PREFIX = "bytes=";
|
||||
|
||||
private static Number sizeFromRangeHeader(String rangeHeader) {
|
||||
if (rangeHeader != null && rangeHeader.startsWith(BYTES_PREFIX)) {
|
||||
String[] values = rangeHeader
|
||||
.substring(BYTES_PREFIX.length())
|
||||
.split("-");
|
||||
if (values.length == 2) {
|
||||
try {
|
||||
long start = Long.parseUnsignedLong(values[0]);
|
||||
long end = Long.parseUnsignedLong(values[0]);
|
||||
return end - start;
|
||||
} catch(NumberFormatException e) {
|
||||
}
|
||||
}
|
||||
}
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.lang.reflect.Constructor;
|
|||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.nio.file.AccessDeniedException;
|
||||
|
||||
import com.amazonaws.HandlerContextAware;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -34,10 +33,12 @@ import org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor;
|
|||
import org.apache.hadoop.fs.s3a.audit.impl.NoopAuditManagerS3A;
|
||||
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
|
||||
|
||||
import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
|
||||
|
||||
import static java.util.Objects.requireNonNull;
|
||||
import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_ENABLED;
|
||||
import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_ENABLED_DEFAULT;
|
||||
import static org.apache.hadoop.fs.s3a.audit.impl.S3AInternalAuditConstants.AUDIT_SPAN_HANDLER_CONTEXT;
|
||||
import static org.apache.hadoop.fs.s3a.audit.impl.S3AInternalAuditConstants.AUDIT_SPAN_EXECUTION_ATTRIBUTE;
|
||||
|
||||
/**
|
||||
* Support for integrating auditing within the S3A code.
|
||||
|
@ -123,25 +124,24 @@ public final class AuditIntegration {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the span from a handler context.
|
||||
* @param request request
|
||||
* @param <T> type of request.
|
||||
* Get the span from the execution attributes.
|
||||
* @param executionAttributes the execution attributes
|
||||
* @return the span callbacks or null
|
||||
*/
|
||||
public static <T extends HandlerContextAware> AWSAuditEventCallbacks
|
||||
retrieveAttachedSpan(final T request) {
|
||||
return request.getHandlerContext(AUDIT_SPAN_HANDLER_CONTEXT);
|
||||
public static AuditSpanS3A
|
||||
retrieveAttachedSpan(final ExecutionAttributes executionAttributes) {
|
||||
return executionAttributes.getAttribute(AUDIT_SPAN_EXECUTION_ATTRIBUTE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Attach a span to a handler context.
|
||||
* @param request request
|
||||
* Attach a span to the execution attributes.
|
||||
* @param executionAttributes the execution attributes
|
||||
* @param span span to attach
|
||||
* @param <T> type of request.
|
||||
*/
|
||||
public static <T extends HandlerContextAware> void attachSpanToRequest(
|
||||
final T request, final AWSAuditEventCallbacks span) {
|
||||
request.addHandlerContext(AUDIT_SPAN_HANDLER_CONTEXT, span);
|
||||
public static void attachSpanToRequest(
|
||||
final ExecutionAttributes executionAttributes,
|
||||
final AuditSpanS3A span) {
|
||||
executionAttributes.putAttribute(AUDIT_SPAN_EXECUTION_ATTRIBUTE, span);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -21,9 +21,6 @@ package org.apache.hadoop.fs.s3a.audit;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import com.amazonaws.handlers.RequestHandler2;
|
||||
import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
|
@ -32,6 +29,9 @@ import org.apache.hadoop.fs.store.audit.ActiveThreadSpanSource;
|
|||
import org.apache.hadoop.fs.store.audit.AuditSpanSource;
|
||||
import org.apache.hadoop.service.Service;
|
||||
|
||||
import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
|
||||
import software.amazon.awssdk.transfer.s3.progress.TransferListener;
|
||||
|
||||
/**
|
||||
* Interface for Audit Managers auditing operations through the
|
||||
* AWS libraries.
|
||||
|
@ -56,24 +56,24 @@ public interface AuditManagerS3A extends Service,
|
|||
OperationAuditor getAuditor();
|
||||
|
||||
/**
|
||||
* Create the request handler(s) for this audit service.
|
||||
* The list returned is mutable; new handlers may be added.
|
||||
* @return list of handlers for the SDK.
|
||||
* Create the execution interceptor(s) for this audit service.
|
||||
* The list returned is mutable; new interceptors may be added.
|
||||
* @return list of interceptors for the SDK.
|
||||
* @throws IOException failure.
|
||||
*/
|
||||
List<RequestHandler2> createRequestHandlers() throws IOException;
|
||||
List<ExecutionInterceptor> createExecutionInterceptors() throws IOException;
|
||||
|
||||
/**
|
||||
* Return a transfer state change callback which
|
||||
* Return a transfer callback which
|
||||
* fixes the active span context to be that in which
|
||||
* the state change listener was created.
|
||||
* the transfer listener was created.
|
||||
* This can be used to audit the creation of the multipart
|
||||
* upload initiation request which the transfer manager
|
||||
* makes when a file to be copied is split up.
|
||||
* This must be invoked/used within the active span.
|
||||
* @return a state change listener.
|
||||
* @return a transfer listener.
|
||||
*/
|
||||
TransferStateChangeListener createStateChangeListener();
|
||||
TransferListener createTransferListener();
|
||||
|
||||
/**
|
||||
* Check for permission to access a path.
|
||||
|
|
|
@ -66,13 +66,20 @@ public final class S3AAuditConstants {
|
|||
"org.apache.hadoop.fs.s3a.audit.impl.NoopAuditor";
|
||||
|
||||
/**
|
||||
* List of extra AWS SDK request handlers: {@value}.
|
||||
* These are added to the SDK request chain <i>after</i>
|
||||
* any audit service.
|
||||
* Deprecated list of extra AWS SDK request handlers: {@value}.
|
||||
* Use {@link #AUDIT_EXECUTION_INTERCEPTORS} instead.
|
||||
*/
|
||||
public static final String AUDIT_REQUEST_HANDLERS =
|
||||
"fs.s3a.audit.request.handlers";
|
||||
|
||||
/**
|
||||
* List of extra AWS SDK execution interceptors: {@value}.
|
||||
* These are added to the SDK request chain <i>after</i>
|
||||
* any audit service.
|
||||
*/
|
||||
public static final String AUDIT_EXECUTION_INTERCEPTORS =
|
||||
"fs.s3a.audit.execution.interceptors";
|
||||
|
||||
/**
|
||||
* Should operations outside spans be rejected?
|
||||
* This is for testing coverage of the span code; if used
|
||||
|
|
|
@ -25,16 +25,6 @@ import java.util.ArrayList;
|
|||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import com.amazonaws.AmazonWebServiceRequest;
|
||||
import com.amazonaws.HandlerContextAware;
|
||||
import com.amazonaws.Request;
|
||||
import com.amazonaws.Response;
|
||||
import com.amazonaws.SdkBaseException;
|
||||
import com.amazonaws.handlers.HandlerAfterAttemptContext;
|
||||
import com.amazonaws.handlers.HandlerBeforeAttemptContext;
|
||||
import com.amazonaws.handlers.RequestHandler2;
|
||||
import com.amazonaws.http.HttpResponse;
|
||||
import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -56,16 +46,27 @@ import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
|
|||
import org.apache.hadoop.fs.s3a.audit.OperationAuditor;
|
||||
import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions;
|
||||
import org.apache.hadoop.fs.s3a.audit.S3AAuditConstants;
|
||||
import org.apache.hadoop.fs.s3a.impl.V2Migration;
|
||||
import org.apache.hadoop.fs.store.LogExactlyOnce;
|
||||
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
|
||||
import org.apache.hadoop.service.CompositeService;
|
||||
import org.apache.hadoop.util.functional.FutureIO;
|
||||
|
||||
import software.amazon.awssdk.core.SdkRequest;
|
||||
import software.amazon.awssdk.core.SdkResponse;
|
||||
import software.amazon.awssdk.core.interceptor.Context;
|
||||
import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
|
||||
import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
|
||||
import software.amazon.awssdk.http.SdkHttpRequest;
|
||||
import software.amazon.awssdk.http.SdkHttpResponse;
|
||||
import software.amazon.awssdk.transfer.s3.progress.TransferListener;
|
||||
|
||||
import static java.util.Objects.requireNonNull;
|
||||
import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_FAILURE;
|
||||
import static org.apache.hadoop.fs.s3a.Statistic.AUDIT_REQUEST_EXECUTION;
|
||||
import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.attachSpanToRequest;
|
||||
import static org.apache.hadoop.fs.s3a.audit.AuditIntegration.retrieveAttachedSpan;
|
||||
import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_EXECUTION_INTERCEPTORS;
|
||||
import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HANDLERS;
|
||||
|
||||
/**
|
||||
|
@ -82,10 +83,11 @@ import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_REQUEST_HAN
|
|||
* will deactivate the wrapped span and then
|
||||
* switch the active span to the unbounded span.
|
||||
*
|
||||
* The inner class {@link AWSAuditEventCallbacks} is returned
|
||||
* as a request handler in {@link #createRequestHandlers()};
|
||||
* this forwards all requests to the outer {@code ActiveAuditManagerS3A},
|
||||
* which then locates the active span and forwards the request.
|
||||
* This class also implements {@link ExecutionInterceptor} and
|
||||
* returns itself in {@link #createExecutionInterceptors()};
|
||||
* once registered with the S3 client, the implemented methods
|
||||
* will be called during different parts of an SDK request lifecycle,
|
||||
* which then locate the active span and forward the request.
|
||||
* If any such invocation raises an {@link AuditFailureException}
|
||||
* then the IOStatistics counter for {@code AUDIT_FAILURE}
|
||||
* is incremented.
|
||||
|
@ -390,25 +392,32 @@ public final class ActiveAuditManagerS3A
|
|||
}
|
||||
|
||||
/**
|
||||
* Return a request handler for the AWS SDK which
|
||||
* Return a list of execution interceptors for the AWS SDK which
|
||||
* relays to this class.
|
||||
* @return a request handler.
|
||||
* @return a list of execution interceptors.
|
||||
*/
|
||||
@Override
|
||||
public List<RequestHandler2> createRequestHandlers()
|
||||
public List<ExecutionInterceptor> createExecutionInterceptors()
|
||||
throws IOException {
|
||||
|
||||
// wire up the AWS SDK To call back into this class when
|
||||
// preparing to make S3 calls.
|
||||
List<RequestHandler2> requestHandlers = new ArrayList<>();
|
||||
requestHandlers.add(new SdkRequestHandler());
|
||||
// now look for any more handlers
|
||||
final Class<?>[] handlers = getConfig().getClasses(AUDIT_REQUEST_HANDLERS);
|
||||
List<ExecutionInterceptor> executionInterceptors = new ArrayList<>();
|
||||
executionInterceptors.add(this);
|
||||
|
||||
final String handlers = getConfig().get(AUDIT_REQUEST_HANDLERS);
|
||||
if (handlers != null) {
|
||||
for (Class<?> handler : handlers) {
|
||||
V2Migration.v1RequestHandlersUsed();
|
||||
}
|
||||
|
||||
// TODO: should we remove this and use Global/Service interceptors, see:
|
||||
// https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/core/interceptor/ExecutionInterceptor.html
|
||||
final Class<?>[] interceptors = getConfig().getClasses(AUDIT_EXECUTION_INTERCEPTORS);
|
||||
if (interceptors != null) {
|
||||
for (Class<?> handler : interceptors) {
|
||||
try {
|
||||
Constructor<?> ctor = handler.getConstructor();
|
||||
requestHandlers.add((RequestHandler2)ctor.newInstance());
|
||||
executionInterceptors.add((ExecutionInterceptor) ctor.newInstance());
|
||||
} catch (ExceptionInInitializerError e) {
|
||||
throw FutureIO.unwrapInnerException(e);
|
||||
} catch (Exception e) {
|
||||
|
@ -416,13 +425,18 @@ public final class ActiveAuditManagerS3A
|
|||
}
|
||||
}
|
||||
}
|
||||
return requestHandlers;
|
||||
return executionInterceptors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TransferStateChangeListener createStateChangeListener() {
|
||||
public TransferListener createTransferListener() {
|
||||
final WrappingAuditSpan span = activeSpan();
|
||||
return (transfer, state) -> switchToActiveSpan(span);
|
||||
return new TransferListener() {
|
||||
@Override
|
||||
public void transferInitiated(Context.TransferInitiated context) {
|
||||
switchToActiveSpan(span);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -434,20 +448,18 @@ public final class ActiveAuditManagerS3A
|
|||
}
|
||||
|
||||
/**
|
||||
* Attach a reference to the active thread span, then
|
||||
* invoke the same callback on that active thread.
|
||||
* Audit the creation of a request and retrieve
|
||||
* a reference to the active thread span.
|
||||
*/
|
||||
@Override
|
||||
public <T extends AmazonWebServiceRequest> T requestCreated(
|
||||
final T request) {
|
||||
public void requestCreated(final SdkRequest.Builder builder) {
|
||||
AuditSpanS3A span = getActiveAuditSpan();
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Created Request {} in span {}",
|
||||
analyzer.analyze(request), span);
|
||||
analyzer.analyze(builder.build()), span);
|
||||
}
|
||||
attachSpanToRequest(request, span);
|
||||
try {
|
||||
return span.requestCreated(request);
|
||||
span.requestCreated(builder);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
|
@ -463,14 +475,13 @@ public final class ActiveAuditManagerS3A
|
|||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public <T extends AmazonWebServiceRequest> T beforeExecution(
|
||||
final T request) {
|
||||
public void beforeExecution(Context.BeforeExecution context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_REQUEST_EXECUTION.getSymbol());
|
||||
|
||||
// identify the span and invoke the callback
|
||||
AuditSpanS3A span = getActiveAuditSpan();
|
||||
attachSpanToRequest(executionAttributes, span);
|
||||
try {
|
||||
return extractAndActivateSpanFromRequest(request)
|
||||
.beforeExecution(request);
|
||||
span.beforeExecution(context, executionAttributes);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
|
@ -479,16 +490,14 @@ public final class ActiveAuditManagerS3A
|
|||
|
||||
/**
|
||||
* Forward to active span.
|
||||
* @param request request
|
||||
* @param response response.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void afterResponse(final Request<?> request,
|
||||
final Response<?> response)
|
||||
throws AuditFailureException, SdkBaseException {
|
||||
public void afterExecution(Context.AfterExecution context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
try {
|
||||
extractAndActivateSpanFromRequest(request)
|
||||
.afterResponse(request, response);
|
||||
extractAndActivateSpanFromRequest(context.request(), executionAttributes)
|
||||
.afterExecution(context, executionAttributes);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
|
@ -496,18 +505,19 @@ public final class ActiveAuditManagerS3A
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the active span from the handler context,
|
||||
* Get the active span from the execution attributes,
|
||||
* falling back to the active thread span if there
|
||||
* is nothing in the context.
|
||||
* Provided the span is a wrapped span, the
|
||||
* is nothing in the attributes.
|
||||
* Provided the span is a wrapped span, the span is
|
||||
* activated.
|
||||
* @param request request
|
||||
* @param <T> type of request.
|
||||
* @return the callbacks
|
||||
* @param executionAttributes the execution attributes
|
||||
* @return the active span
|
||||
*/
|
||||
private <T extends HandlerContextAware> AWSAuditEventCallbacks
|
||||
extractAndActivateSpanFromRequest(final T request) {
|
||||
AWSAuditEventCallbacks span;
|
||||
span = retrieveAttachedSpan(request);
|
||||
private AuditSpanS3A extractAndActivateSpanFromRequest(
|
||||
final SdkRequest request,
|
||||
final ExecutionAttributes executionAttributes) {
|
||||
AuditSpanS3A span = retrieveAttachedSpan(executionAttributes);
|
||||
if (span == null) {
|
||||
// no span is attached. Not unusual for the copy operations,
|
||||
// or for calls to GetBucketLocation made by the AWS client
|
||||
|
@ -531,17 +541,14 @@ public final class ActiveAuditManagerS3A
|
|||
/**
|
||||
* Forward to active span.
|
||||
* @param request request
|
||||
* @param response response.
|
||||
* @param exception exception raised.
|
||||
*/
|
||||
@Override
|
||||
public void afterError(final Request<?> request,
|
||||
final Response<?> response,
|
||||
final Exception exception)
|
||||
throws AuditFailureException, SdkBaseException {
|
||||
* {@inheritDoc}
|
||||
*/@Override
|
||||
public void onExecutionFailure(Context.FailedExecution context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
try {
|
||||
extractAndActivateSpanFromRequest(request)
|
||||
.afterError(request, response, exception);
|
||||
extractAndActivateSpanFromRequest(context.request(),
|
||||
executionAttributes)
|
||||
.onExecutionFailure(context, executionAttributes);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
|
@ -549,11 +556,12 @@ public final class ActiveAuditManagerS3A
|
|||
}
|
||||
|
||||
@Override
|
||||
public AmazonWebServiceRequest beforeMarshalling(
|
||||
final AmazonWebServiceRequest request) {
|
||||
public SdkRequest modifyRequest(Context.ModifyRequest context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
try {
|
||||
return extractAndActivateSpanFromRequest(request)
|
||||
.beforeMarshalling(request);
|
||||
return extractAndActivateSpanFromRequest(context.request(),
|
||||
executionAttributes)
|
||||
.modifyRequest(context, executionAttributes);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
|
@ -561,10 +569,12 @@ public final class ActiveAuditManagerS3A
|
|||
}
|
||||
|
||||
@Override
|
||||
public void beforeRequest(final Request<?> request) {
|
||||
public void beforeMarshalling(Context.BeforeMarshalling context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
try {
|
||||
extractAndActivateSpanFromRequest(request)
|
||||
.beforeRequest(request);
|
||||
extractAndActivateSpanFromRequest(context.request(),
|
||||
executionAttributes)
|
||||
.beforeMarshalling(context, executionAttributes);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
|
@ -572,10 +582,12 @@ public final class ActiveAuditManagerS3A
|
|||
}
|
||||
|
||||
@Override
|
||||
public void beforeAttempt(final HandlerBeforeAttemptContext context) {
|
||||
public void afterMarshalling(Context.AfterMarshalling context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
try {
|
||||
extractAndActivateSpanFromRequest(context.getRequest())
|
||||
.beforeAttempt(context);
|
||||
extractAndActivateSpanFromRequest(context.request(),
|
||||
executionAttributes)
|
||||
.afterMarshalling(context, executionAttributes);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
|
@ -583,10 +595,12 @@ public final class ActiveAuditManagerS3A
|
|||
}
|
||||
|
||||
@Override
|
||||
public void afterAttempt(final HandlerAfterAttemptContext context) {
|
||||
public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
try {
|
||||
extractAndActivateSpanFromRequest(context.getRequest())
|
||||
.afterAttempt(context);
|
||||
return extractAndActivateSpanFromRequest(context.request(),
|
||||
executionAttributes)
|
||||
.modifyHttpRequest(context, executionAttributes);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
|
@ -594,73 +608,80 @@ public final class ActiveAuditManagerS3A
|
|||
}
|
||||
|
||||
@Override
|
||||
public HttpResponse beforeUnmarshalling(final Request<?> request,
|
||||
final HttpResponse httpResponse) {
|
||||
public void beforeTransmission(Context.BeforeTransmission context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
try {
|
||||
extractAndActivateSpanFromRequest(request.getOriginalRequest())
|
||||
.beforeUnmarshalling(request, httpResponse);
|
||||
extractAndActivateSpanFromRequest(context.request(),
|
||||
executionAttributes)
|
||||
.beforeTransmission(context, executionAttributes);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
}
|
||||
return httpResponse;
|
||||
}
|
||||
|
||||
/**
|
||||
* Callbacks from the AWS SDK; all forward to the ActiveAuditManagerS3A.
|
||||
* We need a separate class because the SDK requires the handler list
|
||||
* to be list of {@code RequestHandler2} instances.
|
||||
*/
|
||||
private class SdkRequestHandler extends RequestHandler2 {
|
||||
|
||||
@Override
|
||||
public AmazonWebServiceRequest beforeExecution(
|
||||
final AmazonWebServiceRequest request) {
|
||||
return ActiveAuditManagerS3A.this.beforeExecution(request);
|
||||
@Override
|
||||
public void afterTransmission(Context.AfterTransmission context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
try {
|
||||
extractAndActivateSpanFromRequest(context.request(),
|
||||
executionAttributes)
|
||||
.afterTransmission(context, executionAttributes);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterResponse(final Request<?> request,
|
||||
final Response<?> response) {
|
||||
ActiveAuditManagerS3A.this.afterResponse(request, response);
|
||||
@Override
|
||||
public SdkHttpResponse modifyHttpResponse(Context.ModifyHttpResponse context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
try {
|
||||
return extractAndActivateSpanFromRequest(context.request(),
|
||||
executionAttributes)
|
||||
.modifyHttpResponse(context, executionAttributes);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterError(final Request<?> request,
|
||||
final Response<?> response,
|
||||
final Exception e) {
|
||||
ActiveAuditManagerS3A.this.afterError(request, response, e);
|
||||
@Override
|
||||
public void beforeUnmarshalling(Context.BeforeUnmarshalling context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
try {
|
||||
extractAndActivateSpanFromRequest(context.request(),
|
||||
executionAttributes)
|
||||
.beforeUnmarshalling(context, executionAttributes);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public AmazonWebServiceRequest beforeMarshalling(
|
||||
final AmazonWebServiceRequest request) {
|
||||
return ActiveAuditManagerS3A.this.beforeMarshalling(request);
|
||||
@Override
|
||||
public void afterUnmarshalling(Context.AfterUnmarshalling context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
try {
|
||||
extractAndActivateSpanFromRequest(context.request(),
|
||||
executionAttributes)
|
||||
.afterUnmarshalling(context, executionAttributes);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeRequest(final Request<?> request) {
|
||||
ActiveAuditManagerS3A.this.beforeRequest(request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforeAttempt(
|
||||
final HandlerBeforeAttemptContext context) {
|
||||
ActiveAuditManagerS3A.this.beforeAttempt(context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HttpResponse beforeUnmarshalling(
|
||||
final Request<?> request,
|
||||
final HttpResponse httpResponse) {
|
||||
return ActiveAuditManagerS3A.this.beforeUnmarshalling(request,
|
||||
httpResponse);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterAttempt(
|
||||
final HandlerAfterAttemptContext context) {
|
||||
ActiveAuditManagerS3A.this.afterAttempt(context);
|
||||
@Override
|
||||
public SdkResponse modifyResponse(Context.ModifyResponse context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
try {
|
||||
return extractAndActivateSpanFromRequest(context.request(),
|
||||
executionAttributes)
|
||||
.modifyResponse(context, executionAttributes);
|
||||
} catch (AuditFailureException e) {
|
||||
ioStatisticsStore.incrementCounter(AUDIT_FAILURE.getSymbol());
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -748,9 +769,8 @@ public final class ActiveAuditManagerS3A
|
|||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public <T extends AmazonWebServiceRequest> T requestCreated(
|
||||
final T request) {
|
||||
return span.requestCreated(request);
|
||||
public void requestCreated(final SdkRequest.Builder builder) {
|
||||
span.requestCreated(builder);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -774,79 +794,132 @@ public final class ActiveAuditManagerS3A
|
|||
|
||||
/**
|
||||
* Forward to the inner span.
|
||||
* @param request request
|
||||
* @param <T> type of request
|
||||
* @return an updated request.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public <T extends AmazonWebServiceRequest> T beforeExecution(
|
||||
final T request) {
|
||||
return span.beforeExecution(request);
|
||||
public void beforeExecution(Context.BeforeExecution context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
span.beforeExecution(context, executionAttributes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Forward to the inner span.
|
||||
* @param request request
|
||||
* @param response response.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void afterResponse(final Request<?> request,
|
||||
final Response<?> response) {
|
||||
span.afterResponse(request, response);
|
||||
public void afterExecution(Context.AfterExecution context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
span.afterExecution(context, executionAttributes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Forward to the inner span.
|
||||
* @param request request
|
||||
* @param response response.
|
||||
* @param exception exception raised.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void afterError(final Request<?> request,
|
||||
final Response<?> response,
|
||||
final Exception exception) {
|
||||
span.afterError(request, response, exception);
|
||||
public void onExecutionFailure(Context.FailedExecution context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
span.onExecutionFailure(context, executionAttributes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Forward to the inner span.
|
||||
* @param request request
|
||||
* @return request to marshall
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public AmazonWebServiceRequest beforeMarshalling(
|
||||
final AmazonWebServiceRequest request) {
|
||||
return span.beforeMarshalling(request);
|
||||
public void beforeMarshalling(Context.BeforeMarshalling context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
span.beforeMarshalling(context, executionAttributes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Forward to the inner span.
|
||||
* @param request request
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void beforeRequest(final Request<?> request) {
|
||||
span.beforeRequest(request);
|
||||
public SdkRequest modifyRequest(Context.ModifyRequest context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
return span.modifyRequest(context, executionAttributes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Forward to the inner span.
|
||||
* @param context full context, including the request.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void beforeAttempt(
|
||||
final HandlerBeforeAttemptContext context) {
|
||||
span.beforeAttempt(context);
|
||||
public void afterMarshalling(Context.AfterMarshalling context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
span.afterMarshalling(context, executionAttributes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Forward to the inner span.
|
||||
*
|
||||
* @param context full context, including the request.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void afterAttempt(
|
||||
final HandlerAfterAttemptContext context) {
|
||||
span.afterAttempt(context);
|
||||
public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
return span.modifyHttpRequest(context, executionAttributes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Forward to the inner span.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void beforeTransmission(Context.BeforeTransmission context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
span.beforeTransmission(context, executionAttributes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Forward to the inner span.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void afterTransmission(Context.AfterTransmission context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
span.afterTransmission(context, executionAttributes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Forward to the inner span.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public SdkHttpResponse modifyHttpResponse(Context.ModifyHttpResponse context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
return span.modifyHttpResponse(context, executionAttributes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Forward to the inner span.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void beforeUnmarshalling(Context.BeforeUnmarshalling context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
span.beforeUnmarshalling(context, executionAttributes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Forward to the inner span.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void afterUnmarshalling(Context.AfterUnmarshalling context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
span.afterUnmarshalling(context, executionAttributes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Forward to the inner span.
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public SdkResponse modifyResponse(Context.ModifyResponse context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
return span.modifyResponse(context, executionAttributes);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -859,5 +932,4 @@ public final class ActiveAuditManagerS3A
|
|||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -24,10 +24,11 @@ import java.util.Collection;
|
|||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import com.amazonaws.AmazonWebServiceRequest;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectRequest;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.GetObjectRequest;
|
||||
import software.amazon.awssdk.core.SdkRequest;
|
||||
import software.amazon.awssdk.core.interceptor.Context;
|
||||
import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
|
||||
import software.amazon.awssdk.http.SdkHttpRequest;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -260,21 +261,21 @@ public class LoggingAuditor
|
|||
* Attach Range of data for GetObject Request.
|
||||
* @param request given get object request
|
||||
*/
|
||||
private void attachRangeFromRequest(AmazonWebServiceRequest request) {
|
||||
if (request instanceof GetObjectRequest) {
|
||||
long[] rangeValue = ((GetObjectRequest) request).getRange();
|
||||
if (rangeValue == null || rangeValue.length == 0) {
|
||||
return;
|
||||
}
|
||||
if (rangeValue.length != 2) {
|
||||
WARN_INCORRECT_RANGE.warn("Expected range to contain 0 or 2 elements."
|
||||
+ " Got {} elements. Ignoring.", rangeValue.length);
|
||||
return;
|
||||
}
|
||||
String combinedRangeValue = String.format("%d-%d", rangeValue[0], rangeValue[1]);
|
||||
referrer.set(AuditConstants.PARAM_RANGE, combinedRangeValue);
|
||||
}
|
||||
}
|
||||
// private void attachRangeFromRequest(AmazonWebServiceRequest request) {
|
||||
// if (request instanceof GetObjectRequest) {
|
||||
// long[] rangeValue = ((GetObjectRequest) request).getRange();
|
||||
// if (rangeValue == null || rangeValue.length == 0) {
|
||||
// return;
|
||||
// }
|
||||
// if (rangeValue.length != 2) {
|
||||
// WARN_INCORRECT_RANGE.warn("Expected range to contain 0 or 2 elements."
|
||||
// + " Got {} elements. Ignoring.", rangeValue.length);
|
||||
// return;
|
||||
// }
|
||||
// String combinedRangeValue = String.format("%d-%d", rangeValue[0], rangeValue[1]);
|
||||
// referrer.set(AuditConstants.PARAM_RANGE, combinedRangeValue);
|
||||
// }
|
||||
// }
|
||||
|
||||
private final String description;
|
||||
|
||||
|
@ -346,48 +347,61 @@ public class LoggingAuditor
|
|||
referrer.set(key, value);
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Before execution, the logging auditor always builds
|
||||
* the referrer header, saves to the outer class
|
||||
* (where {@link #getLastHeader()} can retrieve it,
|
||||
* Before transmitting a request, the logging auditor
|
||||
* always builds the referrer header, saves to the outer
|
||||
* class (where {@link #getLastHeader()} can retrieve it,
|
||||
* and logs at debug.
|
||||
* If configured to add the header to the S3 logs, it will
|
||||
* be set as the HTTP referrer.
|
||||
* @param request request
|
||||
* @param <T> type of request.
|
||||
* @return the request with any extra headers.
|
||||
* @param context The current state of the execution,
|
||||
* including the SDK and current HTTP request.
|
||||
* @param executionAttributes A mutable set of attributes scoped
|
||||
* to one specific request/response
|
||||
* cycle that can be used to give data
|
||||
* to future lifecycle methods.
|
||||
* @return The potentially-modified HTTP request that should be
|
||||
* sent to the service. Must not be null.
|
||||
*/
|
||||
@Override
|
||||
public <T extends AmazonWebServiceRequest> T beforeExecution(
|
||||
final T request) {
|
||||
// attach range for GetObject requests
|
||||
attachRangeFromRequest(request);
|
||||
// for delete op, attach the number of files to delete
|
||||
attachDeleteKeySizeAttribute(request);
|
||||
public SdkHttpRequest modifyHttpRequest(Context.ModifyHttpRequest context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
SdkHttpRequest httpRequest = context.httpRequest();
|
||||
|
||||
// attach range for GetObject requests
|
||||
attachRangeFromRequest(httpRequest, executionAttributes);
|
||||
|
||||
// for delete op, attach the number of files to delete
|
||||
attachDeleteKeySizeAttribute(request);
|
||||
|
||||
// build the referrer header
|
||||
final String header = referrer.buildHttpReferrer();
|
||||
// update the outer class's field.
|
||||
setLastHeader(header);
|
||||
if (headerEnabled) {
|
||||
// add the referrer header
|
||||
request.putCustomRequestHeader(HEADER_REFERRER,
|
||||
header);
|
||||
httpRequest = httpRequest.toBuilder()
|
||||
.appendHeader(HEADER_REFERRER, header)
|
||||
.build();
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("[{}] {} Executing {} with {}; {}",
|
||||
currentThreadID(),
|
||||
getSpanId(),
|
||||
getOperationName(),
|
||||
analyzer.analyze(request),
|
||||
analyzer.analyze(context.request()),
|
||||
header);
|
||||
}
|
||||
|
||||
// now see if the request is actually a blocked multipart request
|
||||
if (!isMultipartUploadEnabled && isRequestMultipartIO(request)) {
|
||||
if (!isMultipartUploadEnabled && isRequestMultipartIO(httpRequest)) {
|
||||
throw new AuditOperationRejectedException("Multipart IO request "
|
||||
+ request + " rejected " + header);
|
||||
+ httpRequest + " rejected " + header);
|
||||
}
|
||||
|
||||
return request;
|
||||
return httpRequest;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -460,15 +474,13 @@ public class LoggingAuditor
|
|||
}
|
||||
|
||||
@Override
|
||||
public <T extends AmazonWebServiceRequest> T requestCreated(
|
||||
final T request) {
|
||||
public void requestCreated(final SdkRequest.Builder builder) {
|
||||
String error = "Creating a request outside an audit span "
|
||||
+ analyzer.analyze(request);
|
||||
+ analyzer.analyze(builder.build());
|
||||
LOG.info(error);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(error, new AuditFailureException("unaudited"));
|
||||
}
|
||||
return request;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -476,20 +488,22 @@ public class LoggingAuditor
|
|||
* increment the failure count.
|
||||
* Some requests (e.g. copy part) are not expected in spans due
|
||||
* to how they are executed; these do not trigger failures.
|
||||
* @param request request
|
||||
* @param <T> type of request
|
||||
* @return an updated request.
|
||||
* @throws AuditFailureException if failure is enabled.
|
||||
* @param context The current state of the execution, including
|
||||
* the unmodified SDK request from the service
|
||||
* client call.
|
||||
* @param executionAttributes A mutable set of attributes scoped
|
||||
* to one specific request/response
|
||||
* cycle that can be used to give data
|
||||
* to future lifecycle methods.
|
||||
*/
|
||||
@Override
|
||||
public <T extends AmazonWebServiceRequest> T beforeExecution(
|
||||
final T request) {
|
||||
|
||||
public void beforeExecution(Context.BeforeExecution context,
|
||||
ExecutionAttributes executionAttributes) {
|
||||
String error = "executing a request outside an audit span "
|
||||
+ analyzer.analyze(request);
|
||||
+ analyzer.analyze(context.request());
|
||||
final String unaudited = getSpanId() + " "
|
||||
+ UNAUDITED_OPERATION + " " + error;
|
||||
if (isRequestNotAlwaysInSpan(request)) {
|
||||
if (isRequestNotAlwaysInSpan(context.request())) {
|
||||
// can get by auditing during a copy, so don't overreact
|
||||
LOG.debug(unaudited);
|
||||
} else {
|
||||
|
@ -500,7 +514,7 @@ public class LoggingAuditor
|
|||
}
|
||||
}
|
||||
// now hand off to the superclass for its normal preparation
|
||||
return super.beforeExecution(request);
|
||||
super.beforeExecution(context, executionAttributes);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,10 +24,6 @@ import java.util.ArrayList;
|
|||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
import com.amazonaws.handlers.RequestHandler2;
|
||||
import com.amazonaws.services.s3.transfer.Transfer;
|
||||
import com.amazonaws.services.s3.transfer.internal.TransferStateChangeListener;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -39,6 +35,9 @@ import org.apache.hadoop.fs.s3a.audit.OperationAuditor;
|
|||
import org.apache.hadoop.fs.s3a.audit.OperationAuditorOptions;
|
||||
import org.apache.hadoop.service.CompositeService;
|
||||
|
||||
import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
|
||||
import software.amazon.awssdk.transfer.s3.progress.TransferListener;
|
||||
|
||||
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
|
||||
|
||||
/**
|
||||
|
@ -121,17 +120,13 @@ public class NoopAuditManagerS3A extends CompositeService
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<RequestHandler2> createRequestHandlers() throws IOException {
|
||||
public List<ExecutionInterceptor> createExecutionInterceptors() throws IOException {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TransferStateChangeListener createStateChangeListener() {
|
||||
return new TransferStateChangeListener() {
|
||||
public void transferStateChanged(final Transfer transfer,
|
||||
final Transfer.TransferState state) {
|
||||
}
|
||||
};
|
||||
public TransferListener createTransferListener() {
|
||||
return new TransferListener() {};
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -18,10 +18,10 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a.audit.impl;
|
||||
|
||||
import com.amazonaws.handlers.HandlerContextKey;
|
||||
import software.amazon.awssdk.core.interceptor.ExecutionAttribute;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.s3a.audit.AWSAuditEventCallbacks;
|
||||
import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
|
||||
|
||||
/**
|
||||
* Internal constants; not intended for public use, or
|
||||
|
@ -34,11 +34,11 @@ public final class S3AInternalAuditConstants {
|
|||
}
|
||||
|
||||
/**
|
||||
* Handler key for audit span callbacks.
|
||||
* This is used to bind the handler in the AWS code.
|
||||
* Exceution attribute for audit span callbacks.
|
||||
* This is used to retrieve the span in the AWS code.
|
||||
*/
|
||||
public static final HandlerContextKey<AWSAuditEventCallbacks>
|
||||
AUDIT_SPAN_HANDLER_CONTEXT =
|
||||
new HandlerContextKey<>(
|
||||
"org.apache.hadoop.fs.s3a.audit.AWSAuditEventCallbacks");
|
||||
public static final ExecutionAttribute<AuditSpanS3A>
|
||||
AUDIT_SPAN_EXECUTION_ATTRIBUTE =
|
||||
new ExecutionAttribute<>(
|
||||
"org.apache.hadoop.fs.s3a.audit.AuditSpanS3A");
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.hadoop.fs.s3a.auth;
|
|||
import javax.annotation.Nullable;
|
||||
import java.net.URI;
|
||||
|
||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
|
@ -29,12 +29,9 @@ import org.apache.hadoop.conf.Configuration;
|
|||
* Base class for AWS credential providers which
|
||||
* take a URI and config in their constructor.
|
||||
*
|
||||
* @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider
|
||||
* as part of upgrading S3A to SDK V2. See HADOOP-18073.
|
||||
*/
|
||||
@Deprecated
|
||||
public abstract class AbstractAWSCredentialProvider
|
||||
implements AWSCredentialsProvider {
|
||||
implements AwsCredentialsProvider {
|
||||
|
||||
private final URI binding;
|
||||
|
||||
|
@ -65,10 +62,4 @@ public abstract class AbstractAWSCredentialProvider
|
|||
return binding;
|
||||
}
|
||||
|
||||
/**
|
||||
* Refresh is a no-op by default.
|
||||
*/
|
||||
@Override
|
||||
public void refresh() {
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,29 +23,26 @@ import java.net.URI;
|
|||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import com.amazonaws.SdkBaseException;
|
||||
import com.amazonaws.auth.AWSCredentials;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.s3a.CredentialInitializationException;
|
||||
import org.apache.hadoop.fs.s3a.Invoker;
|
||||
import org.apache.hadoop.fs.s3a.Retries;
|
||||
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentials;
|
||||
import software.amazon.awssdk.core.exception.SdkException;
|
||||
|
||||
/**
|
||||
* Base class for session credential support.
|
||||
*
|
||||
* @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider
|
||||
* as part of upgrading S3A to SDK V2. See HADOOP-18073.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@Deprecated
|
||||
public abstract class AbstractSessionCredentialsProvider
|
||||
extends AbstractAWSCredentialProvider {
|
||||
|
||||
/** Credentials, created in {@link #init()}. */
|
||||
private volatile AWSCredentials awsCredentials;
|
||||
private volatile AwsCredentials awsCredentials;
|
||||
|
||||
/** Atomic flag for on-demand initialization. */
|
||||
private final AtomicBoolean initialized = new AtomicBoolean(false);
|
||||
|
@ -105,7 +102,7 @@ public abstract class AbstractSessionCredentialsProvider
|
|||
* @return the credentials
|
||||
* @throws IOException on any failure.
|
||||
*/
|
||||
protected abstract AWSCredentials createCredentials(Configuration config)
|
||||
protected abstract AwsCredentials createCredentials(Configuration config)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -115,10 +112,10 @@ public abstract class AbstractSessionCredentialsProvider
|
|||
* is thrown here before any attempt to return the credentials
|
||||
* is made.
|
||||
* @return credentials, if set.
|
||||
* @throws SdkBaseException if one was raised during init
|
||||
* @throws SdkException if one was raised during init
|
||||
* @throws CredentialInitializationException on other failures.
|
||||
*/
|
||||
public AWSCredentials getCredentials() throws SdkBaseException {
|
||||
public AwsCredentials resolveCredentials() throws SdkException {
|
||||
// do an on-demand init then raise an AWS SDK exception if
|
||||
// there was a failure.
|
||||
try {
|
||||
|
@ -126,8 +123,8 @@ public abstract class AbstractSessionCredentialsProvider
|
|||
init();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
if (e.getCause() instanceof SdkBaseException) {
|
||||
throw (SdkBaseException) e.getCause();
|
||||
if (e.getCause() instanceof SdkException) {
|
||||
throw (SdkException) e.getCause();
|
||||
} else {
|
||||
throw new CredentialInitializationException(e.getMessage(), e);
|
||||
}
|
||||
|
@ -165,15 +162,16 @@ public abstract class AbstractSessionCredentialsProvider
|
|||
* This will be interpreted as "this provider has no credentials to offer",
|
||||
* rather than an explicit error or anonymous access.
|
||||
*/
|
||||
protected static final class NoCredentials implements AWSCredentials {
|
||||
protected static final class NoCredentials implements AwsCredentials {
|
||||
@Override
|
||||
public String getAWSAccessKeyId() {
|
||||
public String accessKeyId() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getAWSSecretKey() {
|
||||
public String secretAccessKey() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -26,16 +26,18 @@ import java.util.Arrays;
|
|||
import java.util.Locale;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import com.amazonaws.auth.AWSCredentials;
|
||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||
import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
|
||||
import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
|
||||
import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
|
||||
import com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.util.Sets;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
|
||||
import software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider;
|
||||
import software.amazon.awssdk.core.exception.SdkClientException;
|
||||
import software.amazon.awssdk.services.sts.StsClient;
|
||||
import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider;
|
||||
import software.amazon.awssdk.services.sts.model.AssumeRoleRequest;
|
||||
import software.amazon.awssdk.services.sts.model.StsException;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
@ -61,13 +63,10 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.buildAWSProviderList;
|
|||
*
|
||||
* Classname is used in configuration files; do not move.
|
||||
*
|
||||
* @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider
|
||||
* as part of upgrading S3A to SDK V2. See HADOOP-18073.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
@Deprecated
|
||||
public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
|
||||
public class AssumedRoleCredentialProvider implements AwsCredentialsProvider,
|
||||
Closeable {
|
||||
|
||||
private static final Logger LOG =
|
||||
|
@ -78,7 +77,7 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
|
|||
public static final String E_NO_ROLE = "Unset property "
|
||||
+ ASSUMED_ROLE_ARN;
|
||||
|
||||
private final STSAssumeRoleSessionCredentialsProvider stsProvider;
|
||||
private final StsAssumeRoleCredentialsProvider stsProvider;
|
||||
|
||||
private final String sessionName;
|
||||
|
||||
|
@ -90,15 +89,17 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
|
|||
|
||||
private final Invoker invoker;
|
||||
|
||||
private final StsClient stsClient;
|
||||
|
||||
/**
|
||||
* Instantiate.
|
||||
* This calls {@link #getCredentials()} to fail fast on the inner
|
||||
* This calls {@link #resolveCredentials()} to fail fast on the inner
|
||||
* role credential retrieval.
|
||||
* @param fsUri possibly null URI of the filesystem.
|
||||
* @param conf configuration
|
||||
* @throws IOException on IO problems and some parameter checking
|
||||
* @throws IllegalArgumentException invalid parameters
|
||||
* @throws AWSSecurityTokenServiceException problems getting credentials
|
||||
* @throws StsException problems getting credentials
|
||||
*/
|
||||
public AssumedRoleCredentialProvider(@Nullable URI fsUri, Configuration conf)
|
||||
throws IOException {
|
||||
|
@ -125,29 +126,31 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
|
|||
String policy = conf.getTrimmed(ASSUMED_ROLE_POLICY, "");
|
||||
|
||||
LOG.debug("{}", this);
|
||||
STSAssumeRoleSessionCredentialsProvider.Builder builder
|
||||
= new STSAssumeRoleSessionCredentialsProvider.Builder(arn, sessionName);
|
||||
builder.withRoleSessionDurationSeconds((int) duration);
|
||||
|
||||
AssumeRoleRequest.Builder requestBuilder =
|
||||
AssumeRoleRequest.builder().roleArn(arn).roleSessionName(sessionName)
|
||||
.durationSeconds((int) duration);
|
||||
|
||||
if (StringUtils.isNotEmpty(policy)) {
|
||||
LOG.debug("Scope down policy {}", policy);
|
||||
builder.withScopeDownPolicy(policy);
|
||||
requestBuilder.policy(policy);
|
||||
}
|
||||
|
||||
String endpoint = conf.getTrimmed(ASSUMED_ROLE_STS_ENDPOINT, "");
|
||||
String region = conf.getTrimmed(ASSUMED_ROLE_STS_ENDPOINT_REGION,
|
||||
ASSUMED_ROLE_STS_ENDPOINT_REGION_DEFAULT);
|
||||
AWSSecurityTokenServiceClientBuilder stsbuilder =
|
||||
stsClient =
|
||||
STSClientFactory.builder(
|
||||
conf,
|
||||
fsUri != null ? fsUri.getHost() : "",
|
||||
credentialsToSTS,
|
||||
endpoint,
|
||||
region);
|
||||
// the STS client is not tracked for a shutdown in close(), because it
|
||||
// (currently) throws an UnsupportedOperationException in shutdown().
|
||||
builder.withStsClient(stsbuilder.build());
|
||||
region).build();
|
||||
|
||||
//now build the provider
|
||||
stsProvider = builder.build();
|
||||
stsProvider = StsAssumeRoleCredentialsProvider.builder()
|
||||
.refreshRequest(requestBuilder.build())
|
||||
.stsClient(stsClient).build();
|
||||
|
||||
// to handle STS throttling by the AWS account, we
|
||||
// need to retry
|
||||
|
@ -155,21 +158,21 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
|
|||
|
||||
// and force in a fail-fast check just to keep the stack traces less
|
||||
// convoluted
|
||||
getCredentials();
|
||||
resolveCredentials();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get credentials.
|
||||
* @return the credentials
|
||||
* @throws AWSSecurityTokenServiceException if none could be obtained.
|
||||
* @throws StsException if none could be obtained.
|
||||
*/
|
||||
@Override
|
||||
@Retries.RetryRaw
|
||||
public AWSCredentials getCredentials() {
|
||||
public AwsCredentials resolveCredentials() {
|
||||
try {
|
||||
return invoker.retryUntranslated("getCredentials",
|
||||
true,
|
||||
stsProvider::getCredentials);
|
||||
stsProvider::resolveCredentials);
|
||||
} catch (IOException e) {
|
||||
// this is in the signature of retryUntranslated;
|
||||
// its hard to see how this could be raised, but for
|
||||
|
@ -178,24 +181,19 @@ public class AssumedRoleCredentialProvider implements AWSCredentialsProvider,
|
|||
throw new CredentialInitializationException(
|
||||
"getCredentials failed: " + e,
|
||||
e);
|
||||
} catch (AWSSecurityTokenServiceException e) {
|
||||
} catch (SdkClientException e) {
|
||||
LOG.error("Failed to get credentials for role {}",
|
||||
arn, e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void refresh() {
|
||||
stsProvider.refresh();
|
||||
}
|
||||
|
||||
/**
|
||||
* Propagate the close() call to the inner stsProvider.
|
||||
*/
|
||||
@Override
|
||||
public void close() {
|
||||
S3AUtils.closeAutocloseables(LOG, stsProvider, credentialsToSTS);
|
||||
S3AUtils.closeAutocloseables(LOG, stsProvider, credentialsToSTS, stsClient);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,10 +21,11 @@ package org.apache.hadoop.fs.s3a.auth;
|
|||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
import com.amazonaws.AmazonClientException;
|
||||
import com.amazonaws.auth.AWSCredentials;
|
||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||
import com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
|
||||
import software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider;
|
||||
import software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider;
|
||||
import software.amazon.awssdk.core.exception.SdkClientException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
@ -41,17 +42,14 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
* <p>
|
||||
* It is implicitly public; marked evolving as we can change its semantics.
|
||||
*
|
||||
* @deprecated This class will be replaced by one that implements AWS SDK V2's AwsCredentialProvider
|
||||
* as part of upgrading S3A to SDK V2. See HADOOP-18073.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
@Deprecated
|
||||
public class IAMInstanceCredentialsProvider
|
||||
implements AWSCredentialsProvider, Closeable {
|
||||
implements AwsCredentialsProvider, Closeable {
|
||||
|
||||
private final AWSCredentialsProvider provider =
|
||||
new EC2ContainerCredentialsProviderWrapper();
|
||||
private final AwsCredentialsProvider containerCredentialsProvider =
|
||||
ContainerCredentialsProvider.builder().build();
|
||||
|
||||
public IAMInstanceCredentialsProvider() {
|
||||
}
|
||||
|
@ -63,19 +61,29 @@ public class IAMInstanceCredentialsProvider
|
|||
* @throws NoAwsCredentialsException on auth failure to indicate non-recoverable.
|
||||
*/
|
||||
@Override
|
||||
public AWSCredentials getCredentials() {
|
||||
public AwsCredentials resolveCredentials() {
|
||||
try {
|
||||
return provider.getCredentials();
|
||||
} catch (AmazonClientException e) {
|
||||
return getCredentials();
|
||||
} catch (SdkClientException e) {
|
||||
throw new NoAwsCredentialsException("IAMInstanceCredentialsProvider",
|
||||
e.getMessage(),
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void refresh() {
|
||||
provider.refresh();
|
||||
/**
|
||||
* First try {@link ContainerCredentialsProvider}, which will throw an exception if credentials
|
||||
* cannot be retrieved from the container. Then resolve credentials
|
||||
* using {@link InstanceProfileCredentialsProvider}.
|
||||
*
|
||||
* @return credentials
|
||||
*/
|
||||
private AwsCredentials getCredentials() {
|
||||
try {
|
||||
return containerCredentialsProvider.resolveCredentials();
|
||||
} catch (SdkClientException e) {
|
||||
return InstanceProfileCredentialsProvider.create().resolveCredentials();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,19 +24,18 @@ import java.util.Date;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import com.amazonaws.ClientConfiguration;
|
||||
import com.amazonaws.SdkClientException;
|
||||
import com.amazonaws.auth.AWSCredentials;
|
||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||
import com.amazonaws.auth.AWSSessionCredentials;
|
||||
import com.amazonaws.auth.BasicAWSCredentials;
|
||||
import com.amazonaws.auth.BasicSessionCredentials;
|
||||
import com.amazonaws.services.securitytoken.AWSSecurityTokenService;
|
||||
import com.amazonaws.services.securitytoken.model.Credentials;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
|
||||
import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
|
||||
import software.amazon.awssdk.core.exception.SdkException;
|
||||
import software.amazon.awssdk.services.sts.StsClient;
|
||||
import software.amazon.awssdk.services.sts.model.Credentials;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.s3a.Invoker;
|
||||
import org.apache.hadoop.fs.s3a.Retries;
|
||||
|
@ -77,10 +76,10 @@ public final class MarshalledCredentialBinding {
|
|||
public static MarshalledCredentials fromSTSCredentials(
|
||||
final Credentials credentials) {
|
||||
MarshalledCredentials marshalled = new MarshalledCredentials(
|
||||
credentials.getAccessKeyId(),
|
||||
credentials.getSecretAccessKey(),
|
||||
credentials.getSessionToken());
|
||||
Date date = credentials.getExpiration();
|
||||
credentials.accessKeyId(),
|
||||
credentials.secretAccessKey(),
|
||||
credentials.sessionToken());
|
||||
Date date = Date.from(credentials.expiration());
|
||||
marshalled.setExpiration(date != null ? date.getTime() : 0);
|
||||
return marshalled;
|
||||
}
|
||||
|
@ -91,11 +90,11 @@ public final class MarshalledCredentialBinding {
|
|||
* @return a set of marshalled credentials.
|
||||
*/
|
||||
public static MarshalledCredentials fromAWSCredentials(
|
||||
final AWSSessionCredentials credentials) {
|
||||
final AwsSessionCredentials credentials) {
|
||||
return new MarshalledCredentials(
|
||||
credentials.getAWSAccessKeyId(),
|
||||
credentials.getAWSSecretKey(),
|
||||
credentials.getSessionToken());
|
||||
credentials.accessKeyId(),
|
||||
credentials.secretAccessKey(),
|
||||
credentials.sessionToken());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -156,7 +155,7 @@ public final class MarshalledCredentialBinding {
|
|||
* @throws NoAuthWithAWSException validation failure
|
||||
* @throws NoAwsCredentialsException the credentials are actually empty.
|
||||
*/
|
||||
public static AWSCredentials toAWSCredentials(
|
||||
public static AwsCredentials toAWSCredentials(
|
||||
final MarshalledCredentials marshalled,
|
||||
final MarshalledCredentials.CredentialTypeRequired typeRequired,
|
||||
final String component)
|
||||
|
@ -173,18 +172,18 @@ public final class MarshalledCredentialBinding {
|
|||
final String secretKey = marshalled.getSecretKey();
|
||||
if (marshalled.hasSessionToken()) {
|
||||
// a session token was supplied, so return session credentials
|
||||
return new BasicSessionCredentials(accessKey, secretKey,
|
||||
return AwsSessionCredentials.create(accessKey, secretKey,
|
||||
marshalled.getSessionToken());
|
||||
} else {
|
||||
// these are full credentials
|
||||
return new BasicAWSCredentials(accessKey, secretKey);
|
||||
return AwsBasicCredentials.create(accessKey, secretKey);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Request a set of credentials from an STS endpoint.
|
||||
* @param parentCredentials the parent credentials needed to talk to STS
|
||||
* @param awsConf AWS client configuration
|
||||
* @param configuration AWS client configuration
|
||||
* @param stsEndpoint an endpoint, use "" for none
|
||||
* @param stsRegion region; use if the endpoint isn't the AWS default.
|
||||
* @param duration duration of the credentials in seconds. Minimum value: 900.
|
||||
|
@ -194,25 +193,26 @@ public final class MarshalledCredentialBinding {
|
|||
*/
|
||||
@Retries.RetryTranslated
|
||||
public static MarshalledCredentials requestSessionCredentials(
|
||||
final AWSCredentialsProvider parentCredentials,
|
||||
final ClientConfiguration awsConf,
|
||||
final AwsCredentialsProvider parentCredentials,
|
||||
final Configuration configuration,
|
||||
final String stsEndpoint,
|
||||
final String stsRegion,
|
||||
final int duration,
|
||||
final Invoker invoker) throws IOException {
|
||||
final Invoker invoker,
|
||||
final String bucket) throws IOException {
|
||||
try {
|
||||
final AWSSecurityTokenService tokenService =
|
||||
final StsClient tokenService =
|
||||
STSClientFactory.builder(parentCredentials,
|
||||
awsConf,
|
||||
configuration,
|
||||
stsEndpoint.isEmpty() ? null : stsEndpoint,
|
||||
stsRegion)
|
||||
stsRegion, bucket)
|
||||
.build();
|
||||
try (STSClientFactory.STSClient stsClient = STSClientFactory.createClientConnection(
|
||||
tokenService, invoker)) {
|
||||
return fromSTSCredentials(stsClient.requestSessionCredentials(duration,
|
||||
TimeUnit.SECONDS));
|
||||
}
|
||||
} catch (SdkClientException e) {
|
||||
} catch (SdkException e) {
|
||||
if (stsRegion.isEmpty()) {
|
||||
LOG.error("Region must be provided when requesting session credentials.",
|
||||
e);
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.hadoop.fs.s3a.auth;
|
|||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
|
||||
import com.amazonaws.auth.AWSCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentials;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
@ -40,7 +40,6 @@ import static org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding.toAWSCre
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
@SuppressWarnings("deprecation")
|
||||
public class MarshalledCredentialProvider extends
|
||||
AbstractSessionCredentialsProvider {
|
||||
|
||||
|
@ -85,7 +84,7 @@ public class MarshalledCredentialProvider extends
|
|||
* @throws IOException on a failure
|
||||
*/
|
||||
@Override
|
||||
protected AWSCredentials createCredentials(final Configuration config)
|
||||
protected AwsCredentials createCredentials(final Configuration config)
|
||||
throws IOException {
|
||||
return toAWSCredentials(credentials, typeRequired, component);
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.hadoop.fs.s3a.auth;
|
|||
import org.apache.hadoop.fs.s3a.CredentialInitializationException;
|
||||
|
||||
/**
|
||||
* A specific subclass of {@code AmazonClientException} which is
|
||||
* A specific subclass of {@code SdkException} which is
|
||||
* used in the S3A retry policy to fail fast when there is any
|
||||
* authentication problem.
|
||||
*/
|
||||
|
|
|
@ -20,28 +20,33 @@ package org.apache.hadoop.fs.s3a.auth;
|
|||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import com.amazonaws.ClientConfiguration;
|
||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||
import com.amazonaws.client.builder.AwsClientBuilder;
|
||||
import com.amazonaws.services.securitytoken.AWSSecurityTokenService;
|
||||
import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
|
||||
import com.amazonaws.services.securitytoken.model.AssumeRoleRequest;
|
||||
import com.amazonaws.services.securitytoken.model.Credentials;
|
||||
import com.amazonaws.services.securitytoken.model.GetSessionTokenRequest;
|
||||
import org.apache.hadoop.fs.s3a.AWSClientConfig;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
|
||||
import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration;
|
||||
import software.amazon.awssdk.core.retry.RetryPolicy;
|
||||
import software.amazon.awssdk.http.apache.ApacheHttpClient;
|
||||
import software.amazon.awssdk.http.apache.ProxyConfiguration;
|
||||
import software.amazon.awssdk.regions.Region;
|
||||
import software.amazon.awssdk.services.sts.StsClient;
|
||||
import software.amazon.awssdk.services.sts.StsClientBuilder;
|
||||
import software.amazon.awssdk.services.sts.model.AssumeRoleRequest;
|
||||
import software.amazon.awssdk.services.sts.model.Credentials;
|
||||
import software.amazon.awssdk.services.sts.model.GetSessionTokenRequest;
|
||||
import software.amazon.awssdk.thirdparty.org.apache.http.client.utils.URIBuilder;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.s3a.Constants;
|
||||
import org.apache.hadoop.fs.s3a.Invoker;
|
||||
import org.apache.hadoop.fs.s3a.Retries;
|
||||
import org.apache.hadoop.fs.s3a.S3AUtils;
|
||||
|
||||
import static org.apache.commons.lang3.StringUtils.isEmpty;
|
||||
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
|
||||
|
@ -71,17 +76,15 @@ public class STSClientFactory {
|
|||
* @return the builder to call {@code build()}
|
||||
* @throws IOException problem reading proxy secrets
|
||||
*/
|
||||
public static AWSSecurityTokenServiceClientBuilder builder(
|
||||
public static StsClientBuilder builder(
|
||||
final Configuration conf,
|
||||
final String bucket,
|
||||
final AWSCredentialsProvider credentials) throws IOException {
|
||||
final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket,
|
||||
Constants.AWS_SERVICE_IDENTIFIER_STS);
|
||||
final AwsCredentialsProvider credentials) throws IOException {
|
||||
String endpoint = conf.getTrimmed(DELEGATION_TOKEN_ENDPOINT,
|
||||
DEFAULT_DELEGATION_TOKEN_ENDPOINT);
|
||||
String region = conf.getTrimmed(DELEGATION_TOKEN_REGION,
|
||||
DEFAULT_DELEGATION_TOKEN_REGION);
|
||||
return builder(credentials, awsConf, endpoint, region);
|
||||
return builder(credentials, conf, endpoint, region, bucket);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -96,37 +99,55 @@ public class STSClientFactory {
|
|||
* @return the builder to call {@code build()}
|
||||
* @throws IOException problem reading proxy secrets
|
||||
*/
|
||||
public static AWSSecurityTokenServiceClientBuilder builder(
|
||||
public static StsClientBuilder builder(
|
||||
final Configuration conf,
|
||||
final String bucket,
|
||||
final AWSCredentialsProvider credentials,
|
||||
final AwsCredentialsProvider credentials,
|
||||
final String stsEndpoint,
|
||||
final String stsRegion) throws IOException {
|
||||
final ClientConfiguration awsConf = S3AUtils.createAwsConf(conf, bucket,
|
||||
Constants.AWS_SERVICE_IDENTIFIER_STS);
|
||||
return builder(credentials, awsConf, stsEndpoint, stsRegion);
|
||||
return builder(credentials, conf, stsEndpoint, stsRegion, bucket);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the builder ready for any final configuration options.
|
||||
* Picks up connection settings from the Hadoop configuration, including
|
||||
* proxy secrets.
|
||||
* @param awsConf AWS configuration.
|
||||
* @param conf AWS configuration.
|
||||
* @param credentials AWS credential chain to use
|
||||
* @param stsEndpoint optional endpoint "https://sns.us-west-1.amazonaws.com"
|
||||
* @param stsRegion the region, e.g "us-west-1". Must be set if endpoint is.
|
||||
* @return the builder to call {@code build()}
|
||||
*/
|
||||
public static AWSSecurityTokenServiceClientBuilder builder(
|
||||
final AWSCredentialsProvider credentials,
|
||||
final ClientConfiguration awsConf,
|
||||
public static StsClientBuilder builder(
|
||||
final AwsCredentialsProvider credentials,
|
||||
final Configuration conf,
|
||||
final String stsEndpoint,
|
||||
final String stsRegion) {
|
||||
final AWSSecurityTokenServiceClientBuilder builder
|
||||
= AWSSecurityTokenServiceClientBuilder.standard();
|
||||
final String stsRegion,
|
||||
final String bucket) throws IOException {
|
||||
final StsClientBuilder stsClientBuilder = StsClient.builder();
|
||||
|
||||
Preconditions.checkArgument(credentials != null, "No credentials");
|
||||
builder.withClientConfiguration(awsConf);
|
||||
builder.withCredentials(credentials);
|
||||
|
||||
final ClientOverrideConfiguration.Builder clientOverrideConfigBuilder =
|
||||
AWSClientConfig.createClientConfigBuilder(conf);
|
||||
|
||||
final ApacheHttpClient.Builder httpClientBuilder =
|
||||
AWSClientConfig.createHttpClientBuilder(conf);
|
||||
|
||||
final RetryPolicy.Builder retryPolicyBuilder = AWSClientConfig.createRetryPolicyBuilder(conf);
|
||||
|
||||
final ProxyConfiguration proxyConfig =
|
||||
AWSClientConfig.createProxyConfiguration(conf, bucket);
|
||||
|
||||
clientOverrideConfigBuilder.retryPolicy(retryPolicyBuilder.build());
|
||||
httpClientBuilder.proxyConfiguration(proxyConfig);
|
||||
|
||||
stsClientBuilder
|
||||
.httpClientBuilder(httpClientBuilder)
|
||||
.overrideConfiguration(clientOverrideConfigBuilder.build())
|
||||
.credentialsProvider(credentials);
|
||||
|
||||
// TODO: SIGNERS NOT ADDED YET.
|
||||
boolean destIsStandardEndpoint = STS_STANDARD.equals(stsEndpoint);
|
||||
if (isNotEmpty(stsEndpoint) && !destIsStandardEndpoint) {
|
||||
Preconditions.checkArgument(
|
||||
|
@ -134,26 +155,43 @@ public class STSClientFactory {
|
|||
"STS endpoint is set to %s but no signing region was provided",
|
||||
stsEndpoint);
|
||||
LOG.debug("STS Endpoint={}; region='{}'", stsEndpoint, stsRegion);
|
||||
builder.withEndpointConfiguration(
|
||||
new AwsClientBuilder.EndpointConfiguration(stsEndpoint, stsRegion));
|
||||
stsClientBuilder.endpointOverride(getSTSEndpoint(stsEndpoint))
|
||||
.region(Region.of(stsRegion));
|
||||
} else {
|
||||
Preconditions.checkArgument(isEmpty(stsRegion),
|
||||
"STS signing region set set to %s but no STS endpoint specified",
|
||||
stsRegion);
|
||||
}
|
||||
return builder;
|
||||
return stsClientBuilder;
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a endpoint string, create the endpoint URI.
|
||||
*
|
||||
* @param endpoint possibly null endpoint.
|
||||
* @return an endpoint uri
|
||||
*/
|
||||
private static URI getSTSEndpoint(String endpoint) {
|
||||
try {
|
||||
// TODO: The URI builder is currently imported via a shaded dependency. This is due to TM
|
||||
// preview dependency causing some issues.
|
||||
return new URIBuilder().setScheme("https").setHost(endpoint).build();
|
||||
} catch (URISyntaxException e) {
|
||||
throw new IllegalArgumentException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create an STS Client instance.
|
||||
* @param tokenService STS instance
|
||||
* @param stsClient STS instance
|
||||
* @param invoker invoker to use
|
||||
* @return an STS client bonded to that interface.
|
||||
*/
|
||||
public static STSClient createClientConnection(
|
||||
final AWSSecurityTokenService tokenService,
|
||||
final StsClient stsClient,
|
||||
final Invoker invoker) {
|
||||
return new STSClient(tokenService, invoker);
|
||||
return new STSClient(stsClient, invoker);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -161,21 +199,19 @@ public class STSClientFactory {
|
|||
*/
|
||||
public static final class STSClient implements Closeable {
|
||||
|
||||
private final AWSSecurityTokenService tokenService;
|
||||
private final StsClient stsClient;
|
||||
|
||||
private final Invoker invoker;
|
||||
|
||||
private STSClient(final AWSSecurityTokenService tokenService,
|
||||
private STSClient(final StsClient stsClient,
|
||||
final Invoker invoker) {
|
||||
this.tokenService = tokenService;
|
||||
this.stsClient = stsClient;
|
||||
this.invoker = invoker;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
// Since we are not using AbstractAWSSecurityTokenService, we
|
||||
// don't need to worry about catching UnsupportedOperationException.
|
||||
tokenService.shutdown();
|
||||
stsClient.close();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -192,13 +228,13 @@ public class STSClientFactory {
|
|||
final TimeUnit timeUnit) throws IOException {
|
||||
int durationSeconds = (int) timeUnit.toSeconds(duration);
|
||||
LOG.debug("Requesting session token of duration {}", duration);
|
||||
final GetSessionTokenRequest request = new GetSessionTokenRequest();
|
||||
request.setDurationSeconds(durationSeconds);
|
||||
final GetSessionTokenRequest request =
|
||||
GetSessionTokenRequest.builder().durationSeconds(durationSeconds).build();
|
||||
return invoker.retry("request session credentials", "",
|
||||
true,
|
||||
() ->{
|
||||
LOG.info("Requesting Amazon STS Session credentials");
|
||||
return tokenService.getSessionToken(request).getCredentials();
|
||||
return stsClient.getSessionToken(request).credentials();
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -222,15 +258,14 @@ public class STSClientFactory {
|
|||
final TimeUnit timeUnit) throws IOException {
|
||||
LOG.debug("Requesting role {} with duration {}; policy = {}",
|
||||
roleARN, duration, policy);
|
||||
AssumeRoleRequest request = new AssumeRoleRequest();
|
||||
request.setDurationSeconds((int) timeUnit.toSeconds(duration));
|
||||
request.setRoleArn(roleARN);
|
||||
request.setRoleSessionName(sessionName);
|
||||
AssumeRoleRequest.Builder requestBuilder =
|
||||
AssumeRoleRequest.builder().durationSeconds((int) timeUnit.toSeconds(duration))
|
||||
.roleArn(roleARN).roleSessionName(sessionName);
|
||||
if (isNotEmpty(policy)) {
|
||||
request.setPolicy(policy);
|
||||
requestBuilder.policy(policy);
|
||||
}
|
||||
return invoker.retry("request role credentials", "", true,
|
||||
() -> tokenService.assumeRole(request).getCredentials());
|
||||
() -> stsClient.assumeRole(requestBuilder.build()).credentials());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,9 +20,6 @@ package org.apache.hadoop.fs.s3a.auth.delegation;
|
|||
|
||||
import java.util.Optional;
|
||||
|
||||
import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams;
|
||||
import com.amazonaws.services.s3.model.SSECustomerKey;
|
||||
|
||||
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
|
||||
|
||||
/**
|
||||
|
@ -35,37 +32,30 @@ public final class EncryptionSecretOperations {
|
|||
private EncryptionSecretOperations() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Create SSE-C client side key encryption options on demand.
|
||||
* @return an optional key to attach to a request.
|
||||
/***
|
||||
* Gets the SSE-C client side key if present.
|
||||
*
|
||||
* @param secrets source of the encryption secrets.
|
||||
* @return an optional key to attach to a request.
|
||||
*/
|
||||
public static Optional<SSECustomerKey> createSSECustomerKey(
|
||||
final EncryptionSecrets secrets) {
|
||||
if (secrets.hasEncryptionKey() &&
|
||||
secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_C) {
|
||||
return Optional.of(new SSECustomerKey(secrets.getEncryptionKey()));
|
||||
public static Optional<String> getSSECustomerKey(final EncryptionSecrets secrets) {
|
||||
if (secrets.hasEncryptionKey() && secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_C) {
|
||||
return Optional.of(secrets.getEncryptionKey());
|
||||
} else {
|
||||
return Optional.empty();
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create SSE-KMS options for a request, iff the encryption is SSE-KMS.
|
||||
* @return an optional SSE-KMS param to attach to a request.
|
||||
* Gets the SSE-KMS key if present, else let S3 use AWS managed key.
|
||||
*
|
||||
* @param secrets source of the encryption secrets.
|
||||
* @return an optional key to attach to a request.
|
||||
*/
|
||||
public static Optional<SSEAwsKeyManagementParams> createSSEAwsKeyManagementParams(
|
||||
final EncryptionSecrets secrets) {
|
||||
|
||||
//Use specified key, otherwise default to default master aws/s3 key by AWS
|
||||
if (secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_KMS) {
|
||||
if (secrets.hasEncryptionKey()) {
|
||||
return Optional.of(new SSEAwsKeyManagementParams(
|
||||
secrets.getEncryptionKey()));
|
||||
} else {
|
||||
return Optional.of(new SSEAwsKeyManagementParams());
|
||||
}
|
||||
public static Optional<String> getSSEAwsKMSKey(final EncryptionSecrets secrets) {
|
||||
if (secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_KMS
|
||||
&& secrets.hasEncryptionKey()) {
|
||||
return Optional.of(secrets.getEncryptionKey());
|
||||
} else {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
|
|
@ -23,11 +23,11 @@ import java.util.Optional;
|
|||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import com.amazonaws.services.securitytoken.model.Credentials;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import software.amazon.awssdk.services.sts.model.Credentials;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
|
||||
|
|
|
@ -228,7 +228,10 @@ public class S3ADelegationTokens extends AbstractDTService {
|
|||
try {
|
||||
super.serviceStop();
|
||||
} finally {
|
||||
ServiceOperations.stopQuietly(LOG, tokenBinding);
|
||||
// TODO: Importing the transfer manager preview outside of the bundle causes some
|
||||
// issues. Over here, it can no longer find the LOG. We expect this to be fixed with the
|
||||
// release of the TM.
|
||||
// ServiceOperations.stopQuietly(LOG, tokenBinding);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -26,21 +26,18 @@ import java.util.Optional;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import com.amazonaws.ClientConfiguration;
|
||||
import com.amazonaws.auth.AWSCredentials;
|
||||
import com.amazonaws.auth.AWSSessionCredentials;
|
||||
import com.amazonaws.services.securitytoken.AWSSecurityTokenService;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import software.amazon.awssdk.auth.credentials.AwsCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
|
||||
import software.amazon.awssdk.services.sts.StsClient;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.s3a.AWSCredentialProviderList;
|
||||
import org.apache.hadoop.fs.s3a.Constants;
|
||||
import org.apache.hadoop.fs.s3a.Invoker;
|
||||
import org.apache.hadoop.fs.s3a.Retries;
|
||||
import org.apache.hadoop.fs.s3a.S3ARetryPolicy;
|
||||
import org.apache.hadoop.fs.s3a.S3AUtils;
|
||||
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialProvider;
|
||||
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
|
||||
import org.apache.hadoop.fs.s3a.auth.RoleModel;
|
||||
|
@ -292,23 +289,22 @@ public class SessionTokenBinding extends AbstractDelegationTokenBinding {
|
|||
// chain.
|
||||
// As no codepath (session propagation, STS creation) will work,
|
||||
// throw this.
|
||||
final AWSCredentials parentCredentials = once("get credentials",
|
||||
final AwsCredentials parentCredentials = once("get credentials",
|
||||
"",
|
||||
() -> parentAuthChain.getCredentials());
|
||||
hasSessionCreds = parentCredentials instanceof AWSSessionCredentials;
|
||||
() -> parentAuthChain.resolveCredentials());
|
||||
hasSessionCreds = parentCredentials instanceof AwsSessionCredentials;
|
||||
|
||||
if (!hasSessionCreds) {
|
||||
LOG.debug("Creating STS client for {}", getDescription());
|
||||
|
||||
invoker = new Invoker(new S3ARetryPolicy(conf), LOG_EVENT);
|
||||
ClientConfiguration awsConf =
|
||||
S3AUtils.createAwsConf(conf, uri.getHost(),
|
||||
Constants.AWS_SERVICE_IDENTIFIER_STS);
|
||||
AWSSecurityTokenService tokenService =
|
||||
|
||||
StsClient tokenService =
|
||||
STSClientFactory.builder(parentAuthChain,
|
||||
awsConf,
|
||||
conf,
|
||||
endpoint,
|
||||
region)
|
||||
region,
|
||||
uri.getHost())
|
||||
.build();
|
||||
stsClient = Optional.of(
|
||||
STSClientFactory.createClientConnection(tokenService, invoker));
|
||||
|
@ -374,11 +370,11 @@ public class SessionTokenBinding extends AbstractDelegationTokenBinding {
|
|||
+ " -duration unknown", getCanonicalUri());
|
||||
}
|
||||
origin += " " + CREDENTIALS_CONVERTED_TO_DELEGATION_TOKEN;
|
||||
final AWSCredentials awsCredentials
|
||||
= parentAuthChain.getCredentials();
|
||||
if (awsCredentials instanceof AWSSessionCredentials) {
|
||||
final AwsCredentials awsCredentials
|
||||
= parentAuthChain.resolveCredentials();
|
||||
if (awsCredentials instanceof AwsSessionCredentials) {
|
||||
marshalledCredentials = fromAWSCredentials(
|
||||
(AWSSessionCredentials) awsCredentials);
|
||||
(AwsSessionCredentials) awsCredentials);
|
||||
} else {
|
||||
throw new DelegationTokenIOException(
|
||||
"AWS Authentication chain is no longer supplying session secrets");
|
||||
|
|
|
@ -27,10 +27,11 @@ import java.util.Date;
|
|||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
import com.amazonaws.services.s3.model.MultipartUpload;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import software.amazon.awssdk.services.s3.model.MultipartUpload;
|
||||
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
|
@ -975,7 +976,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter
|
|||
.executeWith(commitContext.getOuterSubmitter())
|
||||
.suppressExceptions(suppressExceptions)
|
||||
.run(u -> commitContext.abortMultipartCommit(
|
||||
u.getKey(), u.getUploadId()));
|
||||
u.key(), u.uploadId()));
|
||||
} else {
|
||||
LOG.info("No pending uploads were found");
|
||||
}
|
||||
|
@ -1300,8 +1301,8 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter
|
|||
DateFormat df = DateFormat.getDateTimeInstance();
|
||||
pending.forEach(u ->
|
||||
LOG.info("[{}] {}",
|
||||
df.format(u.getInitiated()),
|
||||
u.getKey()));
|
||||
df.format(Date.from(u.initiated())),
|
||||
u.key()));
|
||||
if (shouldAbortUploadsInCleanup()) {
|
||||
LOG.warn("This committer will abort these uploads in job cleanup");
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.hadoop.fs.s3a.commit;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import com.amazonaws.services.s3.model.PartETag;
|
||||
import software.amazon.awssdk.services.s3.model.CompletedPart;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||
|
@ -76,7 +76,7 @@ public class PutTracker {
|
|||
* @throws IOException I/O problem or validation failure.
|
||||
*/
|
||||
public boolean aboutToComplete(String uploadId,
|
||||
List<PartETag> parts,
|
||||
List<CompletedPart> parts,
|
||||
long bytesWritten,
|
||||
final IOStatistics iostatistics)
|
||||
throws IOException {
|
||||
|
|
|
@ -31,9 +31,10 @@ import java.util.Iterator;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import com.amazonaws.services.s3.model.PartETag;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
import software.amazon.awssdk.services.s3.model.CompletedPart;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
@ -215,13 +216,13 @@ public class SinglePendingCommit extends PersistentCommitData<SinglePendingCommi
|
|||
* @param parts ordered list of etags.
|
||||
* @throws ValidationFailure if the data is invalid
|
||||
*/
|
||||
public void bindCommitData(List<PartETag> parts) throws ValidationFailure {
|
||||
public void bindCommitData(List<CompletedPart> parts) throws ValidationFailure {
|
||||
etags = new ArrayList<>(parts.size());
|
||||
int counter = 1;
|
||||
for (PartETag part : parts) {
|
||||
verify(part.getPartNumber() == counter,
|
||||
"Expected part number %s but got %s", counter, part.getPartNumber());
|
||||
etags.add(part.getETag());
|
||||
for (CompletedPart part : parts) {
|
||||
verify(part.partNumber() == counter,
|
||||
"Expected part number %s but got %s", counter, part.partNumber());
|
||||
etags.add(part.eTag());
|
||||
counter++;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,6 +21,8 @@ package org.apache.hadoop.fs.s3a.commit.impl;
|
|||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -32,13 +34,15 @@ import java.util.stream.IntStream;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import com.amazonaws.services.s3.model.MultipartUpload;
|
||||
import com.amazonaws.services.s3.model.PartETag;
|
||||
import com.amazonaws.services.s3.model.UploadPartRequest;
|
||||
import com.amazonaws.services.s3.model.UploadPartResult;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import software.amazon.awssdk.core.sync.RequestBody;
|
||||
import software.amazon.awssdk.services.s3.model.CompletedPart;
|
||||
import software.amazon.awssdk.services.s3.model.MultipartUpload;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
|
||||
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
|
@ -155,9 +159,9 @@ public class CommitOperations extends AbstractStoreOperation
|
|||
* @param tagIds list of tags
|
||||
* @return same list, now in numbered tuples
|
||||
*/
|
||||
public static List<PartETag> toPartEtags(List<String> tagIds) {
|
||||
public static List<CompletedPart> toPartEtags(List<String> tagIds) {
|
||||
return IntStream.range(0, tagIds.size())
|
||||
.mapToObj(i -> new PartETag(i + 1, tagIds.get(i)))
|
||||
.mapToObj(i -> CompletedPart.builder().partNumber(i + 1).eTag(tagIds.get(i)).build())
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
|
@ -566,26 +570,30 @@ public class CommitOperations extends AbstractStoreOperation
|
|||
numParts, length));
|
||||
}
|
||||
|
||||
List<PartETag> parts = new ArrayList<>((int) numParts);
|
||||
List<CompletedPart> parts = new ArrayList<>((int) numParts);
|
||||
|
||||
LOG.debug("File size is {}, number of parts to upload = {}",
|
||||
length, numParts);
|
||||
for (int partNumber = 1; partNumber <= numParts; partNumber += 1) {
|
||||
progress.progress();
|
||||
long size = Math.min(length - offset, uploadPartSize);
|
||||
UploadPartRequest part;
|
||||
part = writeOperations.newUploadPartRequest(
|
||||
destKey,
|
||||
uploadId,
|
||||
partNumber,
|
||||
(int) size,
|
||||
null,
|
||||
localFile,
|
||||
offset);
|
||||
part.setLastPart(partNumber == numParts);
|
||||
UploadPartResult partResult = writeOperations.uploadPart(part, statistics);
|
||||
offset += uploadPartSize;
|
||||
parts.add(partResult.getPartETag());
|
||||
|
||||
// Open the file to upload.
|
||||
try (InputStream fileStream = Files.newInputStream(localFile.toPath())) {
|
||||
for (int partNumber = 1; partNumber <= numParts; partNumber += 1) {
|
||||
progress.progress();
|
||||
long size = Math.min(length - offset, uploadPartSize);
|
||||
UploadPartRequest part = writeOperations.newUploadPartRequestBuilder(
|
||||
destKey,
|
||||
uploadId,
|
||||
partNumber,
|
||||
size).build();
|
||||
// Read from the file input stream at current position.
|
||||
RequestBody body = RequestBody.fromInputStream(fileStream, size);
|
||||
UploadPartResponse response = writeOperations.uploadPart(part, body, statistics);
|
||||
offset += uploadPartSize;
|
||||
parts.add(CompletedPart.builder()
|
||||
.partNumber(partNumber)
|
||||
.eTag(response.eTag())
|
||||
.build());
|
||||
}
|
||||
}
|
||||
|
||||
commitData.bindCommitData(parts);
|
||||
|
|
|
@ -20,19 +20,22 @@ package org.apache.hadoop.fs.s3a.commit.magic;
|
|||
|
||||
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.CompletedPart;
|
||||
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;
|
||||
|
@ -125,7 +128,7 @@ public class MagicCommitTracker extends PutTracker {
|
|||
*/
|
||||
@Override
|
||||
public boolean aboutToComplete(String uploadId,
|
||||
List<PartETag> parts,
|
||||
List<CompletedPart> parts,
|
||||
long bytesWritten,
|
||||
final IOStatistics iostatistics)
|
||||
throws IOException {
|
||||
|
@ -144,10 +147,9 @@ public class MagicCommitTracker extends PutTracker {
|
|||
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 class MagicCommitTracker extends PutTracker {
|
|||
path, pendingPartKey, commitData);
|
||||
PutObjectRequest put = writer.createPutObjectRequest(
|
||||
pendingPartKey,
|
||||
new ByteArrayInputStream(bytes),
|
||||
bytes.length, null);
|
||||
upload(put);
|
||||
bytes.length, null, false);
|
||||
upload(put, new ByteArrayInputStream(bytes));
|
||||
return false;
|
||||
|
||||
}
|
||||
/**
|
||||
* PUT an object.
|
||||
* @param request the request
|
||||
* @param inputStream input stream of data to be uploaded
|
||||
* @throws IOException on problems
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
private void upload(PutObjectRequest request) throws IOException {
|
||||
trackDurationOfInvocation(trackerStatistics,
|
||||
COMMITTER_MAGIC_MARKER_PUT.getSymbol(), () ->
|
||||
writer.putObject(request, PutObjectOptions.keepingDirs(), null));
|
||||
private void upload(PutObjectRequest request, InputStream inputStream) throws IOException {
|
||||
trackDurationOfInvocation(trackerStatistics, COMMITTER_MAGIC_MARKER_PUT.getSymbol(),
|
||||
() -> writer.putObject(request, PutObjectOptions.keepingDirs(),
|
||||
new S3ADataBlocks.BlockUploadData(inputStream), false, null));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,8 +20,6 @@ package org.apache.hadoop.fs.s3a.impl;
|
|||
|
||||
import java.util.List;
|
||||
|
||||
import com.amazonaws.SdkClientException;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -30,6 +28,10 @@ import org.apache.hadoop.fs.s3a.S3AStorageStatistics;
|
|||
import org.apache.hadoop.fs.s3a.Statistic;
|
||||
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
|
||||
|
||||
import software.amazon.awssdk.core.exception.SdkClientException;
|
||||
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
|
||||
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
|
||||
|
||||
import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException;
|
||||
import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS;
|
||||
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED;
|
||||
|
@ -113,15 +115,15 @@ public class BulkDeleteRetryHandler extends AbstractStoreOperation {
|
|||
* @param deleteRequest request which failed.
|
||||
*/
|
||||
private void onDeleteThrottled(final DeleteObjectsRequest deleteRequest) {
|
||||
final List<DeleteObjectsRequest.KeyVersion> keys = deleteRequest.getKeys();
|
||||
final List<ObjectIdentifier> keys = deleteRequest.delete().objects();
|
||||
final int size = keys.size();
|
||||
incrementStatistic(STORE_IO_THROTTLED, size);
|
||||
instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, size);
|
||||
THROTTLE_LOG.info(
|
||||
"Bulk delete {} keys throttled -first key = {}; last = {}",
|
||||
size,
|
||||
keys.get(0).getKey(),
|
||||
keys.get(size - 1).getKey());
|
||||
keys.get(0).key(),
|
||||
keys.get(size - 1).key());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -20,15 +20,17 @@ package org.apache.hadoop.fs.s3a.impl;
|
|||
|
||||
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.GetObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.GetObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
|
||||
|
||||
import org.apache.commons.lang3.tuple.ImmutablePair;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
@ -211,11 +213,24 @@ public abstract class ChangeDetectionPolicy {
|
|||
* null if the attribute is unavailable (such as when the policy says to use
|
||||
* versionId but object versioning is not enabled for the bucket).
|
||||
*/
|
||||
public abstract String getRevisionId(ObjectMetadata objectMetadata,
|
||||
public abstract String getRevisionId(HeadObjectResponse objectMetadata,
|
||||
String uri);
|
||||
|
||||
/**
|
||||
* Like {{@link #getRevisionId(ObjectMetadata, String)}}, but retrieves the
|
||||
* Like {{@link #getRevisionId(HeadObjectResponse, String)}}, but retrieves the
|
||||
* * revision identifier from {@link GetObjectResponse}.
|
||||
*
|
||||
* @param getObjectResponse the response instance
|
||||
* @param uri the URI of the object
|
||||
* @return the revisionId string as interpreted by this policy, or potentially
|
||||
* null if the attribute is unavailable (such as when the policy says to use
|
||||
* versionId but object versioning is not enabled for the bucket).
|
||||
*/
|
||||
public abstract String getRevisionId(GetObjectResponse getObjectResponse,
|
||||
String uri);
|
||||
|
||||
/**
|
||||
* Like {{@link #getRevisionId(HeadObjectResponse, String)}}, but retrieves the
|
||||
* revision identifier from {@link S3ObjectAttributes}.
|
||||
*
|
||||
* @param s3Attributes the object attributes
|
||||
|
@ -226,44 +241,44 @@ public abstract class ChangeDetectionPolicy {
|
|||
public abstract String getRevisionId(S3ObjectAttributes s3Attributes);
|
||||
|
||||
/**
|
||||
* Like {{@link #getRevisionId(ObjectMetadata, String)}}, but retrieves the
|
||||
* revision identifier from {@link CopyResult}.
|
||||
* Like {{@link #getRevisionId(HeadObjectResponse, String)}}, but retrieves the
|
||||
* revision identifier from {@link CopyObjectResponse}.
|
||||
*
|
||||
* @param copyResult the copy result
|
||||
* @param copyObjectResponse the copy response
|
||||
* @return the revisionId string as interpreted by this policy, or potentially
|
||||
* null if the attribute is unavailable (such as when the policy says to use
|
||||
* versionId but object versioning is not enabled for the bucket).
|
||||
*/
|
||||
public abstract String getRevisionId(CopyResult copyResult);
|
||||
public abstract String getRevisionId(CopyObjectResponse copyObjectResponse);
|
||||
|
||||
/**
|
||||
* Applies the given {@link #getRevisionId(ObjectMetadata, String) revisionId}
|
||||
* as a server-side qualification on the {@code GetObjectRequest}.
|
||||
* Applies the given {@link #getRevisionId(HeadObjectResponse, String) revisionId}
|
||||
* as a server-side qualification on the {@code GetObjectRequest.Builder}.
|
||||
*
|
||||
* @param request the request
|
||||
* @param revisionId the revision id
|
||||
*/
|
||||
public abstract void applyRevisionConstraint(GetObjectRequest request,
|
||||
public abstract void applyRevisionConstraint(GetObjectRequest.Builder request,
|
||||
String revisionId);
|
||||
|
||||
/**
|
||||
* Applies the given {@link #getRevisionId(ObjectMetadata, String) revisionId}
|
||||
* Applies the given {@link #getRevisionId(HeadObjectResponse, String) revisionId}
|
||||
* as a server-side qualification on the {@code CopyObjectRequest}.
|
||||
*
|
||||
* @param request the request
|
||||
* @param requestBuilder the copy object request builder
|
||||
* @param revisionId the revision id
|
||||
*/
|
||||
public abstract void applyRevisionConstraint(CopyObjectRequest request,
|
||||
public abstract void applyRevisionConstraint(CopyObjectRequest.Builder requestBuilder,
|
||||
String revisionId);
|
||||
|
||||
/**
|
||||
* Applies the given {@link #getRevisionId(ObjectMetadata, String) revisionId}
|
||||
* Applies the given {@link #getRevisionId(HeadObjectResponse, String) revisionId}
|
||||
* as a server-side qualification on the {@code GetObjectMetadataRequest}.
|
||||
*
|
||||
* @param request the request
|
||||
* @param requestBuilder the head object request builder
|
||||
* @param revisionId the revision id
|
||||
*/
|
||||
public abstract void applyRevisionConstraint(GetObjectMetadataRequest request,
|
||||
public abstract void applyRevisionConstraint(HeadObjectRequest.Builder requestBuilder,
|
||||
String revisionId);
|
||||
|
||||
/**
|
||||
|
@ -323,7 +338,7 @@ public abstract class ChangeDetectionPolicy {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 +347,13 @@ public abstract class ChangeDetectionPolicy {
|
|||
}
|
||||
|
||||
@Override
|
||||
public String getRevisionId(ObjectMetadata objectMetadata, String uri) {
|
||||
return objectMetadata.getETag();
|
||||
public String getRevisionId(GetObjectResponse objectMetadata, String uri) {
|
||||
return objectMetadata.eTag();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRevisionId(HeadObjectResponse objectMetadata, String uri) {
|
||||
return objectMetadata.eTag();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -342,34 +362,34 @@ public abstract class ChangeDetectionPolicy {
|
|||
}
|
||||
|
||||
@Override
|
||||
public String getRevisionId(CopyResult copyResult) {
|
||||
return copyResult.getETag();
|
||||
public String getRevisionId(CopyObjectResponse copyObjectResponse) {
|
||||
return copyObjectResponse.copyObjectResult().eTag();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyRevisionConstraint(GetObjectRequest request,
|
||||
public void applyRevisionConstraint(GetObjectRequest.Builder builder,
|
||||
String revisionId) {
|
||||
if (revisionId != null) {
|
||||
LOG.debug("Restricting get request to etag {}", revisionId);
|
||||
request.withMatchingETagConstraint(revisionId);
|
||||
builder.ifMatch(revisionId);
|
||||
} else {
|
||||
LOG.debug("No etag revision ID to use as a constraint");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyRevisionConstraint(CopyObjectRequest request,
|
||||
public void applyRevisionConstraint(CopyObjectRequest.Builder requestBuilder,
|
||||
String revisionId) {
|
||||
if (revisionId != null) {
|
||||
LOG.debug("Restricting copy request to etag {}", revisionId);
|
||||
request.withMatchingETagConstraint(revisionId);
|
||||
requestBuilder.copySourceIfMatch(revisionId);
|
||||
} else {
|
||||
LOG.debug("No etag revision ID to use as a constraint");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyRevisionConstraint(GetObjectMetadataRequest request,
|
||||
public void applyRevisionConstraint(HeadObjectRequest.Builder requestBuilder,
|
||||
String revisionId) {
|
||||
LOG.debug("Unable to restrict HEAD request to etag; will check later");
|
||||
}
|
||||
|
@ -388,7 +408,7 @@ public abstract class ChangeDetectionPolicy {
|
|||
|
||||
/**
|
||||
* Change detection policy based on
|
||||
* {@link ObjectMetadata#getVersionId() versionId}.
|
||||
* {@link HeadObjectResponse#versionId()} () versionId}.
|
||||
*/
|
||||
static class VersionIdChangeDetectionPolicy extends
|
||||
ChangeDetectionPolicy {
|
||||
|
@ -398,8 +418,16 @@ public abstract class ChangeDetectionPolicy {
|
|||
}
|
||||
|
||||
@Override
|
||||
public String getRevisionId(ObjectMetadata objectMetadata, String uri) {
|
||||
String versionId = objectMetadata.getVersionId();
|
||||
public String getRevisionId(HeadObjectResponse objectMetadata, String uri) {
|
||||
return logIfNull(objectMetadata.versionId(), uri);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRevisionId(GetObjectResponse getObjectResponse, String uri) {
|
||||
return logIfNull(getObjectResponse.versionId(), uri);
|
||||
}
|
||||
|
||||
private String logIfNull(String versionId, String uri) {
|
||||
if (versionId == null) {
|
||||
// this policy doesn't work if the bucket doesn't have object versioning
|
||||
// enabled (which isn't by default)
|
||||
|
@ -407,8 +435,7 @@ public abstract class ChangeDetectionPolicy {
|
|||
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;
|
||||
}
|
||||
|
@ -419,38 +446,38 @@ public abstract class ChangeDetectionPolicy {
|
|||
}
|
||||
|
||||
@Override
|
||||
public String getRevisionId(CopyResult copyResult) {
|
||||
return copyResult.getVersionId();
|
||||
public String getRevisionId(CopyObjectResponse copyObjectResponse) {
|
||||
return copyObjectResponse.versionId();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyRevisionConstraint(GetObjectRequest request,
|
||||
public void applyRevisionConstraint(GetObjectRequest.Builder builder,
|
||||
String revisionId) {
|
||||
if (revisionId != null) {
|
||||
LOG.debug("Restricting get request to version {}", revisionId);
|
||||
request.withVersionId(revisionId);
|
||||
builder.versionId(revisionId);
|
||||
} else {
|
||||
LOG.debug("No version ID to use as a constraint");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyRevisionConstraint(CopyObjectRequest request,
|
||||
public void applyRevisionConstraint(CopyObjectRequest.Builder requestBuilder,
|
||||
String revisionId) {
|
||||
if (revisionId != null) {
|
||||
LOG.debug("Restricting copy request to version {}", revisionId);
|
||||
request.withSourceVersionId(revisionId);
|
||||
requestBuilder.sourceVersionId(revisionId);
|
||||
} else {
|
||||
LOG.debug("No version ID to use as a constraint");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyRevisionConstraint(GetObjectMetadataRequest request,
|
||||
public void applyRevisionConstraint(HeadObjectRequest.Builder requestBuilder,
|
||||
String revisionId) {
|
||||
if (revisionId != null) {
|
||||
LOG.debug("Restricting metadata request to version {}", revisionId);
|
||||
request.withVersionId(revisionId);
|
||||
requestBuilder.versionId(revisionId);
|
||||
} else {
|
||||
LOG.debug("No version ID to use as a constraint");
|
||||
}
|
||||
|
@ -482,7 +509,13 @@ public abstract class ChangeDetectionPolicy {
|
|||
}
|
||||
|
||||
@Override
|
||||
public String getRevisionId(final ObjectMetadata objectMetadata,
|
||||
public String getRevisionId(final GetObjectResponse objectMetadata,
|
||||
final String uri) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRevisionId(final HeadObjectResponse objectMetadata,
|
||||
final String uri) {
|
||||
return null;
|
||||
}
|
||||
|
@ -493,24 +526,24 @@ public abstract class ChangeDetectionPolicy {
|
|||
}
|
||||
|
||||
@Override
|
||||
public String getRevisionId(CopyResult copyResult) {
|
||||
public String getRevisionId(CopyObjectResponse copyObjectResponse) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyRevisionConstraint(final GetObjectRequest request,
|
||||
public void applyRevisionConstraint(final GetObjectRequest.Builder builder,
|
||||
final String revisionId) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyRevisionConstraint(CopyObjectRequest request,
|
||||
public void applyRevisionConstraint(CopyObjectRequest.Builder requestBuilder,
|
||||
String revisionId) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyRevisionConstraint(GetObjectMetadataRequest request,
|
||||
public void applyRevisionConstraint(HeadObjectRequest.Builder requestBuilder,
|
||||
String revisionId) {
|
||||
|
||||
}
|
||||
|
|
|
@ -18,18 +18,19 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a.impl;
|
||||
|
||||
import com.amazonaws.AmazonServiceException;
|
||||
import com.amazonaws.SdkBaseException;
|
||||
import com.amazonaws.services.s3.model.CopyObjectRequest;
|
||||
import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
|
||||
import com.amazonaws.services.s3.model.GetObjectRequest;
|
||||
import com.amazonaws.services.s3.model.ObjectMetadata;
|
||||
import com.amazonaws.services.s3.model.S3Object;
|
||||
import com.amazonaws.services.s3.transfer.model.CopyResult;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
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.CopyObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.CopyObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.GetObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
|
||||
|
||||
import org.apache.commons.lang3.tuple.ImmutablePair;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
@ -39,6 +40,7 @@ import org.apache.hadoop.fs.s3a.RemoteFileChangedException;
|
|||
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
|
||||
import org.apache.hadoop.fs.s3a.statistics.ChangeTrackerStatistics;
|
||||
|
||||
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_412_PRECONDITION_FAILED;
|
||||
import static org.apache.hadoop.util.Preconditions.checkNotNull;
|
||||
|
||||
/**
|
||||
|
@ -55,8 +57,6 @@ public class ChangeTracker {
|
|||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(ChangeTracker.class);
|
||||
|
||||
/** {@code 412 Precondition Failed} (HTTP/1.1 - RFC 2616) */
|
||||
public static final int SC_PRECONDITION_FAILED = 412;
|
||||
public static final String CHANGE_REPORTED_BY_S3 = "Change reported by S3";
|
||||
|
||||
/** Policy to use. */
|
||||
|
@ -117,15 +117,15 @@ public class ChangeTracker {
|
|||
/**
|
||||
* Apply any revision control set by the policy if it is to be
|
||||
* enforced on the server.
|
||||
* @param request request to modify
|
||||
* @param builder request builder to modify
|
||||
* @return true iff a constraint was added.
|
||||
*/
|
||||
public boolean maybeApplyConstraint(
|
||||
final GetObjectRequest request) {
|
||||
final GetObjectRequest.Builder builder) {
|
||||
|
||||
if (policy.getMode() == ChangeDetectionPolicy.Mode.Server
|
||||
&& revisionId != null) {
|
||||
policy.applyRevisionConstraint(request, revisionId);
|
||||
policy.applyRevisionConstraint(builder, revisionId);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
|
@ -134,26 +134,26 @@ public class ChangeTracker {
|
|||
/**
|
||||
* Apply any revision control set by the policy if it is to be
|
||||
* enforced on the server.
|
||||
* @param request request to modify
|
||||
* @param requestBuilder copy object request builder to modify
|
||||
* @return true iff a constraint was added.
|
||||
*/
|
||||
public boolean maybeApplyConstraint(
|
||||
final CopyObjectRequest request) {
|
||||
final CopyObjectRequest.Builder requestBuilder) {
|
||||
|
||||
if (policy.getMode() == ChangeDetectionPolicy.Mode.Server
|
||||
&& revisionId != null) {
|
||||
policy.applyRevisionConstraint(request, revisionId);
|
||||
policy.applyRevisionConstraint(requestBuilder, revisionId);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public boolean maybeApplyConstraint(
|
||||
final GetObjectMetadataRequest request) {
|
||||
final HeadObjectRequest.Builder requestBuilder) {
|
||||
|
||||
if (policy.getMode() == ChangeDetectionPolicy.Mode.Server
|
||||
&& revisionId != null) {
|
||||
policy.applyRevisionConstraint(request, revisionId);
|
||||
policy.applyRevisionConstraint(requestBuilder, revisionId);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
|
@ -168,7 +168,7 @@ public class ChangeTracker {
|
|||
* @throws PathIOException raised on failure
|
||||
* @throws RemoteFileChangedException if the remote file has changed.
|
||||
*/
|
||||
public void processResponse(final S3Object object,
|
||||
public void processResponse(final GetObjectResponse object,
|
||||
final String operation,
|
||||
final long pos) throws PathIOException {
|
||||
if (object == null) {
|
||||
|
@ -191,30 +191,32 @@ public class ChangeTracker {
|
|||
}
|
||||
}
|
||||
|
||||
processMetadata(object.getObjectMetadata(), operation);
|
||||
processMetadata(object, operation);
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the response from the server for validation against the
|
||||
* change policy.
|
||||
* @param copyResult result of a copy operation
|
||||
* @param copyObjectResponse response of a copy operation
|
||||
* @throws PathIOException raised on failure
|
||||
* @throws RemoteFileChangedException if the remote file has changed.
|
||||
*/
|
||||
public void processResponse(final CopyResult copyResult)
|
||||
public void processResponse(final CopyObjectResponse copyObjectResponse)
|
||||
throws PathIOException {
|
||||
// ETag (sometimes, depending on encryption and/or multipart) is not the
|
||||
// same on the copied object as the original. Version Id seems to never
|
||||
// be the same on the copy. As such, there isn't really anything that
|
||||
// can be verified on the response, except that a revision ID is present
|
||||
// if required.
|
||||
String newRevisionId = policy.getRevisionId(copyResult);
|
||||
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()));
|
||||
// }
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -227,13 +229,14 @@ public class ChangeTracker {
|
|||
* generated (e.g. "copy", "read", "select").
|
||||
* @throws RemoteFileChangedException if the remote file has changed.
|
||||
*/
|
||||
public void processException(SdkBaseException e, String operation) throws
|
||||
public void processException(SdkException e, String operation) throws
|
||||
RemoteFileChangedException {
|
||||
if (e instanceof AmazonServiceException) {
|
||||
AmazonServiceException serviceException = (AmazonServiceException) e;
|
||||
// This isn't really going to be hit due to
|
||||
if (e instanceof AwsServiceException) {
|
||||
AwsServiceException serviceException = (AwsServiceException)e;
|
||||
// TODO: Verify whether this is fixed in SDK v2.
|
||||
// In SDK v1, this wasn't really going to be hit due to
|
||||
// https://github.com/aws/aws-sdk-java/issues/1644
|
||||
if (serviceException.getStatusCode() == SC_PRECONDITION_FAILED) {
|
||||
if (serviceException.statusCode() == SC_412_PRECONDITION_FAILED) {
|
||||
versionMismatches.versionMismatchError();
|
||||
throw new RemoteFileChangedException(uri, operation, String.format(
|
||||
RemoteFileChangedException.PRECONDITIONS_FAILED
|
||||
|
@ -254,12 +257,26 @@ public class ChangeTracker {
|
|||
* @throws PathIOException raised on failure
|
||||
* @throws RemoteFileChangedException if the remote file has changed.
|
||||
*/
|
||||
public void processMetadata(final ObjectMetadata metadata,
|
||||
public void processMetadata(final HeadObjectResponse metadata,
|
||||
final String operation) throws PathIOException {
|
||||
final String newRevisionId = policy.getRevisionId(metadata, uri);
|
||||
processNewRevision(newRevisionId, operation, -1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the response from server for validation against the change
|
||||
* policy.
|
||||
* @param getObjectResponse response returned from server
|
||||
* @param operation operation in progress
|
||||
* @throws PathIOException raised on failure
|
||||
* @throws RemoteFileChangedException if the remote file has changed.
|
||||
*/
|
||||
public void processMetadata(final GetObjectResponse getObjectResponse,
|
||||
final String operation) throws PathIOException {
|
||||
final String newRevisionId = policy.getRevisionId(getObjectResponse, uri);
|
||||
processNewRevision(newRevisionId, operation, -1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate a revision from the server against our expectations.
|
||||
* @param newRevisionId new revision.
|
||||
|
|
|
@ -1,80 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.s3a.impl;
|
||||
|
||||
import com.amazonaws.SdkBaseException;
|
||||
import com.amazonaws.services.s3.transfer.Copy;
|
||||
import com.amazonaws.services.s3.transfer.model.CopyResult;
|
||||
|
||||
/**
|
||||
* Extracts the outcome of a TransferManager-executed copy operation.
|
||||
*/
|
||||
public final class CopyOutcome {
|
||||
|
||||
/**
|
||||
* Result of a successful copy.
|
||||
*/
|
||||
private final CopyResult copyResult;
|
||||
|
||||
/** the copy was interrupted. */
|
||||
private final InterruptedException interruptedException;
|
||||
|
||||
/**
|
||||
* The copy raised an AWS Exception of some form.
|
||||
*/
|
||||
private final SdkBaseException awsException;
|
||||
|
||||
public CopyOutcome(CopyResult copyResult,
|
||||
InterruptedException interruptedException,
|
||||
SdkBaseException awsException) {
|
||||
this.copyResult = copyResult;
|
||||
this.interruptedException = interruptedException;
|
||||
this.awsException = awsException;
|
||||
}
|
||||
|
||||
public CopyResult getCopyResult() {
|
||||
return copyResult;
|
||||
}
|
||||
|
||||
public InterruptedException getInterruptedException() {
|
||||
return interruptedException;
|
||||
}
|
||||
|
||||
public SdkBaseException getAwsException() {
|
||||
return awsException;
|
||||
}
|
||||
|
||||
/**
|
||||
* Calls {@code Copy.waitForCopyResult()} to await the result, converts
|
||||
* it to a copy outcome.
|
||||
* Exceptions caught and
|
||||
* @param copy the copy operation.
|
||||
* @return the outcome.
|
||||
*/
|
||||
public static CopyOutcome waitForCopy(Copy copy) {
|
||||
try {
|
||||
CopyResult result = copy.waitForCopyResult();
|
||||
return new CopyOutcome(result, null, null);
|
||||
} catch (SdkBaseException e) {
|
||||
return new CopyOutcome(null, null, e);
|
||||
} catch (InterruptedException e) {
|
||||
return new CopyOutcome(null, e, null);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -24,7 +24,6 @@ import java.util.List;
|
|||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
||||
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService;
|
||||
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -40,6 +39,8 @@ import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
|
|||
import org.apache.hadoop.fs.s3a.Tristate;
|
||||
import org.apache.hadoop.util.DurationInfo;
|
||||
|
||||
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
|
||||
|
||||
import static org.apache.hadoop.fs.store.audit.AuditingFunctions.callableWithinAuditSpan;
|
||||
import static org.apache.hadoop.util.Preconditions.checkArgument;
|
||||
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.maybeAwaitCompletion;
|
||||
|
@ -386,9 +387,9 @@ public class DeleteOperation extends ExecutingStoreOperation<Boolean> {
|
|||
"Delete page of %d keys", keyList.size())) {
|
||||
if (!keyList.isEmpty()) {
|
||||
// first delete the files.
|
||||
List<DeleteObjectsRequest.KeyVersion> files = keyList.stream()
|
||||
List<ObjectIdentifier> files = keyList.stream()
|
||||
.filter(e -> !e.isDirMarker)
|
||||
.map(e -> e.keyVersion)
|
||||
.map(e -> e.objectIdentifier)
|
||||
.collect(Collectors.toList());
|
||||
LOG.debug("Deleting of {} file objects", files.size());
|
||||
Invoker.once("Remove S3 Files",
|
||||
|
@ -398,9 +399,9 @@ public class DeleteOperation extends ExecutingStoreOperation<Boolean> {
|
|||
false
|
||||
));
|
||||
// now the dirs
|
||||
List<DeleteObjectsRequest.KeyVersion> dirs = keyList.stream()
|
||||
List<ObjectIdentifier> dirs = keyList.stream()
|
||||
.filter(e -> e.isDirMarker)
|
||||
.map(e -> e.keyVersion)
|
||||
.map(e -> e.objectIdentifier)
|
||||
.collect(Collectors.toList());
|
||||
LOG.debug("Deleting of {} directory markers", dirs.size());
|
||||
// This is invoked with deleteFakeDir.
|
||||
|
@ -422,17 +423,17 @@ public class DeleteOperation extends ExecutingStoreOperation<Boolean> {
|
|||
* to choose which statistics to update.
|
||||
*/
|
||||
private static final class DeleteEntry {
|
||||
private final DeleteObjectsRequest.KeyVersion keyVersion;
|
||||
private final ObjectIdentifier objectIdentifier;
|
||||
|
||||
private final boolean isDirMarker;
|
||||
|
||||
private DeleteEntry(final String key, final boolean isDirMarker) {
|
||||
this.keyVersion = new DeleteObjectsRequest.KeyVersion(key);
|
||||
this.objectIdentifier = ObjectIdentifier.builder().key(key).build();
|
||||
this.isDirMarker = isDirMarker;
|
||||
}
|
||||
|
||||
public String getKey() {
|
||||
return keyVersion.getKey();
|
||||
return objectIdentifier.key();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -18,9 +18,9 @@
|
|||
|
||||
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;
|
||||
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404_NOT_FOUND;
|
||||
|
||||
/**
|
||||
* Translate from AWS SDK-wrapped exceptions into IOExceptions with
|
||||
|
@ -49,9 +49,9 @@ public class ErrorTranslation {
|
|||
* @return true if the status code and error code mean that the
|
||||
* remote bucket is unknown.
|
||||
*/
|
||||
public static boolean isUnknownBucket(AmazonServiceException e) {
|
||||
return e.getStatusCode() == SC_404
|
||||
&& AwsErrorCodes.E_NO_SUCH_BUCKET.equals(e.getErrorCode());
|
||||
public static boolean isUnknownBucket(AwsServiceException e) {
|
||||
return e.statusCode() == SC_404_NOT_FOUND
|
||||
&& AwsErrorCodes.E_NO_SUCH_BUCKET.equals(e.awsErrorDetails().errorCode());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -62,8 +62,8 @@ public class ErrorTranslation {
|
|||
* @return true if the status code and error code mean that the
|
||||
* HEAD request returned 404 but the bucket was there.
|
||||
*/
|
||||
public static boolean isObjectNotFound(AmazonServiceException e) {
|
||||
return e.getStatusCode() == SC_404 && !isUnknownBucket(e);
|
||||
public static boolean isObjectNotFound(AwsServiceException e) {
|
||||
return e.statusCode() == SC_404_NOT_FOUND && !isUnknownBucket(e);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -23,15 +23,17 @@ import java.io.FileNotFoundException;
|
|||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import com.amazonaws.services.s3.Headers;
|
||||
import com.amazonaws.services.s3.model.ObjectMetadata;
|
||||
import 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 +277,7 @@ public class HeaderProcessing extends AbstractStoreOperation {
|
|||
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 +289,67 @@ public class HeaderProcessing extends AbstractStoreOperation {
|
|||
callbacks.getObjectMetadata(objectKey + "/"));
|
||||
}
|
||||
// all user metadata
|
||||
Map<String, String> rawHeaders = md.getUserMetadata();
|
||||
Map<String, String> rawHeaders = md.metadata();
|
||||
Map<String, byte[]> 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 +468,51 @@ public class HeaderProcessing extends AbstractStoreOperation {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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<String, String> 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.cacheControl() != null) {
|
||||
copyObjectRequestBuilder.cacheControl(source.cacheControl());
|
||||
}
|
||||
if (source.getContentDisposition() != null) {
|
||||
dest.setContentDisposition(source.getContentDisposition());
|
||||
if (source.contentDisposition() != null) {
|
||||
copyObjectRequestBuilder.contentDisposition(source.contentDisposition());
|
||||
}
|
||||
if (source.getContentEncoding() != null) {
|
||||
dest.setContentEncoding(source.getContentEncoding());
|
||||
if (source.contentEncoding() != null) {
|
||||
copyObjectRequestBuilder.contentEncoding(source.contentEncoding());
|
||||
}
|
||||
if (source.getContentMD5() != null) {
|
||||
dest.setContentMD5(source.getContentMD5());
|
||||
|
||||
if (source.contentType() != null) {
|
||||
copyObjectRequestBuilder.contentType(source.contentType());
|
||||
}
|
||||
if (source.getContentType() != null) {
|
||||
dest.setContentType(source.getContentType());
|
||||
|
||||
if (source.serverSideEncryption() != null) {
|
||||
copyObjectRequestBuilder.serverSideEncryption(source.serverSideEncryption());
|
||||
}
|
||||
if (source.getExpirationTime() != null) {
|
||||
dest.setExpirationTime(source.getExpirationTime());
|
||||
|
||||
if (source.sseCustomerAlgorithm() != null) {
|
||||
copyObjectRequestBuilder.copySourceSSECustomerAlgorithm(source.sseCustomerAlgorithm());
|
||||
}
|
||||
if (source.getExpirationTimeRuleId() != null) {
|
||||
dest.setExpirationTimeRuleId(source.getExpirationTimeRuleId());
|
||||
}
|
||||
if (source.getHttpExpiresDate() != null) {
|
||||
dest.setHttpExpiresDate(source.getHttpExpiresDate());
|
||||
}
|
||||
if (source.getLastModified() != null) {
|
||||
dest.setLastModified(source.getLastModified());
|
||||
}
|
||||
if (source.getOngoingRestore() != null) {
|
||||
dest.setOngoingRestore(source.getOngoingRestore());
|
||||
}
|
||||
if (source.getRestoreExpirationTime() != null) {
|
||||
dest.setRestoreExpirationTime(source.getRestoreExpirationTime());
|
||||
}
|
||||
if (source.getSSEAlgorithm() != null) {
|
||||
dest.setSSEAlgorithm(source.getSSEAlgorithm());
|
||||
}
|
||||
if (source.getSSECustomerAlgorithm() != null) {
|
||||
dest.setSSECustomerAlgorithm(source.getSSECustomerAlgorithm());
|
||||
}
|
||||
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 +525,6 @@ public class HeaderProcessing extends AbstractStoreOperation {
|
|||
* @throws IOException IO and object access problems.
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
ObjectMetadata getObjectMetadata(String key) throws IOException;
|
||||
HeadObjectResponse getObjectMetadata(String key) throws IOException;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -110,11 +110,50 @@ public final class InternalConstants {
|
|||
S3A_OPENFILE_KEYS = Collections.unmodifiableSet(keys);
|
||||
}
|
||||
|
||||
/** 403 error code. */
|
||||
public static final int SC_403 = 403;
|
||||
/** 200 status code: OK. */
|
||||
public static final int SC_200_OK = 200;
|
||||
|
||||
/** 404 error code. */
|
||||
public static final int SC_404 = 404;
|
||||
/** 301 status code: Moved Permanently. */
|
||||
public static final int SC_301_MOVED_PERMANENTLY = 301;
|
||||
|
||||
/** 307 status code: Temporary Redirect. */
|
||||
public static final int SC_307_TEMPORARY_REDIRECT = 307;
|
||||
|
||||
/** 400 status code: Bad Request. */
|
||||
public static final int SC_400_BAD_REQUEST = 400;
|
||||
|
||||
/** 401 status code: Unauthorized. */
|
||||
public static final int SC_401_UNAUTHORIZED = 401;
|
||||
|
||||
/** 403 status code: Forbidden. */
|
||||
public static final int SC_403_FORBIDDEN = 403;
|
||||
|
||||
/** 404 status code: Not Found. */
|
||||
public static final int SC_404_NOT_FOUND = 404;
|
||||
|
||||
/** 405 status code: Method Not Allowed. */
|
||||
public static final int SC_405_METHOD_NOT_ALLOWED = 405;
|
||||
|
||||
/** 410 status code: Gone. */
|
||||
public static final int SC_410_GONE = 410;
|
||||
|
||||
/** 412 status code: Precondition Failed. */
|
||||
public static final int SC_412_PRECONDITION_FAILED = 412;
|
||||
|
||||
/** 416 status code: Range Not Satisfiable. */
|
||||
public static final int SC_416_RANGE_NOT_SATISFIABLE = 416;
|
||||
|
||||
/** 443 status code: No Response (unofficial). */
|
||||
public static final int SC_443_NO_RESPONSE = 443;
|
||||
|
||||
/** 444 status code: No Response (unofficial). */
|
||||
public static final int SC_444_NO_RESPONSE = 444;
|
||||
|
||||
/** 500 status code: Internal Server Error. */
|
||||
public static final int SC_500_INTERNAL_SERVER_ERROR = 500;
|
||||
|
||||
/** 503 status code: Service Unavailable. */
|
||||
public static final int SC_503_SERVICE_UNAVAILABLE = 503;
|
||||
|
||||
/** Name of the log for throttling events. Value: {@value}. */
|
||||
public static final String THROTTLE_LOG_NAME =
|
||||
|
|
|
@ -22,15 +22,15 @@ import java.io.IOException;
|
|||
import java.io.InterruptedIOException;
|
||||
import java.util.List;
|
||||
|
||||
import com.amazonaws.AmazonClientException;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
|
||||
import com.amazonaws.services.s3.transfer.model.CopyResult;
|
||||
import software.amazon.awssdk.awscore.exception.AwsServiceException;
|
||||
import software.amazon.awssdk.services.s3.model.CopyObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.InvalidRequestException;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.s3a.MultiObjectDeleteException;
|
||||
import org.apache.hadoop.fs.s3a.Retries;
|
||||
import org.apache.hadoop.fs.s3a.S3AFileStatus;
|
||||
import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
|
||||
|
@ -127,7 +127,7 @@ public interface OperationCallbacks {
|
|||
* @throws IOException Other IO problems
|
||||
*/
|
||||
@Retries.RetryTranslated
|
||||
CopyResult copyFile(String srcKey,
|
||||
CopyObjectResponse copyFile(String srcKey,
|
||||
String destKey,
|
||||
S3ObjectAttributes srcAttributes,
|
||||
S3AReadOpContext readContext)
|
||||
|
@ -142,14 +142,14 @@ public interface OperationCallbacks {
|
|||
* a mistaken attempt to delete the root directory.
|
||||
* @throws MultiObjectDeleteException one or more of the keys could not
|
||||
* be deleted in a multiple object delete operation.
|
||||
* @throws AmazonClientException amazon-layer failure.
|
||||
* @throws AwsServiceException amazon-layer failure.
|
||||
* @throws IOException other IO Exception.
|
||||
*/
|
||||
@Retries.RetryRaw
|
||||
void removeKeys(
|
||||
List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
||||
List<ObjectIdentifier> keysToDelete,
|
||||
boolean deleteFakeDir)
|
||||
throws MultiObjectDeleteException, AmazonClientException,
|
||||
throws MultiObjectDeleteException, AwsServiceException,
|
||||
IOException;
|
||||
|
||||
/**
|
||||
|
|
|
@ -25,9 +25,6 @@ import java.util.Map;
|
|||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import com.amazonaws.AmazonClientException;
|
||||
import com.amazonaws.SdkBaseException;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -44,6 +41,9 @@ import org.apache.hadoop.fs.s3a.Tristate;
|
|||
import org.apache.hadoop.util.DurationInfo;
|
||||
import org.apache.hadoop.util.OperationDuration;
|
||||
|
||||
import software.amazon.awssdk.core.exception.SdkException;
|
||||
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
|
||||
|
||||
import static org.apache.hadoop.fs.s3a.S3AUtils.translateException;
|
||||
import static org.apache.hadoop.fs.store.audit.AuditingFunctions.callableWithinAuditSpan;
|
||||
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
|
||||
|
@ -122,7 +122,7 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
|
|||
/**
|
||||
* list of keys to delete on the next (bulk) delete call.
|
||||
*/
|
||||
private final List<DeleteObjectsRequest.KeyVersion> keysToDelete =
|
||||
private final List<ObjectIdentifier> keysToDelete =
|
||||
new ArrayList<>();
|
||||
|
||||
/**
|
||||
|
@ -199,7 +199,7 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
|
|||
*/
|
||||
private void queueToDelete(Path path, String key) {
|
||||
LOG.debug("Queueing to delete {}", path);
|
||||
keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key));
|
||||
keysToDelete.add(ObjectIdentifier.builder().key(key).build());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -268,7 +268,7 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
|
|||
} else {
|
||||
recursiveDirectoryRename();
|
||||
}
|
||||
} catch (AmazonClientException | IOException ex) {
|
||||
} catch (SdkException | IOException ex) {
|
||||
// rename failed.
|
||||
// block for all ongoing copies to complete, successfully or not
|
||||
try {
|
||||
|
@ -572,7 +572,7 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
|
|||
*/
|
||||
@Retries.RetryTranslated
|
||||
private void removeSourceObjects(
|
||||
final List<DeleteObjectsRequest.KeyVersion> keys)
|
||||
final List<ObjectIdentifier> keys)
|
||||
throws IOException {
|
||||
// remove the keys
|
||||
|
||||
|
@ -580,9 +580,9 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
|
|||
// who is trying to debug why objects are no longer there.
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Initiating delete operation for {} objects", keys.size());
|
||||
for (DeleteObjectsRequest.KeyVersion key : keys) {
|
||||
LOG.debug(" {} {}", key.getKey(),
|
||||
key.getVersion() != null ? key.getVersion() : "");
|
||||
for (ObjectIdentifier objectIdentifier : keys) {
|
||||
LOG.debug(" {} {}", objectIdentifier.key(),
|
||||
objectIdentifier.versionId() != null ? objectIdentifier.versionId() : "");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -619,10 +619,10 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
|
|||
protected IOException convertToIOException(final Exception ex) {
|
||||
if (ex instanceof IOException) {
|
||||
return (IOException) ex;
|
||||
} else if (ex instanceof SdkBaseException) {
|
||||
} else if (ex instanceof SdkException) {
|
||||
return translateException("rename " + sourcePath + " to " + destPath,
|
||||
sourcePath.toString(),
|
||||
(SdkBaseException) ex);
|
||||
(SdkException) ex);
|
||||
} else {
|
||||
// should never happen, but for completeness
|
||||
return new IOException(ex);
|
||||
|
|
|
@ -18,42 +18,41 @@
|
|||
|
||||
package org.apache.hadoop.fs.s3a.impl;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Base64;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import com.amazonaws.AmazonWebServiceRequest;
|
||||
import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.CannedAccessControlList;
|
||||
import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.CopyObjectRequest;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectRequest;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
|
||||
import com.amazonaws.services.s3.model.GetObjectRequest;
|
||||
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
|
||||
import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
|
||||
import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.ListObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.ListObjectsV2Request;
|
||||
import com.amazonaws.services.s3.model.ObjectListing;
|
||||
import com.amazonaws.services.s3.model.ObjectMetadata;
|
||||
import com.amazonaws.services.s3.model.PartETag;
|
||||
import com.amazonaws.services.s3.model.PutObjectRequest;
|
||||
import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams;
|
||||
import com.amazonaws.services.s3.model.SSECustomerKey;
|
||||
import com.amazonaws.services.s3.model.SelectObjectContentRequest;
|
||||
import com.amazonaws.services.s3.model.StorageClass;
|
||||
import com.amazonaws.services.s3.model.UploadPartRequest;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import software.amazon.awssdk.core.SdkRequest;
|
||||
import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest;
|
||||
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
|
||||
import software.amazon.awssdk.services.s3.model.CompletedMultipartUpload;
|
||||
import software.amazon.awssdk.services.s3.model.CompletedPart;
|
||||
import software.amazon.awssdk.services.s3.model.CopyObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
|
||||
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest;
|
||||
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest;
|
||||
import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
|
||||
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
|
||||
import software.amazon.awssdk.services.s3.model.MetadataDirective;
|
||||
import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
|
||||
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
|
||||
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest;
|
||||
import software.amazon.awssdk.services.s3.model.ServerSideEncryption;
|
||||
import software.amazon.awssdk.services.s3.model.StorageClass;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
|
||||
import software.amazon.awssdk.utils.Md5Utils;
|
||||
|
||||
import org.apache.hadoop.fs.PathIOException;
|
||||
import org.apache.hadoop.fs.s3a.Retries;
|
||||
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
|
||||
|
@ -79,8 +78,8 @@ import static org.apache.hadoop.util.Preconditions.checkNotNull;
|
|||
* This is where audit span information is added to the requests,
|
||||
* until it is done in the AWS SDK itself.
|
||||
*
|
||||
* All created requests will be passed through
|
||||
* {@link PrepareRequest#prepareRequest(AmazonWebServiceRequest)} before
|
||||
* All created request builders will be passed to
|
||||
* {@link PrepareRequest#prepareRequest(SdkRequest.Builder)} before
|
||||
* being returned to the caller.
|
||||
*/
|
||||
public class RequestFactoryImpl implements RequestFactory {
|
||||
|
@ -101,7 +100,7 @@ public class RequestFactoryImpl implements RequestFactory {
|
|||
/**
|
||||
* ACL For new objects.
|
||||
*/
|
||||
private final CannedAccessControlList cannedACL;
|
||||
private final ObjectCannedACL cannedACL;
|
||||
|
||||
/**
|
||||
* Max number of multipart entries allowed in a large
|
||||
|
@ -147,14 +146,15 @@ public class RequestFactoryImpl implements RequestFactory {
|
|||
|
||||
/**
|
||||
* Preflight preparation of AWS request.
|
||||
* @param <T> web service request
|
||||
* @return prepared entry.
|
||||
* @param <T> web service request builder
|
||||
* @return prepared builder.
|
||||
*/
|
||||
@Retries.OnceRaw
|
||||
private <T extends AmazonWebServiceRequest> T prepareRequest(T t) {
|
||||
return requestPreparer != null
|
||||
? requestPreparer.prepareRequest(t)
|
||||
: t;
|
||||
private <T extends SdkRequest.Builder> T prepareRequest(T t) {
|
||||
if (requestPreparer != null) {
|
||||
requestPreparer.prepareRequest(t);
|
||||
}
|
||||
return t;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -162,7 +162,7 @@ public class RequestFactoryImpl implements RequestFactory {
|
|||
* @return an ACL, if any
|
||||
*/
|
||||
@Override
|
||||
public CannedAccessControlList getCannedACL() {
|
||||
public ObjectCannedACL getCannedACL() {
|
||||
return cannedACL;
|
||||
}
|
||||
|
||||
|
@ -174,29 +174,6 @@ public class RequestFactoryImpl implements RequestFactory {
|
|||
return bucket;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the AWS SDK structure used to configure SSE,
|
||||
* if the encryption secrets contain the information/settings for this.
|
||||
* @return an optional set of KMS Key settings
|
||||
*/
|
||||
@Override
|
||||
public Optional<SSEAwsKeyManagementParams> generateSSEAwsKeyParams() {
|
||||
return EncryptionSecretOperations.createSSEAwsKeyManagementParams(
|
||||
encryptionSecrets);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the SSE-C structure for the AWS SDK, if the encryption secrets
|
||||
* contain the information/settings for this.
|
||||
* This will contain a secret extracted from the bucket/configuration.
|
||||
* @return an optional customer key.
|
||||
*/
|
||||
@Override
|
||||
public Optional<SSECustomerKey> generateSSECustomerKey() {
|
||||
return EncryptionSecretOperations.createSSECustomerKey(
|
||||
encryptionSecrets);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the encryption algorithm of this endpoint.
|
||||
* @return the encryption algorithm.
|
||||
|
@ -229,307 +206,301 @@ public class RequestFactoryImpl implements RequestFactory {
|
|||
* request when encryption is enabled.
|
||||
* @param request upload part request
|
||||
*/
|
||||
protected void setOptionalUploadPartRequestParameters(
|
||||
UploadPartRequest request) {
|
||||
generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
|
||||
protected void uploadPartEncryptionParameters(
|
||||
UploadPartRequest.Builder builder) {
|
||||
// TODO: review/refactor together with similar methods for other requests.
|
||||
// need to set key to get objects encrypted with SSE_C
|
||||
EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets).ifPresent(base64customerKey -> {
|
||||
builder.sseCustomerAlgorithm(ServerSideEncryption.AES256.name())
|
||||
.sseCustomerKey(base64customerKey)
|
||||
.sseCustomerKeyMD5(Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey)));
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets server side encryption parameters to the GET reuquest.
|
||||
* request when encryption is enabled.
|
||||
* @param request upload part request
|
||||
*/
|
||||
protected void setOptionalGetObjectMetadataParameters(
|
||||
GetObjectMetadataRequest request) {
|
||||
generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
|
||||
}
|
||||
private CopyObjectRequest.Builder buildCopyObjectRequest() {
|
||||
|
||||
/**
|
||||
* Set the optional parameters when initiating the request (encryption,
|
||||
* headers, storage, etc).
|
||||
* @param request request to patch.
|
||||
*/
|
||||
protected void setOptionalMultipartUploadRequestParameters(
|
||||
InitiateMultipartUploadRequest request) {
|
||||
generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams);
|
||||
generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
|
||||
}
|
||||
CopyObjectRequest.Builder copyObjectRequestBuilder = CopyObjectRequest.builder();
|
||||
|
||||
/**
|
||||
* Set the optional parameters for a PUT request.
|
||||
* @param request request to patch.
|
||||
*/
|
||||
protected void setOptionalPutRequestParameters(PutObjectRequest request) {
|
||||
generateSSEAwsKeyParams().ifPresent(request::setSSEAwsKeyManagementParams);
|
||||
generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the optional metadata for an object being created or copied.
|
||||
* @param metadata to update.
|
||||
* @param isDirectoryMarker is this for a directory marker?
|
||||
*/
|
||||
protected void setOptionalObjectMetadata(ObjectMetadata metadata,
|
||||
boolean isDirectoryMarker) {
|
||||
final S3AEncryptionMethods algorithm
|
||||
= getServerSideEncryptionAlgorithm();
|
||||
if (S3AEncryptionMethods.SSE_S3 == algorithm) {
|
||||
metadata.setSSEAlgorithm(algorithm.getMethod());
|
||||
if (contentEncoding != null) {
|
||||
copyObjectRequestBuilder.contentEncoding(contentEncoding);
|
||||
}
|
||||
if (contentEncoding != null && !isDirectoryMarker) {
|
||||
metadata.setContentEncoding(contentEncoding);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
return om;
|
||||
return copyObjectRequestBuilder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CopyObjectRequest newCopyObjectRequest(String srcKey,
|
||||
public CopyObjectRequest.Builder newCopyObjectRequestBuilder(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);
|
||||
HeadObjectResponse srcom) {
|
||||
|
||||
CopyObjectRequest.Builder copyObjectRequestBuilder = buildCopyObjectRequest();
|
||||
|
||||
Map<String, String> dstom = new HashMap<>();
|
||||
HeaderProcessing.cloneObjectMetadata(srcom, dstom, copyObjectRequestBuilder);
|
||||
copyEncryptionParameters(copyObjectRequestBuilder);
|
||||
|
||||
copyObjectRequestBuilder
|
||||
.metadata(dstom)
|
||||
.metadataDirective(MetadataDirective.REPLACE)
|
||||
.acl(cannedACL);
|
||||
|
||||
if (srcom.storageClass() != null) {
|
||||
copyObjectRequestBuilder.storageClass(srcom.storageClass());
|
||||
}
|
||||
|
||||
copyObjectRequestBuilder.destinationBucket(getBucket())
|
||||
.destinationKey(dstKey).sourceBucket(getBucket()).sourceKey(srcKey);
|
||||
|
||||
return prepareRequest(copyObjectRequestBuilder);
|
||||
}
|
||||
|
||||
/**
|
||||
* Propagate encryption parameters from source file if set else use the
|
||||
* current filesystem encryption settings.
|
||||
* @param 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;
|
||||
protected void copyEncryptionParameters(CopyObjectRequest.Builder copyObjectRequestBuilder) {
|
||||
|
||||
case SSE_C:
|
||||
generateSSECustomerKey().ifPresent(customerKey -> {
|
||||
copyObjectRequest.setSourceSSECustomerKey(customerKey);
|
||||
copyObjectRequest.setDestinationSSECustomerKey(customerKey);
|
||||
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)));
|
||||
});
|
||||
break;
|
||||
|
||||
case SSE_KMS:
|
||||
generateSSEAwsKeyParams().ifPresent(
|
||||
copyObjectRequest::setSSEAwsKeyManagementParams);
|
||||
break;
|
||||
default:
|
||||
}
|
||||
}
|
||||
/**
|
||||
* 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);
|
||||
if (storageClass != null) {
|
||||
putObjectRequest.setStorageClass(storageClass);
|
||||
}
|
||||
putObjectRequest.setMetadata(metadata);
|
||||
return prepareRequest(putObjectRequest);
|
||||
}
|
||||
long length,
|
||||
boolean isDirectoryMarker) {
|
||||
|
||||
/**
|
||||
* 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);
|
||||
|
||||
PutObjectRequest.Builder putObjectRequestBuilder =
|
||||
buildPutObjectRequest(length, isDirectoryMarker);
|
||||
putObjectRequestBuilder.bucket(getBucket()).key(key);
|
||||
|
||||
if (options != null) {
|
||||
putObjectRequestBuilder.metadata(options.getHeaders());
|
||||
}
|
||||
|
||||
putEncryptionParameters(putObjectRequestBuilder);
|
||||
|
||||
if (storageClass != null) {
|
||||
putObjectRequestBuilder.storageClass(storageClass);
|
||||
}
|
||||
|
||||
return prepareRequest(putObjectRequestBuilder);
|
||||
}
|
||||
|
||||
private PutObjectRequest.Builder buildPutObjectRequest(long length, boolean isDirectoryMarker) {
|
||||
|
||||
PutObjectRequest.Builder putObjectRequestBuilder = PutObjectRequest.builder();
|
||||
|
||||
putObjectRequestBuilder.acl(cannedACL);
|
||||
|
||||
if (length >= 0) {
|
||||
putObjectRequestBuilder.contentLength(length);
|
||||
}
|
||||
|
||||
if (contentEncoding != null && !isDirectoryMarker) {
|
||||
putObjectRequestBuilder.contentEncoding(contentEncoding);
|
||||
}
|
||||
|
||||
return putObjectRequestBuilder;
|
||||
}
|
||||
|
||||
private void putEncryptionParameters(PutObjectRequest.Builder putObjectRequestBuilder) {
|
||||
final S3AEncryptionMethods algorithm
|
||||
= getServerSideEncryptionAlgorithm();
|
||||
|
||||
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);
|
||||
|
||||
PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key,
|
||||
inputStream, metadata);
|
||||
setOptionalPutRequestParameters(putObjectRequest);
|
||||
putObjectRequest.setCannedAcl(cannedACL);
|
||||
return prepareRequest(putObjectRequest);
|
||||
putObjectRequestBuilder.bucket(getBucket()).key(key)
|
||||
.contentType(HeaderProcessing.CONTENT_TYPE_X_DIRECTORY);
|
||||
|
||||
putEncryptionParameters(putObjectRequestBuilder);
|
||||
|
||||
return prepareRequest(putObjectRequestBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListMultipartUploadsRequest
|
||||
newListMultipartUploadsRequest(String prefix) {
|
||||
ListMultipartUploadsRequest request = new ListMultipartUploadsRequest(
|
||||
getBucket());
|
||||
public ListMultipartUploadsRequest.Builder
|
||||
newListMultipartUploadsRequestBuilder(String prefix) {
|
||||
|
||||
ListMultipartUploadsRequest.Builder requestBuilder = ListMultipartUploadsRequest.builder();
|
||||
|
||||
requestBuilder.bucket(getBucket());
|
||||
if (prefix != null) {
|
||||
request.setPrefix(prefix);
|
||||
requestBuilder.prefix(prefix);
|
||||
}
|
||||
return prepareRequest(request);
|
||||
return prepareRequest(requestBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbortMultipartUploadRequest newAbortMultipartUploadRequest(
|
||||
public AbortMultipartUploadRequest.Builder newAbortMultipartUploadRequestBuilder(
|
||||
String destKey,
|
||||
String uploadId) {
|
||||
return prepareRequest(new AbortMultipartUploadRequest(getBucket(),
|
||||
destKey,
|
||||
uploadId));
|
||||
AbortMultipartUploadRequest.Builder requestBuilder =
|
||||
AbortMultipartUploadRequest.builder().bucket(getBucket()).key(destKey).uploadId(uploadId);
|
||||
|
||||
return prepareRequest(requestBuilder);
|
||||
}
|
||||
|
||||
private void multipartUploadEncryptionParameters(CreateMultipartUploadRequest.Builder mpuRequestBuilder) {
|
||||
final S3AEncryptionMethods algorithm
|
||||
= getServerSideEncryptionAlgorithm();
|
||||
|
||||
if (S3AEncryptionMethods.SSE_S3 == algorithm) {
|
||||
mpuRequestBuilder.serverSideEncryption(algorithm.getMethod());
|
||||
} else if (S3AEncryptionMethods.SSE_KMS == algorithm) {
|
||||
mpuRequestBuilder.serverSideEncryption(ServerSideEncryption.AWS_KMS);
|
||||
// Set the KMS key if present, else S3 uses AWS managed key.
|
||||
EncryptionSecretOperations.getSSEAwsKMSKey(encryptionSecrets)
|
||||
.ifPresent(kmsKey -> mpuRequestBuilder.ssekmsKeyId(kmsKey));
|
||||
} else if (S3AEncryptionMethods.SSE_C == algorithm) {
|
||||
EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets)
|
||||
.ifPresent(base64customerKey -> {
|
||||
mpuRequestBuilder.sseCustomerAlgorithm(ServerSideEncryption.AES256.name())
|
||||
.sseCustomerKey(base64customerKey).sseCustomerKeyMD5(
|
||||
Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey)));
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public InitiateMultipartUploadRequest newMultipartUploadRequest(
|
||||
public CreateMultipartUploadRequest.Builder newMultipartUploadRequestBuilder(
|
||||
final String destKey,
|
||||
@Nullable final PutObjectOptions options) throws PathIOException {
|
||||
if (!isMultipartUploadEnabled) {
|
||||
throw new PathIOException(destKey, "Multipart uploads are disabled.");
|
||||
}
|
||||
final ObjectMetadata objectMetadata = newObjectMetadata(-1);
|
||||
maybeSetMetadata(options, objectMetadata);
|
||||
final InitiateMultipartUploadRequest initiateMPURequest =
|
||||
new InitiateMultipartUploadRequest(getBucket(),
|
||||
destKey,
|
||||
objectMetadata);
|
||||
initiateMPURequest.setCannedACL(getCannedACL());
|
||||
if (getStorageClass() != null) {
|
||||
initiateMPURequest.withStorageClass(getStorageClass());
|
||||
|
||||
CreateMultipartUploadRequest.Builder requestBuilder = CreateMultipartUploadRequest.builder();
|
||||
|
||||
if (contentEncoding != null) {
|
||||
requestBuilder.contentEncoding(contentEncoding);
|
||||
}
|
||||
setOptionalMultipartUploadRequestParameters(initiateMPURequest);
|
||||
return prepareRequest(initiateMPURequest);
|
||||
|
||||
if (options != null) {
|
||||
requestBuilder.metadata(options.getHeaders());
|
||||
}
|
||||
|
||||
requestBuilder.bucket(getBucket()).key(destKey).acl(cannedACL);
|
||||
|
||||
multipartUploadEncryptionParameters(requestBuilder);
|
||||
|
||||
if (storageClass != null) {
|
||||
requestBuilder.storageClass(storageClass);
|
||||
}
|
||||
|
||||
return prepareRequest(requestBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompleteMultipartUploadRequest newCompleteMultipartUploadRequest(
|
||||
public CompleteMultipartUploadRequest.Builder newCompleteMultipartUploadRequestBuilder(
|
||||
String destKey,
|
||||
String uploadId,
|
||||
List<PartETag> partETags) {
|
||||
List<CompletedPart> partETags) {
|
||||
// a copy of the list is required, so that the AWS SDK doesn't
|
||||
// attempt to sort an unmodifiable list.
|
||||
return prepareRequest(new CompleteMultipartUploadRequest(bucket,
|
||||
destKey, uploadId, new ArrayList<>(partETags)));
|
||||
CompleteMultipartUploadRequest.Builder requestBuilder =
|
||||
CompleteMultipartUploadRequest.builder().bucket(bucket).key(destKey).uploadId(uploadId)
|
||||
.multipartUpload(CompletedMultipartUpload.builder().parts(partETags).build());
|
||||
|
||||
return prepareRequest(requestBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetObjectMetadataRequest newGetObjectMetadataRequest(String key) {
|
||||
GetObjectMetadataRequest request =
|
||||
new GetObjectMetadataRequest(getBucket(), key);
|
||||
//SSE-C requires to be filled in if enabled for object metadata
|
||||
setOptionalGetObjectMetadataParameters(request);
|
||||
return prepareRequest(request);
|
||||
public HeadObjectRequest.Builder newHeadObjectRequestBuilder(String key) {
|
||||
|
||||
HeadObjectRequest.Builder headObjectRequestBuilder =
|
||||
HeadObjectRequest.builder().bucket(getBucket()).key(key);
|
||||
|
||||
// need to set key to get metadata for objects encrypted with SSE_C
|
||||
EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets).ifPresent(base64customerKey -> {
|
||||
headObjectRequestBuilder.sseCustomerAlgorithm(ServerSideEncryption.AES256.name())
|
||||
.sseCustomerKey(base64customerKey)
|
||||
.sseCustomerKeyMD5(Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey)));
|
||||
});
|
||||
|
||||
return prepareRequest(headObjectRequestBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public 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 prepareRequest(builder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public UploadPartRequest newUploadPartRequest(
|
||||
public UploadPartRequest.Builder newUploadPartRequestBuilder(
|
||||
String destKey,
|
||||
String uploadId,
|
||||
int partNumber,
|
||||
long size,
|
||||
InputStream uploadStream,
|
||||
File sourceFile,
|
||||
long offset) throws PathIOException {
|
||||
long size) throws PathIOException {
|
||||
checkNotNull(uploadId);
|
||||
// exactly one source must be set; xor verifies this
|
||||
checkArgument((uploadStream != null) ^ (sourceFile != null),
|
||||
"Data source");
|
||||
checkArgument(size >= 0, "Invalid partition size %s", size);
|
||||
checkArgument(partNumber > 0,
|
||||
"partNumber must be between 1 and %s inclusive, but is %s",
|
||||
DEFAULT_UPLOAD_PART_COUNT_LIMIT, partNumber);
|
||||
multipartPartCountLimit, partNumber);
|
||||
|
||||
LOG.debug("Creating part upload request for {} #{} size {}",
|
||||
uploadId, partNumber, size);
|
||||
|
@ -539,88 +510,81 @@ public class RequestFactoryImpl implements RequestFactory {
|
|||
throw new PathIOException(destKey,
|
||||
String.format(pathErrorMsg, partNumber, multipartPartCountLimit));
|
||||
}
|
||||
UploadPartRequest request = new UploadPartRequest()
|
||||
.withBucketName(getBucket())
|
||||
.withKey(destKey)
|
||||
.withUploadId(uploadId)
|
||||
.withPartNumber(partNumber)
|
||||
.withPartSize(size);
|
||||
if (uploadStream != null) {
|
||||
// there's an upload stream. Bind to it.
|
||||
request.setInputStream(uploadStream);
|
||||
} else {
|
||||
checkArgument(sourceFile.exists(),
|
||||
"Source file does not exist: %s", sourceFile);
|
||||
checkArgument(sourceFile.isFile(),
|
||||
"Source is not a file: %s", sourceFile);
|
||||
checkArgument(offset >= 0, "Invalid offset %s", offset);
|
||||
long length = sourceFile.length();
|
||||
checkArgument(offset == 0 || offset < length,
|
||||
"Offset %s beyond length of file %s", offset, length);
|
||||
request.setFile(sourceFile);
|
||||
request.setFileOffset(offset);
|
||||
}
|
||||
setOptionalUploadPartRequestParameters(request);
|
||||
return prepareRequest(request);
|
||||
UploadPartRequest.Builder builder = UploadPartRequest.builder()
|
||||
.bucket(getBucket())
|
||||
.key(destKey)
|
||||
.uploadId(uploadId)
|
||||
.partNumber(partNumber)
|
||||
.contentLength(size);
|
||||
uploadPartEncryptionParameters(builder);
|
||||
return prepareRequest(builder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SelectObjectContentRequest newSelectRequest(String key) {
|
||||
SelectObjectContentRequest request = new SelectObjectContentRequest();
|
||||
request.setBucketName(bucket);
|
||||
request.setKey(key);
|
||||
generateSSECustomerKey().ifPresent(request::setSSECustomerKey);
|
||||
return prepareRequest(request);
|
||||
public SelectObjectContentRequest.Builder newSelectRequestBuilder(String key) {
|
||||
SelectObjectContentRequest.Builder requestBuilder =
|
||||
SelectObjectContentRequest.builder()
|
||||
.bucket(bucket)
|
||||
.key(key);
|
||||
|
||||
EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets)
|
||||
.ifPresent(base64customerKey -> {
|
||||
requestBuilder
|
||||
.sseCustomerAlgorithm(ServerSideEncryption.AES256.name())
|
||||
.sseCustomerKey(base64customerKey)
|
||||
.sseCustomerKeyMD5(Md5Utils.md5AsBase64(
|
||||
Base64.getDecoder().decode(base64customerKey)));
|
||||
});
|
||||
|
||||
return prepareRequest(requestBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListObjectsRequest newListObjectsV1Request(
|
||||
public ListObjectsRequest.Builder newListObjectsV1RequestBuilder(
|
||||
final String key,
|
||||
final String delimiter,
|
||||
final int maxKeys) {
|
||||
ListObjectsRequest request = new ListObjectsRequest()
|
||||
.withBucketName(bucket)
|
||||
.withMaxKeys(maxKeys)
|
||||
.withPrefix(key);
|
||||
|
||||
ListObjectsRequest.Builder requestBuilder =
|
||||
ListObjectsRequest.builder().bucket(bucket).maxKeys(maxKeys).prefix(key);
|
||||
|
||||
if (delimiter != null) {
|
||||
request.setDelimiter(delimiter);
|
||||
requestBuilder.delimiter(delimiter);
|
||||
}
|
||||
return prepareRequest(request);
|
||||
|
||||
return prepareRequest(requestBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListNextBatchOfObjectsRequest newListNextBatchOfObjectsRequest(
|
||||
ObjectListing prev) {
|
||||
return prepareRequest(new ListNextBatchOfObjectsRequest(prev));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListObjectsV2Request newListObjectsV2Request(
|
||||
public ListObjectsV2Request.Builder newListObjectsV2RequestBuilder(
|
||||
final String key,
|
||||
final String delimiter,
|
||||
final int maxKeys) {
|
||||
final ListObjectsV2Request request = new ListObjectsV2Request()
|
||||
.withBucketName(bucket)
|
||||
.withMaxKeys(maxKeys)
|
||||
.withPrefix(key);
|
||||
|
||||
final ListObjectsV2Request.Builder requestBuilder = ListObjectsV2Request.builder()
|
||||
.bucket(bucket)
|
||||
.maxKeys(maxKeys)
|
||||
.prefix(key);
|
||||
|
||||
if (delimiter != null) {
|
||||
request.setDelimiter(delimiter);
|
||||
requestBuilder.delimiter(delimiter);
|
||||
}
|
||||
return prepareRequest(request);
|
||||
|
||||
return prepareRequest(requestBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DeleteObjectRequest newDeleteObjectRequest(String key) {
|
||||
return prepareRequest(new DeleteObjectRequest(bucket, key));
|
||||
public DeleteObjectRequest.Builder newDeleteObjectRequestBuilder(String key) {
|
||||
return prepareRequest(DeleteObjectRequest.builder().bucket(bucket).key(key));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DeleteObjectsRequest newBulkDeleteRequest(
|
||||
List<DeleteObjectsRequest.KeyVersion> keysToDelete) {
|
||||
return prepareRequest(
|
||||
new DeleteObjectsRequest(bucket)
|
||||
.withKeys(keysToDelete)
|
||||
.withQuiet(true));
|
||||
public DeleteObjectsRequest.Builder newBulkDeleteRequestBuilder(
|
||||
List<ObjectIdentifier> keysToDelete) {
|
||||
return prepareRequest(DeleteObjectsRequest
|
||||
.builder()
|
||||
.bucket(bucket)
|
||||
.delete(d -> d.objects(keysToDelete).quiet(true)));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -628,23 +592,6 @@ public class RequestFactoryImpl implements RequestFactory {
|
|||
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<String, String> headers = options.getHeaders();
|
||||
if (headers != null) {
|
||||
objectMetadata.setUserMetadata(headers);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a builder.
|
||||
* @return new builder.
|
||||
|
@ -671,7 +618,7 @@ public class RequestFactoryImpl implements RequestFactory {
|
|||
/**
|
||||
* ACL For new objects.
|
||||
*/
|
||||
private CannedAccessControlList cannedACL = null;
|
||||
private ObjectCannedACL cannedACL = null;
|
||||
|
||||
/** Content Encoding. */
|
||||
private String contentEncoding;
|
||||
|
@ -754,7 +701,7 @@ public class RequestFactoryImpl implements RequestFactory {
|
|||
* @return the builder
|
||||
*/
|
||||
public RequestFactoryBuilder withCannedACL(
|
||||
final CannedAccessControlList value) {
|
||||
final ObjectCannedACL value) {
|
||||
cannedACL = value;
|
||||
return this;
|
||||
}
|
||||
|
@ -806,11 +753,9 @@ public class RequestFactoryImpl implements RequestFactory {
|
|||
|
||||
/**
|
||||
* Post-creation preparation of AWS request.
|
||||
* @param t request
|
||||
* @param <T> request type.
|
||||
* @return prepared entry.
|
||||
* @param t request builder
|
||||
*/
|
||||
@Retries.OnceRaw
|
||||
<T extends AmazonWebServiceRequest> T prepareRequest(T t);
|
||||
void prepareRequest(SdkRequest.Builder t);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,10 +34,12 @@ import java.util.Objects;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
||||
import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
|
||||
import com.amazonaws.services.s3.model.PartETag;
|
||||
import com.amazonaws.services.s3.model.UploadPartRequest;
|
||||
import com.amazonaws.services.s3.model.UploadPartResult;
|
||||
import software.amazon.awssdk.core.sync.RequestBody;
|
||||
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse;
|
||||
import software.amazon.awssdk.services.s3.model.CompletedPart;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartRequest;
|
||||
import software.amazon.awssdk.services.s3.model.UploadPartResponse;
|
||||
|
||||
import org.apache.hadoop.thirdparty.com.google.common.base.Charsets;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
@ -152,18 +154,18 @@ class S3AMultipartUploader extends AbstractMultipartUploader {
|
|||
Charsets.UTF_8);
|
||||
return context.submit(new CompletableFuture<>(),
|
||||
() -> {
|
||||
UploadPartRequest request = writeOperations.newUploadPartRequest(key,
|
||||
uploadIdString, partNumber, (int) lengthInBytes, inputStream,
|
||||
null, 0L);
|
||||
UploadPartResult result = writeOperations.uploadPart(request, statistics);
|
||||
UploadPartRequest request = writeOperations.newUploadPartRequestBuilder(key,
|
||||
uploadIdString, partNumber, lengthInBytes).build();
|
||||
RequestBody body = RequestBody.fromInputStream(inputStream, lengthInBytes);
|
||||
UploadPartResponse response = writeOperations.uploadPart(request, body, statistics);
|
||||
statistics.partPut(lengthInBytes);
|
||||
String eTag = result.getETag();
|
||||
String eTag = response.eTag();
|
||||
return BBPartHandle.from(
|
||||
ByteBuffer.wrap(
|
||||
buildPartHandlePayload(
|
||||
filePath.toUri().toString(),
|
||||
uploadIdString,
|
||||
result.getPartNumber(),
|
||||
partNumber,
|
||||
eTag,
|
||||
lengthInBytes)));
|
||||
});
|
||||
|
@ -188,7 +190,7 @@ class S3AMultipartUploader extends AbstractMultipartUploader {
|
|||
|
||||
String uploadIdStr = new String(uploadIdBytes, 0, uploadIdBytes.length,
|
||||
Charsets.UTF_8);
|
||||
ArrayList<PartETag> eTags = new ArrayList<>();
|
||||
ArrayList<CompletedPart> eTags = new ArrayList<>();
|
||||
eTags.ensureCapacity(handles.size());
|
||||
long totalLength = 0;
|
||||
// built up to identify duplicates -if the size of this set is
|
||||
|
@ -201,7 +203,8 @@ class S3AMultipartUploader extends AbstractMultipartUploader {
|
|||
payload.validate(uploadIdStr, filePath);
|
||||
ids.add(payload.getPartNumber());
|
||||
totalLength += payload.getLen();
|
||||
eTags.add(new PartETag(handle.getKey(), payload.getEtag()));
|
||||
eTags.add(
|
||||
CompletedPart.builder().partNumber(handle.getKey()).eTag(payload.getEtag()).build());
|
||||
}
|
||||
Preconditions.checkArgument(ids.size() == count,
|
||||
"Duplicate PartHandles");
|
||||
|
@ -210,7 +213,7 @@ class S3AMultipartUploader extends AbstractMultipartUploader {
|
|||
long finalLen = totalLength;
|
||||
return context.submit(new CompletableFuture<>(),
|
||||
trackDurationOfCallable(statistics, MULTIPART_UPLOAD_COMPLETED.getSymbol(), () -> {
|
||||
CompleteMultipartUploadResult result =
|
||||
CompleteMultipartUploadResponse result =
|
||||
writeOperations.commitUpload(
|
||||
key,
|
||||
uploadIdStr,
|
||||
|
@ -218,7 +221,7 @@ class S3AMultipartUploader extends AbstractMultipartUploader {
|
|||
finalLen
|
||||
);
|
||||
|
||||
byte[] eTag = result.getETag().getBytes(Charsets.UTF_8);
|
||||
byte[] eTag = result.eTag().getBytes(Charsets.UTF_8);
|
||||
statistics.uploadCompleted();
|
||||
return (PathHandle) () -> ByteBuffer.wrap(eTag);
|
||||
}));
|
||||
|
|
|
@ -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.classification.VisibleForTesting;
|
|||
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<Boolean> {
|
||||
public class SDKStreamDrainer<TStream extends InputStream & Abortable>
|
||||
implements CallableRaisingIOE<Boolean> {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(
|
||||
SDKStreamDrainer.class);
|
||||
|
@ -58,17 +51,9 @@ public class SDKStreamDrainer implements CallableRaisingIOE<Boolean> {
|
|||
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<Boolean> {
|
|||
/**
|
||||
* 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<Boolean> {
|
|||
* @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 @@ public class SDKStreamDrainer implements CallableRaisingIOE<Boolean> {
|
|||
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 @@ public class SDKStreamDrainer implements CallableRaisingIOE<Boolean> {
|
|||
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 class SDKStreamDrainer implements CallableRaisingIOE<Boolean> {
|
|||
return uri;
|
||||
}
|
||||
|
||||
public Object getRequestObject() {
|
||||
return requestObject;
|
||||
}
|
||||
|
||||
public SdkFilterInputStream getSdkStream() {
|
||||
public TStream getSdkStream() {
|
||||
return sdkStream;
|
||||
}
|
||||
|
||||
|
|
|
@ -47,6 +47,9 @@ public final class V2Migration {
|
|||
private static final LogExactlyOnce WARN_OF_CUSTOM_SIGNER =
|
||||
new LogExactlyOnce(SDK_V2_UPGRADE_LOG);
|
||||
|
||||
private static final LogExactlyOnce WARN_OF_REQUEST_HANDLERS =
|
||||
new LogExactlyOnce(SDK_V2_UPGRADE_LOG);
|
||||
|
||||
private static final LogExactlyOnce WARN_ON_GET_OBJECT_METADATA =
|
||||
new LogExactlyOnce(SDK_V2_UPGRADE_LOG);
|
||||
|
||||
|
@ -87,6 +90,15 @@ public final class V2Migration {
|
|||
+ "once S3A is upgraded to SDK V2");
|
||||
}
|
||||
|
||||
/**
|
||||
* Warns on use of request handlers.
|
||||
*/
|
||||
public static void v1RequestHandlersUsed() {
|
||||
WARN_OF_REQUEST_HANDLERS.warn(
|
||||
"The request handler interface has changed in AWS SDK V2, use exception interceptors "
|
||||
+ "once S3A is upgraded to SDK V2");
|
||||
}
|
||||
|
||||
/**
|
||||
* Warns on use of getObjectMetadata.
|
||||
*/
|
||||
|
|
|
@ -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.impl.prefetch.Validate;
|
|||
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<InputStream, S3Object> s3Objects;
|
||||
|
||||
/**
|
||||
* uri of the object being read.
|
||||
*/
|
||||
|
@ -123,7 +115,6 @@ public class S3ARemoteObject {
|
|||
this.client = client;
|
||||
this.streamStatistics = streamStatistics;
|
||||
this.changeTracker = changeTracker;
|
||||
this.s3Objects = new IdentityHashMap<>();
|
||||
this.uri = this.getPath();
|
||||
}
|
||||
|
||||
|
@ -187,21 +178,23 @@ public class S3ARemoteObject {
|
|||
* @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<GetObjectResponse> 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<GetObjectResponse> object = null;
|
||||
|
||||
try {
|
||||
object = Invoker.once(operation, uri, () -> client.getObject(request));
|
||||
|
@ -212,27 +205,14 @@ public class S3ARemoteObject {
|
|||
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<GetObjectResponse> inputStream, int numRemainingBytes) {
|
||||
SDKStreamDrainer drainer = new SDKStreamDrainer(
|
||||
uri,
|
||||
obj,
|
||||
(S3ObjectInputStream)inputStream,
|
||||
inputStream,
|
||||
false,
|
||||
numRemainingBytes,
|
||||
streamStatistics,
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.hadoop.fs.s3a.prefetch;
|
|||
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.impl.prefetch.Validate;
|
|||
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 @@ public class S3ARemoteObjectReader implements Closeable {
|
|||
return;
|
||||
}
|
||||
|
||||
InputStream inputStream = remoteObject.openForRead(offset, readSize);
|
||||
ResponseInputStream<GetObjectResponse> inputStream =
|
||||
remoteObject.openForRead(offset, readSize);
|
||||
int numRemainingBytes = readSize;
|
||||
byte[] bytes = new byte[READ_BUFFER_SIZE];
|
||||
|
||||
|
|
|
@ -33,11 +33,11 @@ import java.util.Scanner;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import com.amazonaws.services.s3.model.MultipartUpload;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import software.amazon.awssdk.services.s3.model.MultipartUpload;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
@ -694,11 +694,11 @@ public abstract class S3GuardTool extends Configured implements Tool,
|
|||
count++;
|
||||
if (mode == Mode.ABORT || mode == Mode.LIST || verbose) {
|
||||
println(out, "%s%s %s", mode == Mode.ABORT ? "Deleting: " : "",
|
||||
upload.getKey(), upload.getUploadId());
|
||||
upload.key(), upload.uploadId());
|
||||
}
|
||||
if (mode == Mode.ABORT) {
|
||||
writeOperationHelper
|
||||
.abortMultipartUpload(upload.getKey(), upload.getUploadId(),
|
||||
.abortMultipartUpload(upload.key(), upload.uploadId(),
|
||||
true, LOG_EVENT);
|
||||
}
|
||||
}
|
||||
|
@ -726,7 +726,7 @@ public abstract class S3GuardTool extends Configured implements Tool,
|
|||
return true;
|
||||
}
|
||||
Date ageDate = new Date(System.currentTimeMillis() - msec);
|
||||
return ageDate.compareTo(u.getInitiated()) >= 0;
|
||||
return ageDate.compareTo(Date.from(u.initiated())) >= 0;
|
||||
}
|
||||
|
||||
private void processArgs(List<String> args, PrintStream out)
|
||||
|
|
|
@ -0,0 +1,151 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.s3a.select;
|
||||
|
||||
import java.util.Enumeration;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
||||
import org.reactivestreams.Subscriber;
|
||||
import org.reactivestreams.Subscription;
|
||||
|
||||
import software.amazon.awssdk.core.async.SdkPublisher;
|
||||
import software.amazon.awssdk.core.exception.SdkException;
|
||||
|
||||
/**
|
||||
* Implements the {@link Enumeration} interface by subscribing to a
|
||||
* {@link SdkPublisher} instance. The enumeration will buffer a fixed
|
||||
* number of elements and only request new ones from the publisher
|
||||
* when they are consumed. Calls to {@link #hasMoreElements()} and
|
||||
* {@link #nextElement()} may block while waiting for new elements.
|
||||
* @param <T> the type of element.
|
||||
*/
|
||||
public final class BlockingEnumeration<T> implements Enumeration<T> {
|
||||
private static final class Signal<T> {
|
||||
public final T element;
|
||||
public final Throwable error;
|
||||
|
||||
public Signal(T element) {
|
||||
this.element = element;
|
||||
this.error = null;
|
||||
}
|
||||
|
||||
public Signal(Throwable error) {
|
||||
this.element = null;
|
||||
this.error = error;
|
||||
}
|
||||
}
|
||||
|
||||
private final Signal<T> END_SIGNAL = new Signal<>((Throwable)null);
|
||||
private final CompletableFuture<Subscription> subscription = new CompletableFuture<>();
|
||||
private final BlockingQueue<Signal<T>> signalQueue;
|
||||
private final int bufferSize;
|
||||
private Signal<T> current = null;
|
||||
|
||||
/**
|
||||
* Create an enumeration with a fixed buffer size and an
|
||||
* optional injected first element.
|
||||
* @param publisher the publisher feeding the enumeration.
|
||||
* @param bufferSize the buffer size.
|
||||
* @param firstElement (optional) first element the enumeration will return.
|
||||
*/
|
||||
public BlockingEnumeration(SdkPublisher<T> publisher,
|
||||
final int bufferSize,
|
||||
final T firstElement) {
|
||||
this.signalQueue = new LinkedBlockingQueue<>();
|
||||
this.bufferSize = bufferSize;
|
||||
if (firstElement != null) {
|
||||
this.current = new Signal<>(firstElement);
|
||||
}
|
||||
publisher.subscribe(new EnumerationSubscriber());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create an enumeration with a fixed buffer size.
|
||||
* @param publisher the publisher feeding the enumeration.
|
||||
* @param bufferSize the buffer size.
|
||||
*/
|
||||
public BlockingEnumeration(SdkPublisher<T> publisher,
|
||||
final int bufferSize) {
|
||||
this(publisher, bufferSize, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMoreElements() {
|
||||
if (current == null) {
|
||||
try {
|
||||
current = signalQueue.take();
|
||||
} catch (InterruptedException e) {
|
||||
current = new Signal<>(e);
|
||||
subscription.thenAccept(Subscription::cancel);
|
||||
}
|
||||
}
|
||||
if (current.error != null) {
|
||||
if (current.error instanceof SdkException) {
|
||||
throw (SdkException)current.error;
|
||||
} else {
|
||||
throw SdkException.create("Unexpected error", current.error);
|
||||
}
|
||||
}
|
||||
return current != END_SIGNAL;
|
||||
}
|
||||
|
||||
@Override
|
||||
public T nextElement() {
|
||||
if (!hasMoreElements()) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
T element = current.element;
|
||||
current = null;
|
||||
subscription.thenAccept(s -> s.request(1));
|
||||
return element;
|
||||
}
|
||||
|
||||
private final class EnumerationSubscriber implements Subscriber<T> {
|
||||
|
||||
@Override
|
||||
public void onSubscribe(Subscription s) {
|
||||
long request = bufferSize;
|
||||
if (current != null) {
|
||||
request--;
|
||||
}
|
||||
if (request > 0) {
|
||||
s.request(request);
|
||||
}
|
||||
subscription.complete(s);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNext(T t) {
|
||||
signalQueue.add(new Signal<>(t));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onError(Throwable t) {
|
||||
signalQueue.add(new Signal<>(t));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onComplete() {
|
||||
signalQueue.add(END_SIGNAL);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -21,13 +21,6 @@ package org.apache.hadoop.fs.s3a.select;
|
|||
import java.io.IOException;
|
||||
import java.util.Locale;
|
||||
|
||||
import com.amazonaws.services.s3.model.CSVInput;
|
||||
import com.amazonaws.services.s3.model.CSVOutput;
|
||||
import com.amazonaws.services.s3.model.ExpressionType;
|
||||
import com.amazonaws.services.s3.model.InputSerialization;
|
||||
import com.amazonaws.services.s3.model.OutputSerialization;
|
||||
import com.amazonaws.services.s3.model.QuoteFields;
|
||||
import com.amazonaws.services.s3.model.SelectObjectContentRequest;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -42,6 +35,14 @@ import org.apache.hadoop.fs.s3a.S3AReadOpContext;
|
|||
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
|
||||
import org.apache.hadoop.fs.s3a.WriteOperationHelper;
|
||||
|
||||
import software.amazon.awssdk.services.s3.model.CSVInput;
|
||||
import software.amazon.awssdk.services.s3.model.CSVOutput;
|
||||
import software.amazon.awssdk.services.s3.model.ExpressionType;
|
||||
import software.amazon.awssdk.services.s3.model.InputSerialization;
|
||||
import software.amazon.awssdk.services.s3.model.OutputSerialization;
|
||||
import software.amazon.awssdk.services.s3.model.QuoteFields;
|
||||
import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest;
|
||||
|
||||
import static org.apache.hadoop.util.Preconditions.checkNotNull;
|
||||
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
|
||||
import static org.apache.hadoop.fs.s3a.select.SelectConstants.*;
|
||||
|
@ -145,9 +146,9 @@ public class SelectBinding {
|
|||
Preconditions.checkState(isEnabled(),
|
||||
"S3 Select is not enabled for %s", path);
|
||||
|
||||
SelectObjectContentRequest request = operations.newSelectRequest(path);
|
||||
SelectObjectContentRequest.Builder request = operations.newSelectRequestBuilder(path);
|
||||
buildRequest(request, expression, builderOptions);
|
||||
return request;
|
||||
return request.build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -175,14 +176,14 @@ public class SelectBinding {
|
|||
}
|
||||
boolean sqlInErrors = builderOptions.getBoolean(SELECT_ERRORS_INCLUDE_SQL,
|
||||
errorsIncludeSql);
|
||||
String expression = request.getExpression();
|
||||
String expression = request.expression();
|
||||
final String errorText = sqlInErrors ? expression : "Select";
|
||||
if (sqlInErrors) {
|
||||
LOG.info("Issuing SQL request {}", expression);
|
||||
}
|
||||
SelectEventStreamPublisher selectPublisher = operations.select(path, request, errorText);
|
||||
return new SelectInputStream(readContext,
|
||||
objectAttributes,
|
||||
operations.select(path, request, errorText));
|
||||
objectAttributes, selectPublisher);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -197,14 +198,14 @@ public class SelectBinding {
|
|||
* <li>The default values in {@link SelectConstants}</li>
|
||||
* </ol>
|
||||
*
|
||||
* @param request request to build up
|
||||
* @param requestBuilder request to build up
|
||||
* @param expression SQL expression
|
||||
* @param builderOptions the options which came in from the openFile builder.
|
||||
* @throws IllegalArgumentException if an option is somehow invalid.
|
||||
* @throws IOException if an option is somehow invalid.
|
||||
*/
|
||||
void buildRequest(
|
||||
final SelectObjectContentRequest request,
|
||||
final SelectObjectContentRequest.Builder requestBuilder,
|
||||
final String expression,
|
||||
final Configuration builderOptions)
|
||||
throws IllegalArgumentException, IOException {
|
||||
|
@ -213,7 +214,6 @@ public class SelectBinding {
|
|||
|
||||
final Configuration ownerConf = operations.getConf();
|
||||
|
||||
|
||||
String inputFormat = builderOptions.get(SELECT_INPUT_FORMAT,
|
||||
SELECT_FORMAT_CSV).toLowerCase(Locale.ENGLISH);
|
||||
Preconditions.checkArgument(SELECT_FORMAT_CSV.equals(inputFormat),
|
||||
|
@ -224,34 +224,24 @@ public class SelectBinding {
|
|||
Preconditions.checkArgument(SELECT_FORMAT_CSV.equals(outputFormat),
|
||||
"Unsupported output format %s", outputFormat);
|
||||
|
||||
request.setExpressionType(ExpressionType.SQL);
|
||||
request.setExpression(expandBackslashChars(expression));
|
||||
|
||||
InputSerialization inputSerialization = buildCsvInputRequest(ownerConf,
|
||||
builderOptions);
|
||||
String compression = opt(builderOptions,
|
||||
ownerConf,
|
||||
SELECT_INPUT_COMPRESSION,
|
||||
COMPRESSION_OPT_NONE,
|
||||
true).toUpperCase(Locale.ENGLISH);
|
||||
if (isNotEmpty(compression)) {
|
||||
inputSerialization.setCompressionType(compression);
|
||||
}
|
||||
request.setInputSerialization(inputSerialization);
|
||||
|
||||
request.setOutputSerialization(buildCSVOutput(ownerConf, builderOptions));
|
||||
requestBuilder.expressionType(ExpressionType.SQL);
|
||||
requestBuilder.expression(expandBackslashChars(expression));
|
||||
|
||||
requestBuilder.inputSerialization(
|
||||
buildCsvInput(ownerConf, builderOptions));
|
||||
requestBuilder.outputSerialization(
|
||||
buildCSVOutput(ownerConf, builderOptions));
|
||||
}
|
||||
|
||||
/**
|
||||
* Build the CSV input request.
|
||||
* Build the CSV input format for a request.
|
||||
* @param ownerConf FS owner configuration
|
||||
* @param builderOptions options on the specific request
|
||||
* @return the constructed request
|
||||
* @return the input format
|
||||
* @throws IllegalArgumentException argument failure
|
||||
* @throws IOException validation failure
|
||||
*/
|
||||
public InputSerialization buildCsvInputRequest(
|
||||
public InputSerialization buildCsvInput(
|
||||
final Configuration ownerConf,
|
||||
final Configuration builderOptions)
|
||||
throws IllegalArgumentException, IOException {
|
||||
|
@ -283,28 +273,35 @@ public class SelectBinding {
|
|||
CSV_INPUT_QUOTE_ESCAPE_CHARACTER_DEFAULT);
|
||||
|
||||
// CSV input
|
||||
CSVInput csv = new CSVInput();
|
||||
csv.setFieldDelimiter(fieldDelimiter);
|
||||
csv.setRecordDelimiter(recordDelimiter);
|
||||
csv.setComments(commentMarker);
|
||||
csv.setQuoteCharacter(quoteCharacter);
|
||||
CSVInput.Builder csvBuilder = CSVInput.builder()
|
||||
.fieldDelimiter(fieldDelimiter)
|
||||
.recordDelimiter(recordDelimiter)
|
||||
.comments(commentMarker)
|
||||
.quoteCharacter(quoteCharacter);
|
||||
if (StringUtils.isNotEmpty(quoteEscapeCharacter)) {
|
||||
csv.setQuoteEscapeCharacter(quoteEscapeCharacter);
|
||||
csvBuilder.quoteEscapeCharacter(quoteEscapeCharacter);
|
||||
}
|
||||
csv.setFileHeaderInfo(headerInfo);
|
||||
|
||||
InputSerialization inputSerialization = new InputSerialization();
|
||||
inputSerialization.setCsv(csv);
|
||||
|
||||
return inputSerialization;
|
||||
csvBuilder.fileHeaderInfo(headerInfo);
|
||||
|
||||
InputSerialization.Builder inputSerialization =
|
||||
InputSerialization.builder()
|
||||
.csv(csvBuilder.build());
|
||||
String compression = opt(builderOptions,
|
||||
ownerConf,
|
||||
SELECT_INPUT_COMPRESSION,
|
||||
COMPRESSION_OPT_NONE,
|
||||
true).toUpperCase(Locale.ENGLISH);
|
||||
if (isNotEmpty(compression)) {
|
||||
inputSerialization.compressionType(compression);
|
||||
}
|
||||
return inputSerialization.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Build CSV output for a request.
|
||||
* Build CSV output format for a request.
|
||||
* @param ownerConf FS owner configuration
|
||||
* @param builderOptions options on the specific request
|
||||
* @return the constructed request
|
||||
* @return the output format
|
||||
* @throws IllegalArgumentException argument failure
|
||||
* @throws IOException validation failure
|
||||
*/
|
||||
|
@ -333,21 +330,19 @@ public class SelectBinding {
|
|||
CSV_OUTPUT_QUOTE_FIELDS,
|
||||
CSV_OUTPUT_QUOTE_FIELDS_ALWAYS).toUpperCase(Locale.ENGLISH);
|
||||
|
||||
// output is CSV, always
|
||||
OutputSerialization outputSerialization
|
||||
= new OutputSerialization();
|
||||
CSVOutput csvOut = new CSVOutput();
|
||||
csvOut.setQuoteCharacter(quoteCharacter);
|
||||
csvOut.setQuoteFields(
|
||||
QuoteFields.fromValue(quoteFields));
|
||||
csvOut.setFieldDelimiter(fieldDelimiter);
|
||||
csvOut.setRecordDelimiter(recordDelimiter);
|
||||
CSVOutput.Builder csvOutputBuilder = CSVOutput.builder()
|
||||
.quoteCharacter(quoteCharacter)
|
||||
.quoteFields(QuoteFields.fromValue(quoteFields))
|
||||
.fieldDelimiter(fieldDelimiter)
|
||||
.recordDelimiter(recordDelimiter);
|
||||
if (!quoteEscapeCharacter.isEmpty()) {
|
||||
csvOut.setQuoteEscapeCharacter(quoteEscapeCharacter);
|
||||
csvOutputBuilder.quoteEscapeCharacter(quoteEscapeCharacter);
|
||||
}
|
||||
|
||||
outputSerialization.setCsv(csvOut);
|
||||
return outputSerialization;
|
||||
// output is CSV, always
|
||||
return OutputSerialization.builder()
|
||||
.csv(csvOutputBuilder.build())
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -359,18 +354,18 @@ public class SelectBinding {
|
|||
public static String toString(final SelectObjectContentRequest request) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("SelectObjectContentRequest{")
|
||||
.append("bucket name=").append(request.getBucketName())
|
||||
.append("; key=").append(request.getKey())
|
||||
.append("; expressionType=").append(request.getExpressionType())
|
||||
.append("; expression=").append(request.getExpression());
|
||||
InputSerialization input = request.getInputSerialization();
|
||||
.append("bucket name=").append(request.bucket())
|
||||
.append("; key=").append(request.key())
|
||||
.append("; expressionType=").append(request.expressionType())
|
||||
.append("; expression=").append(request.expression());
|
||||
InputSerialization input = request.inputSerialization();
|
||||
if (input != null) {
|
||||
sb.append("; Input")
|
||||
.append(input.toString());
|
||||
} else {
|
||||
sb.append("; Input Serialization: none");
|
||||
}
|
||||
OutputSerialization out = request.getOutputSerialization();
|
||||
OutputSerialization out = request.outputSerialization();
|
||||
if (out != null) {
|
||||
sb.append("; Output")
|
||||
.append(out.toString());
|
||||
|
|
|
@ -0,0 +1,124 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.s3a.select;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.InputStream;
|
||||
import java.io.SequenceInputStream;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
import org.reactivestreams.Subscriber;
|
||||
|
||||
import software.amazon.awssdk.core.async.SdkPublisher;
|
||||
import software.amazon.awssdk.http.AbortableInputStream;
|
||||
import software.amazon.awssdk.services.s3.model.EndEvent;
|
||||
import software.amazon.awssdk.services.s3.model.RecordsEvent;
|
||||
import software.amazon.awssdk.services.s3.model.SelectObjectContentEventStream;
|
||||
import software.amazon.awssdk.services.s3.model.SelectObjectContentResponse;
|
||||
import software.amazon.awssdk.utils.ToString;
|
||||
|
||||
/**
|
||||
* Async publisher of {@link SelectObjectContentEventStream}s returned
|
||||
* from a SelectObjectContent call.
|
||||
*/
|
||||
public final class SelectEventStreamPublisher implements
|
||||
SdkPublisher<SelectObjectContentEventStream> {
|
||||
|
||||
private final CompletableFuture<Void> selectOperationFuture;
|
||||
private final SelectObjectContentResponse response;
|
||||
private final SdkPublisher<SelectObjectContentEventStream> publisher;
|
||||
|
||||
/**
|
||||
* Create the publisher.
|
||||
* @param selectOperationFuture SelectObjectContent future
|
||||
* @param response SelectObjectContent response
|
||||
* @param publisher SelectObjectContentEventStream publisher to wrap
|
||||
*/
|
||||
public SelectEventStreamPublisher(
|
||||
CompletableFuture<Void> selectOperationFuture,
|
||||
SelectObjectContentResponse response,
|
||||
SdkPublisher<SelectObjectContentEventStream> publisher) {
|
||||
this.selectOperationFuture = selectOperationFuture;
|
||||
this.response = response;
|
||||
this.publisher = publisher;
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve an input stream to the subset of the S3 object that matched the select query.
|
||||
* This is equivalent to loading the content of all RecordsEvents into an InputStream.
|
||||
* This will lazily-load the content from S3, minimizing the amount of memory used.
|
||||
* @param onEndEvent callback on the end event
|
||||
* @return the input stream
|
||||
*/
|
||||
public AbortableInputStream toRecordsInputStream(Consumer<EndEvent> onEndEvent) {
|
||||
SdkPublisher<InputStream> recordInputStreams = this.publisher
|
||||
.filter(e -> {
|
||||
if (e instanceof RecordsEvent) {
|
||||
return true;
|
||||
} else if (e instanceof EndEvent) {
|
||||
onEndEvent.accept((EndEvent) e);
|
||||
}
|
||||
return false;
|
||||
})
|
||||
.map(e -> ((RecordsEvent) e).payload().asInputStream());
|
||||
|
||||
// Subscribe to the async publisher using an enumeration that will
|
||||
// buffer a single chunk (RecordsEvent's payload) at a time and
|
||||
// block until it is consumed.
|
||||
// Also inject an empty stream as the first element that
|
||||
// SequenceInputStream will request on construction.
|
||||
BlockingEnumeration enumeration =
|
||||
new BlockingEnumeration(recordInputStreams, 1, EMPTY_STREAM);
|
||||
return AbortableInputStream.create(
|
||||
new SequenceInputStream(enumeration),
|
||||
this::cancel);
|
||||
}
|
||||
|
||||
/**
|
||||
* The response from the SelectObjectContent call.
|
||||
* @return the response object
|
||||
*/
|
||||
public SelectObjectContentResponse response() {
|
||||
return response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void subscribe(Subscriber<? super SelectObjectContentEventStream> subscriber) {
|
||||
publisher.subscribe(subscriber);
|
||||
}
|
||||
|
||||
/**
|
||||
* Cancel the operation.
|
||||
*/
|
||||
public void cancel() {
|
||||
selectOperationFuture.cancel(true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return ToString.builder("SelectObjectContentEventStream")
|
||||
.add("response", response)
|
||||
.add("publisher", publisher)
|
||||
.build();
|
||||
}
|
||||
|
||||
private static final InputStream EMPTY_STREAM =
|
||||
new ByteArrayInputStream(new byte[0]);
|
||||
}
|
|
@ -23,12 +23,8 @@ import java.io.IOException;
|
|||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import com.amazonaws.AbortedException;
|
||||
import com.amazonaws.services.s3.model.SelectObjectContentEvent;
|
||||
import com.amazonaws.services.s3.model.SelectObjectContentEventVisitor;
|
||||
import com.amazonaws.services.s3.model.SelectObjectContentResult;
|
||||
import com.amazonaws.services.s3.model.SelectRecordsInputStream;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -44,6 +40,9 @@ import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
|
|||
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
|
||||
import software.amazon.awssdk.core.exception.AbortedException;
|
||||
import software.amazon.awssdk.http.AbortableInputStream;
|
||||
|
||||
import static org.apache.hadoop.util.Preconditions.checkNotNull;
|
||||
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
|
||||
import static org.apache.hadoop.fs.s3a.Invoker.once;
|
||||
|
@ -93,7 +92,7 @@ public class SelectInputStream extends FSInputStream implements
|
|||
* Abortable response stream.
|
||||
* This is guaranteed to never be null.
|
||||
*/
|
||||
private final SelectRecordsInputStream wrappedStream;
|
||||
private final AbortableInputStream wrappedStream;
|
||||
|
||||
private final String bucket;
|
||||
|
||||
|
@ -112,14 +111,14 @@ public class SelectInputStream extends FSInputStream implements
|
|||
* The read attempt is initiated immediately.
|
||||
* @param readContext read context
|
||||
* @param objectAttributes object attributes from a HEAD request
|
||||
* @param selectResponse response from the already executed call
|
||||
* @param selectPublisher event stream publisher from the already executed call
|
||||
* @throws IOException failure
|
||||
*/
|
||||
@Retries.OnceTranslated
|
||||
public SelectInputStream(
|
||||
final S3AReadOpContext readContext,
|
||||
final S3ObjectAttributes objectAttributes,
|
||||
final SelectObjectContentResult selectResponse) throws IOException {
|
||||
final SelectEventStreamPublisher selectPublisher) throws IOException {
|
||||
Preconditions.checkArgument(isNotEmpty(objectAttributes.getBucket()),
|
||||
"No Bucket");
|
||||
Preconditions.checkArgument(isNotEmpty(objectAttributes.getKey()),
|
||||
|
@ -132,17 +131,17 @@ public class SelectInputStream extends FSInputStream implements
|
|||
this.readahead = readContext.getReadahead();
|
||||
this.streamStatistics = readContext.getS3AStatisticsContext()
|
||||
.newInputStreamStatistics();
|
||||
SelectRecordsInputStream stream = once(
|
||||
|
||||
AbortableInputStream stream = once(
|
||||
"S3 Select",
|
||||
uri,
|
||||
() -> selectResponse.getPayload()
|
||||
.getRecordsInputStream(new SelectObjectContentEventVisitor() {
|
||||
@Override
|
||||
public void visit(final SelectObjectContentEvent.EndEvent event) {
|
||||
LOG.debug("Completed successful S3 select read from {}", uri);
|
||||
completedSuccessfully.set(true);
|
||||
}
|
||||
}));
|
||||
() -> {
|
||||
return selectPublisher.toRecordsInputStream(e -> {
|
||||
LOG.debug("Completed successful S3 select read from {}", uri);
|
||||
completedSuccessfully.set(true);
|
||||
});
|
||||
});
|
||||
|
||||
this.wrappedStream = checkNotNull(stream);
|
||||
// this stream is already opened, so mark as such in the statistics.
|
||||
streamStatistics.streamOpened();
|
||||
|
|
|
@ -0,0 +1,111 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.s3a.select;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionException;
|
||||
|
||||
import software.amazon.awssdk.core.async.SdkPublisher;
|
||||
import software.amazon.awssdk.core.exception.SdkException;
|
||||
import software.amazon.awssdk.services.s3.model.SelectObjectContentEventStream;
|
||||
import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest;
|
||||
import software.amazon.awssdk.services.s3.model.SelectObjectContentResponse;
|
||||
import software.amazon.awssdk.services.s3.model.SelectObjectContentResponseHandler;
|
||||
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.s3a.S3AUtils;
|
||||
|
||||
import static org.apache.hadoop.fs.s3a.WriteOperationHelper.WriteOperationHelperCallbacks;
|
||||
|
||||
/**
|
||||
* Helper for SelectObjectContent queries against an S3 Bucket.
|
||||
*/
|
||||
public final class SelectObjectContentHelper {
|
||||
|
||||
/**
|
||||
* Execute an S3 Select operation.
|
||||
* @param writeOperationHelperCallbacks helper callbacks
|
||||
* @param source source for selection
|
||||
* @param request Select request to issue.
|
||||
* @param action the action for use in exception creation
|
||||
* @return the select response event stream publisher
|
||||
* @throws IOException on failure
|
||||
*/
|
||||
public static SelectEventStreamPublisher select(
|
||||
WriteOperationHelperCallbacks writeOperationHelperCallbacks,
|
||||
Path source,
|
||||
SelectObjectContentRequest request,
|
||||
String action)
|
||||
throws IOException {
|
||||
try {
|
||||
Handler handler = new Handler();
|
||||
CompletableFuture<Void> selectOperationFuture =
|
||||
writeOperationHelperCallbacks.selectObjectContent(request, handler);
|
||||
return handler.eventPublisher(selectOperationFuture).join();
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof CompletionException) {
|
||||
e = e.getCause();
|
||||
}
|
||||
IOException translated;
|
||||
if (e instanceof SdkException) {
|
||||
translated = S3AUtils.translateException(action, source,
|
||||
(SdkException)e);
|
||||
} else {
|
||||
translated = new IOException(e);
|
||||
}
|
||||
throw translated;
|
||||
}
|
||||
}
|
||||
|
||||
private static class Handler implements SelectObjectContentResponseHandler {
|
||||
private volatile CompletableFuture<Pair<SelectObjectContentResponse,
|
||||
SdkPublisher<SelectObjectContentEventStream>>> responseAndPublisherFuture =
|
||||
new CompletableFuture<>();
|
||||
|
||||
private volatile SelectObjectContentResponse response;
|
||||
|
||||
public CompletableFuture<SelectEventStreamPublisher> eventPublisher(
|
||||
CompletableFuture<Void> selectOperationFuture) {
|
||||
return responseAndPublisherFuture.thenApply(p ->
|
||||
new SelectEventStreamPublisher(selectOperationFuture,
|
||||
p.getLeft(), p.getRight()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void responseReceived(SelectObjectContentResponse response) {
|
||||
this.response = response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onEventStream(SdkPublisher<SelectObjectContentEventStream> publisher) {
|
||||
responseAndPublisherFuture.complete(Pair.of(response, publisher));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void exceptionOccurred(Throwable error) {
|
||||
responseAndPublisherFuture.completeExceptionally(error);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void complete() {
|
||||
}
|
||||
}
|
||||
}
|
|
@ -21,23 +21,18 @@ package org.apache.hadoop.fs.s3a.statistics.impl;
|
|||
import java.time.Duration;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.LongConsumer;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import com.amazonaws.Request;
|
||||
import com.amazonaws.Response;
|
||||
import com.amazonaws.metrics.RequestMetricCollector;
|
||||
import com.amazonaws.util.TimingInfo;
|
||||
import software.amazon.awssdk.core.metrics.CoreMetric;
|
||||
import software.amazon.awssdk.http.HttpMetric;
|
||||
import software.amazon.awssdk.http.HttpStatusCode;
|
||||
import software.amazon.awssdk.metrics.MetricCollection;
|
||||
import software.amazon.awssdk.metrics.MetricPublisher;
|
||||
import software.amazon.awssdk.metrics.SdkMetric;
|
||||
|
||||
import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk;
|
||||
|
||||
import static com.amazonaws.util.AWSRequestMetrics.Field.ClientExecuteTime;
|
||||
import static com.amazonaws.util.AWSRequestMetrics.Field.HttpClientRetryCount;
|
||||
import static com.amazonaws.util.AWSRequestMetrics.Field.HttpRequestTime;
|
||||
import static com.amazonaws.util.AWSRequestMetrics.Field.RequestCount;
|
||||
import static com.amazonaws.util.AWSRequestMetrics.Field.RequestMarshallTime;
|
||||
import static com.amazonaws.util.AWSRequestMetrics.Field.RequestSigningTime;
|
||||
import static com.amazonaws.util.AWSRequestMetrics.Field.ResponseProcessingTime;
|
||||
import static com.amazonaws.util.AWSRequestMetrics.Field.ThrottleException;
|
||||
|
||||
/**
|
||||
* Collect statistics from the AWS SDK and forward to an instance of
|
||||
* {@link StatisticsFromAwsSdk} and thence into the S3A statistics.
|
||||
|
@ -45,9 +40,9 @@ import static com.amazonaws.util.AWSRequestMetrics.Field.ThrottleException;
|
|||
* See {@code com.facebook.presto.hive.s3.PrestoS3FileSystemMetricCollector}
|
||||
* for the inspiration for this.
|
||||
* <p>
|
||||
* See {@code com.amazonaws.util.AWSRequestMetrics} for metric names.
|
||||
* See {@code software.amazon.awssdk.core.metrics.CoreMetric} for metric names.
|
||||
*/
|
||||
public class AwsStatisticsCollector extends RequestMetricCollector {
|
||||
public class AwsStatisticsCollector implements MetricPublisher {
|
||||
|
||||
/**
|
||||
* final destination of updates.
|
||||
|
@ -65,65 +60,122 @@ public class AwsStatisticsCollector extends RequestMetricCollector {
|
|||
/**
|
||||
* This is the callback from the AWS SDK where metrics
|
||||
* can be collected.
|
||||
* @param request AWS request
|
||||
* @param response AWS response
|
||||
* @param metricCollection metrics collection
|
||||
*/
|
||||
@Override
|
||||
public void collectMetrics(
|
||||
final Request<?> request,
|
||||
final Response<?> response) {
|
||||
public void publish(MetricCollection metricCollection) {
|
||||
// MetricCollections are nested, so we need to traverse through their
|
||||
// "children" to collect the desired metrics. E.g.:
|
||||
//
|
||||
// ApiCall
|
||||
// ┌─────────────────────────────────────────┐
|
||||
// │ MarshallingDuration=PT0.002808333S │
|
||||
// │ RetryCount=0 │
|
||||
// │ ApiCallSuccessful=true │
|
||||
// │ OperationName=DeleteObject │
|
||||
// │ ApiCallDuration=PT0.079801458S │
|
||||
// │ CredentialsFetchDuration=PT0.000007083S │
|
||||
// │ ServiceId=S3 │
|
||||
// └─────────────────────────────────────────┘
|
||||
// ApiCallAttempt
|
||||
// ┌─────────────────────────────────────────────────────────────────┐
|
||||
// │ SigningDuration=PT0.000319375S │
|
||||
// │ ServiceCallDuration=PT0.078908584S │
|
||||
// │ AwsExtendedRequestId=Kmvb2Sz8NuDgIFJPKzLLBhuHgQGmpAjVYBMrSHDvy= │
|
||||
// │ HttpStatusCode=204 │
|
||||
// │ BackoffDelayDuration=PT0S │
|
||||
// │ AwsRequestId=KR0XZCSX │
|
||||
// └─────────────────────────────────────────────────────────────────┘
|
||||
// HttpClient
|
||||
// ┌─────────────────────────────────┐
|
||||
// │ AvailableConcurrency=1 │
|
||||
// │ LeasedConcurrency=0 │
|
||||
// │ ConcurrencyAcquireDuration=PT0S │
|
||||
// │ PendingConcurrencyAcquires=0 │
|
||||
// │ MaxConcurrency=96 │
|
||||
// │ HttpClientName=Apache │
|
||||
// └─────────────────────────────────┘
|
||||
|
||||
TimingInfo timingInfo = request.getAWSRequestMetrics().getTimingInfo();
|
||||
final long[] throttling = {0};
|
||||
recurseThroughChildren(metricCollection)
|
||||
.collect(Collectors.toList())
|
||||
.forEach(m -> {
|
||||
counter(m, CoreMetric.RETRY_COUNT, retries -> {
|
||||
collector.updateAwsRetryCount(retries);
|
||||
collector.updateAwsRequestCount(retries + 1);
|
||||
});
|
||||
|
||||
counter(timingInfo, HttpClientRetryCount.name(),
|
||||
collector::updateAwsRetryCount);
|
||||
counter(timingInfo, RequestCount.name(),
|
||||
collector::updateAwsRequestCount);
|
||||
counter(timingInfo, ThrottleException.name(),
|
||||
collector::updateAwsThrottleExceptionsCount);
|
||||
counter(m, HttpMetric.HTTP_STATUS_CODE, statusCode -> {
|
||||
if (statusCode == HttpStatusCode.THROTTLING) {
|
||||
throttling[0] += 1;
|
||||
}
|
||||
});
|
||||
|
||||
timing(m, CoreMetric.API_CALL_DURATION,
|
||||
collector::noteAwsClientExecuteTime);
|
||||
|
||||
timing(m, CoreMetric.SERVICE_CALL_DURATION,
|
||||
collector::noteAwsRequestTime);
|
||||
|
||||
timing(m, CoreMetric.MARSHALLING_DURATION,
|
||||
collector::noteRequestMarshallTime);
|
||||
|
||||
timing(m, CoreMetric.SIGNING_DURATION,
|
||||
collector::noteRequestSigningTime);
|
||||
|
||||
timing(m, CoreMetric.UNMARSHALLING_DURATION,
|
||||
collector::noteResponseProcessingTime);
|
||||
});
|
||||
|
||||
collector.updateAwsThrottleExceptionsCount(throttling[0]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
|
||||
timing(timingInfo, ClientExecuteTime.name(),
|
||||
collector::noteAwsClientExecuteTime);
|
||||
timing(timingInfo, HttpRequestTime.name(),
|
||||
collector::noteAwsRequestTime);
|
||||
timing(timingInfo, RequestMarshallTime.name(),
|
||||
collector::noteRequestMarshallTime);
|
||||
timing(timingInfo, RequestSigningTime.name(),
|
||||
collector::noteRequestSigningTime);
|
||||
timing(timingInfo, ResponseProcessingTime.name(),
|
||||
collector::noteResponseProcessingTime);
|
||||
}
|
||||
|
||||
/**
|
||||
* Process a timing.
|
||||
* @param timingInfo timing info
|
||||
* @param subMeasurementName sub measurement
|
||||
* @param collection metric collection
|
||||
* @param metric metric
|
||||
* @param durationConsumer consumer
|
||||
*/
|
||||
private void timing(
|
||||
TimingInfo timingInfo,
|
||||
String subMeasurementName,
|
||||
MetricCollection collection,
|
||||
SdkMetric<Duration> metric,
|
||||
Consumer<Duration> durationConsumer) {
|
||||
TimingInfo t1 = timingInfo.getSubMeasurement(subMeasurementName);
|
||||
if (t1 != null && t1.getTimeTakenMillisIfKnown() != null) {
|
||||
durationConsumer.accept(Duration.ofMillis(
|
||||
t1.getTimeTakenMillisIfKnown().longValue()));
|
||||
}
|
||||
collection
|
||||
.metricValues(metric)
|
||||
.forEach(v -> durationConsumer.accept(v));
|
||||
}
|
||||
|
||||
/**
|
||||
* Process a counter.
|
||||
* @param timingInfo timing info
|
||||
* @param subMeasurementName sub measurement
|
||||
* @param collection metric collection
|
||||
* @param metric metric
|
||||
* @param consumer consumer
|
||||
*/
|
||||
private void counter(
|
||||
TimingInfo timingInfo,
|
||||
String subMeasurementName,
|
||||
MetricCollection collection,
|
||||
SdkMetric<Integer> metric,
|
||||
LongConsumer consumer) {
|
||||
Number n = timingInfo.getCounter(subMeasurementName);
|
||||
if (n != null) {
|
||||
consumer.accept(n.longValue());
|
||||
}
|
||||
collection
|
||||
.metricValues(metric)
|
||||
.forEach(v -> consumer.accept(v.longValue()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Metric collections can be nested. Exposes a stream of the given
|
||||
* collection and its nested children.
|
||||
* @param metrics initial collection
|
||||
* @return a stream of all nested metric collections
|
||||
*/
|
||||
private static Stream<MetricCollection> recurseThroughChildren(
|
||||
MetricCollection metrics) {
|
||||
return Stream.concat(
|
||||
Stream.of(metrics),
|
||||
metrics.children().stream()
|
||||
.flatMap(c -> recurseThroughChildren(c)));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,10 +32,8 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import com.amazonaws.AmazonClientException;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.fs.s3a.MultiObjectDeleteException;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -61,6 +59,9 @@ import org.apache.hadoop.fs.shell.CommandFormat;
|
|||
import org.apache.hadoop.util.DurationInfo;
|
||||
import org.apache.hadoop.util.ExitUtil;
|
||||
|
||||
import software.amazon.awssdk.awscore.exception.AwsServiceException;
|
||||
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
|
||||
|
||||
import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH;
|
||||
import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE;
|
||||
import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT;
|
||||
|
@ -784,7 +785,7 @@ public final class MarkerTool extends S3GuardTool {
|
|||
private MarkerPurgeSummary purgeMarkers(
|
||||
final DirMarkerTracker tracker,
|
||||
final int deletePageSize)
|
||||
throws MultiObjectDeleteException, AmazonClientException, IOException {
|
||||
throws MultiObjectDeleteException, AwsServiceException, IOException {
|
||||
|
||||
MarkerPurgeSummary summary = new MarkerPurgeSummary();
|
||||
// we get a map of surplus markers to delete.
|
||||
|
@ -792,13 +793,13 @@ public final class MarkerTool extends S3GuardTool {
|
|||
= tracker.getSurplusMarkers();
|
||||
int size = markers.size();
|
||||
// build a list from the strings in the map
|
||||
List<DeleteObjectsRequest.KeyVersion> collect =
|
||||
List<ObjectIdentifier> collect =
|
||||
markers.values().stream()
|
||||
.map(p -> new DeleteObjectsRequest.KeyVersion(p.getKey()))
|
||||
.map(p -> ObjectIdentifier.builder().key(p.getKey()).build())
|
||||
.collect(Collectors.toList());
|
||||
// build an array list for ease of creating the lists of
|
||||
// keys in each page through the subList() method.
|
||||
List<DeleteObjectsRequest.KeyVersion> markerKeys =
|
||||
List<ObjectIdentifier> markerKeys =
|
||||
new ArrayList<>(collect);
|
||||
|
||||
// now randomize. Why so? if the list spans multiple S3 partitions,
|
||||
|
@ -819,7 +820,7 @@ public final class MarkerTool extends S3GuardTool {
|
|||
while (start < size) {
|
||||
// end is one past the end of the page
|
||||
int end = Math.min(start + deletePageSize, size);
|
||||
List<DeleteObjectsRequest.KeyVersion> page = markerKeys.subList(start,
|
||||
List<ObjectIdentifier> page = markerKeys.subList(start,
|
||||
end);
|
||||
once("Remove S3 Keys",
|
||||
tracker.getBasePath().toString(), () ->
|
||||
|
|
|
@ -21,16 +21,16 @@ package org.apache.hadoop.fs.s3a.tools;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import com.amazonaws.AmazonClientException;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
|
||||
|
||||
import org.apache.hadoop.fs.InvalidRequestException;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.s3a.MultiObjectDeleteException;
|
||||
import org.apache.hadoop.fs.s3a.Retries;
|
||||
import org.apache.hadoop.fs.s3a.S3AFileStatus;
|
||||
|
||||
import software.amazon.awssdk.awscore.exception.AwsServiceException;
|
||||
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
|
||||
|
||||
/**
|
||||
* Operations which must be offered by the store for {@link MarkerTool}.
|
||||
* These are a proper subset of {@code OperationCallbacks}; this interface
|
||||
|
@ -62,14 +62,14 @@ public interface MarkerToolOperations {
|
|||
* a mistaken attempt to delete the root directory.
|
||||
* @throws MultiObjectDeleteException one or more of the keys could not
|
||||
* be deleted in a multiple object delete operation.
|
||||
* @throws AmazonClientException amazon-layer failure.
|
||||
* @throws AwsServiceException amazon-layer failure.
|
||||
* @throws IOException other IO Exception.
|
||||
*/
|
||||
@Retries.RetryMixed
|
||||
void removeKeys(
|
||||
List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
||||
List<ObjectIdentifier> keysToDelete,
|
||||
boolean deleteFakeDir)
|
||||
throws MultiObjectDeleteException, AmazonClientException,
|
||||
throws MultiObjectDeleteException, AwsServiceException,
|
||||
IOException;
|
||||
|
||||
}
|
||||
|
|
|
@ -21,15 +21,15 @@ package org.apache.hadoop.fs.s3a.tools;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import com.amazonaws.AmazonClientException;
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
||||
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.s3a.MultiObjectDeleteException;
|
||||
import org.apache.hadoop.fs.s3a.S3AFileStatus;
|
||||
import org.apache.hadoop.fs.s3a.impl.OperationCallbacks;
|
||||
|
||||
import software.amazon.awssdk.awscore.exception.AwsServiceException;
|
||||
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
|
||||
|
||||
/**
|
||||
* Implement the marker tool operations by forwarding to the
|
||||
* {@link OperationCallbacks} instance provided in the constructor.
|
||||
|
@ -55,9 +55,9 @@ public class MarkerToolOperationsImpl implements MarkerToolOperations {
|
|||
|
||||
@Override
|
||||
public void removeKeys(
|
||||
final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
||||
final List<ObjectIdentifier> keysToDelete,
|
||||
final boolean deleteFakeDir)
|
||||
throws MultiObjectDeleteException, AmazonClientException, IOException {
|
||||
throws MultiObjectDeleteException, AwsServiceException, IOException {
|
||||
operationCallbacks.removeKeys(keysToDelete, deleteFakeDir
|
||||
);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,340 @@
|
|||
<!---
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License. See accompanying LICENSE file.
|
||||
-->
|
||||
|
||||
# Upgrade S3A to AWS SDK V2: Changelog
|
||||
|
||||
Note: This document is not meant to be committed as part of the final merge, and instead just serves
|
||||
as a guide to help with reviewing the PR.
|
||||
|
||||
This document tracks changes to S3A during the upgrade to AWS SDK V2. Once the upgrade
|
||||
is complete, some of its content will be added to the existing document
|
||||
[Upcoming upgrade to AWS Java SDK V2](./aws_sdk_upgrade.html).
|
||||
|
||||
This work is tracked in [HADOOP-18073](https://issues.apache.org/jira/browse/HADOOP-18073).
|
||||
|
||||
## Contents
|
||||
|
||||
* [Client Configuration](#client-configuration)
|
||||
* [Endpoint and region configuration](#endpoint-and-region-configuration)
|
||||
* [List Object](#list-object)
|
||||
* [EncryptionSecretOperations](#encryptionsecretoperations)
|
||||
* [GetObjectMetadata](#getobjectmetadata)
|
||||
* [PutObject](#putobject)
|
||||
* [CopyObject](#copyobject)
|
||||
* [MultipartUpload](#multipartupload)
|
||||
* [GetObject](#getObject)
|
||||
* [DeleteObject](#deleteobject)
|
||||
* [Select](#select)
|
||||
* [CredentialsProvider](#credentialsprovider)
|
||||
* [Auditing](#auditing)
|
||||
* [Metric Collection](#metric-collection)
|
||||
* [Exception Handling](#exception-handling)
|
||||
* [Failure Injection](#failure-injection)
|
||||
|
||||
### Client Configuration:
|
||||
|
||||
* We now have two clients, a sync S3 Client and an async S3 Client. The async s3 client is required
|
||||
as the select operation is currently only supported on the async client. Once we are confident in
|
||||
the current set of changes, we will also be exploring moving other operations to the async client
|
||||
as this could provide potential performance benefits. However those changes are not in the scope
|
||||
of this PR, and will be done separately.
|
||||
* The [createAwsConf](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java#L1190)
|
||||
method is now split into:
|
||||
```
|
||||
createClientConfigBuilder // sets request timeout, user agent*
|
||||
createHttpClientBuilder* // sets max connections, connection timeout, socket timeout
|
||||
createProxyConfigurationBuilder // sets proxy config, defined in table below
|
||||
```
|
||||
|
||||
The table below lists the configurations S3A was using and what they now map to.
|
||||
|
||||
|SDK V1 |SDK V2 |
|
||||
|--- |--- |
|
||||
|setMaxConnections |httpClientBuilder.maxConnections |
|
||||
|setProtocol |The protocol is now HTTPS by default, and can only be modified by setting an HTTP endpoint on the client builder. This is done when setting the endpoint in getS3Endpoint() |
|
||||
|setMaxErrorRetry |createRetryPolicyBuilder |
|
||||
|setConnectionTimeout |httpClientBuilder.connectionTimeout |
|
||||
|setSocketTimeout |httpClientBuilder.socketTimeout |
|
||||
|setRequestTimeout |overrideConfigBuilder.apiCallAttemptTimeout |
|
||||
|setSocketBufferSizeHints |Not supported |
|
||||
|setSignerOverride |Not done yet |
|
||||
|setProxyHost |proxyConfigBuilder.endpoint |
|
||||
|setProxyPort |set when setting proxy host with .endpoint |
|
||||
|setProxyUsername |proxyConfigBuilder.username |
|
||||
|setProxyPassword |proxyConfigBuilder.password |
|
||||
|setProxyDomain |proxyConfigBuilder.ntlmDomain, not supported in async client |
|
||||
|setProxyWorkstation |proxyConfigBuilder.ntlmWorkstation, not supported in async client |
|
||||
|setUserAgentPrefix |overrideConfigBuilder.putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX, userAgent); |
|
||||
|addHeader |overrideConfigBuilder.putHeader |
|
||||
|setUseThrottleRetries |not supported |
|
||||
|
||||
### Endpoint and region configuration
|
||||
|
||||
Previously, if no endpoint and region was configured, fall back to using us-east-1. Set
|
||||
withForceGlobalBucketAccessEnabled(true) which will allow access to buckets not in this region too.
|
||||
Since the SDK V2 no longer supports cross region access, we need to set the region and endpoint of
|
||||
the bucket. The behaviour has now been changed to:
|
||||
|
||||
* If no endpoint is specified, use s3.amazonaws.com.
|
||||
* When setting the endpoint, also set the protocol (HTTP or HTTPS)
|
||||
* When setting the region, first initiate a default S3 Client with region eu-west-2. Call headBucket
|
||||
using this client. If the bucket is also in eu-west-2, then this will return a successful
|
||||
response. Otherwise it will throw an error with status code 301 permanently moved. This error
|
||||
contains the region of the bucket in its header, which we can then use to configure the client.
|
||||
|
||||
### List Object:
|
||||
|
||||
There is no way to paginate the listObject V1 result, we are
|
||||
doing [this](https://github.com/ahmarsuhail/hadoop/pull/23/files#diff-4050f95b7e3912145415b6e2f9cd3b0760fcf2ce96bf0980c6c30a6edad2d0fbR2745)
|
||||
instead. We are trying to get pagination to listObject V1 in the SDK, but will have to use this
|
||||
workaround for now.
|
||||
|
||||
### EncryptionSecretOperations:
|
||||
|
||||
Two new methods have been added, `getSSECustomerKey` and `getSSEAwsKMSKey`. Previously SDK V1 had
|
||||
specific classes for these keys `SSECustomerKey` and `SSEAwsKeyManagementParams` . There are no such
|
||||
classes with V2, and things need to be set manually. For this reason, we simply just return keys as
|
||||
strings now. And will have to calculate and set md5’s ourselves when building the request.
|
||||
|
||||
|
||||
### GetObjectMetadata:
|
||||
|
||||
* `RequestFactory.newGetObjectMetadataRequest` is now `RequestFactory.newHeadObjectRequestBuilder`.
|
||||
* In `HeaderProcessing.retrieveHeaders()`, called by `getXAttrs()`,
|
||||
removed `maybeSetHeader(headers, XA_CONTENT_MD5, md.getContentMD5())` as S3 doesn’t ever actually
|
||||
return an md5 header, regardless of whether you set it during a putObject. It does return
|
||||
an `etag` which may or may not be an md5 depending on certain conditions. `getContentMD5()` is
|
||||
always empty, there does not seem to be a need to set this header.
|
||||
* `RequestFactoryImpl.setOptionalGetObjectMetadataParameters` : Method has been removed and this
|
||||
logic has been moved to `RequestFactoryImpl.newHeadObjectRequestBuilder()`
|
||||
* `RequestFactoryImpl.generateSSECustomerKey()` has been removed, and instead
|
||||
call `EncryptionSecretOperations.createSSECustomerKey` directly in `newHeadObjectRequestBuilder()`
|
||||
|
||||
|
||||
|
||||
### PutObject
|
||||
|
||||
* Previously, when creating the `putObjectRequest`, you would also give it the data to be uploaded.
|
||||
So it would be of the form `PutObjectRequest(bucket, key, file/inputstream)`, this is no longer
|
||||
the case. Instead, the data now needs to be passed in while making the `s3Client.putObject()`
|
||||
call. For this reason, the data is now part of
|
||||
the `S3AFileSystem.putObject(putObjectRequest, file, listener)`
|
||||
and `S3AFileSystem.putObjectDirect(putObjectRequest, putOptions, uploadData, isFile)`.
|
||||
* `S3ADataBlocks`: Need to make this class public as it’s now used to pass in data
|
||||
to `putObjectDirect()`, sometimes from outside the package (`MagicCommitTracker`
|
||||
, `ITestS3AMiscOperations`).
|
||||
* `ProgressableProgressListener`: You can no longer pass in the `Upload` while initialising the
|
||||
listener
|
||||
as `ProgressableProgressListener listener = new ProgressableProgressListener(this, key, upload, progress);`
|
||||
The upload is now only available after initialising the listener, since the listener needs to be
|
||||
initialised during creation of the Transfer Manager upload. Previously, you could create the
|
||||
listener after the starting the TM upload, and attach it.
|
||||
* The `Upload` is now passed into the progress listener later,
|
||||
in `listener.uploadCompleted(uploadInfo.getFileUpload());`.
|
||||
* `UploadInfo`: Previously, since the data to be uploaded was part of `putObjectRequest`, the
|
||||
transfer manager only returned a single `Upload` type, which could be used to track the upload.
|
||||
Now, depending on the upload type (eg: File or InputStream), it returns different types. This
|
||||
class has been updated to return FileUpload info, as it’s only ever used for file uploads
|
||||
currently. It can be extended to store different transfer types in the future.
|
||||
* `WriteOperationHelper.createPutObjectRequest() `: Previously the data to be uploaded was part
|
||||
of `PutObjectRequest`, and so we required two methods to create the request. One for input streams
|
||||
and one for files. Since the data to be uploaded is no longer part of the request, but instead an
|
||||
argument in `putObject` , we only need one method now.
|
||||
* `WriteOperationHelper.newObjectMetadata()`: This method has been removed, as standard metadata,
|
||||
instead of being part of the `ObjectMetadata`, is now just added while building the request, for
|
||||
example `putObjectRequestBuilder.serverSideEncryption().`
|
||||
* `RequestFactory`: Similar to WriteOperationHelper, there is now a single putObjectRequest,
|
||||
and `newObjectMetadata` has been removed. Instead, all standard metadata is now set in the new
|
||||
method `buildPutObjectRequest`.
|
||||
* `RequestFactoryImpl.newObjectMetadata()`: Previously, object metadata was created
|
||||
using `newObjectMetadata()` and passed into the `newPutObjectRequest()` call. This method has been
|
||||
removed, as standard metadata, instead of being part of the `ObjectMetadata`, is now just added
|
||||
while building the request, in `putObjectRequestBuilder.serverSideEncryption().` Content length
|
||||
and content encoding set in this method is now set in `buildPutObjectRequest()` , and SSE is set
|
||||
in `putEncryptionParameters()`.
|
||||
* `RequestFactoryImpl.maybeSetMetadata()` : was a generic method to set user metadata on object
|
||||
metadata. user metadata now gets set on the request builder, so method has been removed.
|
||||
* `RequestFactoryImpl.setOptionalPutRequestParameters()` : Method has been removed, and this logic
|
||||
has been moved to `putEncryptionParameters()` .
|
||||
|
||||
### CopyObject
|
||||
|
||||
* `RequestFactoryImpl.buildPutObjectRequest` : Destination metadata is no longer built
|
||||
using `newObjectMetadata()` and instead set on the request builder. The logic has a couple of
|
||||
differences:
|
||||
* content encoding is set in `buildCopyObjectRequest`,
|
||||
the `if (contentEncoding != null && !isDirectoryMarker)` can just
|
||||
be `if (contentEncoding != null)` for copy, as for this `isDirectoryMarker` was always false.
|
||||
* contentLength is not set, as this is a system defined header, and copied over automatically by
|
||||
S3 during copy.
|
||||
* `HeaderProcessing.cloneObjectMetadata`: This was previously also setting a lot of system defined
|
||||
metadata, eg: `setHttpExpiresDate` and `setLastModified`. These have been removed as they are set
|
||||
by S3 during the copy. Have tested, and can see they are set automatically regardless of the
|
||||
metadataDirective (copy or replace).
|
||||
* `RequestFactoryImpl. copyEncryptionParameters()` : Due to the changes
|
||||
in `EncryptionSecretOperations`, source and destination encryption params have to be set manually.
|
||||
|
||||
### MultipartUpload
|
||||
|
||||
* `RequestFactoryImpl.newObjectMetdata()` : Metadata is now set on the request builder. For MPU, only
|
||||
content encoding needs to be set, as per per previous behaviour. Encryption params are set
|
||||
in ` multipartUploadEncryptionParameters`.
|
||||
|
||||
### GetObject
|
||||
|
||||
* Previously, GetObject returned a `S3Object` response which exposed its content in a
|
||||
`S3ObjectInputStream` through the `getObjectContent()` method. In SDK v2, the response is
|
||||
directly a `ResponseInputStream<GetObjectResponse>` with the content, while the
|
||||
`GetObjectResponse` instance can be retrieved by calling `response()` on it.
|
||||
* The above change simplifies managing the lifetime of the response input stream. In v1,
|
||||
`S3AInputStream` had to keep a reference to the `S3Object` while holding the wrapped
|
||||
`S3ObjectInputStream`. When upgraded to SDK v2, it can simply wrap the new
|
||||
`ResponseInputStream<GetObjectResponse>`, which handles lifetime correctly. Same applies
|
||||
to `SDKStreamDrainer`. Furthermore, the map in `S3ARemoteObject` associating input streams and
|
||||
`S3Object` instances is no longer needed.
|
||||
* The range header on a `GetObject` request is now specified as a string, rather than a
|
||||
`start`-`end` pair. `S3AUtils.formatRange` was introduced to format it.
|
||||
|
||||
### DeleteObject
|
||||
|
||||
In SDK v1, bulk delete would throw a `com.amazonaws.services.s3.model.MultiObjectDeleteException`
|
||||
in case of partial failure. In v2, instead, it returns a `DeleteObjectsResponse` containing a
|
||||
list of errors. A new `MultiObjectDeleteException` class was introduced in
|
||||
`org.apache.hadoop.fs.s3a` and is thrown when appropriate to reproduce the previous behaviour.
|
||||
* `MultiObjectDeleteSupport.translateDeleteException` was moved into `MultiObjectDeleteException`.
|
||||
* `ObjectIdentifier` replaces DeleteObjectsRequest.KeyVersion.
|
||||
|
||||
### Select
|
||||
|
||||
In SDK v2, Handling of select requests has changes significantly since SelectObjectContent is
|
||||
only supported on the new async S3 client. In previous versions, the response to a
|
||||
SelectObjectContent request exposed the results in a `SelectRecordsInputStream`, which S3A
|
||||
could wrap in `SelectInputStream`. In v2, instead, the response needs to be handled by an object
|
||||
implementing `SelectObjectContentResponseHandler`, which can receive an async publisher of
|
||||
the "events" returned by the service (`SdkPublisher<SelectObjectContentEventStream>`).
|
||||
|
||||
In order to adapt the new API in S3A, three new classes have been introduced in
|
||||
`org.apache.hadoop.fs.s3a.select`:
|
||||
|
||||
* `SelectObjectContentHelper`: wraps the `selectObjectContent()` call, provides a custom
|
||||
response handler to receive the response, and exposes a `SelectEventStreamPublisher`.
|
||||
* `SelectEventStreamPublisher`: a publisher of select event stream events, which handles the
|
||||
future returned by the select call and wraps the original publisher. This class provides
|
||||
a `toRecordsInputStream()` method which returns an input stream containing the results,
|
||||
reproducing the behaviour of the old `SelectRecordsInputStream`.
|
||||
* `BlockingEnumeration`: an adapter which lazily requests new elements from the publisher and
|
||||
exposes them through an `Enumeration` interface. Used in
|
||||
`SelectEventStreamPublisher.toRecordsInputStream()` to adapt the event publisher into
|
||||
an enumeration of input streams, eventually passed to a `SequenceInputStream`.
|
||||
Note that the "lazy" behaviour means that new elements are requested only on `read()` calls on
|
||||
the input stream.
|
||||
|
||||
|
||||
|
||||
### CredentialsProvider
|
||||
|
||||
* All credential provider classes implemented in Hadoop now implement V2's `AwsCredentialProvider`
|
||||
* New adapter class `org.apache.hadoop.fs.s3a.adapter.V1ToV2AwsCredentialProviderAdapter` has been
|
||||
added. This converts SDK V1 credential providers to SDK V2’s which
|
||||
implement `AwsCredentialsProvider`.
|
||||
* `AWSCredentialProviderList` also implements `AwsCredentialProvider`. But keeps existing
|
||||
constructors and add methods for V1 credential providers, and wraps V1 cred providers in the
|
||||
adapter here. This means that custom binding classes in delegation tokens, as well as any custom
|
||||
credential providers will continue to work.
|
||||
* Added a new `getCredentials()` method in `AWSCredentialProviderList`, which ensured that custom
|
||||
binding classes which are calling `AWSCredentialProviderList.getCredentials()`, continue to work.
|
||||
* The following values `fs.s3a.aws.credentials.provider` are mapped:
|
||||
as `com.amazonaws.auth.EnvironmentVariableCredentialsProvider`, then map it to V2’s
|
||||
|
||||
|`fs.s3a.aws.credentials.provider` value |Mapped to |
|
||||
|--- |--- |
|
||||
|`com.amazonaws.auth.EnvironmentVariableCredentialsProvider` |`software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider` |
|
||||
|`com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper` |`org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` |
|
||||
|`com.amazonaws.auth.`InstanceProfileCredentialsProvider`` |`org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` |
|
||||
|
||||
|
||||
### Auditing
|
||||
|
||||
The SDK v2 offers a new `ExecutionInterceptor`
|
||||
[interface](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/core/interceptor/ExecutionInterceptor.html)
|
||||
which broadly replaces the `RequestHandler2` abstract class from v1.
|
||||
Switching to the new mechanism in S3A brings:
|
||||
|
||||
* Simplification in `AWSAuditEventCallbacks` (and implementors) which can now extend
|
||||
`ExecutionInterceptor`
|
||||
* "Registering" a Span with a request has moved from `requestCreated` to `beforeExecution`
|
||||
(where an `ExecutionAttributes` instance is first available)
|
||||
* The ReferrerHeader is built and added to the http request in `modifyHttpRequest`,
|
||||
rather than in `beforeExecution`, where no http request is yet available
|
||||
* Dynamic loading of interceptors has been implemented to reproduce previous behaviour
|
||||
with `RequestHandler2`s. The AWS SDK v2 offers an alternative mechanism, described
|
||||
[here](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/core/interceptor/ExecutionInterceptor.html)
|
||||
under "Interceptor Registration", which could make it redundant.
|
||||
|
||||
In the Transfer Manager, `TransferListener` replaces `TransferStateChangeListener`. S3A code
|
||||
has been updated and `AuditManagerS3A` implementations now provide an instance of the former to
|
||||
switch to the active span, but registration of the new listeners is currently commented out because
|
||||
it causes an incompatibility issue with the internal logger, resulting in `NoSuchMethodError`s,
|
||||
at least in the current TransferManager Preview release.
|
||||
|
||||
|
||||
### Metric Collection
|
||||
|
||||
`AwsStatisticsCollector` has been updated to implement the new `MetricPublisher` interface
|
||||
and collect the metrics from a `MetricCollection` object.
|
||||
The following table maps SDK v2 metrics to their equivalent in v1:
|
||||
|
||||
| v2 Metrics | com.amazonaws.util.AWSRequestMetrics.Field | Comment |
|
||||
|-------------------------------------------------------------|---------------------------------------------|--------------------------------|
|
||||
| CoreMetric.RETRY_COUNT | HttpClientRetryCount | |
|
||||
| CoreMetric.RETRY_COUNT | RequestCount | always HttpClientRetryCount+1 |
|
||||
| HttpMetric.HTTP_STATUS_CODE with HttpStatusCode.THROTTLING | ThrottleException | to be confirmed |
|
||||
| CoreMetric.API_CALL_DURATION | ClientExecuteTime | |
|
||||
| CoreMetric.SERVICE_CALL_DURATION | HttpRequestTime | |
|
||||
| CoreMetric.MARSHALLING_DURATION | RequestMarshallTime | |
|
||||
| CoreMetric.SIGNING_DURATION | RequestSigningTime | |
|
||||
| CoreMetric.UNMARSHALLING_DURATION | ResponseProcessingTime | to be confirmed |
|
||||
|
||||
Note that none of the timing metrics (`*_DURATION`) are currently collected in S3A.
|
||||
|
||||
### Exception Handling
|
||||
|
||||
The code to handle exceptions thrown by the SDK has been updated to reflect the changes in v2:
|
||||
|
||||
* `com.amazonaws.SdkBaseException` and `com.amazonaws.AmazonClientException` changes:
|
||||
* These classes have combined and replaced with
|
||||
`software.amazon.awssdk.core.exception.SdkException`.
|
||||
* `com.amazonaws.SdkClientException` changes:
|
||||
* This class has been replaced with `software.amazon.awssdk.core.exception.SdkClientException`.
|
||||
* This class now extends `software.amazon.awssdk.core.exception.SdkException`.
|
||||
* `com.amazonaws.AmazonServiceException` changes:
|
||||
* This class has been replaced with
|
||||
`software.amazon.awssdk.awscore.exception.AwsServiceException`.
|
||||
* This class now extends `software.amazon.awssdk.core.exception.SdkServiceException`,
|
||||
a new exception type that extends `software.amazon.awssdk.core.exception.SdkException`.
|
||||
|
||||
See also the
|
||||
[SDK changelog](https://github.com/aws/aws-sdk-java-v2/blob/master/docs/LaunchChangelog.md#3-exception-changes).
|
||||
|
||||
|
||||
### Failure Injection
|
||||
|
||||
While using the SDK v1, failure injection was implemented in `InconsistentAmazonS3CClient`,
|
||||
which extended the S3 client. In SDK v2, reproducing this approach would not be straightforward,
|
||||
since the default S3 client is an internal final class. Instead, the same fault injection strategy
|
||||
is now performed by a `FailureInjectionInterceptor` (see
|
||||
[ExecutionInterceptor](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/core/interceptor/ExecutionInterceptor.html))
|
||||
registered on the default client by `InconsistentS3CClientFactory`.
|
||||
`InconsistentAmazonS3CClient` has been removed. No changes to the user configuration are required.
|
||||
|
|
@ -1004,20 +1004,18 @@ using an absolute XInclude reference to it.
|
|||
**Warning do not enable any type of failure injection in production. The
|
||||
following settings are for testing only.**
|
||||
|
||||
One of the challenges with S3A integration tests was the fact that S3 was an
|
||||
eventually-consistent storage system. To simulate inconsistencies more
|
||||
frequently than they would normally surface, S3A supports a shim layer on top of the `AmazonS3Client`
|
||||
class which artificially delays certain paths from appearing in listings.
|
||||
This is implemented in the class `InconsistentAmazonS3Client`.
|
||||
S3A provides an "Inconsistent S3 Client Factory" that can be used to
|
||||
simulate throttling by injecting random failures on S3 client requests.
|
||||
|
||||
Now that S3 is consistent, injecting inconsistency is no longer needed
|
||||
during testing.
|
||||
However, it is stil useful to use the other feature of the client:
|
||||
throttling simulation.
|
||||
|
||||
## Simulating List Inconsistencies
|
||||
**Note**
|
||||
|
||||
### Enabling the InconsistentAmazonS3CClient
|
||||
In previous releases, this factory could also be used to simulate
|
||||
inconsistencies during testing of S3Guard. Now that S3 is consistent,
|
||||
injecting inconsistency is no longer needed during testing.
|
||||
|
||||
|
||||
### Enabling the InconsistentS3CClientFactory
|
||||
|
||||
|
||||
To enable the fault-injecting client via configuration, switch the
|
||||
|
@ -1047,7 +1045,7 @@ These exceptions are returned to S3; they do not test the
|
|||
AWS SDK retry logic.
|
||||
|
||||
|
||||
### Using the `InconsistentAmazonS3CClient` in downstream integration tests
|
||||
### Using the `InconsistentS3CClientFactory` in downstream integration tests
|
||||
|
||||
The inconsistent client is shipped in the `hadoop-aws` JAR, so it can
|
||||
be used in integration tests.
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue