Make private methods static
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / behaviors / FollowerTest.java
index 1e931109c3e9234e73cf3c94f31b1a3e7ecf3978..1b15ecb135a89f87c212ecf3a080cf8181ba921e 100644 (file)
@@ -1,3 +1,11 @@
+/*
+ * Copyright (c) 2014, 2015 Cisco Systems, Inc. and others.  All rights reserved.
+ *
+ * This program and the accompanying materials are made available under the
+ * terms of the Eclipse Public License v1.0 which accompanies this distribution,
+ * and is available at http://www.eclipse.org/legal/epl-v10.html
+ */
+
 package org.opendaylight.controller.cluster.raft.behaviors;
 
 import static org.junit.Assert.assertEquals;
@@ -64,7 +72,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
 
     @Override
     protected RaftActorBehavior createBehavior(RaftActorContext actorContext) {
-        return new Follower(actorContext);
+        return new TestFollower(actorContext);
     }
 
     @Override
@@ -79,6 +87,13 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
         return context;
     }
 
+    private static int getElectionTimeoutCount(RaftActorBehavior follower){
+        if(follower instanceof TestFollower){
+            return ((TestFollower) follower).getElectionTimeoutCount();
+        }
+        return -1;
+    }
+
     @Test
     public void testThatAnElectionTimeoutIsTriggered(){
         MockRaftActorContext actorContext = createActorContext();
@@ -115,6 +130,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
 
         assertEquals("isVoteGranted", true, reply.isVoteGranted());
         assertEquals("getTerm", term, reply.getTerm());
+        assertEquals("schedule election", 1, getElectionTimeoutCount(follower));
     }
 
     @Test
@@ -132,6 +148,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
         RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, RequestVoteReply.class);
 
         assertEquals("isVoteGranted", false, reply.isVoteGranted());
+        assertEquals("schedule election", 0, getElectionTimeoutCount(follower));
     }
 
 
@@ -140,10 +157,15 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
         logStart("testHandleFirstAppendEntries");
 
         MockRaftActorContext context = createActorContext();
+        context.getReplicatedLog().clear(0,2);
+        context.getReplicatedLog().append(newReplicatedLogEntry(1,100, "bar"));
+        context.getReplicatedLog().setSnapshotIndex(99);
 
         List<ReplicatedLogEntry> entries = Arrays.asList(
                 newReplicatedLogEntry(2, 101, "foo"));
 
+        Assert.assertEquals(1, context.getReplicatedLog().size());
+
         // The new commitIndex is 101
         AppendEntries appendEntries = new AppendEntries(2, "leader-1", 100, 1, entries, 101, 100, (short)0);
 
@@ -151,8 +173,105 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
         follower.handleMessage(leaderActor, appendEntries);
 
         FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+        AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, AppendEntriesReply.class);
+
+        assertFalse(syncStatus.isInitialSyncDone());
+        assertTrue("append entries reply should be true", reply.isSuccess());
+    }
+
+    @Test
+    public void testHandleFirstAppendEntriesWithPrevIndexMinusOne() 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", -1, -1, entries, 101, 100, (short) 0);
+
+        follower = createBehavior(context);
+        follower.handleMessage(leaderActor, appendEntries);
+
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+        AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, AppendEntriesReply.class);
+
+        assertFalse(syncStatus.isInitialSyncDone());
+        assertFalse("append entries reply should be false", reply.isSuccess());
+    }
+
+    @Test
+    public void testHandleFirstAppendEntriesWithPrevIndexMinusOneAndReplicatedToAllIndexPresentInLog() throws Exception {
+        logStart("testHandleFirstAppendEntries");
+
+        MockRaftActorContext context = createActorContext();
+        context.getReplicatedLog().clear(0,2);
+        context.getReplicatedLog().append(newReplicatedLogEntry(1, 100, "bar"));
+        context.getReplicatedLog().setSnapshotIndex(99);
+
+        List<ReplicatedLogEntry> entries = Arrays.asList(
+                newReplicatedLogEntry(2, 101, "foo"));
+
+        // The new commitIndex is 101
+        AppendEntries appendEntries = new AppendEntries(2, "leader-1", -1, -1, entries, 101, 100, (short) 0);
+
+        follower = createBehavior(context);
+        follower.handleMessage(leaderActor, appendEntries);
+
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+        AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, AppendEntriesReply.class);
+
+        assertFalse(syncStatus.isInitialSyncDone());
+        assertTrue("append entries reply should be true", reply.isSuccess());
+    }
+
+    @Test
+    public void testHandleFirstAppendEntriesWithPrevIndexMinusOneAndReplicatedToAllIndexPresentInSnapshot() throws Exception {
+        logStart("testHandleFirstAppendEntries");
+
+        MockRaftActorContext context = createActorContext();
+        context.getReplicatedLog().clear(0,2);
+        context.getReplicatedLog().setSnapshotIndex(100);
+
+        List<ReplicatedLogEntry> entries = Arrays.asList(
+                newReplicatedLogEntry(2, 101, "foo"));
+
+        // The new commitIndex is 101
+        AppendEntries appendEntries = new AppendEntries(2, "leader-1", -1, -1, entries, 101, 100, (short) 0);
+
+        follower = createBehavior(context);
+        follower.handleMessage(leaderActor, appendEntries);
+
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+        AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, AppendEntriesReply.class);
+
+        assertFalse(syncStatus.isInitialSyncDone());
+        assertTrue("append entries reply should be true", reply.isSuccess());
+    }
+
+    @Test
+    public void testHandleFirstAppendEntriesWithPrevIndexMinusOneAndReplicatedToAllIndexPresentInSnapshotButCalculatedPreviousEntryMissing() throws Exception {
+        logStart("testHandleFirstAppendEntries");
+
+        MockRaftActorContext context = createActorContext();
+        context.getReplicatedLog().clear(0,2);
+        context.getReplicatedLog().setSnapshotIndex(100);
+
+        List<ReplicatedLogEntry> entries = Arrays.asList(
+                newReplicatedLogEntry(2, 105, "foo"));
+
+        // The new commitIndex is 101
+        AppendEntries appendEntries = new AppendEntries(2, "leader-1", -1, -1, entries, 105, 100, (short) 0);
+
+        follower = createBehavior(context);
+        follower.handleMessage(leaderActor, appendEntries);
+
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+        AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, AppendEntriesReply.class);
 
         assertFalse(syncStatus.isInitialSyncDone());
+        assertFalse("append entries reply should be false", reply.isSuccess());
     }
 
     @Test
@@ -480,6 +599,44 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
         expectAndVerifyAppendEntriesReply(2, true, context.getId(), 2, 3);
     }
 
+    @Test
+    public void testHandleAppendEntriesWhenOutOfSyncLogDetectedRequestForceInstallSnapshot() {
+        logStart("testHandleAppendEntriesWhenOutOfSyncLogDetectedRequestForceInstallSnapshot");
+
+        MockRaftActorContext context = createActorContext();
+
+        // First set the receivers term to lower number
+        context.getTermInformation().update(1, "test");
+
+        // Prepare the receivers log
+        MockRaftActorContext.SimpleReplicatedLog log = new MockRaftActorContext.SimpleReplicatedLog();
+        log.append(newReplicatedLogEntry(1, 0, "zero"));
+        log.append(newReplicatedLogEntry(1, 1, "one"));
+        log.append(newReplicatedLogEntry(1, 2, "two"));
+
+        context.setReplicatedLog(log);
+
+        // Prepare the entries to be sent with AppendEntries
+        List<ReplicatedLogEntry> entries = new ArrayList<>();
+        entries.add(newReplicatedLogEntry(2, 2, "two-1"));
+        entries.add(newReplicatedLogEntry(2, 3, "three"));
+
+        // Send appendEntries with the same term as was set on the receiver
+        // before the new behavior was created (1 in this case)
+        // This will not work for a Candidate because as soon as a Candidate
+        // is created it increments the term
+        AppendEntries appendEntries = new AppendEntries(2, "leader", 1, 1, entries, 3, -1, (short)0);
+
+        context.setRaftPolicy(createRaftPolicy(false, true));
+        follower = createBehavior(context);
+
+        RaftActorBehavior newBehavior = follower.handleMessage(leaderActor, appendEntries);
+
+        Assert.assertSame(follower, newBehavior);
+
+        expectAndVerifyAppendEntriesReply(2, false, context.getId(), 1, 2, true);
+    }
+
     @Test
     public void testHandleAppendEntriesPreviousLogEntryMissing(){
         logStart("testHandleAppendEntriesPreviousLogEntryMissing");
@@ -592,6 +749,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
         logStart("testHandleInstallSnapshot");
 
         MockRaftActorContext context = createActorContext();
+        context.getTermInformation().update(1, "leader");
 
         follower = createBehavior(context);
 
@@ -625,6 +783,9 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
                 snapshot.getLastAppliedIndex());
         assertEquals("getLastTerm", lastInstallSnapshot.getLastIncludedTerm(), snapshot.getLastTerm());
         Assert.assertArrayEquals("getState", bsSnapshot.toByteArray(), snapshot.getState());
+        assertEquals("getElectionTerm", 1, snapshot.getElectionTerm());
+        assertEquals("getElectionVotedFor", "leader", snapshot.getElectionVotedFor());
+        applySnapshot.getCallback().onSuccess();
 
         List<InstallSnapshotReply> replies = MessageCollectorActor.getAllMatching(
                 leaderActor, InstallSnapshotReply.class);
@@ -801,6 +962,26 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
         MessageCollectorActor.assertNoneMatching(followerActor, ElectionTimeout.class, 500);
     }
 
+    @Test
+    public void testElectionScheduledWhenAnyRaftRPCReceived(){
+        MockRaftActorContext context = createActorContext();
+        follower = createBehavior(context);
+        follower.handleMessage(leaderActor, new RaftRPC() {
+            @Override
+            public long getTerm() {
+                return 100;
+            }
+        });
+        assertEquals("schedule election", 1, getElectionTimeoutCount(follower));
+    }
+
+    @Test
+    public void testElectionNotScheduledWhenNonRaftRPCMessageReceived(){
+        MockRaftActorContext context = createActorContext();
+        follower = createBehavior(context);
+        follower.handleMessage(leaderActor, "non-raft-rpc");
+        assertEquals("schedule election", 0, getElectionTimeoutCount(follower));
+    }
 
     public ByteString getNextChunk (ByteString bs, int offset, int chunkSize){
         int snapshotLength = bs.size();
@@ -818,6 +999,12 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
 
     private void expectAndVerifyAppendEntriesReply(int expTerm, boolean expSuccess,
             String expFollowerId, long expLogLastTerm, long expLogLastIndex) {
+        expectAndVerifyAppendEntriesReply(expTerm, expSuccess, expFollowerId, expLogLastTerm, expLogLastIndex, false);
+    }
+
+    private void expectAndVerifyAppendEntriesReply(int expTerm, boolean expSuccess,
+                                                   String expFollowerId, long expLogLastTerm, long expLogLastIndex,
+                                                   boolean expForceInstallSnapshot) {
 
         AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor,
                 AppendEntriesReply.class);
@@ -828,9 +1015,11 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
         assertEquals("getLogLastTerm", expLogLastTerm, reply.getLogLastTerm());
         assertEquals("getLogLastIndex", expLogLastIndex, reply.getLogLastIndex());
         assertEquals("getPayloadVersion", payloadVersion, reply.getPayloadVersion());
+        assertEquals("isForceInstallSnapshot", expForceInstallSnapshot, reply.isForceInstallSnapshot());
     }
 
-    private ReplicatedLogEntry newReplicatedLogEntry(long term, long index, String data) {
+
+    private static ReplicatedLogEntry newReplicatedLogEntry(long term, long index, String data) {
         return new MockRaftActorContext.MockReplicatedLogEntry(term, index,
                 new MockRaftActorContext.MockPayload(data));
     }
@@ -859,4 +1048,23 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest {
         AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(replyActor, AppendEntriesReply.class);
         assertEquals("isSuccess", true, reply.isSuccess());
     }
+
+    private static class TestFollower extends Follower {
+
+        int electionTimeoutCount = 0;
+
+        public TestFollower(RaftActorContext context) {
+            super(context);
+        }
+
+        @Override
+        protected void scheduleElection(FiniteDuration interval) {
+            electionTimeoutCount++;
+            super.scheduleElection(interval);
+        }
+
+        public int getElectionTimeoutCount() {
+            return electionTimeoutCount;
+        }
+    }
 }