Introduce a mechanism for a Follower to signal it's sync up status
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / behaviors / FollowerTest.java
index 4e8e7fe11bad4fdeb085ffc1ac60450cbbfeda11..29fb613327f23b72755514ce6f2c256d447f8a83 100644 (file)
@@ -1,6 +1,8 @@
 package org.opendaylight.controller.cluster.raft.behaviors;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import akka.actor.ActorRef;
 import akka.actor.Props;
@@ -19,6 +21,7 @@ import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
 import org.opendaylight.controller.cluster.raft.Snapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
+import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
@@ -118,6 +121,178 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
         assertEquals("isVoteGranted", false, reply.isVoteGranted());
     }
 
+
+    @Test
+    public void testHandleFirstAppendEntries() throws Exception {
+        logStart("testHandleFirstAppendEntries");
+
+        MockRaftActorContext context = createActorContext();
+
+        List<ReplicatedLogEntry> entries = Arrays.asList(
+                newReplicatedLogEntry(2, 101, "foo"));
+
+        // The new commitIndex is 101
+        AppendEntries appendEntries = new AppendEntries(2, "leader-1", 100, 1, entries, 101, 100);
+
+        follower = createBehavior(context);
+        follower.handleMessage(leaderActor, appendEntries);
+
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+
+        assertFalse(syncStatus.isInitialSyncDone());
+    }
+
+    @Test
+    public void testHandleSyncUpAppendEntries() throws Exception {
+        logStart("testHandleSyncUpAppendEntries");
+
+        MockRaftActorContext context = createActorContext();
+
+        List<ReplicatedLogEntry> entries = Arrays.asList(
+                newReplicatedLogEntry(2, 101, "foo"));
+
+        // The new commitIndex is 101
+        AppendEntries appendEntries = new AppendEntries(2, "leader-1", 100, 1, entries, 101, 100);
+
+        follower = createBehavior(context);
+        follower.handleMessage(leaderActor, appendEntries);
+
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+
+        assertFalse(syncStatus.isInitialSyncDone());
+
+        // Clear all the messages
+        followerActor.underlyingActor().clear();
+
+        context.setLastApplied(101);
+        context.setCommitIndex(101);
+        setLastLogEntry(context, 1, 101,
+                new MockRaftActorContext.MockPayload(""));
+
+        entries = Arrays.asList(
+                newReplicatedLogEntry(2, 101, "foo"));
+
+        // The new commitIndex is 101
+        appendEntries = new AppendEntries(2, "leader-1", 101, 1, entries, 102, 101);
+        follower.handleMessage(leaderActor, appendEntries);
+
+        syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+
+        assertTrue(syncStatus.isInitialSyncDone());
+
+        followerActor.underlyingActor().clear();
+
+        // Sending the same message again should not generate another message
+
+        follower.handleMessage(leaderActor, appendEntries);
+
+        syncStatus = MessageCollectorActor.getFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+
+        assertNull(syncStatus);
+
+    }
+
+    @Test
+    public void testHandleAppendEntriesLeaderChangedBeforeSyncUpComplete() throws Exception {
+        logStart("testHandleAppendEntriesLeaderChangedBeforeSyncUpComplete");
+
+        MockRaftActorContext context = createActorContext();
+
+        List<ReplicatedLogEntry> entries = Arrays.asList(
+                newReplicatedLogEntry(2, 101, "foo"));
+
+        // The new commitIndex is 101
+        AppendEntries appendEntries = new AppendEntries(2, "leader-1", 100, 1, entries, 101, 100);
+
+        follower = createBehavior(context);
+        follower.handleMessage(leaderActor, appendEntries);
+
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+
+        assertFalse(syncStatus.isInitialSyncDone());
+
+        // Clear all the messages
+        followerActor.underlyingActor().clear();
+
+        context.setLastApplied(100);
+        setLastLogEntry(context, 1, 100,
+                new MockRaftActorContext.MockPayload(""));
+
+        entries = Arrays.asList(
+                newReplicatedLogEntry(2, 101, "foo"));
+
+        // leader-2 is becoming the leader now and it says the commitIndex is 45
+        appendEntries = new AppendEntries(2, "leader-2", 45, 1, entries, 46, 100);
+        follower.handleMessage(leaderActor, appendEntries);
+
+        syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+
+        // We get a new message saying initial status is not done
+        assertFalse(syncStatus.isInitialSyncDone());
+
+    }
+
+
+    @Test
+    public void testHandleAppendEntriesLeaderChangedAfterSyncUpComplete() throws Exception {
+        logStart("testHandleAppendEntriesLeaderChangedAfterSyncUpComplete");
+
+        MockRaftActorContext context = createActorContext();
+
+        List<ReplicatedLogEntry> entries = Arrays.asList(
+                newReplicatedLogEntry(2, 101, "foo"));
+
+        // The new commitIndex is 101
+        AppendEntries appendEntries = new AppendEntries(2, "leader-1", 100, 1, entries, 101, 100);
+
+        follower = createBehavior(context);
+        follower.handleMessage(leaderActor, appendEntries);
+
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+
+        assertFalse(syncStatus.isInitialSyncDone());
+
+        // Clear all the messages
+        followerActor.underlyingActor().clear();
+
+        context.setLastApplied(101);
+        context.setCommitIndex(101);
+        setLastLogEntry(context, 1, 101,
+                new MockRaftActorContext.MockPayload(""));
+
+        entries = Arrays.asList(
+                newReplicatedLogEntry(2, 101, "foo"));
+
+        // The new commitIndex is 101
+        appendEntries = new AppendEntries(2, "leader-1", 101, 1, entries, 102, 101);
+        follower.handleMessage(leaderActor, appendEntries);
+
+        syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+
+        assertTrue(syncStatus.isInitialSyncDone());
+
+        // Clear all the messages
+        followerActor.underlyingActor().clear();
+
+        context.setLastApplied(100);
+        setLastLogEntry(context, 1, 100,
+                new MockRaftActorContext.MockPayload(""));
+
+        entries = Arrays.asList(
+                newReplicatedLogEntry(2, 101, "foo"));
+
+        // leader-2 is becoming the leader now and it says the commitIndex is 45
+        appendEntries = new AppendEntries(2, "leader-2", 45, 1, entries, 46, 100);
+        follower.handleMessage(leaderActor, appendEntries);
+
+        syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+
+        // We get a new message saying initial status is not done
+        assertFalse(syncStatus.isInitialSyncDone());
+
+    }
+
+
     /**
      * This test verifies that when an AppendEntries RPC is received by a RaftActor
      * with a commitIndex that is greater than what has been applied to the
@@ -358,7 +533,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
     }
 
     @Test
-    public void testHandleAppendAfterInstallingSnapshot(){
+    public void testHandleAppendEntriesAfterInstallingSnapshot(){
         logStart("testHandleAppendAfterInstallingSnapshot");
 
         MockRaftActorContext context = createActorContext();
@@ -449,7 +624,64 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
             assertEquals("getFollowerId", context.getId(), reply.getFollowerId());
         }
 
-        Assert.assertNull("Expected null SnapshotTracker", ((Follower)follower).getSnapshotTracker());
+        assertNull("Expected null SnapshotTracker", ((Follower) follower).getSnapshotTracker());
+    }
+
+    @Test
+    public void testInitialSyncUpWithHandleInstallSnapshotFollowedByAppendEntries() throws Exception {
+        logStart("testInitialSyncUpWithHandleInstallSnapshot");
+
+        MockRaftActorContext context = createActorContext();
+
+        follower = createBehavior(context);
+
+        HashMap<String, String> followerSnapshot = new HashMap<>();
+        followerSnapshot.put("1", "A");
+        followerSnapshot.put("2", "B");
+        followerSnapshot.put("3", "C");
+
+        ByteString bsSnapshot  = toByteString(followerSnapshot);
+        int offset = 0;
+        int snapshotLength = bsSnapshot.size();
+        int chunkSize = 50;
+        int totalChunks = (snapshotLength / chunkSize) + ((snapshotLength % chunkSize) > 0 ? 1 : 0);
+        int lastIncludedIndex = 1;
+        int chunkIndex = 1;
+        InstallSnapshot lastInstallSnapshot = null;
+
+        for(int i = 0; i < totalChunks; i++) {
+            ByteString chunkData = getNextChunk(bsSnapshot, offset, chunkSize);
+            lastInstallSnapshot = new InstallSnapshot(1, "leader", lastIncludedIndex, 1,
+                    chunkData, chunkIndex, totalChunks);
+            follower.handleMessage(leaderActor, lastInstallSnapshot);
+            offset = offset + 50;
+            lastIncludedIndex++;
+            chunkIndex++;
+        }
+
+        FollowerInitialSyncUpStatus syncStatus =
+                MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+
+        assertFalse(syncStatus.isInitialSyncDone());
+
+        // Clear all the messages
+        followerActor.underlyingActor().clear();
+
+        context.setLastApplied(101);
+        context.setCommitIndex(101);
+        setLastLogEntry(context, 1, 101,
+                new MockRaftActorContext.MockPayload(""));
+
+        List<ReplicatedLogEntry> entries = Arrays.asList(
+                newReplicatedLogEntry(2, 101, "foo"));
+
+        // The new commitIndex is 101
+        AppendEntries appendEntries = new AppendEntries(2, "leader", 101, 1, entries, 102, 101);
+        follower.handleMessage(leaderActor, appendEntries);
+
+        syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+
+        assertTrue(syncStatus.isInitialSyncDone());
     }
 
     @Test
@@ -479,7 +711,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
         assertEquals("getTerm", 1, reply.getTerm());
         assertEquals("getFollowerId", context.getId(), reply.getFollowerId());
 
-        Assert.assertNull("Expected null SnapshotTracker", ((Follower)follower).getSnapshotTracker());
+        assertNull("Expected null SnapshotTracker", ((Follower) follower).getSnapshotTracker());
     }
 
     public ByteString getNextChunk (ByteString bs, int offset, int chunkSize){