Remove MockReplicatedLogEntry
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / behaviors / CandidateTest.java
1 /*
2  * Copyright (c) 2014, 2015 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8
9 package org.opendaylight.controller.cluster.raft.behaviors;
10
11 import static org.junit.Assert.assertEquals;
12 import static org.junit.Assert.assertTrue;
13
14 import akka.actor.ActorRef;
15 import akka.actor.Props;
16 import akka.testkit.TestActorRef;
17 import com.google.common.base.Stopwatch;
18 import java.util.ArrayList;
19 import java.util.Collections;
20 import java.util.HashMap;
21 import java.util.List;
22 import java.util.Map;
23 import java.util.concurrent.TimeUnit;
24 import org.junit.After;
25 import org.junit.Before;
26 import org.junit.Test;
27 import org.mockito.Mockito;
28 import org.opendaylight.controller.cluster.NonPersistentDataProvider;
29 import org.opendaylight.controller.cluster.raft.DefaultConfigParamsImpl;
30 import org.opendaylight.controller.cluster.raft.ElectionTerm;
31 import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
32 import org.opendaylight.controller.cluster.raft.RaftActorContext;
33 import org.opendaylight.controller.cluster.raft.RaftActorContextImpl;
34 import org.opendaylight.controller.cluster.raft.RaftState;
35 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
36 import org.opendaylight.controller.cluster.raft.VotingState;
37 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
38 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
39 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
40 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
41 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
42 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
43 import org.opendaylight.controller.cluster.raft.persisted.SimpleReplicatedLogEntry;
44 import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
45 import org.slf4j.Logger;
46 import org.slf4j.LoggerFactory;
47
48 public class CandidateTest extends AbstractRaftActorBehaviorTest<Candidate> {
49     static final Logger LOG = LoggerFactory.getLogger(CandidateTest.class);
50
51     private final TestActorRef<MessageCollectorActor> candidateActor = actorFactory.createTestActor(
52             Props.create(MessageCollectorActor.class), actorFactory.generateActorId("candidate"));
53
54     private TestActorRef<MessageCollectorActor>[] peerActors;
55
56     private RaftActorBehavior candidate;
57
58     @Before
59     public void setUp(){
60     }
61
62     @Override
63     @After
64     public void tearDown() throws Exception {
65         if (candidate != null) {
66             candidate.close();
67         }
68
69         super.tearDown();
70     }
71
72     @Test
73     public void testWhenACandidateIsCreatedItIncrementsTheCurrentTermAndVotesForItself() {
74         RaftActorContext raftActorContext = createActorContext();
75         long expectedTerm = raftActorContext.getTermInformation().getCurrentTerm();
76
77         candidate = new Candidate(raftActorContext);
78
79         assertEquals("getCurrentTerm", expectedTerm + 1, raftActorContext.getTermInformation().getCurrentTerm());
80         assertEquals("getVotedFor", raftActorContext.getId(), raftActorContext.getTermInformation().getVotedFor());
81     }
82
83     @Test
84     public void testThatAnElectionTimeoutIsTriggered() {
85         MockRaftActorContext actorContext = createActorContext();
86         candidate = new Candidate(actorContext);
87
88         MessageCollectorActor.expectFirstMatching(candidateActor, ElectionTimeout.class,
89                 actorContext.getConfigParams().getElectionTimeOutInterval().$times(6).toMillis());
90     }
91
92     @Test
93     public void testHandleElectionTimeoutWhenThereAreZeroPeers() {
94         RaftActorContext raftActorContext = createActorContext();
95         candidate = new Candidate(raftActorContext);
96
97         RaftActorBehavior newBehavior =
98             candidate.handleMessage(candidateActor, ElectionTimeout.INSTANCE);
99
100         assertEquals("Behavior", RaftState.Leader, newBehavior.state());
101     }
102
103     @Test
104     public void testHandleElectionTimeoutWhenThereAreTwoNodeCluster() {
105         MockRaftActorContext raftActorContext = createActorContext();
106         raftActorContext.setPeerAddresses(setupPeers(1));
107         candidate = new Candidate(raftActorContext);
108
109         candidate = candidate.handleMessage(candidateActor, ElectionTimeout.INSTANCE);
110
111         assertEquals("Behavior", RaftState.Candidate, candidate.state());
112     }
113
114     @Test
115     public void testBecomeLeaderOnReceivingMajorityVotesInThreeNodeCluster() {
116         MockRaftActorContext raftActorContext = createActorContext();
117         raftActorContext.setLastApplied(raftActorContext.getReplicatedLog().lastIndex());
118         raftActorContext.setPeerAddresses(setupPeers(2));
119         candidate = new Candidate(raftActorContext);
120
121         candidate = candidate.handleMessage(peerActors[0], new RequestVoteReply(1, true));
122
123         assertEquals("Behavior", RaftState.Leader, candidate.state());
124     }
125
126     @Test
127     public void testBecomePreLeaderOnReceivingMajorityVotesInThreeNodeCluster() {
128         MockRaftActorContext raftActorContext = createActorContext();
129         raftActorContext.setLastApplied(-1);
130         raftActorContext.setPeerAddresses(setupPeers(2));
131         candidate = new Candidate(raftActorContext);
132
133         candidate = candidate.handleMessage(peerActors[0], new RequestVoteReply(1, true));
134
135         // LastApplied is -1 and behind the last index.
136         assertEquals("Behavior", RaftState.PreLeader, candidate.state());
137     }
138
139     @Test
140     public void testBecomeLeaderOnReceivingMajorityVotesInFiveNodeCluster() {
141         MockRaftActorContext raftActorContext = createActorContext();
142         raftActorContext.getTermInformation().update(2L, "other");
143         raftActorContext.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder()
144                 .createEntries(0, 5, 1).build());
145         raftActorContext.setCommitIndex(raftActorContext.getReplicatedLog().lastIndex());
146         raftActorContext.setLastApplied(raftActorContext.getReplicatedLog().lastIndex());
147         raftActorContext.setPeerAddresses(setupPeers(4));
148         candidate = new Candidate(raftActorContext);
149
150         RequestVote requestVote = MessageCollectorActor.expectFirstMatching(peerActors[0], RequestVote.class);
151         assertEquals("getTerm", 3L, requestVote.getTerm());
152         assertEquals("getCandidateId", raftActorContext.getId(), requestVote.getCandidateId());
153         assertEquals("getLastLogTerm", 1L, requestVote.getLastLogTerm());
154         assertEquals("getLastLogIndex", 4L, requestVote.getLastLogIndex());
155
156         MessageCollectorActor.expectFirstMatching(peerActors[1], RequestVote.class);
157         MessageCollectorActor.expectFirstMatching(peerActors[2], RequestVote.class);
158         MessageCollectorActor.expectFirstMatching(peerActors[3], RequestVote.class);
159
160         // First peers denies the vote.
161         candidate = candidate.handleMessage(peerActors[0], new RequestVoteReply(1, false));
162
163         assertEquals("Behavior", RaftState.Candidate, candidate.state());
164
165         candidate = candidate.handleMessage(peerActors[1], new RequestVoteReply(1, true));
166
167         assertEquals("Behavior", RaftState.Candidate, candidate.state());
168
169         candidate = candidate.handleMessage(peerActors[2], new RequestVoteReply(1, true));
170
171         assertEquals("Behavior", RaftState.Leader, candidate.state());
172     }
173
174     @Test
175     public void testBecomeLeaderOnReceivingMajorityVotesWithNonVotingPeers() {
176         ElectionTerm mockElectionTerm = Mockito.mock(ElectionTerm.class);
177         Mockito.doReturn(1L).when(mockElectionTerm).getCurrentTerm();
178         RaftActorContext raftActorContext = new RaftActorContextImpl(candidateActor, candidateActor.actorContext(),
179                 "candidate", mockElectionTerm, -1, -1, setupPeers(4), new DefaultConfigParamsImpl(),
180                 new NonPersistentDataProvider(), LOG);
181         raftActorContext.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().build());
182         raftActorContext.getPeerInfo("peer1").setVotingState(VotingState.NON_VOTING);
183         raftActorContext.getPeerInfo("peer4").setVotingState(VotingState.NON_VOTING);
184         candidate = new Candidate(raftActorContext);
185
186         MessageCollectorActor.expectFirstMatching(peerActors[1], RequestVote.class);
187         MessageCollectorActor.expectFirstMatching(peerActors[2], RequestVote.class);
188         MessageCollectorActor.assertNoneMatching(peerActors[0], RequestVote.class, 300);
189         MessageCollectorActor.assertNoneMatching(peerActors[3], RequestVote.class, 100);
190
191         candidate = candidate.handleMessage(peerActors[1], new RequestVoteReply(1, false));
192
193         assertEquals("Behavior", RaftState.Candidate, candidate.state());
194
195         candidate = candidate.handleMessage(peerActors[2], new RequestVoteReply(1, true));
196
197         assertEquals("Behavior", RaftState.Leader, candidate.state());
198     }
199
200     @Test
201     public void testResponseToHandleAppendEntriesWithLowerTerm() {
202         candidate = new Candidate(createActorContext());
203
204         setupPeers(1);
205         RaftActorBehavior newBehavior = candidate.handleMessage(peerActors[0], new AppendEntries(1, "test", 0, 0,
206                 Collections.<ReplicatedLogEntry>emptyList(), 0, -1, (short) 0));
207
208         AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(
209                 peerActors[0], AppendEntriesReply.class);
210         assertEquals("isSuccess", false, reply.isSuccess());
211         assertEquals("getTerm", 2, reply.getTerm());
212         assertTrue("New Behavior : " + newBehavior, newBehavior instanceof Candidate);
213     }
214
215     @Test
216     public void testResponseToHandleAppendEntriesWithHigherTerm() {
217         candidate = new Candidate(createActorContext());
218
219         setupPeers(1);
220         RaftActorBehavior newBehavior = candidate.handleMessage(peerActors[0], new AppendEntries(5, "test", 0, 0,
221                 Collections.<ReplicatedLogEntry>emptyList(), 0, -1, (short) 0));
222
223         assertTrue("New Behavior : " + newBehavior, newBehavior instanceof Follower);
224     }
225
226     @Test
227     public void testResponseToHandleAppendEntriesWithEqualTerm() {
228         MockRaftActorContext actorContext = createActorContext();
229
230         candidate = new Candidate(actorContext);
231
232         setupPeers(1);
233         RaftActorBehavior newBehavior = candidate.handleMessage(peerActors[0], new AppendEntries(2, "test", 0, 0,
234                 Collections.<ReplicatedLogEntry>emptyList(), 0, -1, (short) 0));
235
236         assertTrue("New Behavior : " + newBehavior + " term = " + actorContext.getTermInformation().getCurrentTerm(),
237                 newBehavior instanceof Follower);
238     }
239
240
241     @Test
242     public void testResponseToRequestVoteWithLowerTerm() {
243         candidate = new Candidate(createActorContext());
244
245         setupPeers(1);
246         candidate.handleMessage(peerActors[0], new RequestVote(1, "test", 0, 0));
247
248         RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(
249                 peerActors[0], RequestVoteReply.class);
250         assertEquals("isVoteGranted", false, reply.isVoteGranted());
251         assertEquals("getTerm", 2, reply.getTerm());
252     }
253
254     @Test
255     public void testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForMatches() {
256         MockRaftActorContext context = createActorContext();
257         context.getTermInformation().update(1000, null);
258
259         // Once a candidate is created it will immediately increment the current term so after
260         // construction the currentTerm should be 1001
261         candidate = new Candidate(context);
262
263         setupPeers(1);
264         candidate.handleMessage(peerActors[0], new RequestVote(1001, context.getId(), 10000, 999));
265
266         RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(
267                 peerActors[0], RequestVoteReply.class);
268         assertEquals("isVoteGranted", true, reply.isVoteGranted());
269         assertEquals("getTerm", 1001, reply.getTerm());
270     }
271
272     @Test
273     public void testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForDoesNotMatch() {
274         MockRaftActorContext context = createActorContext();
275         context.getTermInformation().update(1000, null);
276
277         // Once a candidate is created it will immediately increment the current term so after
278         // construction the currentTerm should be 1001
279         candidate = new Candidate(context);
280
281         setupPeers(1);
282
283         // RequestVote candidate ID ("candidate2") does not match this candidate's votedFor
284         // (it votes for itself)
285         candidate.handleMessage(peerActors[0], new RequestVote(1001, "candidate2", 10000, 999));
286
287         RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(
288                 peerActors[0], RequestVoteReply.class);
289         assertEquals("isVoteGranted", false, reply.isVoteGranted());
290         assertEquals("getTerm", 1001, reply.getTerm());
291     }
292
293     @Test
294     public void testCandidateSchedulesElectionTimeoutImmediatelyWhenItHasNoPeers() {
295         MockRaftActorContext context = createActorContext();
296
297         Stopwatch stopwatch = Stopwatch.createStarted();
298
299         candidate = createBehavior(context);
300
301         MessageCollectorActor.expectFirstMatching(candidateActor, ElectionTimeout.class);
302
303         long elapsed = stopwatch.elapsed(TimeUnit.MILLISECONDS);
304
305         assertTrue(elapsed < context.getConfigParams().getElectionTimeOutInterval().toMillis());
306     }
307
308     @Test
309     @Override
310     public void testHandleAppendEntriesAddSameEntryToLog() throws Exception {
311         MockRaftActorContext context = createActorContext();
312
313         context.getTermInformation().update(2, "test");
314
315         // Prepare the receivers log
316         MockRaftActorContext.MockPayload payload = new MockRaftActorContext.MockPayload("zero");
317         setLastLogEntry(context, 2, 0, payload);
318
319         List<ReplicatedLogEntry> entries = new ArrayList<>();
320         entries.add(new SimpleReplicatedLogEntry(0, 2, payload));
321
322         final AppendEntries appendEntries = new AppendEntries(2, "leader-1", -1, -1, entries, 2, -1, (short)0);
323
324         behavior = createBehavior(context);
325
326         // Resetting the Candidates term to make sure it will match
327         // the term sent by AppendEntries. If this was not done then
328         // the test will fail because the Candidate will assume that
329         // the message was sent to it from a lower term peer and will
330         // thus respond with a failure
331         context.getTermInformation().update(2, "test");
332
333         // Send an unknown message so that the state of the RaftActor remains unchanged
334         behavior.handleMessage(candidateActor, "unknown");
335
336         RaftActorBehavior raftBehavior = behavior.handleMessage(candidateActor, appendEntries);
337
338         assertEquals("Raft state", RaftState.Follower, raftBehavior.state());
339
340         assertEquals("ReplicatedLog size", 1, context.getReplicatedLog().size());
341
342         handleAppendEntriesAddSameEntryToLogReply(candidateActor);
343     }
344
345     @Override
346     protected Candidate createBehavior(final RaftActorContext actorContext) {
347         return new Candidate(actorContext);
348     }
349
350     @Override protected MockRaftActorContext createActorContext() {
351         return new MockRaftActorContext("candidate", getSystem(), candidateActor);
352     }
353
354     @SuppressWarnings("unchecked")
355     private Map<String, String> setupPeers(final int count) {
356         Map<String, String> peerMap = new HashMap<>();
357         peerActors = new TestActorRef[count];
358         for (int i = 0; i < count; i++) {
359             peerActors[i] = actorFactory.createTestActor(Props.create(MessageCollectorActor.class),
360                     actorFactory.generateActorId("peer"));
361             peerMap.put("peer" + (i + 1), peerActors[i].path().toString());
362         }
363
364         return peerMap;
365     }
366
367     @Override
368     protected void assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(final MockRaftActorContext actorContext,
369             final ActorRef actorRef, final RaftRPC rpc) throws Exception {
370         super.assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(actorContext, actorRef, rpc);
371         if (rpc instanceof RequestVote) {
372             assertEquals("New votedFor", ((RequestVote)rpc).getCandidateId(),
373                     actorContext.getTermInformation().getVotedFor());
374         } else {
375             assertEquals("New votedFor", null, actorContext.getTermInformation().getVotedFor());
376         }
377     }
378 }