You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by dr...@apache.org on 2023/02/09 08:20:05 UTC
[ratis] branch master updated: RATIS-1778. Fix NPE in readIndexHeartbeats (#819)
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 d4126cf81 RATIS-1778. Fix NPE in readIndexHeartbeats (#819)
d4126cf81 is described below
commit d4126cf818d5c8f14bfd989ad6037a815c103538
Author: Yaolong Liu <ly...@163.com>
AuthorDate: Thu Feb 9 16:19:58 2023 +0800
RATIS-1778. Fix NPE in readIndexHeartbeats (#819)
Co-authored-by: dragonyliu <dr...@tencent.com>
---
.../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..92513086d 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)).filter(HeartbeatAck::isAcknowledged).isPresent();
+ }
+
boolean isCompletedNormally() {
return future.isDone() && !future.isCancelled() && !future.isCompletedExceptionally();
}