drewschleit commented on code in PR #11052: URL: https://github.com/apache/iceberg/pull/11052#discussion_r1777451480
########## aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java: ########## @@ -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))) + + // Workaround: exclude all 503s from consuming retry tokens. Review Comment: 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. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org