Add more debug output in AbstractLeader and Follower
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractLeader.java
index b80a1bab15a9d248fa43a8628fdf810ef5bb0289..0d15e756354a9cb541e24f6b5e8e242fc5fee0e8 100644 (file)
@@ -28,12 +28,13 @@ import javax.annotation.Nullable;
 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
-import org.opendaylight.controller.cluster.raft.FollowerLogInformation.FollowerState;
 import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
+import org.opendaylight.controller.cluster.raft.PeerInfo;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
 import org.opendaylight.controller.cluster.raft.Snapshot;
+import org.opendaylight.controller.cluster.raft.VotingState;
 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
@@ -88,8 +89,6 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
     private int minReplicationCount;
 
-    private int minIsolatedLeaderPeerCount;
-
     private Optional<SnapshotHolder> snapshot;
 
     public AbstractLeader(RaftActorContext context) {
@@ -97,18 +96,16 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
         setLeaderPayloadVersion(context.getPayloadVersion());
 
-        for (String followerId : context.getPeerIds()) {
-            FollowerLogInformation followerLogInformation =
-                new FollowerLogInformationImpl(followerId, -1, context);
-
-            followerToLog.put(followerId, followerLogInformation);
+        for(PeerInfo peerInfo: context.getPeers()) {
+            FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(peerInfo, -1, context);
+            followerToLog.put(peerInfo.getId(), followerLogInformation);
         }
 
         leaderId = context.getId();
 
         LOG.debug("{}: Election: Leader has following peers: {}", logName(), getFollowerIds());
 
-        updateMinReplicaCountAndMinIsolatedLeaderPeerCount();
+        updateMinReplicaCount();
 
         snapshot = Optional.absent();
 
@@ -131,9 +128,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return followerToLog.keySet();
     }
 
-    public void addFollower(String followerId, FollowerState followerState) {
-        FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(followerId, -1, context);
-        followerLogInformation.setFollowerState(followerState);
+    public void addFollower(String followerId) {
+        FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(
+                context.getPeerInfo(followerId), -1, context);
         followerToLog.put(followerId, followerLogInformation);
 
         if(heartbeatSchedule == null) {
@@ -143,22 +140,29 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
     public void removeFollower(String followerId) {
         followerToLog.remove(followerId);
+        mapFollowerToSnapshot.remove(followerId);
     }
 
-    public void updateMinReplicaCountAndMinIsolatedLeaderPeerCount(){
-        minReplicationCount = getMajorityVoteCount(getFollowerIds().size());
+    public void updateMinReplicaCount() {
+        int numVoting = 0;
+        for(PeerInfo peer: context.getPeers()) {
+            if(peer.isVoting()) {
+                numVoting++;
+            }
+        }
 
-        //the isolated Leader peer count will be 1 less than the majority vote count.
+        minReplicationCount = getMajorityVoteCount(numVoting);
+    }
+
+    protected int getMinIsolatedLeaderPeerCount(){
+      //the isolated Leader peer count will be 1 less than the majority vote count.
         //this is because the vote count has the self vote counted in it
         //for e.g
         //0 peers = 1 votesRequired , minIsolatedLeaderPeerCount = 0
         //2 peers = 2 votesRequired , minIsolatedLeaderPeerCount = 1
         //4 peers = 3 votesRequired, minIsolatedLeaderPeerCount = 2
-        minIsolatedLeaderPeerCount = minReplicationCount > 0 ? (minReplicationCount - 1) : 0;
-    }
 
-    protected int getMinIsolatedLeaderPeerCount(){
-        return minIsolatedLeaderPeerCount;
+        return minReplicationCount > 0 ? (minReplicationCount - 1) : 0;
     }
 
     @VisibleForTesting
@@ -247,23 +251,51 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         // If there exists an N such that N > commitIndex, a majority
         // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
         // set commitIndex = N (§5.3, §5.4).
+        if(LOG.isTraceEnabled()) {
+            LOG.trace("{}: handleAppendEntriesReply from {}: commitIndex: {}, lastAppliedIndex: {}, currentTerm: {}",
+                    logName(), followerId, context.getCommitIndex(), context.getLastApplied(), currentTerm());
+        }
+
         for (long N = context.getCommitIndex() + 1; ; N++) {
             int replicatedCount = 1;
 
+            LOG.trace("{}: checking Nth index {}", logName(), N);
             for (FollowerLogInformation info : followerToLog.values()) {
-                if (info.getMatchIndex() >= N) {
+                final PeerInfo peerInfo = context.getPeerInfo(info.getId());
+                if(info.getMatchIndex() >= N && (peerInfo != null && peerInfo.isVoting())) {
                     replicatedCount++;
+                } else if(LOG.isDebugEnabled()) {
+                    LOG.debug("{}: Not counting follower {} - matchIndex: {}, {}", logName(), info.getId(),
+                            info.getMatchIndex(), peerInfo);
                 }
             }
 
+            if(LOG.isTraceEnabled()) {
+                LOG.trace("{}: replicatedCount {}, minReplicationCount: {}", logName(), replicatedCount, minReplicationCount);
+            }
+
             if (replicatedCount >= minReplicationCount) {
                 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
-                if (replicatedLogEntry != null && replicatedLogEntry.getTerm() == currentTerm()) {
+                if (replicatedLogEntry == null) {
+                    LOG.debug("{}: ReplicatedLogEntry not found for index {} - snapshotIndex: {}, journal size: {}",
+                            logName(), N, context.getReplicatedLog().getSnapshotIndex(), context.getReplicatedLog().size());
+                    break;
+                }
+
+                // Don't update the commit index if the log entry is from a previous term, as per §5.4.1:
+                // "Raft never commits log entries from previous terms by counting replicas".
+                // However we keep looping so we can make progress when new entries in the current term
+                // reach consensus, as per §5.4.1: "once an entry from the current term is committed by
+                // counting replicas, then all prior entries are committed indirectly".
+                if (replicatedLogEntry.getTerm() == currentTerm()) {
+                    LOG.trace("{}: Setting commit index to {}", logName(), N);
                     context.setCommitIndex(N);
                 } else {
-                    break;
+                    LOG.debug("{}: Not updating commit index to {} - retrieved log entry with index {}, term {} does not match the current term {}",
+                            logName(), N, replicatedLogEntry.getIndex(), replicatedLogEntry.getTerm(), currentTerm());
                 }
             } else {
+                LOG.trace("{}: minReplicationCount not reached - breaking", logName());
                 break;
             }
         }
@@ -396,12 +428,20 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
 
         if (followerToSnapshot == null) {
-            LOG.error("{}: FollowerId {} in InstallSnapshotReply not known to Leader",
+            LOG.error("{}: FollowerToSnapshot not found for follower {} in InstallSnapshotReply",
                     logName(), followerId);
             return;
         }
 
         FollowerLogInformation followerLogInformation = followerToLog.get(followerId);
+        if(followerLogInformation == null) {
+            // This can happen during AddServer if it times out.
+            LOG.error("{}: FollowerLogInformation not found for follower {} in InstallSnapshotReply",
+                    logName(), followerId);
+            mapFollowerToSnapshot.remove(followerId);
+            return;
+        }
+
         followerLogInformation.markFollowerActive();
 
         if (followerToSnapshot.getChunkIndex() == reply.getChunkIndex()) {
@@ -432,8 +472,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                         setSnapshot(null);
                     }
                     wasLastChunk = true;
-                    FollowerState followerState = followerLogInformation.getFollowerState();
-                    if(followerState == FollowerState.VOTING_NOT_INITIALIZED){
+                    if(context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED){
                         UnInitializedFollowerSnapshotReply unInitFollowerSnapshotSuccess =
                                              new UnInitializedFollowerSnapshotReply(followerId);
                         context.getActor().tell(unInitFollowerSnapshotSuccess, context.getActor());
@@ -499,7 +538,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    private void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
+    protected void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
         // Send an AppendEntries to all followers
         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
             final String followerId = e.getKey();
@@ -621,17 +660,16 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      * then send the existing snapshot in chunks to the follower.
      * @param followerId
      */
-    public void initiateCaptureSnapshot(String followerId) {
+    public boolean initiateCaptureSnapshot(String followerId) {
         if (snapshot.isPresent()) {
             // if a snapshot is present in the memory, most likely another install is in progress
             // no need to capture snapshot.
             // This could happen if another follower needs an install when one is going on.
             final ActorSelection followerActor = context.getPeerActorSelection(followerId);
             sendSnapshotChunk(followerActor, followerId);
-
-
+            return true;
         } else {
-            context.getSnapshotManager().captureToInstall(context.getReplicatedLog().last(),
+            return context.getSnapshotManager().captureToInstall(context.getReplicatedLog().last(),
                     this.getReplicatedToAllIndex(), followerId);
         }
     }
@@ -650,14 +688,15 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     private void sendInstallSnapshot() {
         LOG.debug("{}: sendInstallSnapshot", logName());
         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
-            ActorSelection followerActor = context.getPeerActorSelection(e.getKey());
+            String followerId = e.getKey();
+            ActorSelection followerActor = context.getPeerActorSelection(followerId);
             FollowerLogInformation followerLogInfo = e.getValue();
 
             if (followerActor != null) {
-                long nextIndex = e.getValue().getNextIndex();
-                if (followerLogInfo.getFollowerState() == FollowerState.VOTING_NOT_INITIALIZED ||
+                long nextIndex = followerLogInfo.getNextIndex();
+                if (context.getPeerInfo(followerId).getVotingState() == VotingState.VOTING_NOT_INITIALIZED ||
                         canInstallSnapshot(nextIndex)) {
-                    sendSnapshotChunk(followerActor, e.getKey());
+                    sendSnapshotChunk(followerActor, followerId);
                 }
             }
         }
@@ -759,7 +798,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     protected boolean isLeaderIsolated() {
-        int minPresent = minIsolatedLeaderPeerCount;
+        int minPresent = getMinIsolatedLeaderPeerCount();
         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
             if (followerLogInformation.isFollowerActive()) {
                 --minPresent;