Refactor CandidateTest
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / behaviors / CandidateTest.java
index 8bcee58afe11b40c03bf8fa36eeb0b338a1ae3ff..9d323b6ab1e2213718a6d25ebbdec14cca5102e2 100644 (file)
@@ -1,68 +1,48 @@
 package org.opendaylight.controller.cluster.raft.behaviors;
 
-import akka.actor.ActorRef;
+import static org.junit.Assert.assertEquals;
 import akka.actor.Props;
-import akka.testkit.JavaTestKit;
-import junit.framework.Assert;
+import akka.testkit.TestActorRef;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
-import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout;
+import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
+import org.opendaylight.controller.cluster.raft.TestActorFactory;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
-import org.opendaylight.controller.cluster.raft.utils.DoNothingActor;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
+import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
 
 public class CandidateTest extends AbstractRaftActorBehaviorTest {
 
-    private final ActorRef candidateActor = getSystem().actorOf(Props.create(
-        DoNothingActor.class));
-
-    private final ActorRef peerActor1 = getSystem().actorOf(Props.create(
-        DoNothingActor.class));
-
-    private final ActorRef peerActor2 = getSystem().actorOf(Props.create(
-        DoNothingActor.class));
+    private final TestActorFactory actorFactory = new TestActorFactory(getSystem());
 
-    private final ActorRef peerActor3 = getSystem().actorOf(Props.create(
-        DoNothingActor.class));
+    private final TestActorRef<MessageCollectorActor> candidateActor = actorFactory.createTestActor(
+            Props.create(MessageCollectorActor.class), actorFactory.generateActorId("candidate"));
 
-    private final ActorRef peerActor4 = getSystem().actorOf(Props.create(
-        DoNothingActor.class));
+    private TestActorRef<MessageCollectorActor>[] peerActors;
 
-    private final Map<String, String> onePeer = new HashMap<>();
-    private final Map<String, String> twoPeers = new HashMap<>();
-    private final Map<String, String> fourPeers = new HashMap<>();
+    private RaftActorBehavior candidate;
 
     @Before
     public void setUp(){
-        onePeer.put(peerActor1.path().toString(),
-            peerActor1.path().toString());
-
-        twoPeers.put(peerActor1.path().toString(),
-            peerActor1.path().toString());
-        twoPeers.put(peerActor2.path().toString(),
-            peerActor2.path().toString());
-
-        fourPeers.put(peerActor1.path().toString(),
-            peerActor1.path().toString());
-        fourPeers.put(peerActor2.path().toString(),
-            peerActor2.path().toString());
-        fourPeers.put(peerActor3.path().toString(),
-            peerActor3.path().toString());
-        fourPeers.put(peerActor4.path().toString(),
-            peerActor3.path().toString());
+    }
 
+    @After
+    public void tearDown() throws Exception {
+        if(candidate != null) {
+            candidate.close();
+        }
 
+        actorFactory.close();
     }
 
     @Test
@@ -70,230 +50,160 @@ public class CandidateTest extends AbstractRaftActorBehaviorTest {
         RaftActorContext raftActorContext = createActorContext();
         long expectedTerm = raftActorContext.getTermInformation().getCurrentTerm();
 
-        new Candidate(raftActorContext);
+        candidate = new Candidate(raftActorContext);
 
-        assertEquals(expectedTerm+1, raftActorContext.getTermInformation().getCurrentTerm());
-        assertEquals(raftActorContext.getId(), raftActorContext.getTermInformation().getVotedFor());
+        assertEquals("getCurrentTerm", expectedTerm+1, raftActorContext.getTermInformation().getCurrentTerm());
+        assertEquals("getVotedFor", raftActorContext.getId(), raftActorContext.getTermInformation().getVotedFor());
     }
 
     @Test
     public void testThatAnElectionTimeoutIsTriggered(){
-        new JavaTestKit(getSystem()) {{
-
-            new Within(duration("1 seconds")) {
-                protected void run() {
-
-                    Candidate candidate = new Candidate(createActorContext(getTestActor()));
-
-                    final Boolean out = new ExpectMsg<Boolean>(duration("1 seconds"), "ElectionTimeout") {
-                        // do not put code outside this method, will run afterwards
-                        protected Boolean match(Object in) {
-                            if (in instanceof ElectionTimeout) {
-                                 return true;
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get();
-
-                    assertEquals(true, out);
-                }
-            };
-        }};
+         MockRaftActorContext actorContext = createActorContext();
+         candidate = new Candidate(actorContext);
+
+         MessageCollectorActor.expectFirstMatching(candidateActor, ElectionTimeout.class,
+                 actorContext.getConfigParams().getElectionTimeOutInterval().$times(6).toMillis());
     }
 
     @Test
     public void testHandleElectionTimeoutWhenThereAreZeroPeers(){
         RaftActorContext raftActorContext = createActorContext();
-        Candidate candidate =
-            new Candidate(raftActorContext);
+        candidate = new Candidate(raftActorContext);
 
-        RaftState raftState =
+        RaftActorBehavior newBehavior =
             candidate.handleMessage(candidateActor, new ElectionTimeout());
 
-        Assert.assertEquals(RaftState.Leader, raftState);
+        assertEquals("Behavior", RaftState.Leader, newBehavior.state());
     }
 
     @Test
-    public void testHandleElectionTimeoutWhenThereAreTwoNodesInCluster(){
-        MockRaftActorContext raftActorContext =
-            (MockRaftActorContext) createActorContext();
-        raftActorContext.setPeerAddresses(onePeer);
-        Candidate candidate =
-            new Candidate(raftActorContext);
-
-        RaftState raftState =
-            candidate.handleMessage(candidateActor, new ElectionTimeout());
+    public void testHandleElectionTimeoutWhenThereAreTwoNodeCluster(){
+        MockRaftActorContext raftActorContext = createActorContext();
+        raftActorContext.setPeerAddresses(setupPeers(1));
+        candidate = new Candidate(raftActorContext);
 
-        Assert.assertEquals(RaftState.Candidate, raftState);
+        candidate = candidate.handleMessage(candidateActor, new ElectionTimeout());
+
+        assertEquals("Behavior", RaftState.Candidate, candidate.state());
     }
 
     @Test
-    public void testBecomeLeaderOnReceivingMajorityVotesInThreeNodesInCluster(){
-        MockRaftActorContext raftActorContext =
-            (MockRaftActorContext) createActorContext();
-        raftActorContext.setPeerAddresses(twoPeers);
-        Candidate candidate =
-            new Candidate(raftActorContext);
-
-        RaftState stateOnFirstVote = candidate.handleMessage(peerActor1, new RequestVoteReply(0, true));
+    public void testBecomeLeaderOnReceivingMajorityVotesInThreeNodeCluster(){
+        MockRaftActorContext raftActorContext = createActorContext();
+        raftActorContext.setPeerAddresses(setupPeers(2));
+        candidate = new Candidate(raftActorContext);
 
-        Assert.assertEquals(RaftState.Leader, stateOnFirstVote);
+        candidate = candidate.handleMessage(peerActors[0], new RequestVoteReply(1, true));
 
+        assertEquals("Behavior", RaftState.Leader, candidate.state());
     }
 
     @Test
-    public void testBecomeLeaderOnReceivingMajorityVotesInFiveNodesInCluster(){
-        MockRaftActorContext raftActorContext =
-            (MockRaftActorContext) createActorContext();
-        raftActorContext.setPeerAddresses(fourPeers);
-        Candidate candidate =
-            new Candidate(raftActorContext);
+    public void testBecomeLeaderOnReceivingMajorityVotesInFiveNodeCluster(){
+        MockRaftActorContext raftActorContext = createActorContext();
+        raftActorContext.setPeerAddresses(setupPeers(4));
+        candidate = new Candidate(raftActorContext);
 
-        RaftState stateOnFirstVote = candidate.handleMessage(peerActor1, new RequestVoteReply(0, true));
+        // First peers denies the vote.
+        candidate = candidate.handleMessage(peerActors[0], new RequestVoteReply(1, false));
 
-        RaftState stateOnSecondVote = candidate.handleMessage(peerActor2, new RequestVoteReply(0, true));
+        assertEquals("Behavior", RaftState.Candidate, candidate.state());
 
-        Assert.assertEquals(RaftState.Candidate, stateOnFirstVote);
-        Assert.assertEquals(RaftState.Leader, stateOnSecondVote);
+        candidate = candidate.handleMessage(peerActors[1], new RequestVoteReply(1, true));
 
-    }
+        assertEquals("Behavior", RaftState.Candidate, candidate.state());
 
-    @Test
-    public void testResponseToAppendEntriesWithLowerTerm(){
-        new JavaTestKit(getSystem()) {{
-
-            new Within(duration("1 seconds")) {
-                protected void run() {
-
-                    Candidate candidate = new Candidate(createActorContext(getTestActor()));
-
-                    candidate.handleMessage(getTestActor(), new AppendEntries(0, "test", 0,0,Collections.EMPTY_LIST, 0));
-
-                    final Boolean out = new ExpectMsg<Boolean>(duration("1 seconds"), "AppendEntriesResponse") {
-                        // do not put code outside this method, will run afterwards
-                        protected Boolean match(Object in) {
-                            if (in instanceof AppendEntriesReply) {
-                                AppendEntriesReply reply = (AppendEntriesReply) in;
-                                return reply.isSuccess();
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get();
-
-                    assertEquals(false, out);
-                }
-            };
-        }};
+        candidate = candidate.handleMessage(peerActors[2], new RequestVoteReply(1, true));
+
+        assertEquals("Behavior", RaftState.Leader, candidate.state());
     }
 
     @Test
-    public void testResponseToRequestVoteWithLowerTerm(){
-        new JavaTestKit(getSystem()) {{
-
-            new Within(duration("1 seconds")) {
-                protected void run() {
-
-                    Candidate candidate = new Candidate(createActorContext(getTestActor()));
-
-                    candidate.handleMessage(getTestActor(), new RequestVote(0, "test", 0, 0));
-
-                    final Boolean out = new ExpectMsg<Boolean>(duration("1 seconds"), "AppendEntriesResponse") {
-                        // do not put code outside this method, will run afterwards
-                        protected Boolean match(Object in) {
-                            if (in instanceof RequestVoteReply) {
-                                RequestVoteReply reply = (RequestVoteReply) in;
-                                return reply.isVoteGranted();
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get();
-
-                    assertEquals(false, out);
-                }
-            };
-        }};
+    public void testResponseToHandleAppendEntriesWithLowerTerm() {
+        candidate = new Candidate(createActorContext());
+
+        setupPeers(1);
+        candidate.handleMessage(peerActors[0], new AppendEntries(1, "test", 0, 0,
+                Collections.<ReplicatedLogEntry>emptyList(), 0, -1));
+
+        AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(
+                peerActors[0], AppendEntriesReply.class);
+        assertEquals("isSuccess", false, reply.isSuccess());
+        assertEquals("getTerm", 2, reply.getTerm());
     }
 
     @Test
-    public void testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForIsNull(){
-        new JavaTestKit(getSystem()) {{
-
-            new Within(duration("1 seconds")) {
-                protected void run() {
-
-                    RaftActorContext context = createActorContext(getTestActor());
-
-                    context.getTermInformation().update(1000, null);
-
-                    // Once a candidate is created it will immediately increment the current term so after
-                    // construction the currentTerm should be 1001
-                    RaftActorBehavior follower = createBehavior(context);
-
-                    follower.handleMessage(getTestActor(), new RequestVote(1001, "test", 10000, 999));
-
-                    final Boolean out = new ExpectMsg<Boolean>(duration("1 seconds"), "RequestVoteReply") {
-                        // do not put code outside this method, will run afterwards
-                        protected Boolean match(Object in) {
-                            if (in instanceof RequestVoteReply) {
-                                RequestVoteReply reply = (RequestVoteReply) in;
-                                return reply.isVoteGranted();
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get();
-
-                    assertEquals(true, out);
-                }
-            };
-        }};
+    public void testResponseToRequestVoteWithLowerTerm() {
+        candidate = new Candidate(createActorContext());
+
+        setupPeers(1);
+        candidate.handleMessage(peerActors[0], new RequestVote(1, "test", 0, 0));
+
+        RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(
+                peerActors[0], RequestVoteReply.class);
+        assertEquals("isVoteGranted", false, reply.isVoteGranted());
+        assertEquals("getTerm", 2, reply.getTerm());
     }
 
     @Test
-    public void testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForIsNotTheSameAsCandidateId(){
-        new JavaTestKit(getSystem()) {{
+    public void testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForMatches() {
+        MockRaftActorContext context = createActorContext();
+        context.getTermInformation().update(1000, null);
 
-            new Within(duration("1 seconds")) {
-                protected void run() {
+        // Once a candidate is created it will immediately increment the current term so after
+        // construction the currentTerm should be 1001
+        candidate = new Candidate(context);
 
-                    RaftActorContext context = createActorContext(getTestActor());
+        setupPeers(1);
+        candidate.handleMessage(peerActors[0], new RequestVote(1001, context.getId(), 10000, 999));
 
-                    context.getTermInformation().update(1000, "test");
+        RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(
+                peerActors[0], RequestVoteReply.class);
+        assertEquals("isVoteGranted", true, reply.isVoteGranted());
+        assertEquals("getTerm", 1001, reply.getTerm());
+    }
 
-                    RaftActorBehavior follower = createBehavior(context);
+    @Test
+    public void testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForDoesNotMatch() {
+        MockRaftActorContext context = createActorContext();
+        context.getTermInformation().update(1000, null);
+
+        // Once a candidate is created it will immediately increment the current term so after
+        // construction the currentTerm should be 1001
+        candidate = new Candidate(context);
 
-                    follower.handleMessage(getTestActor(), new RequestVote(1001, "candidate", 10000, 999));
+        setupPeers(1);
 
-                    final Boolean out = new ExpectMsg<Boolean>(duration("1 seconds"), "RequestVoteReply") {
-                        // do not put code outside this method, will run afterwards
-                        protected Boolean match(Object in) {
-                            if (in instanceof RequestVoteReply) {
-                                RequestVoteReply reply = (RequestVoteReply) in;
-                                return reply.isVoteGranted();
-                            } else {
-                                throw noMatch();
-                            }
-                        }
-                    }.get();
+        // RequestVote candidate ID ("candidate2") does not match this candidate's votedFor
+        // (it votes for itself)
+        candidate.handleMessage(peerActors[0], new RequestVote(1001, "candidate2", 10000, 999));
 
-                    assertEquals(false, out);
-                }
-            };
-        }};
+        RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(
+                peerActors[0], RequestVoteReply.class);
+        assertEquals("isVoteGranted", false, reply.isVoteGranted());
+        assertEquals("getTerm", 1001, reply.getTerm());
     }
 
 
 
-    @Override protected RaftActorBehavior createBehavior(RaftActorContext actorContext) {
+    @Override
+    protected RaftActorBehavior createBehavior(RaftActorContext actorContext) {
         return new Candidate(actorContext);
     }
 
-    @Override protected RaftActorContext createActorContext() {
-        return new MockRaftActorContext("test", getSystem(), candidateActor);
+    @Override protected MockRaftActorContext createActorContext() {
+        return new MockRaftActorContext("candidate", getSystem(), candidateActor);
     }
 
+    private Map<String, String> setupPeers(int count) {
+        Map<String, String> peerMap = new HashMap<>();
+        peerActors = new TestActorRef[count];
+        for(int i = 0; i < count; i++) {
+            peerActors[i] = actorFactory.createTestActor(Props.create(MessageCollectorActor.class),
+                    actorFactory.generateActorId("peer"));
+            peerMap.put("peer" + (i+1), peerActors[i].path().toString());
+        }
 
+        return peerMap;
+    }
 }