@@ -631,6 +631,15 @@ synchronized void changeToLeader() {
631631
632632 @ Override
633633 public Collection <CommitInfoProto > getCommitInfos () {
634+ try {
635+ return getCommitInfosImpl ();
636+ } catch (Throwable t ) {
637+ LOG .warn ("{} Failed to getCommitInfos" , getMemberId (), t );
638+ return Collections .emptyList ();
639+ }
640+ }
641+
642+ private Collection <CommitInfoProto > getCommitInfosImpl () {
634643 final List <CommitInfoProto > infos = new ArrayList <>();
635644 // add the commit info of this server
636645 final long commitIndex = updateCommitInfoCache ();
@@ -922,17 +931,10 @@ CompletableFuture<RaftClientReply> executeSubmitClientRequestAsync(
922931 public CompletableFuture <RaftClientReply > submitClientRequestAsync (
923932 ReferenceCountedObject <RaftClientRequest > requestRef ) {
924933 final RaftClientRequest request = requestRef .retain ();
925- LOG .debug ("{}: receive client request({})" , getMemberId (), request );
926-
927934 try {
935+ LOG .debug ("{}: receive client request({})" , getMemberId (), request );
928936 assertLifeCycleState (LifeCycle .States .RUNNING );
929- } catch (ServerNotReadyException e ) {
930- final RaftClientReply reply = newExceptionReply (request , e );
931- requestRef .release ();
932- return CompletableFuture .completedFuture (reply );
933- }
934937
935- try {
936938 RaftClientRequest .Type type = request .getType ();
937939 final Timekeeper timer = raftServerMetrics .getClientRequestTimer (type );
938940 final Optional <Timekeeper .Context > timerContext = Optional .ofNullable (timer ).map (Timekeeper ::time );
@@ -942,6 +944,11 @@ public CompletableFuture<RaftClientReply> submitClientRequestAsync(
942944 raftServerMetrics .incFailedRequestCount (type );
943945 }
944946 });
947+ } catch (RaftException e ) {
948+ return CompletableFuture .completedFuture (newExceptionReply (request , e ));
949+ } catch (Throwable t ) {
950+ LOG .error ("{} Failed to submitClientRequestAsync for {}" , getMemberId (), request , t );
951+ return CompletableFuture .completedFuture (newExceptionReply (request , new RaftException (t )));
945952 } finally {
946953 requestRef .release ();
947954 }
0 commit comments