Revert "Add mockito-configuration to tests"
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / behaviors / LeaderTest.java
index 6664600073f2c727936ece4da93f6cd5b76bb290..b51f0a70b1dc7ef54da2f837b55354fdffaf02af 100644 (file)
@@ -11,6 +11,9 @@ package org.opendaylight.controller.cluster.raft.behaviors;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
 import akka.actor.ActorRef;
 import akka.actor.PoisonPill;
 import akka.actor.Props;
@@ -32,6 +35,7 @@ import org.opendaylight.controller.cluster.raft.DefaultConfigParamsImpl;
 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
 import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
+import org.opendaylight.controller.cluster.raft.RaftActorLeadershipTransferCohort;
 import org.opendaylight.controller.cluster.raft.RaftState;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogImplEntry;
@@ -41,6 +45,7 @@ import org.opendaylight.controller.cluster.raft.VotingState;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.base.messages.IsolatedLeaderCheck;
 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
@@ -1943,6 +1948,152 @@ public class LeaderTest extends AbstractLeaderTest {
         MessageCollectorActor.expectFirstMatching(leaderActor, ApplyState.class);
     }
 
+    @Test
+    public void testTransferLeadershipWithFollowerInSync() {
+        logStart("testTransferLeadershipWithFollowerInSync");
+
+        MockRaftActorContext leaderActorContext = createActorContextWithFollower();
+        ((DefaultConfigParamsImpl)leaderActorContext.getConfigParams()).setHeartBeatInterval(
+                new FiniteDuration(1000, TimeUnit.SECONDS));
+        leaderActorContext.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().build());
+
+        leader = new Leader(leaderActorContext);
+
+        // Initial heartbeat
+        MessageCollectorActor.expectFirstMatching(followerActor, AppendEntries.class);
+        leader.handleMessage(leaderActor, new AppendEntriesReply(FOLLOWER_ID, 1, true, -1, -1, (short)0));
+        MessageCollectorActor.clearMessages(followerActor);
+
+        sendReplicate(leaderActorContext, 0);
+        MessageCollectorActor.expectFirstMatching(followerActor, AppendEntries.class);
+
+        leader.handleMessage(leaderActor, new AppendEntriesReply(FOLLOWER_ID, 1, true, 0, 1, (short)0));
+        MessageCollectorActor.expectFirstMatching(leaderActor, ApplyState.class);
+        MessageCollectorActor.clearMessages(followerActor);
+
+        RaftActorLeadershipTransferCohort mockTransferCohort = mock(RaftActorLeadershipTransferCohort.class);
+        leader.transferLeadership(mockTransferCohort);
+
+        verify(mockTransferCohort, never()).transferComplete();
+        MessageCollectorActor.expectFirstMatching(followerActor, AppendEntries.class);
+        leader.handleMessage(leaderActor, new AppendEntriesReply(FOLLOWER_ID, 1, true, 0, 1, (short)0));
+
+        // Expect a final AppendEntries to ensure the follower's lastApplied index is up-to-date
+        MessageCollectorActor.expectMatching(followerActor, AppendEntries.class, 2);
+
+        // Leader should force an election timeout
+        MessageCollectorActor.expectFirstMatching(followerActor, ElectionTimeout.class);
+
+        verify(mockTransferCohort).transferComplete();
+    }
+
+    @Test
+    public void testTransferLeadershipWithEmptyLog() {
+        logStart("testTransferLeadershipWithEmptyLog");
+
+        MockRaftActorContext leaderActorContext = createActorContextWithFollower();
+        ((DefaultConfigParamsImpl)leaderActorContext.getConfigParams()).setHeartBeatInterval(
+                new FiniteDuration(1000, TimeUnit.SECONDS));
+        leaderActorContext.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().build());
+
+        leader = new Leader(leaderActorContext);
+
+        // Initial heartbeat
+        MessageCollectorActor.expectFirstMatching(followerActor, AppendEntries.class);
+        leader.handleMessage(leaderActor, new AppendEntriesReply(FOLLOWER_ID, 1, true, -1, -1, (short)0));
+        MessageCollectorActor.clearMessages(followerActor);
+
+        RaftActorLeadershipTransferCohort mockTransferCohort = mock(RaftActorLeadershipTransferCohort.class);
+        leader.transferLeadership(mockTransferCohort);
+
+        verify(mockTransferCohort, never()).transferComplete();
+        MessageCollectorActor.expectFirstMatching(followerActor, AppendEntries.class);
+        leader.handleMessage(leaderActor, new AppendEntriesReply(FOLLOWER_ID, 1, true, -1, -1, (short)0));
+
+        // Expect a final AppendEntries to ensure the follower's lastApplied index is up-to-date
+        MessageCollectorActor.expectFirstMatching(followerActor, AppendEntries.class);
+
+        // Leader should force an election timeout
+        MessageCollectorActor.expectFirstMatching(followerActor, ElectionTimeout.class);
+
+        verify(mockTransferCohort).transferComplete();
+    }
+
+    @Test
+    public void testTransferLeadershipWithFollowerInitiallyOutOfSync() {
+        logStart("testTransferLeadershipWithFollowerInitiallyOutOfSync");
+
+        MockRaftActorContext leaderActorContext = createActorContextWithFollower();
+        ((DefaultConfigParamsImpl)leaderActorContext.getConfigParams()).setHeartBeatInterval(
+                new FiniteDuration(200, TimeUnit.MILLISECONDS));
+
+        leader = new Leader(leaderActorContext);
+
+        // Initial heartbeat
+        MessageCollectorActor.expectFirstMatching(followerActor, AppendEntries.class);
+        MessageCollectorActor.clearMessages(followerActor);
+
+        RaftActorLeadershipTransferCohort mockTransferCohort = mock(RaftActorLeadershipTransferCohort.class);
+        leader.transferLeadership(mockTransferCohort);
+
+        verify(mockTransferCohort, never()).transferComplete();
+
+        // Sync up the follower.
+        MessageCollectorActor.expectFirstMatching(followerActor, AppendEntries.class);
+        leader.handleMessage(leaderActor, new AppendEntriesReply(FOLLOWER_ID, 1, true, -1, -1, (short)0));
+        MessageCollectorActor.clearMessages(followerActor);
+
+        Uninterruptibles.sleepUninterruptibly(leaderActorContext.getConfigParams().
+                getHeartBeatInterval().toMillis() + 1, TimeUnit.MILLISECONDS);
+        leader.handleMessage(leaderActor, new SendHeartBeat());
+        MessageCollectorActor.expectFirstMatching(followerActor, AppendEntries.class);
+        leader.handleMessage(leaderActor, new AppendEntriesReply(FOLLOWER_ID, 1, true, 1, 1, (short)0));
+
+        // Leader should force an election timeout
+        MessageCollectorActor.expectFirstMatching(followerActor, ElectionTimeout.class);
+
+        verify(mockTransferCohort).transferComplete();
+    }
+
+    @Test
+    public void testTransferLeadershipWithFollowerSyncTimeout() {
+        logStart("testTransferLeadershipWithFollowerSyncTimeout");
+
+        MockRaftActorContext leaderActorContext = createActorContextWithFollower();
+        ((DefaultConfigParamsImpl)leaderActorContext.getConfigParams()).setHeartBeatInterval(
+                new FiniteDuration(200, TimeUnit.MILLISECONDS));
+        ((DefaultConfigParamsImpl)leaderActorContext.getConfigParams()).setElectionTimeoutFactor(2);
+        leaderActorContext.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().build());
+
+        leader = new Leader(leaderActorContext);
+
+        // Initial heartbeat
+        MessageCollectorActor.expectFirstMatching(followerActor, AppendEntries.class);
+        leader.handleMessage(leaderActor, new AppendEntriesReply(FOLLOWER_ID, 1, true, -1, -1, (short)0));
+        MessageCollectorActor.clearMessages(followerActor);
+
+        sendReplicate(leaderActorContext, 0);
+        MessageCollectorActor.expectFirstMatching(followerActor, AppendEntries.class);
+
+        MessageCollectorActor.clearMessages(followerActor);
+
+        RaftActorLeadershipTransferCohort mockTransferCohort = mock(RaftActorLeadershipTransferCohort.class);
+        leader.transferLeadership(mockTransferCohort);
+
+        verify(mockTransferCohort, never()).transferComplete();
+
+        // Send heartbeats to time out the transfer.
+        for(int i = 0; i < leaderActorContext.getConfigParams().getElectionTimeoutFactor(); i++) {
+            Uninterruptibles.sleepUninterruptibly(leaderActorContext.getConfigParams().
+                    getHeartBeatInterval().toMillis() + 1, TimeUnit.MILLISECONDS);
+            leader.handleMessage(leaderActor, new SendHeartBeat());
+        }
+
+        verify(mockTransferCohort).abortTransfer();
+        verify(mockTransferCohort, never()).transferComplete();
+        MessageCollectorActor.assertNoneMatching(followerActor, ElectionTimeout.class, 100);
+    }
+
     @Override
     protected void assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(RaftActorContext actorContext,
             ActorRef actorRef, RaftRPC rpc) throws Exception {