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

dragonyliu 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 e852362f3 RATIS-1778. Fix NPE in readIndexHeartbeats (#817)
e852362f3 is described below

commit e852362f30fc57a8123f1dbcb98d3870d5e186e8
Author: Yaolong Liu <[email protected]>
AuthorDate: Wed Feb 1 21:18:29 2023 +0800

    RATIS-1778. Fix NPE in readIndexHeartbeats (#817)
    
    * RATIS-1778. Fix NPE in readIndexHeartbeats
    
    * address review comment
    
    ---------
    
    Co-authored-by: dragonyliu <[email protected]>
---
 .../java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java     | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)

diff --git 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java
 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java
index f7671e2f6..b07ccaa8e 100644
--- 
a/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java
+++ 
b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java
@@ -28,6 +28,7 @@ import org.slf4j.LoggerFactory;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.NavigableMap;
+import java.util.Optional;
 import java.util.TreeMap;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
@@ -101,7 +102,7 @@ class ReadIndexHeartbeats {
       final HeartbeatAck reply = replies.computeIfAbsent(
           logAppender.getFollowerId(), key -> new HeartbeatAck(logAppender));
       if (reply.receive(proto)) {
-        if (hasMajority.test(id -> replies.get(id).isAcknowledged())) {
+        if (hasMajority.test(this::isAcknowledged)) {
           future.complete(commitIndex);
           return true;
         }
@@ -110,6 +111,10 @@ class ReadIndexHeartbeats {
       return isCompletedNormally();
     }
 
+    boolean isAcknowledged(RaftPeerId id) {
+      return 
Optional.ofNullable(replies.get(id)).map(HeartbeatAck::isAcknowledged).isPresent();
+    }
+
     boolean isCompletedNormally() {
       return future.isDone() && !future.isCancelled() && 
!future.isCompletedExceptionally();
     }

Reply via email to