X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2Fbehaviors%2FAbstractLeader.java;fp=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2Fbehaviors%2FAbstractLeader.java;h=4ce84bd11f8ed8edccc4d8699a5202850dbef5be;hp=ec465935039eb99a0c34341e05aa93bd2340e34d;hb=615798c6573f1689068d6da14963112174c0702a;hpb=49f3024ee0a621729e31279f8b2d46051fc003f1 diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractLeader.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractLeader.java index ec46593503..4ce84bd11f 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractLeader.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractLeader.java @@ -677,7 +677,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } else if (installSnapshotState.canSendNextChunk()) { sendSnapshotChunk(followerActor, followerLogInformation); } - } else if (sendHeartbeat) { + } else if (sendHeartbeat || followerLogInformation.hasStaleCommitIndex(context.getCommitIndex())) { // we send a heartbeat even if we have not received a reply for the last chunk sendAppendEntries = true; } @@ -698,7 +698,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { log.debug("{}: sendAppendEntries: {} is present for follower {}", logName(), followerNextIndex, followerId); - if (followerLogInformation.okToReplicate()) { + if (followerLogInformation.okToReplicate(context.getCommitIndex())) { entries = getEntriesToSend(followerLogInformation, followerActor); sendAppendEntries = true; } @@ -726,7 +726,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { context.getReplicatedLog().size()); } - } else if (sendHeartbeat) { + } else if (sendHeartbeat || followerLogInformation.hasStaleCommitIndex(context.getCommitIndex())) { // we send an AppendEntries, even if the follower is inactive // in-order to update the followers timestamp, in case it becomes active again sendAppendEntries = true; @@ -837,6 +837,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { appendEntries); } + followerLogInformation.setSentCommitIndex(leaderCommitIndex); followerActor.tell(appendEntries, actor()); }