amogh-jahagirdar commented on code in PR #10140:
URL: https://github.com/apache/iceberg/pull/10140#discussion_r1565006119


##########
core/src/main/java/org/apache/iceberg/ClientPoolImpl.java:
##########
@@ -56,26 +67,36 @@ public <R> R run(Action<R, C, E> action, boolean retry) 
throws E, InterruptedExc
     C client = get();
     try {
       return action.run(client);
-
     } catch (Exception exc) {
-      if (retry && isConnectionException(exc)) {
-        try {
-          client = reconnect(client);
-        } catch (Exception ignored) {
-          // if reconnection throws any exception, rethrow the original failure
-          throw reconnectExc.cast(exc);
-        }
-
-        return action.run(client);
+      if (!isConnectionException(exc)) {
+        throw exc;
       }
 
-      throw exc;
+      return retryAction(action, exc, client);
 
     } finally {
       release(client);
     }
   }
 
+  private <R> R retryAction(Action<R, C, E> action, Exception originalFailure, 
C client) throws E {
+    int retryAttempts = 0;
+    while (retryAttempts < maxRetries) {
+      try {
+        C reconnectedClient = reconnect(client);
+        return action.run(reconnectedClient);
+      } catch (Exception exc) {
+        if (isConnectionException(exc)) {
+          retryAttempts++;
+        } else {
+          throw reconnectExc.cast(originalFailure);
+        }
+      }
+    }
+
+    throw reconnectExc.cast(originalFailure);

Review Comment:
   I wanted to use `Tasks` for all this :(, but it's seemingly infeasible 
without passing through the actual class for `E` so that we can handle that 
appropriately during task failure (the generic E and the handling at Tasks are 
just impossible to reconcile without fundamental API changes). That is kind of 
a hack and also adds an unneccessary API change. I think the best option is 
just to write the retry logic.



##########
core/src/main/java/org/apache/iceberg/ClientPoolImpl.java:
##########
@@ -56,26 +67,36 @@ public <R> R run(Action<R, C, E> action, boolean retry) 
throws E, InterruptedExc
     C client = get();
     try {
       return action.run(client);
-
     } catch (Exception exc) {
-      if (retry && isConnectionException(exc)) {
-        try {
-          client = reconnect(client);
-        } catch (Exception ignored) {
-          // if reconnection throws any exception, rethrow the original failure
-          throw reconnectExc.cast(exc);
-        }
-
-        return action.run(client);
+      if (!isConnectionException(exc)) {
+        throw exc;
       }
 
-      throw exc;
+      return retryAction(action, exc, client);
 
     } finally {
       release(client);
     }
   }
 
+  private <R> R retryAction(Action<R, C, E> action, Exception originalFailure, 
C client) throws E {
+    int retryAttempts = 0;
+    while (retryAttempts < maxRetries) {
+      try {
+        C reconnectedClient = reconnect(client);
+        return action.run(reconnectedClient);
+      } catch (Exception exc) {
+        if (isConnectionException(exc)) {
+          retryAttempts++;
+        } else {
+          throw reconnectExc.cast(originalFailure);
+        }
+      }
+    }
+
+    throw reconnectExc.cast(originalFailure);

Review Comment:
   Maybe add some backoff/jitter? 



##########
core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java:
##########
@@ -43,8 +53,16 @@ public JdbcClientPool(String dbUrl, Map<String, String> 
props) {
   }
 
   public JdbcClientPool(int poolSize, String dbUrl, Map<String, String> props) 
{
-    super(poolSize, SQLNonTransientConnectionException.class, true);
+    super(poolSize, SQLTransientException.class, true);
     properties = props;
+    retryableStatusCodes = Sets.newHashSet();
+    retryableStatusCodes.addAll(COMMON_RETRYABLE_CONNECTION_SQL_STATES);
+    String configuredRetryableStatuses = 
props.get(JdbcCatalog.RETRYABLE_STATUS_CODES);
+    if (configuredRetryableStatuses != null) {

Review Comment:
   I'm presenting this configuration option since at the moment I believe it's 
the most practical way for people to work around production issues they hit 
today.
   
   Long term, if we separate JDBC catalog into a separate library (I forgot 
where we landed on where JDBC fits in in terms of the catalog discussion that's 
happened in the past) that library could define status codes for each database 
so a user doesn't have to specify anything.



-- 
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

Reply via email to