Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

AWS: Set better defaults for S3 retry behaviour #11052

Merged
merged 1 commit into from
Oct 1, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ public S3Client s3() {
.applyMutation(s3FileIOProperties::applyEndpointConfigurations)
.applyMutation(s3FileIOProperties::applyServiceConfigurations)
.applyMutation(s3FileIOProperties::applySignerConfiguration)
.applyMutation(s3FileIOProperties::applyRetryConfigurations)
.build();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -114,6 +114,7 @@ public S3Client s3() {
.applyMutation(s3FileIOProperties::applySignerConfiguration)
.applyMutation(s3FileIOProperties::applyS3AccessGrantsConfigurations)
.applyMutation(s3FileIOProperties::applyUserAgentConfigurations)
.applyMutation(s3FileIOProperties::applyRetryConfigurations)
.build();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,7 @@ public S3Client s3() {
.applyMutation(httpClientProperties()::applyHttpClientConfigurations)
.applyMutation(s3FileIOProperties()::applyEndpointConfigurations)
.applyMutation(s3FileIOProperties()::applyServiceConfigurations)
.applyMutation(s3FileIOProperties()::applyRetryConfigurations)
.credentialsProvider(
new LakeFormationCredentialsProvider(lakeFormation(), buildTableArn()))
.region(Region.of(region()))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ public S3Client s3() {
.applyMutation(s3FileIOProperties::applySignerConfiguration)
.applyMutation(s3FileIOProperties::applyS3AccessGrantsConfigurations)
.applyMutation(s3FileIOProperties::applyUserAgentConfigurations)
.applyMutation(s3FileIOProperties::applyRetryConfigurations)
.build();
}
}
124 changes: 124 additions & 0 deletions aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,12 @@

import java.io.Serializable;
import java.net.URI;
import java.time.Duration;
import java.util.Collections;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import javax.xml.stream.XMLStreamException;
import org.apache.iceberg.EnvironmentContext;
import org.apache.iceberg.aws.AwsClientProperties;
import org.apache.iceberg.aws.glue.GlueCatalog;
Expand All @@ -38,6 +40,14 @@
import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration;
import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption;
import software.amazon.awssdk.core.exception.SdkServiceException;
import software.amazon.awssdk.core.retry.RetryMode;
import software.amazon.awssdk.core.retry.RetryPolicy;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Any reason why not use RetryStrategy from the retries API introduced in version 2.26? The current version used in iceberg supports that and technically RetryPolicy has already been flagged as deprecated.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is mainly for the retryCapacityCondition that we use in retryPolicy to workaround the HEAD 503 problem which I don't find a good replacement in RetryStrategy. We have an issue open with the SDK to fix that, after that we should be able to move.

Just double checking the support level, it seems like there is no immediate date to stop supporting RetryPolicy as well: https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/retry-strategy.html#migrate-from-retry-policies

import software.amazon.awssdk.core.retry.backoff.EqualJitterBackoffStrategy;
import software.amazon.awssdk.core.retry.conditions.OrRetryCondition;
import software.amazon.awssdk.core.retry.conditions.RetryCondition;
import software.amazon.awssdk.core.retry.conditions.RetryOnExceptionsCondition;
import software.amazon.awssdk.core.retry.conditions.TokenBucketRetryCondition;
import software.amazon.awssdk.services.s3.S3ClientBuilder;
import software.amazon.awssdk.services.s3.S3Configuration;
import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
Expand Down Expand Up @@ -393,6 +403,21 @@ public class S3FileIOProperties implements Serializable {
*/
private static final String S3_FILE_IO_USER_AGENT = "s3fileio/" + EnvironmentContext.get();

/** Number of times to retry S3 operations. */
public static final String S3_RETRY_NUM_RETRIES = "s3.retry.num-retries";

public static final int S3_RETRY_NUM_RETRIES_DEFAULT = 5;

/** Minimum wait time to retry a S3 operation */
public static final String S3_RETRY_MIN_WAIT_MS = "s3.retry.min-wait-ms";

public static final long S3_RETRY_MIN_WAIT_MS_DEFAULT = 2_000; // 2 seconds
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

would 500 ms make more sense maybe?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This was the original value that we did our load tests with but we did not see any benefit from lower value and it caused significantly higher throttle rates


/** Maximum wait time to retry a S3 read operation */
public static final String S3_RETRY_MAX_WAIT_MS = "s3.retry.max-wait-ms";

public static final long S3_RETRY_MAX_WAIT_MS_DEFAULT = 20_000; // 20 seconds

private String sseType;
private String sseKey;
private String sseMd5;
Expand Down Expand Up @@ -423,6 +448,9 @@ public class S3FileIOProperties implements Serializable {
private final String endpoint;
private final boolean isRemoteSigningEnabled;
private String writeStorageClass;
private int s3RetryNumRetries;
private long s3RetryMinWaitMs;
private long s3RetryMaxWaitMs;
private final Map<String, String> allProperties;

public S3FileIOProperties() {
Expand Down Expand Up @@ -455,6 +483,9 @@ public S3FileIOProperties() {
this.isRemoteSigningEnabled = REMOTE_SIGNING_ENABLED_DEFAULT;
this.isS3AccessGrantsEnabled = S3_ACCESS_GRANTS_ENABLED_DEFAULT;
this.isS3AccessGrantsFallbackToIamEnabled = S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED_DEFAULT;
this.s3RetryNumRetries = S3_RETRY_NUM_RETRIES_DEFAULT;
this.s3RetryMinWaitMs = S3_RETRY_MIN_WAIT_MS_DEFAULT;
this.s3RetryMaxWaitMs = S3_RETRY_MAX_WAIT_MS_DEFAULT;
this.allProperties = Maps.newHashMap();

ValidationException.check(
Expand Down Expand Up @@ -553,6 +584,12 @@ public S3FileIOProperties(Map<String, String> properties) {
properties,
S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED,
S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED_DEFAULT);
this.s3RetryNumRetries =
PropertyUtil.propertyAsInt(properties, S3_RETRY_NUM_RETRIES, S3_RETRY_NUM_RETRIES_DEFAULT);
this.s3RetryMinWaitMs =
PropertyUtil.propertyAsLong(properties, S3_RETRY_MIN_WAIT_MS, S3_RETRY_MIN_WAIT_MS_DEFAULT);
this.s3RetryMaxWaitMs =
PropertyUtil.propertyAsLong(properties, S3_RETRY_MAX_WAIT_MS, S3_RETRY_MAX_WAIT_MS_DEFAULT);

ValidationException.check(
keyIdAccessKeyBothConfigured(),
Expand Down Expand Up @@ -753,6 +790,34 @@ public void setS3AccessGrantsFallbackToIamEnabled(boolean s3AccessGrantsFallback
this.isS3AccessGrantsFallbackToIamEnabled = s3AccessGrantsFallbackToIamEnabled;
}

public int s3RetryNumRetries() {
return s3RetryNumRetries;
}

public void setS3RetryNumRetries(int s3RetryNumRetries) {
this.s3RetryNumRetries = s3RetryNumRetries;
}

public long s3RetryMinWaitMs() {
return s3RetryMinWaitMs;
}

public void setS3RetryMinWaitMs(long s3RetryMinWaitMs) {
this.s3RetryMinWaitMs = s3RetryMinWaitMs;
}

public long s3RetryMaxWaitMs() {
return s3RetryMaxWaitMs;
}

public void setS3RetryMaxWaitMs(long s3RetryMaxWaitMs) {
this.s3RetryMaxWaitMs = s3RetryMaxWaitMs;
}

public long s3RetryTotalWaitMs() {
return (long) s3RetryNumRetries() * s3RetryMaxWaitMs();
}

private boolean keyIdAccessKeyBothConfigured() {
return (accessKeyId == null) == (secretAccessKey == null);
}
Expand Down Expand Up @@ -824,6 +889,65 @@ public <T extends S3ClientBuilder> void applyEndpointConfigurations(T builder) {
}
}

/**
* Override the retry configurations for an S3 client.
*
* <p>Sample usage:
*
* <pre>
* S3Client.builder().applyMutation(s3FileIOProperties::applyRetryConfigurations)
* </pre>
*/
public <T extends S3ClientBuilder> void applyRetryConfigurations(T builder) {
builder.overrideConfiguration(
config ->
config.retryPolicy(
// Use a retry strategy which will persistently retry throttled exceptions with
// exponential backoff, to give S3 a chance to autoscale.
// LEGACY mode works best here, as it will allow throttled exceptions to use all of
// the configured retry attempts.
RetryPolicy.builder(RetryMode.LEGACY)
.numRetries(s3RetryNumRetries)
.throttlingBackoffStrategy(
EqualJitterBackoffStrategy.builder()
.baseDelay(Duration.ofMillis(s3RetryMinWaitMs))
.maxBackoffTime(Duration.ofMillis(s3RetryMaxWaitMs))
.build())

// Workaround: add XMLStreamException as a retryable exception.
// https://github.com/aws/aws-sdk-java-v2/issues/5442
// Without this workaround, we see SDK failures if there's a socket exception
// while parsing an error XML response.
.retryCondition(
OrRetryCondition.create(
RetryCondition.defaultRetryCondition(),
RetryOnExceptionsCondition.create(XMLStreamException.class)))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think this is comprehensive. We've seen issues with exceptions thrown as UncheckedIOException when using the urlconnect client which throws directly the UncheckedIOException and is not catched by the RetryableStage, which leads to these failures not being retried even if we set them as retryable conditions on exceptions (since they are not recognized/catched at all).

Example:

java.io.UncheckedIOException: java.net.SocketException: Unexpected end of file from server
 at software.amazon.awssdk.utils.FunctionalUtils.asRuntimeException(FunctionalUtils.java:180)
 at software.amazon.awssdk.utils.FunctionalUtils.lambda$safeSupplier$4(FunctionalUtils.java:110)
 at software.amazon.awssdk.utils.FunctionalUtils.invokeSafely(FunctionalUtils.java:136)
 at software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient$RequestCallable.lambda$tryGetOutputStream$0(UrlConnectionHttpClient.java:324)
 at software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient$RequestCallable.getAndHandle100Bug(UrlConnectionHttpClient.java:360)
 at software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient$RequestCallable.tryGetOutputStream(UrlConnectionHttpClient.java:324)
 at software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient$RequestCallable.call(UrlConnectionHttpClient.java:301)
 at software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient$RequestCallable.call(UrlConnectionHttpClient.java:274)
 at software.amazon.awssdk.core.internal.util.MetricUtils.measureDurationUnsafe(MetricUtils.java:67)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.executeHttpRequest(MakeHttpRequestStage.java:77)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:56)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:39)
 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:72)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:52)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:37)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)
 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:56)
 at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:36)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.executeWithTimer(ApiCallTimeoutTrackingStage.java:80)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:60)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:42)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:50)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:32)
 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:37)
 at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:26)
 at software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient$RequestExecutionBuilderImpl.execute(AmazonSyncHttpClient.java:196)
 at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)
 at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:171)
 at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$1(BaseSyncClientHandler.java:82)
 at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:179)
 at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:76)
 at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:45)
 at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:56)
 at software.amazon.awssdk.services.s3.DefaultS3Client.putObject(DefaultS3Client.java:8783)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You are right, it seems RetryableStage doesn't even catch all the default exceptions mentioned here particularly the UncheckedIO one. Seems like we need to solve this on the SDK side regardless but I am leaning towards leaving them here so that we don't need to come back and revisit this later on. What do you think?

I can update aws/aws-sdk-java-v2#5442 with this info as well, not sure whether you know a separate issue open regarding this.


// Workaround: exclude all 503s from consuming retry tokens.
Copy link
Contributor

@danielcweeks danielcweeks Sep 26, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Based on this it sounds like you're proposing unlimited retries on 503s and 32 retries on other failures? This feels far too aggressive, but I'm not sure if I'm reading this correctly. Why would we treat bucket HEAD requests differently anyway? (Also, I'm not sure that we actually make that request currently, so I don't think this is necessary)

Copy link
Contributor

@drewschleit drewschleit Sep 26, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The retry policy is configured with both a max retry count and a token bucket. Throttling exceptions (503) won't consume tokens from the token bucket, so the retry count is limited by the max retry count of 32. Retryable but non-throttling exceptions will consume tokens from the token bucket. We're reusing the SDK default values for the token bucket, which is 500 tokens, and a non-throttling exception will consume 5 tokens. So, if there's a spike in non-throttling exceptions, we'd expect the token bucket to kick in and effectively disable retries.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why would we treat bucket HEAD requests differently anyway?

It's a workaround to an SDK bug where 503s from HEAD requests aren't properly categorized as throttling exceptions.

HEAD is used here: https://github.com/apache/iceberg/blob/main/aws/src/main/java/org/apache/iceberg/aws/s3/BaseS3File.java#L80

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, I confused the HEAD request here with a Bucket HEAD request (not an object HEAD request).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So, if there's a spike in non-throttling exceptions, we'd expect the token bucket to kick in and effectively disable retries.

I see, this is the part I missed. I tried digging into the AWS SDK to see if there's a simpler way to express all this but don't think there is. But this implementation does seem correct for achieving this.

// https://github.com/aws/aws-sdk-java-v2/issues/5414
// Without this workaround, workloads which see 503s from S3 HEAD will fail
// prematurely.
.retryCapacityCondition(
TokenBucketRetryCondition.builder()
.tokenBucketSize(500) // 500 is the SDK default
.exceptionCostFunction(
e -> {
if (e instanceof SdkServiceException) {
SdkServiceException sdkServiceException =
(SdkServiceException) e;
if (sdkServiceException.isThrottlingException()
|| sdkServiceException.statusCode() == 503) {
return 0;
}
}

// 5 is the SDK default for non-throttling exceptions
return 5;
})
.build())
.build()));
}

/**
* Add the S3 Access Grants Plugin for an S3 client.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,8 @@
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration;
import software.amazon.awssdk.core.retry.RetryPolicy;
import software.amazon.awssdk.services.s3.S3Client;
import software.amazon.awssdk.services.s3.S3ClientBuilder;
import software.amazon.awssdk.services.s3.S3Configuration;
import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
Expand Down Expand Up @@ -491,4 +493,17 @@ public void testApplyUserAgentConfigurations() {
Mockito.verify(mockS3ClientBuilder)
.overrideConfiguration(Mockito.any(ClientOverrideConfiguration.class));
}

@Test
public void testApplyRetryConfiguration() {
Map<String, String> properties = Maps.newHashMap();
properties.put(S3FileIOProperties.S3_RETRY_NUM_RETRIES, "999");
S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(properties);

S3ClientBuilder builder = S3Client.builder();
s3FileIOProperties.applyRetryConfigurations(builder);

RetryPolicy retryPolicy = builder.overrideConfiguration().retryPolicy().get();
assertThat(retryPolicy.numRetries()).as("retries was not set").isEqualTo(999);
}
}
14 changes: 14 additions & 0 deletions docs/docs/aws.md
Original file line number Diff line number Diff line change
Expand Up @@ -378,6 +378,20 @@ However, for the older versions up to 0.12.0, the logic is as follows:

For more details, please refer to the [LocationProvider Configuration](custom-catalog.md#custom-location-provider-implementation) section.

### S3 Retries

Workloads which encounter S3 throttling should persistently retry, with exponential backoff, to make progress while S3
automatically scales. We provide the configurations below to adjust S3 retries for this purpose. For workloads that encounter
throttling and fail due to retry exhaustion, we recommend retry count to set 32 in order allow S3 to auto-scale. Note that
workloads with exceptionally high throughput against tables that S3 has not yet scaled, it may be necessary to increase the retry count further.


| Property | Default | Description |
|----------------------|---------|---------------------------------------------------------------------------------------|
| s3.retry.num-retries | 5 | Number of times to retry S3 operations. Recommended 32 for high-throughput workloads. |
| s3.retry.min-wait-ms | 2s | Minimum wait time to retry a S3 operation. |
| s3.retry.max-wait-ms | 20s | Maximum wait time to retry a S3 read operation. |

### S3 Strong Consistency

In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all read operations, and Iceberg is updated to fully leverage this feature.
Expand Down