Skip to content

Commit

Permalink
Introduce RetryingJdbcClient
Browse files Browse the repository at this point in the history
So far JDBC connectors were able to retry in case of transient issues
when establishing connections. RetryingJdbcClient is able to retry an
operation on remote database during other situations.

Here we can have nested retrying. One in jdbc client and other in connection factory.
It is ok as as in both places we are using the retry policy with max duration to 30
seconds. The outer retries won't retry if the operation takes longer than 30s.
  • Loading branch information
kokosing committed Sep 12, 2024
1 parent 4f1fa43 commit cb2cd50
Show file tree
Hide file tree
Showing 13 changed files with 710 additions and 94 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import io.airlift.configuration.AbstractConfigurationAwareModule;
import io.trino.plugin.base.mapping.IdentifierMappingModule;
import io.trino.plugin.base.session.SessionPropertiesProvider;
import io.trino.plugin.jdbc.jmx.StatisticsAwareJdbcClient;
import io.trino.plugin.jdbc.logging.RemoteQueryModifierModule;
import io.trino.plugin.jdbc.procedure.ExecuteProcedure;
import io.trino.plugin.jdbc.procedure.FlushJdbcMetadataCacheProcedure;
Expand Down Expand Up @@ -52,7 +51,7 @@ public void setup(Binder binder)
install(new JdbcDiagnosticModule());
install(new IdentifierMappingModule());
install(new RemoteQueryModifierModule());
install(new RetryingConnectionFactoryModule());
install(new RetryingModule());

newOptionalBinder(binder, ConnectorAccessControl.class);
newOptionalBinder(binder, QueryBuilder.class).setDefault().to(DefaultQueryBuilder.class).in(Scopes.SINGLETON);
Expand Down Expand Up @@ -86,7 +85,7 @@ public void setup(Binder binder)
newExporter(binder).export(DynamicFilteringStats.class)
.as(generator -> generator.generatedNameOf(DynamicFilteringStats.class, catalogName.get().toString()));

binder.bind(JdbcClient.class).annotatedWith(ForCaching.class).to(Key.get(StatisticsAwareJdbcClient.class)).in(Scopes.SINGLETON);
binder.bind(JdbcClient.class).annotatedWith(ForCaching.class).to(Key.get(RetryingJdbcClient.class)).in(Scopes.SINGLETON);
binder.bind(CachingJdbcClient.class).in(Scopes.SINGLETON);
binder.bind(JdbcClient.class).to(Key.get(CachingJdbcClient.class)).in(Scopes.SINGLETON);

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.trino.plugin.jdbc;

public interface RetryStrategy
{
boolean isExceptionRecoverable(Throwable exception);
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,48 +13,28 @@
*/
package io.trino.plugin.jdbc;

import com.google.common.base.Throwables;
import com.google.inject.Inject;
import dev.failsafe.Failsafe;
import dev.failsafe.FailsafeException;
import dev.failsafe.RetryPolicy;
import io.trino.spi.TrinoException;
import io.trino.spi.connector.ConnectorSession;

import java.sql.Connection;
import java.sql.SQLException;
import java.sql.SQLTransientException;
import java.util.Set;

import static java.time.temporal.ChronoUnit.MILLIS;
import static java.time.temporal.ChronoUnit.SECONDS;
import static java.util.Objects.requireNonNull;

public class RetryingConnectionFactory
implements ConnectionFactory
{
private final RetryPolicy<Object> retryPolicy;

private final ConnectionFactory delegate;
private final RetryPolicy<Object> retryPolicy;

@Inject
public RetryingConnectionFactory(@ForRetrying ConnectionFactory delegate, Set<RetryStrategy> retryStrategies)
public RetryingConnectionFactory(@ForRetrying ConnectionFactory delegate, RetryPolicy<Object> retryPolicy)
{
requireNonNull(retryStrategies);
this.delegate = requireNonNull(delegate, "delegate is null");
this.retryPolicy = RetryPolicy.builder()
.withMaxDuration(java.time.Duration.of(30, SECONDS))
.withMaxAttempts(5)
.withBackoff(50, 5_000, MILLIS, 4)
.handleIf(throwable -> isExceptionRecoverable(retryStrategies, throwable))
.abortOn(TrinoException.class)
.build();
}

private static boolean isExceptionRecoverable(Set<RetryStrategy> retryStrategies, Throwable throwable)
{
return retryStrategies.stream()
.anyMatch(retryStrategy -> retryStrategy.isExceptionRecoverable(throwable));
this.retryPolicy = requireNonNull(retryPolicy, "retryPolicy is null");
}

@Override
Expand All @@ -79,20 +59,4 @@ public void close()
{
delegate.close();
}

public interface RetryStrategy
{
boolean isExceptionRecoverable(Throwable exception);
}

public static class DefaultRetryStrategy
implements RetryStrategy
{
@Override
public boolean isExceptionRecoverable(Throwable exception)
{
return Throwables.getCausalChain(exception).stream()
.anyMatch(SQLTransientException.class::isInstance);
}
}
}

This file was deleted.

Loading

0 comments on commit cb2cd50

Please sign in to comment.