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

szetszwo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git


The following commit(s) were added to refs/heads/master by this push:
     new 1970565  RATIS-1417. catch user statemachine 
notifyInstallSnapshotFromLeader exception (#520)
1970565 is described below

commit 1970565bac3d39737079b779a73838753d8b7ec9
Author: Jackson Yao <[email protected]>
AuthorDate: Fri Nov 5 18:34:52 2021 +0800

    RATIS-1417. catch user statemachine notifyInstallSnapshotFromLeader 
exception (#520)
---
 .../org/apache/ratis/server/impl/RaftServerImpl.java  | 19 +++++++++++++------
 1 file changed, 13 insertions(+), 6 deletions(-)

diff --git 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
index 11d13d5..31df3da 100644
--- 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
+++ 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
@@ -1600,14 +1600,21 @@ class RaftServerImpl implements RaftServer.Division,
                     LOG.debug("{}: StateMachine could not install snapshot as 
it is not available", this);
                   }
                 }
-              // wait for 10 seconds for statemachine to install snapshot
+              // wait for 1 seconds for statemachine to install snapshot
               }).get(1, TimeUnit.SECONDS);
-        } catch (ExecutionException t) {
+        } catch (InterruptedException | TimeoutException t) {
+          //nothing to do
+        } catch (Exception t) {
+          // there are two cases:
+          //1 `get()` may throw ExecutionException if `whenComplete` throw an 
exception
+          //2 when generating completeFuture, 
`statemachine#notifyInstallSnapshotFromLeader`
+          // may throw an uncertain exception, which is determined by the 
implementation of
+          // user statemachine.
           
inProgressInstallSnapshotRequest.compareAndSet(firstAvailableLogIndex, 0);
-          LOG.warn("{}: Failed to notify StateMachine to InstallSnapshot. 
ExecutionException: {}",
-              getMemberId(), t.getMessage());
-          throw new IOException("Failed to install snapshot");
-        } catch (InterruptedException | TimeoutException t) {}
+          final String err = getMemberId() + ": Failed to notify StateMachine 
to InstallSnapshot.";
+          LOG.warn(err + " " + t);
+          throw new IOException(err, t);
+        }
 
         if (LOG.isDebugEnabled()) {
           LOG.debug("{}: StateMachine is processing Snapshot Installation 
Request.", getMemberId());

Reply via email to