+ AUDIT_SPAN_EXECUTION_ATTRIBUTE =
+ new ExecutionAttribute<>(
+ "org.apache.hadoop.fs.s3a.audit.AuditSpanS3A");
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractAWSCredentialProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractAWSCredentialProvider.java
index 1815285738b..4754427a4b1 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractAWSCredentialProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractAWSCredentialProvider.java
@@ -21,7 +21,7 @@ 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() {
- }
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractSessionCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractSessionCredentialsProvider.java
index 5b1829e0961..365885cc70a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractSessionCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AbstractSessionCredentialsProvider.java
@@ -23,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;
}
}
+
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java
index 1e2ac16075a..eb32ed8afc8 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java
@@ -26,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
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java
index ca9c518d300..f505cfcab5d 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/IAMInstanceCredentialsProvider.java
@@ -21,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;
*
* 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
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialBinding.java
index 29e815560a8..e91f8b08240 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialBinding.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialBinding.java
@@ -24,19 +24,18 @@ import java.util.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);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialProvider.java
index 8bd04744cd8..4bb5f65e14c 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/MarshalledCredentialProvider.java
@@ -21,7 +21,7 @@ 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);
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAuthWithAWSException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAuthWithAWSException.java
index 7ec13b092c9..8f92153b2e1 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAuthWithAWSException.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/NoAuthWithAWSException.java
@@ -21,7 +21,7 @@ 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.
*/
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java
index 82d4fa58816..ebd8ad9fddc 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/STSClientFactory.java
@@ -20,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());
}
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java
index 6526f9a9478..889e1e2c4af 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/EncryptionSecretOperations.java
@@ -20,9 +20,6 @@ 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 createSSECustomerKey(
- final EncryptionSecrets secrets) {
- if (secrets.hasEncryptionKey() &&
- secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_C) {
- return Optional.of(new SSECustomerKey(secrets.getEncryptionKey()));
+ public static Optional getSSECustomerKey(final EncryptionSecrets secrets) {
+ if (secrets.hasEncryptionKey() && secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_C) {
+ return Optional.of(secrets.getEncryptionKey());
} else {
- return Optional.empty();
+ return Optional.empty();
}
}
/**
- * Create SSE-KMS options for a request, iff the encryption is SSE-KMS.
- * @return an optional SSE-KMS param to attach to a request.
+ * Gets the SSE-KMS key if present, else let S3 use AWS managed key.
+ *
* @param secrets source of the encryption secrets.
+ * @return an optional key to attach to a request.
*/
- public static Optional createSSEAwsKeyManagementParams(
- final EncryptionSecrets secrets) {
-
- //Use specified key, otherwise default to default master aws/s3 key by AWS
- if (secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_KMS) {
- if (secrets.hasEncryptionKey()) {
- return Optional.of(new SSEAwsKeyManagementParams(
- secrets.getEncryptionKey()));
- } else {
- return Optional.of(new SSEAwsKeyManagementParams());
- }
+ public static Optional getSSEAwsKMSKey(final EncryptionSecrets secrets) {
+ if (secrets.getEncryptionMethod() == S3AEncryptionMethods.SSE_KMS
+ && secrets.hasEncryptionKey()) {
+ return Optional.of(secrets.getEncryptionKey());
} else {
return Optional.empty();
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/RoleTokenBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/RoleTokenBinding.java
index 9b06031d586..cb0cb64233a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/RoleTokenBinding.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/RoleTokenBinding.java
@@ -23,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;
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java
index bfb7e696645..73123a0d71e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/S3ADelegationTokens.java
@@ -228,7 +228,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);
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java
index 2f0a71767ed..434ec5b2467 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/delegation/SessionTokenBinding.java
@@ -26,21 +26,18 @@ import java.util.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");
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
index e53c690431e..54d3bc2e24e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
@@ -27,10 +27,11 @@ import java.util.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");
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java
index 10440f77e72..6c3cf3942d5 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/PutTracker.java
@@ -21,7 +21,7 @@ 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 parts,
+ List parts,
long bytesWritten,
final IOStatistics iostatistics)
throws IOException {
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java
index 77c3fed11fb..8801c8bdce7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java
@@ -31,9 +31,10 @@ import java.util.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 parts) throws ValidationFailure {
+ public void bindCommitData(List parts) throws ValidationFailure {
etags = new ArrayList<>(parts.size());
int counter = 1;
- for (PartETag part : parts) {
- verify(part.getPartNumber() == counter,
- "Expected part number %s but got %s", counter, part.getPartNumber());
- etags.add(part.getETag());
+ for (CompletedPart part : parts) {
+ verify(part.partNumber() == counter,
+ "Expected part number %s but got %s", counter, part.partNumber());
+ etags.add(part.eTag());
counter++;
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java
index ef56d829781..0dc0db24baf 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java
@@ -21,6 +21,8 @@ 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 toPartEtags(List tagIds) {
+ public static List toPartEtags(List tagIds) {
return IntStream.range(0, tagIds.size())
- .mapToObj(i -> new PartETag(i + 1, tagIds.get(i)))
+ .mapToObj(i -> CompletedPart.builder().partNumber(i + 1).eTag(tagIds.get(i)).build())
.collect(Collectors.toList());
}
@@ -566,26 +570,30 @@ public class CommitOperations extends AbstractStoreOperation
numParts, length));
}
- List parts = new ArrayList<>((int) numParts);
+ List parts = new ArrayList<>((int) numParts);
LOG.debug("File size is {}, number of parts to upload = {}",
length, numParts);
- for (int partNumber = 1; partNumber <= numParts; partNumber += 1) {
- progress.progress();
- long size = Math.min(length - offset, uploadPartSize);
- UploadPartRequest part;
- part = writeOperations.newUploadPartRequest(
- destKey,
- uploadId,
- partNumber,
- (int) size,
- null,
- localFile,
- offset);
- part.setLastPart(partNumber == numParts);
- UploadPartResult partResult = writeOperations.uploadPart(part, statistics);
- offset += uploadPartSize;
- parts.add(partResult.getPartETag());
+
+ // Open the file to upload.
+ try (InputStream fileStream = Files.newInputStream(localFile.toPath())) {
+ for (int partNumber = 1; partNumber <= numParts; partNumber += 1) {
+ progress.progress();
+ long size = Math.min(length - offset, uploadPartSize);
+ UploadPartRequest part = writeOperations.newUploadPartRequestBuilder(
+ destKey,
+ uploadId,
+ partNumber,
+ size).build();
+ // Read from the file input stream at current position.
+ RequestBody body = RequestBody.fromInputStream(fileStream, size);
+ UploadPartResponse response = writeOperations.uploadPart(part, body, statistics);
+ offset += uploadPartSize;
+ parts.add(CompletedPart.builder()
+ .partNumber(partNumber)
+ .eTag(response.eTag())
+ .build());
+ }
}
commitData.bindCommitData(parts);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java
index 1a5451df801..135adf0de39 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java
@@ -20,19 +20,22 @@ 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 parts,
+ List 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
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java
index 4169a9899cb..f076a4b701e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteRetryHandler.java
@@ -20,8 +20,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 keys = deleteRequest.getKeys();
+ final List keys = deleteRequest.delete().objects();
final int size = keys.size();
incrementStatistic(STORE_IO_THROTTLED, size);
instrumentation.addValueToQuantiles(STORE_IO_THROTTLE_RATE, size);
THROTTLE_LOG.info(
"Bulk delete {} keys throttled -first key = {}; last = {}",
size,
- keys.get(0).getKey(),
- keys.get(size - 1).getKey());
+ keys.get(0).key(),
+ keys.get(size - 1).key());
}
/**
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java
index dcb538dc668..f9d673e6579 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeDetectionPolicy.java
@@ -20,15 +20,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) {
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java
index e7dd75c5811..6020f979fa3 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java
@@ -18,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.
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CopyOutcome.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CopyOutcome.java
deleted file mode 100644
index 16459ac45b8..00000000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CopyOutcome.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3a.impl;
-
-import com.amazonaws.SdkBaseException;
-import com.amazonaws.services.s3.transfer.Copy;
-import com.amazonaws.services.s3.transfer.model.CopyResult;
-
-/**
- * Extracts the outcome of a TransferManager-executed copy operation.
- */
-public final class CopyOutcome {
-
- /**
- * Result of a successful copy.
- */
- private final CopyResult copyResult;
-
- /** the copy was interrupted. */
- private final InterruptedException interruptedException;
-
- /**
- * The copy raised an AWS Exception of some form.
- */
- private final SdkBaseException awsException;
-
- public CopyOutcome(CopyResult copyResult,
- InterruptedException interruptedException,
- SdkBaseException awsException) {
- this.copyResult = copyResult;
- this.interruptedException = interruptedException;
- this.awsException = awsException;
- }
-
- public CopyResult getCopyResult() {
- return copyResult;
- }
-
- public InterruptedException getInterruptedException() {
- return interruptedException;
- }
-
- public SdkBaseException getAwsException() {
- return awsException;
- }
-
- /**
- * Calls {@code Copy.waitForCopyResult()} to await the result, converts
- * it to a copy outcome.
- * Exceptions caught and
- * @param copy the copy operation.
- * @return the outcome.
- */
- public static CopyOutcome waitForCopy(Copy copy) {
- try {
- CopyResult result = copy.waitForCopyResult();
- return new CopyOutcome(result, null, null);
- } catch (SdkBaseException e) {
- return new CopyOutcome(null, null, e);
- } catch (InterruptedException e) {
- return new CopyOutcome(null, e, null);
- }
- }
-}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java
index a45bfe46f16..8f919897372 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java
@@ -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 {
"Delete page of %d keys", keyList.size())) {
if (!keyList.isEmpty()) {
// first delete the files.
- List files = keyList.stream()
+ List files = keyList.stream()
.filter(e -> !e.isDirMarker)
- .map(e -> e.keyVersion)
+ .map(e -> e.objectIdentifier)
.collect(Collectors.toList());
LOG.debug("Deleting of {} file objects", files.size());
Invoker.once("Remove S3 Files",
@@ -398,9 +399,9 @@ public class DeleteOperation extends ExecutingStoreOperation {
false
));
// now the dirs
- List dirs = keyList.stream()
+ List dirs = keyList.stream()
.filter(e -> e.isDirMarker)
- .map(e -> e.keyVersion)
+ .map(e -> e.objectIdentifier)
.collect(Collectors.toList());
LOG.debug("Deleting of {} directory markers", dirs.size());
// This is invoked with deleteFakeDir.
@@ -422,17 +423,17 @@ public class DeleteOperation extends ExecutingStoreOperation {
* to choose which statistics to update.
*/
private static final class DeleteEntry {
- private final DeleteObjectsRequest.KeyVersion keyVersion;
+ private final ObjectIdentifier objectIdentifier;
private final boolean isDirMarker;
private DeleteEntry(final String key, final boolean isDirMarker) {
- this.keyVersion = new DeleteObjectsRequest.KeyVersion(key);
+ this.objectIdentifier = ObjectIdentifier.builder().key(key).build();
this.isDirMarker = isDirMarker;
}
public String getKey() {
- return keyVersion.getKey();
+ return objectIdentifier.key();
}
@Override
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java
index f7e06413a37..54a91323bc2 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ErrorTranslation.java
@@ -18,9 +18,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);
}
/**
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java
index f75066e049d..275ad40c08d 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/HeaderProcessing.java
@@ -23,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 rawHeaders = md.getUserMetadata();
+ Map rawHeaders = md.metadata();
Map headers = new TreeMap<>();
rawHeaders.forEach((key, value) ->
headers.put(XA_HEADER_PREFIX + key, encodeBytes(value)));
// and add the usual content length &c, if set
maybeSetHeader(headers, XA_CACHE_CONTROL,
- md.getCacheControl());
+ md.cacheControl());
maybeSetHeader(headers, XA_CONTENT_DISPOSITION,
- md.getContentDisposition());
+ md.contentDisposition());
maybeSetHeader(headers, XA_CONTENT_ENCODING,
- md.getContentEncoding());
+ md.contentEncoding());
maybeSetHeader(headers, XA_CONTENT_LANGUAGE,
- md.getContentLanguage());
+ md.contentLanguage());
// If CSE is enabled, use the unencrypted content length.
- if (md.getUserMetaDataOf(Headers.CRYPTO_CEK_ALGORITHM) != null
- && md.getUserMetaDataOf(Headers.UNENCRYPTED_CONTENT_LENGTH) != null) {
- maybeSetHeader(headers, XA_CONTENT_LENGTH,
- md.getUserMetaDataOf(Headers.UNENCRYPTED_CONTENT_LENGTH));
- } else {
- maybeSetHeader(headers, XA_CONTENT_LENGTH,
- md.getContentLength());
+ // TODO: CSE is not supported yet, add these headers in during CSE work.
+// if (md.getUserMetaDataOf(Headers.CRYPTO_CEK_ALGORITHM) != null
+// && md.getUserMetaDataOf(Headers.UNENCRYPTED_CONTENT_LENGTH) != null) {
+// maybeSetHeader(headers, XA_CONTENT_LENGTH,
+// md.getUserMetaDataOf(Headers.UNENCRYPTED_CONTENT_LENGTH));
+// } else {
+// maybeSetHeader(headers, XA_CONTENT_LENGTH,
+// md.contentLength());
+// }
+// maybeSetHeader(headers, XA_CONTENT_MD5,
+// md.getContentMD5());
+ // TODO: Add back in else block during CSE work.
+ maybeSetHeader(headers, XA_CONTENT_LENGTH,
+ md.contentLength());
+ if (md.sdkHttpResponse() != null && md.sdkHttpResponse().headers() != null
+ && md.sdkHttpResponse().headers().get("Content-Range") != null) {
+ maybeSetHeader(headers, XA_CONTENT_RANGE,
+ md.sdkHttpResponse().headers().get("Content-Range").get(0));
}
- maybeSetHeader(headers, XA_CONTENT_MD5,
- md.getContentMD5());
- maybeSetHeader(headers, XA_CONTENT_RANGE,
- md.getContentRange());
maybeSetHeader(headers, XA_CONTENT_TYPE,
- md.getContentType());
+ md.contentType());
maybeSetHeader(headers, XA_ETAG,
- md.getETag());
+ md.eTag());
maybeSetHeader(headers, XA_LAST_MODIFIED,
- md.getLastModified());
+ Date.from(md.lastModified()));
// AWS custom headers
maybeSetHeader(headers, XA_ARCHIVE_STATUS,
- md.getArchiveStatus());
+ md.archiveStatus());
maybeSetHeader(headers, XA_OBJECT_LOCK_LEGAL_HOLD_STATUS,
- md.getObjectLockLegalHoldStatus());
+ md.objectLockLegalHoldStatus());
maybeSetHeader(headers, XA_OBJECT_LOCK_MODE,
- md.getObjectLockMode());
+ md.objectLockMode());
maybeSetHeader(headers, XA_OBJECT_LOCK_RETAIN_UNTIL_DATE,
- md.getObjectLockRetainUntilDate());
+ md.objectLockRetainUntilDate());
maybeSetHeader(headers, XA_OBJECT_REPLICATION_STATUS,
- md.getReplicationStatus());
+ md.replicationStatus());
maybeSetHeader(headers, XA_S3_VERSION_ID,
- md.getVersionId());
+ md.versionId());
maybeSetHeader(headers, XA_SERVER_SIDE_ENCRYPTION,
- md.getSSEAlgorithm());
+ md.serverSideEncryptionAsString());
maybeSetHeader(headers, XA_STORAGE_CLASS,
- md.getStorageClass());
- maybeSetHeader(headers, XA_STORAGE_CLASS,
- md.getReplicationStatus());
+ md.storageClassAsString());
+ // TODO: check this, looks wrong.
+ // maybeSetHeader(headers, XA_STORAGE_CLASS,
+// md.getReplicationStatus());
return headers;
}
@@ -458,70 +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 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;
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
index 2c34e7b9b6e..14e1fdc8cb5 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java
@@ -110,11 +110,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 =
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java
index 5d17ae91b81..70c6165c635 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java
@@ -22,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 keysToDelete,
+ List keysToDelete,
boolean deleteFakeDir)
- throws MultiObjectDeleteException, AmazonClientException,
+ throws MultiObjectDeleteException, AwsServiceException,
IOException;
/**
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java
index ae4d2fe7a34..62ed2ba6145 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java
@@ -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 {
/**
* list of keys to delete on the next (bulk) delete call.
*/
- private final List keysToDelete =
+ private final List keysToDelete =
new ArrayList<>();
/**
@@ -199,7 +199,7 @@ public class RenameOperation extends ExecutingStoreOperation {
*/
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 {
} 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 {
*/
@Retries.RetryTranslated
private void removeSourceObjects(
- final List keys)
+ final List keys)
throws IOException {
// remove the keys
@@ -580,9 +580,9 @@ public class RenameOperation extends ExecutingStoreOperation {
// 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 {
protected IOException convertToIOException(final Exception ex) {
if (ex instanceof IOException) {
return (IOException) ex;
- } else if (ex instanceof SdkBaseException) {
+ } else if (ex instanceof SdkException) {
return translateException("rename " + sourcePath + " to " + destPath,
sourcePath.toString(),
- (SdkBaseException) ex);
+ (SdkException) ex);
} else {
// should never happen, but for completeness
return new IOException(ex);
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java
index 7227941e344..1e01253bbf2 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java
@@ -18,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 web service request
- * @return prepared entry.
+ * @param web service request builder
+ * @return prepared builder.
*/
@Retries.OnceRaw
- private T prepareRequest(T t) {
- return requestPreparer != null
- ? requestPreparer.prepareRequest(t)
- : t;
+ private T prepareRequest(T t) {
+ if (requestPreparer != null) {
+ requestPreparer.prepareRequest(t);
+ }
+ return t;
}
/**
@@ -162,7 +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 generateSSEAwsKeyParams() {
- return EncryptionSecretOperations.createSSEAwsKeyManagementParams(
- encryptionSecrets);
- }
-
- /**
- * Create the SSE-C structure for the AWS SDK, if the encryption secrets
- * contain the information/settings for this.
- * This will contain a secret extracted from the bucket/configuration.
- * @return an optional customer key.
- */
- @Override
- public Optional generateSSECustomerKey() {
- return EncryptionSecretOperations.createSSECustomerKey(
- encryptionSecrets);
- }
-
/**
* Get the encryption algorithm of this endpoint.
* @return the encryption algorithm.
@@ -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 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 partETags) {
+ List partETags) {
// a copy of the list is required, so that the AWS SDK doesn't
// attempt to sort an unmodifiable list.
- return prepareRequest(new CompleteMultipartUploadRequest(bucket,
- destKey, uploadId, new ArrayList<>(partETags)));
+ CompleteMultipartUploadRequest.Builder requestBuilder =
+ CompleteMultipartUploadRequest.builder().bucket(bucket).key(destKey).uploadId(uploadId)
+ .multipartUpload(CompletedMultipartUpload.builder().parts(partETags).build());
+ return prepareRequest(requestBuilder);
}
@Override
- public GetObjectMetadataRequest newGetObjectMetadataRequest(String key) {
- GetObjectMetadataRequest request =
- new GetObjectMetadataRequest(getBucket(), key);
- //SSE-C requires to be filled in if enabled for object metadata
- setOptionalGetObjectMetadataParameters(request);
- return prepareRequest(request);
+ public HeadObjectRequest.Builder newHeadObjectRequestBuilder(String key) {
+
+ HeadObjectRequest.Builder headObjectRequestBuilder =
+ HeadObjectRequest.builder().bucket(getBucket()).key(key);
+
+ // need to set key to get metadata for objects encrypted with SSE_C
+ EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets).ifPresent(base64customerKey -> {
+ headObjectRequestBuilder.sseCustomerAlgorithm(ServerSideEncryption.AES256.name())
+ .sseCustomerKey(base64customerKey)
+ .sseCustomerKeyMD5(Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey)));
+ });
+
+ return prepareRequest(headObjectRequestBuilder);
}
@Override
- public 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 keysToDelete) {
- return prepareRequest(
- new DeleteObjectsRequest(bucket)
- .withKeys(keysToDelete)
- .withQuiet(true));
+ public DeleteObjectsRequest.Builder newBulkDeleteRequestBuilder(
+ List keysToDelete) {
+ return prepareRequest(DeleteObjectsRequest
+ .builder()
+ .bucket(bucket)
+ .delete(d -> d.objects(keysToDelete).quiet(true)));
}
@Override
@@ -628,23 +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 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 request type.
- * @return prepared entry.
+ * @param t request builder
*/
@Retries.OnceRaw
- T prepareRequest(T t);
+ void prepareRequest(SdkRequest.Builder t);
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java
index 4ab5bc6a992..b7eae8ead70 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java
@@ -34,10 +34,12 @@ import java.util.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 eTags = new ArrayList<>();
+ ArrayList eTags = new ArrayList<>();
eTags.ensureCapacity(handles.size());
long totalLength = 0;
// built up to identify duplicates -if the size of this set is
@@ -201,7 +203,8 @@ 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);
}));
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/SDKStreamDrainer.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/SDKStreamDrainer.java
index b566f9ad427..206d74e549d 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/SDKStreamDrainer.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/SDKStreamDrainer.java
@@ -18,12 +18,9 @@
package org.apache.hadoop.fs.s3a.impl;
-import java.io.Closeable;
+import java.io.InputStream;
import java.util.concurrent.atomic.AtomicBoolean;
-import javax.annotation.Nullable;
-
-import com.amazonaws.internal.SdkFilterInputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -31,23 +28,19 @@ import org.apache.hadoop.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 {
+public class SDKStreamDrainer
+ implements CallableRaisingIOE {
private static final Logger LOG = LoggerFactory.getLogger(
SDKStreamDrainer.class);
@@ -58,17 +51,9 @@ public class SDKStreamDrainer implements CallableRaisingIOE {
private final String uri;
/**
- * Request object; usually S3Object
- * Never used, but needed to keep the http connection
- * open long enough for draining to take place.
+ * Stream from the getObject response for draining and closing.
*/
- @Nullable
- private final Closeable requestObject;
-
- /**
- * Stream from the {@link #requestObject} for draining and closing.
- */
- private final SdkFilterInputStream sdkStream;
+ private final TStream sdkStream;
/**
* Should the request be aborted?
@@ -118,7 +103,6 @@ public class SDKStreamDrainer implements CallableRaisingIOE {
/**
* Prepare to drain the stream.
* @param uri URI for messages
- * @param requestObject http request object; needed to avoid GC issues.
* @param sdkStream stream to close.
* @param shouldAbort force an abort; used if explicitly requested.
* @param streamStatistics stats to update
@@ -126,14 +110,12 @@ public class SDKStreamDrainer implements CallableRaisingIOE {
* @param remaining remaining bytes
*/
public SDKStreamDrainer(final String uri,
- @Nullable final Closeable requestObject,
- final SdkFilterInputStream sdkStream,
+ final TStream sdkStream,
final boolean shouldAbort,
final int remaining,
final S3AInputStreamStatistics streamStatistics,
final String reason) {
this.uri = uri;
- this.requestObject = requestObject;
this.sdkStream = requireNonNull(sdkStream);
this.shouldAbort = shouldAbort;
this.remaining = remaining;
@@ -233,7 +215,6 @@ public class SDKStreamDrainer implements CallableRaisingIOE {
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 {
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 {
return uri;
}
- public Object getRequestObject() {
- return requestObject;
- }
-
- public SdkFilterInputStream getSdkStream() {
+ public TStream getSdkStream() {
return sdkStream;
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/V2Migration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/V2Migration.java
index 3aa8ad270ee..c9156f42047 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/V2Migration.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/V2Migration.java
@@ -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.
*/
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java
index 3ab0022bb08..65b5a423513 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java
@@ -19,15 +19,8 @@
package org.apache.hadoop.fs.s3a.prefetch;
-
import java.io.IOException;
-import java.io.InputStream;
-import java.util.IdentityHashMap;
-import java.util.Map;
-import com.amazonaws.services.s3.model.GetObjectRequest;
-import com.amazonaws.services.s3.model.S3Object;
-import com.amazonaws.services.s3.model.S3ObjectInputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -35,12 +28,17 @@ import org.apache.hadoop.fs.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 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 openForRead(long offset, int size)
+ throws IOException {
Validate.checkNotNegative(offset, "offset");
Validate.checkLessOrEqual(offset, "offset", size(), "size()");
Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset");
streamStatistics.streamOpened();
- final GetObjectRequest request =
- client.newGetRequest(s3Attributes.getKey())
- .withRange(offset, offset + size - 1);
- changeTracker.maybeApplyConstraint(request);
+ final GetObjectRequest request = client
+ .newGetRequestBuilder(s3Attributes.getKey())
+ .range(S3AUtils.formatRange(offset, offset + size - 1))
+ .applyMutation(changeTracker::maybeApplyConstraint)
+ .build();
String operation = String.format(
"%s %s at %d", S3AInputStream.OPERATION_OPEN, uri, offset);
DurationTracker tracker = streamStatistics.initiateGetRequest();
- S3Object object = null;
+ ResponseInputStream object = null;
try {
object = Invoker.once(operation, uri, () -> client.getObject(request));
@@ -212,27 +205,14 @@ public 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 inputStream, int numRemainingBytes) {
SDKStreamDrainer drainer = new SDKStreamDrainer(
uri,
- obj,
- (S3ObjectInputStream)inputStream,
+ inputStream,
false,
numRemainingBytes,
streamStatistics,
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java
index 89ea77d6d0e..b49b2699f91 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java
@@ -22,7 +22,6 @@ 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 inputStream =
+ remoteObject.openForRead(offset, readSize);
int numRemainingBytes = readSize;
byte[] bytes = new byte[READ_BUFFER_SIZE];
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
index 608f9168c24..63913afac79 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
@@ -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 args, PrintStream out)
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/BlockingEnumeration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/BlockingEnumeration.java
new file mode 100644
index 00000000000..6ff195609cb
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/BlockingEnumeration.java
@@ -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 the type of element.
+ */
+public final class BlockingEnumeration implements Enumeration {
+ private static final class Signal {
+ 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 END_SIGNAL = new Signal<>((Throwable)null);
+ private final CompletableFuture subscription = new CompletableFuture<>();
+ private final BlockingQueue> signalQueue;
+ private final int bufferSize;
+ private Signal current = null;
+
+ /**
+ * Create an enumeration with a fixed buffer size and an
+ * optional injected first element.
+ * @param publisher the publisher feeding the enumeration.
+ * @param bufferSize the buffer size.
+ * @param firstElement (optional) first element the enumeration will return.
+ */
+ public BlockingEnumeration(SdkPublisher publisher,
+ final int bufferSize,
+ final T firstElement) {
+ this.signalQueue = new LinkedBlockingQueue<>();
+ this.bufferSize = bufferSize;
+ if (firstElement != null) {
+ this.current = new Signal<>(firstElement);
+ }
+ publisher.subscribe(new EnumerationSubscriber());
+ }
+
+ /**
+ * Create an enumeration with a fixed buffer size.
+ * @param publisher the publisher feeding the enumeration.
+ * @param bufferSize the buffer size.
+ */
+ public BlockingEnumeration(SdkPublisher publisher,
+ final int bufferSize) {
+ this(publisher, bufferSize, null);
+ }
+
+ @Override
+ public boolean hasMoreElements() {
+ if (current == null) {
+ try {
+ current = signalQueue.take();
+ } catch (InterruptedException e) {
+ current = new Signal<>(e);
+ subscription.thenAccept(Subscription::cancel);
+ }
+ }
+ 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 {
+
+ @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);
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java
index 9c79cc1004c..95cad543383 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java
@@ -21,13 +21,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 {
* The default values in {@link SelectConstants}
*
*
- * @param request request to build up
+ * @param requestBuilder request to build up
* @param expression SQL expression
* @param builderOptions the options which came in from the openFile builder.
* @throws IllegalArgumentException if an option is somehow invalid.
* @throws IOException if an option is somehow invalid.
*/
void buildRequest(
- final SelectObjectContentRequest request,
+ final SelectObjectContentRequest.Builder requestBuilder,
final String expression,
final Configuration builderOptions)
throws IllegalArgumentException, IOException {
@@ -213,7 +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());
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectEventStreamPublisher.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectEventStreamPublisher.java
new file mode 100644
index 00000000000..c71ea5f1623
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectEventStreamPublisher.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.select;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.io.SequenceInputStream;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+
+import org.reactivestreams.Subscriber;
+
+import software.amazon.awssdk.core.async.SdkPublisher;
+import software.amazon.awssdk.http.AbortableInputStream;
+import software.amazon.awssdk.services.s3.model.EndEvent;
+import software.amazon.awssdk.services.s3.model.RecordsEvent;
+import software.amazon.awssdk.services.s3.model.SelectObjectContentEventStream;
+import software.amazon.awssdk.services.s3.model.SelectObjectContentResponse;
+import software.amazon.awssdk.utils.ToString;
+
+/**
+ * Async publisher of {@link SelectObjectContentEventStream}s returned
+ * from a SelectObjectContent call.
+ */
+public final class SelectEventStreamPublisher implements
+ SdkPublisher {
+
+ private final CompletableFuture selectOperationFuture;
+ private final SelectObjectContentResponse response;
+ private final SdkPublisher publisher;
+
+ /**
+ * Create the publisher.
+ * @param selectOperationFuture SelectObjectContent future
+ * @param response SelectObjectContent response
+ * @param publisher SelectObjectContentEventStream publisher to wrap
+ */
+ public SelectEventStreamPublisher(
+ CompletableFuture selectOperationFuture,
+ SelectObjectContentResponse response,
+ SdkPublisher publisher) {
+ this.selectOperationFuture = selectOperationFuture;
+ this.response = response;
+ this.publisher = publisher;
+ }
+
+ /**
+ * Retrieve an input stream to the subset of the S3 object that matched the select query.
+ * This is equivalent to loading the content of all RecordsEvents into an InputStream.
+ * This will lazily-load the content from S3, minimizing the amount of memory used.
+ * @param onEndEvent callback on the end event
+ * @return the input stream
+ */
+ public AbortableInputStream toRecordsInputStream(Consumer onEndEvent) {
+ SdkPublisher recordInputStreams = this.publisher
+ .filter(e -> {
+ if (e instanceof RecordsEvent) {
+ return true;
+ } else if (e instanceof EndEvent) {
+ onEndEvent.accept((EndEvent) e);
+ }
+ return false;
+ })
+ .map(e -> ((RecordsEvent) e).payload().asInputStream());
+
+ // Subscribe to the async publisher using an enumeration that will
+ // buffer a single chunk (RecordsEvent's payload) at a time and
+ // block until it is consumed.
+ // Also inject an empty stream as the first element that
+ // SequenceInputStream will request on construction.
+ BlockingEnumeration enumeration =
+ new BlockingEnumeration(recordInputStreams, 1, EMPTY_STREAM);
+ return AbortableInputStream.create(
+ new SequenceInputStream(enumeration),
+ this::cancel);
+ }
+
+ /**
+ * The response from the SelectObjectContent call.
+ * @return the response object
+ */
+ public SelectObjectContentResponse response() {
+ return response;
+ }
+
+ @Override
+ public void subscribe(Subscriber 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]);
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java
index f6ae52eba53..a2f5f28dc4c 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java
@@ -23,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();
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectObjectContentHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectObjectContentHelper.java
new file mode 100644
index 00000000000..c08793defaa
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectObjectContentHelper.java
@@ -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 selectOperationFuture =
+ writeOperationHelperCallbacks.selectObjectContent(request, handler);
+ return handler.eventPublisher(selectOperationFuture).join();
+ } catch (Throwable e) {
+ if (e instanceof CompletionException) {
+ e = e.getCause();
+ }
+ IOException translated;
+ if (e instanceof SdkException) {
+ translated = S3AUtils.translateException(action, source,
+ (SdkException)e);
+ } else {
+ translated = new IOException(e);
+ }
+ throw translated;
+ }
+ }
+
+ private static class Handler implements SelectObjectContentResponseHandler {
+ private volatile CompletableFuture>> responseAndPublisherFuture =
+ new CompletableFuture<>();
+
+ private volatile SelectObjectContentResponse response;
+
+ public CompletableFuture eventPublisher(
+ CompletableFuture selectOperationFuture) {
+ return responseAndPublisherFuture.thenApply(p ->
+ new SelectEventStreamPublisher(selectOperationFuture,
+ p.getLeft(), p.getRight()));
+ }
+
+ @Override
+ public void responseReceived(SelectObjectContentResponse response) {
+ this.response = response;
+ }
+
+ @Override
+ public void onEventStream(SdkPublisher publisher) {
+ responseAndPublisherFuture.complete(Pair.of(response, publisher));
+ }
+
+ @Override
+ public void exceptionOccurred(Throwable error) {
+ responseAndPublisherFuture.completeExceptionally(error);
+ }
+
+ @Override
+ public void complete() {
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java
index c002a4a6dee..188bb83ddad 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/AwsStatisticsCollector.java
@@ -21,23 +21,18 @@ 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.
*
- * 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 metric,
Consumer durationConsumer) {
- TimingInfo t1 = timingInfo.getSubMeasurement(subMeasurementName);
- if (t1 != null && t1.getTimeTakenMillisIfKnown() != null) {
- durationConsumer.accept(Duration.ofMillis(
- t1.getTimeTakenMillisIfKnown().longValue()));
- }
+ collection
+ .metricValues(metric)
+ .forEach(v -> durationConsumer.accept(v));
}
/**
* Process a counter.
- * @param timingInfo timing info
- * @param subMeasurementName sub measurement
+ * @param collection metric collection
+ * @param metric metric
* @param consumer consumer
*/
private void counter(
- TimingInfo timingInfo,
- String subMeasurementName,
+ MetricCollection collection,
+ SdkMetric metric,
LongConsumer consumer) {
- Number n = timingInfo.getCounter(subMeasurementName);
- if (n != null) {
- consumer.accept(n.longValue());
- }
+ collection
+ .metricValues(metric)
+ .forEach(v -> consumer.accept(v.longValue()));
+ }
+
+ /**
+ * Metric collections can be nested. Exposes a stream of the given
+ * collection and its nested children.
+ * @param metrics initial collection
+ * @return a stream of all nested metric collections
+ */
+ private static Stream recurseThroughChildren(
+ MetricCollection metrics) {
+ return Stream.concat(
+ Stream.of(metrics),
+ metrics.children().stream()
+ .flatMap(c -> recurseThroughChildren(c)));
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java
index 4ddc5f9478b..58078d1c6e0 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java
@@ -32,10 +32,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 collect =
+ List collect =
markers.values().stream()
- .map(p -> new DeleteObjectsRequest.KeyVersion(p.getKey()))
+ .map(p -> ObjectIdentifier.builder().key(p.getKey()).build())
.collect(Collectors.toList());
// build an array list for ease of creating the lists of
// keys in each page through the subList() method.
- List markerKeys =
+ List markerKeys =
new ArrayList<>(collect);
// now randomize. Why so? if the list spans multiple S3 partitions,
@@ -819,7 +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 page = markerKeys.subList(start,
+ List