This is an automated email from the ASF dual-hosted git repository.

tanxinyu pushed a commit to branch rel/1.1
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/rel/1.1 by this push:
     new 0e48a216831 IOTDB-5848] Allow ratis-client retry when gRPC IO 
Unavailable (#9898) (#9910)
0e48a216831 is described below

commit 0e48a216831de544b04d46e7ca9c3c59a9b0fb96
Author: William Song <[email protected]>
AuthorDate: Mon May 22 22:37:20 2023 +0800

    IOTDB-5848] Allow ratis-client retry when gRPC IO Unavailable (#9898) 
(#9910)
---
 .../org/apache/iotdb/consensus/ratis/RatisClient.java  | 18 ++++++++++++++----
 1 file changed, 14 insertions(+), 4 deletions(-)

diff --git 
a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisClient.java 
b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisClient.java
index 9e870338f5e..fe80a486062 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisClient.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisClient.java
@@ -31,11 +31,13 @@ import org.apache.ratis.protocol.RaftGroup;
 import org.apache.ratis.protocol.exceptions.RaftException;
 import org.apache.ratis.retry.ExponentialBackoffRetry;
 import org.apache.ratis.retry.RetryPolicy;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
 import org.apache.ratis.util.TimeDuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 
 class RatisClient {
@@ -141,10 +143,18 @@ class RatisClient {
 
     @Override
     public Action handleAttemptFailure(Event event) {
-
-      if (event.getCause() instanceof IOException && !(event.getCause() 
instanceof RaftException)) {
-        // unexpected. may be caused by statemachine.
-        logger.info("raft client request failed and caught exception: ", 
event.getCause());
+      // Ratis guarantees that event.getCause() is instance of IOException.
+      // We should allow RaftException or IOException(StatusRuntimeException, 
thrown by gRPC) to be
+      // retried.
+      Optional<Throwable> unexpectedCause =
+          Optional.ofNullable(event.getCause())
+              .filter(t -> t instanceof RaftException)
+              .map(Throwable::getCause)
+              .filter(t -> t instanceof StatusRuntimeException);
+
+      if (unexpectedCause.isPresent()) {
+        logger.info(
+            "{}: raft client request failed and caught exception: ", this, 
unexpectedCause.get());
         return NO_RETRY_ACTION;
       }
 

Reply via email to