BUG-5626: do not allow overriding of RaftActor.handleCommand()
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractLeader.java
index 497d98cdce13cded64be94696591ffdb171f22d3..d914154f8bb5565c64836f7f97282a7390aa37e4 100644 (file)
@@ -16,6 +16,7 @@ import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -91,18 +92,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
     private Optional<SnapshotHolder> snapshot;
 
-    public AbstractLeader(RaftActorContext context) {
-        super(context, RaftState.Leader);
-
-        setLeaderPayloadVersion(context.getPayloadVersion());
+    protected AbstractLeader(RaftActorContext context, RaftState state) {
+        super(context, state);
 
         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());
 
         updateMinReplicaCount();
@@ -140,6 +137,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
     public void removeFollower(String followerId) {
         followerToLog.remove(followerId);
+        mapFollowerToSnapshot.remove(followerId);
     }
 
     public void updateMinReplicaCount() {
@@ -210,6 +208,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
         followerLogInformation.markFollowerActive();
         followerLogInformation.setPayloadVersion(appendEntriesReply.getPayloadVersion());
+        followerLogInformation.setRaftVersion(appendEntriesReply.getRaftVersion());
 
         boolean updated = false;
         if (appendEntriesReply.isSuccess()) {
@@ -254,18 +253,26 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             int replicatedCount = 1;
 
             for (FollowerLogInformation info : followerToLog.values()) {
-                if ((info.getMatchIndex() >= N) && (context.getPeerInfo(followerId).isVoting())) {
+                final PeerInfo peerInfo = context.getPeerInfo(info.getId());
+                if(info.getMatchIndex() >= N && (peerInfo != null && peerInfo.isVoting())) {
                     replicatedCount++;
                 }
             }
 
             if (replicatedCount >= minReplicationCount) {
                 ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(N);
-                if (replicatedLogEntry != null && replicatedLogEntry.getTerm() == currentTerm()) {
-                    context.setCommitIndex(N);
-                } else {
+                if (replicatedLogEntry == null) {
                     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()) {
+                    context.setCommitIndex(N);
+                }
             } else {
                 break;
             }
@@ -373,23 +380,19 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             beforeSendHeartbeat();
             sendHeartBeat();
             scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
-            return this;
-
         } else if(message instanceof SendInstallSnapshot) {
             // received from RaftActor
             setSnapshot(((SendInstallSnapshot) message).getSnapshot());
             sendInstallSnapshot();
-
         } else if (message instanceof Replicate) {
             replicate((Replicate) message);
-
-        } else if (message instanceof InstallSnapshotReply){
+        } else if (message instanceof InstallSnapshotReply) {
             handleInstallSnapshotReply((InstallSnapshotReply) message);
-
+        } else {
+            return super.handleMessage(sender, message);
         }
 
-
-        return super.handleMessage(sender, message);
+        return this;
     }
 
     private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
@@ -509,7 +512,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();
@@ -615,7 +618,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                     appendEntries);
         }
 
-        followerActor.tell(appendEntries.toSerializable(), actor());
+        followerActor.tell(appendEntries, actor());
     }
 
     /**
@@ -680,7 +683,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     private void sendSnapshotChunk(ActorSelection followerActor, String followerId) {
         try {
             if (snapshot.isPresent()) {
-                ByteString nextSnapshotChunk = getNextSnapshotChunk(followerId, snapshot.get().getSnapshotBytes());
+                byte[] nextSnapshotChunk = getNextSnapshotChunk(followerId, snapshot.get().getSnapshotBytes());
 
                 // Note: the previous call to getNextSnapshotChunk has the side-effect of adding
                 // followerId to the followerToSnapshot map.
@@ -694,7 +697,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                         followerToSnapshot.incrementChunkIndex(),
                         followerToSnapshot.getTotalChunks(),
                         Optional.of(followerToSnapshot.getLastChunkHashCode())
-                    ).toSerializable(),
+                    ).toSerializable(followerToLog.get(followerId).getRaftVersion()),
                     actor()
                 );
 
@@ -713,15 +716,15 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      * Acccepts snaphot as ByteString, enters into map for future chunks
      * creates and return a ByteString chunk
      */
-    private ByteString getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
+    private byte[] getNextSnapshotChunk(String followerId, ByteString snapshotBytes) throws IOException {
         FollowerToSnapshot followerToSnapshot = mapFollowerToSnapshot.get(followerId);
         if (followerToSnapshot == null) {
             followerToSnapshot = new FollowerToSnapshot(snapshotBytes);
             mapFollowerToSnapshot.put(followerId, followerToSnapshot);
         }
-        ByteString nextChunk = followerToSnapshot.getNextChunk();
+        byte[] nextChunk = followerToSnapshot.getNextChunk();
 
-        LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.size());
+        LOG.debug("{}: next snapshot chunk size for follower {}: {}", logName(), followerId, nextChunk.length);
 
         return nextChunk;
     }
@@ -754,20 +757,25 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         // need to be sent if there are other messages being sent to the remote
         // actor.
         heartbeatSchedule = context.getActorSystem().scheduler().scheduleOnce(
-            interval, context.getActor(), new SendHeartBeat(),
+            interval, context.getActor(), SendHeartBeat.INSTANCE,
             context.getActorSystem().dispatcher(), context.getActor());
     }
 
     @Override
-    public void close() throws Exception {
+    public void close() {
         stopHeartBeat();
     }
 
     @Override
-    public String getLeaderId() {
+    public final String getLeaderId() {
         return context.getId();
     }
 
+    @Override
+    public final short getLeaderPayloadVersion() {
+        return context.getPayloadVersion();
+    }
+
     protected boolean isLeaderIsolated() {
         int minPresent = getMinIsolatedLeaderPeerCount();
         for (FollowerLogInformation followerLogInformation : followerToLog.values()) {
@@ -860,25 +868,23 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             }
         }
 
-        public ByteString getNextChunk() {
+        public byte[] getNextChunk() {
             int snapshotLength = getSnapshotBytes().size();
             int start = incrementOffset();
             int size = context.getConfigParams().getSnapshotChunkSize();
             if (context.getConfigParams().getSnapshotChunkSize() > snapshotLength) {
                 size = snapshotLength;
-            } else {
-                if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
-                    size = snapshotLength - start;
-                }
+            } else if ((start + context.getConfigParams().getSnapshotChunkSize()) > snapshotLength) {
+                size = snapshotLength - start;
             }
 
+            byte[] nextChunk = new byte[size];
+            getSnapshotBytes().copyTo(nextChunk, start, 0, size);
+            nextChunkHashCode = Arrays.hashCode(nextChunk);
 
-            LOG.debug("{}: Next chunk: length={}, offset={},size={}", logName(),
-                    snapshotLength, start, size);
-
-            ByteString substring = getSnapshotBytes().substring(start, start + size);
-            nextChunkHashCode = substring.hashCode();
-            return substring;
+            LOG.debug("{}: Next chunk: total length={}, offset={}, size={}, hashCode={}", logName(),
+                    snapshotLength, start, size, nextChunkHashCode);
+            return nextChunk;
         }
 
         /**