Skip to content

Commit

Permalink
Use exponential backoff retry strategy for glue client
Browse files Browse the repository at this point in the history
Default strategy was not good enough for handling
concurrent modifications, which led to flaky behavior
in concurrent modifications tests like
TestIcebergGlueCatalogConnectorSmokeTest#testDeleteRowsConcurrently
  • Loading branch information
oskar-szwajkowski authored and ebyhr committed Aug 27, 2024
1 parent 0c5b0cf commit 784a131
Show file tree
Hide file tree
Showing 2 changed files with 8 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import software.amazon.awssdk.http.apache.ProxyConfiguration;
import software.amazon.awssdk.regions.Region;
import software.amazon.awssdk.regions.providers.DefaultAwsRegionProviderChain;
import software.amazon.awssdk.retries.api.BackoffStrategy;
import software.amazon.awssdk.services.glue.GlueClient;
import software.amazon.awssdk.services.glue.GlueClientBuilder;
import software.amazon.awssdk.services.glue.model.ConcurrentModificationException;
Expand Down Expand Up @@ -131,6 +132,9 @@ public static GlueClient createGlueClient(GlueHiveMetastoreConfig config, OpenTe
.build().newExecutionInterceptor())
.retryStrategy(retryBuilder -> retryBuilder
.retryOnException(throwable -> throwable instanceof ConcurrentModificationException)
.backoffStrategy(BackoffStrategy.exponentialDelay(
java.time.Duration.ofMillis(20),
java.time.Duration.ofMillis(1500)))
.maxAttempts(config.getMaxGlueErrorRetries())));

Optional<StaticCredentialsProvider> staticCredentialsProvider = getStaticCredentialsProvider(config);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,10 @@
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.handlers.RequestHandler2;
import com.amazonaws.metrics.RequestMetricCollector;
import com.amazonaws.retry.PredefinedBackoffStrategies.ExponentialBackoffStrategy;
import com.amazonaws.retry.PredefinedRetryPolicies;
import com.amazonaws.retry.RetryPolicy;
import com.amazonaws.retry.RetryPolicy.BackoffStrategy;
import com.amazonaws.retry.RetryPolicy.RetryCondition;
import com.amazonaws.services.glue.AWSGlueAsync;
import com.amazonaws.services.glue.AWSGlueAsyncClientBuilder;
Expand All @@ -45,11 +47,12 @@ public static AWSGlueAsync createAsyncGlueClient(
RetryCondition customRetryCondition = (requestContext, exception, retriesAttempted) ->
defaultRetryPolicy.getRetryCondition().shouldRetry(requestContext, exception, retriesAttempted)
|| exception instanceof ConcurrentModificationException;
BackoffStrategy customBackoffStrategy = new ExponentialBackoffStrategy(20, 1500);

RetryPolicy glueRetryPolicy = RetryPolicy.builder()
.withRetryMode(defaultRetryPolicy.getRetryMode())
.withRetryCondition(customRetryCondition)
.withBackoffStrategy(defaultRetryPolicy.getBackoffStrategy())
.withBackoffStrategy(customBackoffStrategy)
.withFastFailRateLimiting(defaultRetryPolicy.isFastFailRateLimiting())
.withMaxErrorRetry(config.getMaxGlueErrorRetries())
.build();
Expand Down

0 comments on commit 784a131

Please sign in to comment.