Merge "BUG 2773 : Transition Shard to Leader state when it has no peers"
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / behaviors / CandidateTest.java
1 package org.opendaylight.controller.cluster.raft.behaviors;
2
3 import static org.junit.Assert.assertEquals;
4 import static org.junit.Assert.assertTrue;
5 import akka.actor.ActorRef;
6 import akka.actor.Props;
7 import akka.testkit.TestActorRef;
8 import com.google.common.base.Stopwatch;
9 import java.util.Collections;
10 import java.util.HashMap;
11 import java.util.Map;
12 import java.util.concurrent.TimeUnit;
13 import org.junit.After;
14 import org.junit.Before;
15 import org.junit.Test;
16 import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
17 import org.opendaylight.controller.cluster.raft.RaftActorContext;
18 import org.opendaylight.controller.cluster.raft.RaftState;
19 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
20 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
21 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
22 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
23 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
24 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
25 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
26 import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
27
28 public class CandidateTest extends AbstractRaftActorBehaviorTest {
29
30     private final TestActorRef<MessageCollectorActor> candidateActor = actorFactory.createTestActor(
31             Props.create(MessageCollectorActor.class), actorFactory.generateActorId("candidate"));
32
33     private TestActorRef<MessageCollectorActor>[] peerActors;
34
35     private RaftActorBehavior candidate;
36
37     @Before
38     public void setUp(){
39     }
40
41     @Override
42     @After
43     public void tearDown() throws Exception {
44         if(candidate != null) {
45             candidate.close();
46         }
47
48         super.tearDown();
49     }
50
51     @Test
52     public void testWhenACandidateIsCreatedItIncrementsTheCurrentTermAndVotesForItself(){
53         RaftActorContext raftActorContext = createActorContext();
54         long expectedTerm = raftActorContext.getTermInformation().getCurrentTerm();
55
56         candidate = new Candidate(raftActorContext);
57
58         assertEquals("getCurrentTerm", expectedTerm+1, raftActorContext.getTermInformation().getCurrentTerm());
59         assertEquals("getVotedFor", raftActorContext.getId(), raftActorContext.getTermInformation().getVotedFor());
60     }
61
62     @Test
63     public void testThatAnElectionTimeoutIsTriggered(){
64          MockRaftActorContext actorContext = createActorContext();
65          candidate = new Candidate(actorContext);
66
67          MessageCollectorActor.expectFirstMatching(candidateActor, ElectionTimeout.class,
68                  actorContext.getConfigParams().getElectionTimeOutInterval().$times(6).toMillis());
69     }
70
71     @Test
72     public void testHandleElectionTimeoutWhenThereAreZeroPeers(){
73         RaftActorContext raftActorContext = createActorContext();
74         candidate = new Candidate(raftActorContext);
75
76         RaftActorBehavior newBehavior =
77             candidate.handleMessage(candidateActor, new ElectionTimeout());
78
79         assertEquals("Behavior", RaftState.Leader, newBehavior.state());
80     }
81
82     @Test
83     public void testHandleElectionTimeoutWhenThereAreTwoNodeCluster(){
84         MockRaftActorContext raftActorContext = createActorContext();
85         raftActorContext.setPeerAddresses(setupPeers(1));
86         candidate = new Candidate(raftActorContext);
87
88         candidate = candidate.handleMessage(candidateActor, new ElectionTimeout());
89
90         assertEquals("Behavior", RaftState.Candidate, candidate.state());
91     }
92
93     @Test
94     public void testBecomeLeaderOnReceivingMajorityVotesInThreeNodeCluster(){
95         MockRaftActorContext raftActorContext = createActorContext();
96         raftActorContext.setPeerAddresses(setupPeers(2));
97         candidate = new Candidate(raftActorContext);
98
99         candidate = candidate.handleMessage(peerActors[0], new RequestVoteReply(1, true));
100
101         assertEquals("Behavior", RaftState.Leader, candidate.state());
102     }
103
104     @Test
105     public void testBecomeLeaderOnReceivingMajorityVotesInFiveNodeCluster(){
106         MockRaftActorContext raftActorContext = createActorContext();
107         raftActorContext.setPeerAddresses(setupPeers(4));
108         candidate = new Candidate(raftActorContext);
109
110         // First peers denies the vote.
111         candidate = candidate.handleMessage(peerActors[0], new RequestVoteReply(1, false));
112
113         assertEquals("Behavior", RaftState.Candidate, candidate.state());
114
115         candidate = candidate.handleMessage(peerActors[1], new RequestVoteReply(1, true));
116
117         assertEquals("Behavior", RaftState.Candidate, candidate.state());
118
119         candidate = candidate.handleMessage(peerActors[2], new RequestVoteReply(1, true));
120
121         assertEquals("Behavior", RaftState.Leader, candidate.state());
122     }
123
124     @Test
125     public void testResponseToHandleAppendEntriesWithLowerTerm() {
126         candidate = new Candidate(createActorContext());
127
128         setupPeers(1);
129         candidate.handleMessage(peerActors[0], new AppendEntries(1, "test", 0, 0,
130                 Collections.<ReplicatedLogEntry>emptyList(), 0, -1));
131
132         AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(
133                 peerActors[0], AppendEntriesReply.class);
134         assertEquals("isSuccess", false, reply.isSuccess());
135         assertEquals("getTerm", 2, reply.getTerm());
136     }
137
138     @Test
139     public void testResponseToRequestVoteWithLowerTerm() {
140         candidate = new Candidate(createActorContext());
141
142         setupPeers(1);
143         candidate.handleMessage(peerActors[0], new RequestVote(1, "test", 0, 0));
144
145         RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(
146                 peerActors[0], RequestVoteReply.class);
147         assertEquals("isVoteGranted", false, reply.isVoteGranted());
148         assertEquals("getTerm", 2, reply.getTerm());
149     }
150
151     @Test
152     public void testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForMatches() {
153         MockRaftActorContext context = createActorContext();
154         context.getTermInformation().update(1000, null);
155
156         // Once a candidate is created it will immediately increment the current term so after
157         // construction the currentTerm should be 1001
158         candidate = new Candidate(context);
159
160         setupPeers(1);
161         candidate.handleMessage(peerActors[0], new RequestVote(1001, context.getId(), 10000, 999));
162
163         RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(
164                 peerActors[0], RequestVoteReply.class);
165         assertEquals("isVoteGranted", true, reply.isVoteGranted());
166         assertEquals("getTerm", 1001, reply.getTerm());
167     }
168
169     @Test
170     public void testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForDoesNotMatch() {
171         MockRaftActorContext context = createActorContext();
172         context.getTermInformation().update(1000, null);
173
174         // Once a candidate is created it will immediately increment the current term so after
175         // construction the currentTerm should be 1001
176         candidate = new Candidate(context);
177
178         setupPeers(1);
179
180         // RequestVote candidate ID ("candidate2") does not match this candidate's votedFor
181         // (it votes for itself)
182         candidate.handleMessage(peerActors[0], new RequestVote(1001, "candidate2", 10000, 999));
183
184         RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(
185                 peerActors[0], RequestVoteReply.class);
186         assertEquals("isVoteGranted", false, reply.isVoteGranted());
187         assertEquals("getTerm", 1001, reply.getTerm());
188     }
189
190     @Test
191     public void testCandidateSchedulesElectionTimeoutImmediatelyWhenItHasNoPeers(){
192         MockRaftActorContext context = createActorContext();
193
194         Stopwatch stopwatch = Stopwatch.createStarted();
195
196         candidate = createBehavior(context);
197
198         MessageCollectorActor.expectFirstMatching(candidateActor, ElectionTimeout.class);
199
200         long elapsed = stopwatch.elapsed(TimeUnit.MILLISECONDS);
201
202         assertTrue(elapsed < context.getConfigParams().getElectionTimeOutInterval().toMillis());
203     }
204
205
206     @Override
207     protected RaftActorBehavior createBehavior(RaftActorContext actorContext) {
208         return new Candidate(actorContext);
209     }
210
211     @Override protected MockRaftActorContext createActorContext() {
212         return new MockRaftActorContext("candidate", getSystem(), candidateActor);
213     }
214
215     private Map<String, String> setupPeers(int count) {
216         Map<String, String> peerMap = new HashMap<>();
217         peerActors = new TestActorRef[count];
218         for(int i = 0; i < count; i++) {
219             peerActors[i] = actorFactory.createTestActor(Props.create(MessageCollectorActor.class),
220                     actorFactory.generateActorId("peer"));
221             peerMap.put("peer" + (i+1), peerActors[i].path().toString());
222         }
223
224         return peerMap;
225     }
226
227     @Override
228     protected void assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(RaftActorContext actorContext,
229             ActorRef actorRef, RaftRPC rpc) throws Exception {
230         super.assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(actorContext, actorRef, rpc);
231         assertEquals("New votedFor", null, actorContext.getTermInformation().getVotedFor());
232     }
233 }