Cleanup warnings
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / behaviors / PartitionedCandidateOnStartupElectionScenarioTest.java
1 /*
2  * Copyright (c) 2015 Brocade Communications 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 package org.opendaylight.controller.cluster.raft.behaviors;
9
10 import static org.junit.Assert.assertEquals;
11
12 import akka.actor.ActorRef;
13 import com.google.common.collect.ImmutableMap;
14 import org.junit.Test;
15 import org.opendaylight.controller.cluster.raft.DefaultConfigParamsImpl;
16 import org.opendaylight.controller.cluster.raft.MockRaftActorContext.MockPayload;
17 import org.opendaylight.controller.cluster.raft.MockRaftActorContext.SimpleReplicatedLog;
18 import org.opendaylight.controller.cluster.raft.RaftState;
19 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
20 import org.opendaylight.controller.cluster.raft.base.messages.TimeoutNow;
21 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
22 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
23 import org.opendaylight.controller.cluster.raft.persisted.SimpleReplicatedLogEntry;
24
25 /**
26  * A leader election scenario test that partitions a candidate when trying to join a cluster on startup.
27  *
28  * @author Thomas Pantelis
29  */
30 public class PartitionedCandidateOnStartupElectionScenarioTest extends AbstractLeaderElectionScenarioTest {
31
32     private final int numCandidateElections = 5;
33     private long candidateElectionTerm;
34
35     @Test
36     public void runTest() {
37         testLog.info("PartitionedCandidateOnStartupElectionScenarioTest starting");
38
39         setupInitialMember1AndMember2Behaviors();
40
41         setupPartitionedCandidateMember3AndSendElectionTimeouts();
42
43         resolvePartitionAndSendElectionTimeoutsToCandidateMember3();
44
45         sendElectionTimeoutToFollowerMember1();
46
47         testLog.info("PartitionedCandidateOnStartupElectionScenarioTest ending");
48     }
49
50     private void sendElectionTimeoutToFollowerMember1() {
51         testLog.info("sendElectionTimeoutToFollowerMember1 starting");
52
53         // At this point we have no leader. Candidate member 3 would continue to start new elections
54         // but wouldn't be granted a vote. One of the 2 followers would eventually time out from
55         // not having received a heartbeat from a leader and switch to candidate and start a new
56         // election. We'll simulate that here by sending an ElectionTimeout to member 1.
57
58         member1Actor.clear();
59         member1Actor.expectMessageClass(RequestVoteReply.class, 1);
60         member2Actor.clear();
61         member2Actor.expectMessageClass(RequestVote.class, 1);
62         member3Actor.clear();
63         member3Actor.expectMessageClass(RequestVote.class, 1);
64         member3Actor.expectBehaviorStateChange();
65
66         member1ActorRef.tell(TimeoutNow.INSTANCE, ActorRef.noSender());
67
68         member2Actor.waitForExpectedMessages(RequestVote.class);
69         member3Actor.waitForExpectedMessages(RequestVote.class);
70
71         // The RequestVoteReply should come from Follower member 2 and the vote should be granted
72         // since member 2's last term and index matches member 1's.
73
74         member1Actor.waitForExpectedMessages(RequestVoteReply.class);
75
76         RequestVoteReply requestVoteReply = member1Actor.getCapturedMessage(RequestVoteReply.class);
77         assertEquals("getTerm", member1Context.getTermInformation().getCurrentTerm(), requestVoteReply.getTerm());
78         assertEquals("isVoteGranted", true, requestVoteReply.isVoteGranted());
79
80         // Candidate member 3 should change to follower as its term should be less than the
81         // RequestVote term (member 1 started a new term higher than the other member's terms).
82
83         member3Actor.waitForBehaviorStateChange();
84
85         verifyBehaviorState("member 1", member1Actor, RaftState.Leader);
86         verifyBehaviorState("member 2", member2Actor, RaftState.Follower);
87         verifyBehaviorState("member 3", member3Actor, RaftState.Follower);
88
89         // newTerm should be 10.
90
91         long newTerm = candidateElectionTerm + 1;
92         assertEquals("member 1 election term", newTerm, member1Context.getTermInformation().getCurrentTerm());
93         assertEquals("member 2 election term", newTerm, member2Context.getTermInformation().getCurrentTerm());
94         assertEquals("member 3 election term", newTerm, member3Context.getTermInformation().getCurrentTerm());
95
96         testLog.info("sendElectionTimeoutToFollowerMember1 ending");
97     }
98
99     private void resolvePartitionAndSendElectionTimeoutsToCandidateMember3() {
100         testLog.info("resolvePartitionAndSendElectionTimeoutsToCandidateMember3 starting");
101
102         // Now send a couple more ElectionTimeouts to Candidate member 3 with the partition resolved.
103         //
104         // On the first RequestVote, Leader member 1 should switch to Follower as its term (s) is less than
105         // the RequestVote's term (8) from member 3. No RequestVoteReply should be sent by member 1.
106         // Follower member 2 should update its term since it less than the RequestVote's term and
107         // should return a RequestVoteReply but should not grant the vote as its last term and index
108         // is greater than the RequestVote's lastLogTerm and lastLogIndex, ie member 2's log is later
109         // or more up to date than member 3's.
110         //
111         // On the second RequestVote, both member 1 and 2 are followers so they should update their
112         // term and return a RequestVoteReply but should not grant the vote.
113
114         candidateElectionTerm += 2;
115         for (int i = 0; i < 2; i++) {
116             member1Actor.clear();
117             member1Actor.expectMessageClass(RequestVote.class, 1);
118             member2Actor.clear();
119             member2Actor.expectMessageClass(RequestVote.class, 1);
120             member3Actor.clear();
121             member3Actor.expectMessageClass(RequestVoteReply.class, 1);
122
123             member3ActorRef.tell(ElectionTimeout.INSTANCE, ActorRef.noSender());
124
125             member1Actor.waitForExpectedMessages(RequestVote.class);
126             member2Actor.waitForExpectedMessages(RequestVote.class);
127
128             member3Actor.waitForExpectedMessages(RequestVoteReply.class);
129
130             RequestVoteReply requestVoteReply = member3Actor.getCapturedMessage(RequestVoteReply.class);
131             assertEquals("getTerm", member3Context.getTermInformation().getCurrentTerm(), requestVoteReply.getTerm());
132             assertEquals("isVoteGranted", false, requestVoteReply.isVoteGranted());
133         }
134
135         verifyBehaviorState("member 1", member1Actor, RaftState.Follower);
136         verifyBehaviorState("member 2", member2Actor, RaftState.Follower);
137         verifyBehaviorState("member 3", member3Actor, RaftState.Candidate);
138
139         // Even though member 3 didn't get voted for, member 1 and 2 should have updated their term
140         // to member 3's.
141
142         assertEquals("member 1 election term", candidateElectionTerm,
143                 member1Context.getTermInformation().getCurrentTerm());
144         assertEquals("member 2 election term", candidateElectionTerm,
145                 member2Context.getTermInformation().getCurrentTerm());
146         assertEquals("member 3 election term", candidateElectionTerm,
147                 member3Context.getTermInformation().getCurrentTerm());
148
149         testLog.info("resolvePartitionAndSendElectionTimeoutsToCandidateMember3 ending");
150     }
151
152     private void setupPartitionedCandidateMember3AndSendElectionTimeouts() {
153         testLog.info("setupPartitionedCandidateMember3AndSendElectionTimeouts starting");
154
155         // Create member 3's behavior initially as a Candidate.
156
157         member3Context = newRaftActorContext("member3", member3ActorRef,
158                 ImmutableMap.<String,String>builder()
159                     .put("member1", member1ActorRef.path().toString())
160                     .put("member2", member2ActorRef.path().toString()).build());
161
162         DefaultConfigParamsImpl member3ConfigParams = newConfigParams();
163         member3Context.setConfigParams(member3ConfigParams);
164
165         // Initialize the ReplicatedLog and election term info for Candidate member 3. The current term
166         // will be 2 and the last term will be 1 so it is behind the leader's log.
167
168         SimpleReplicatedLog candidateReplicatedLog = new SimpleReplicatedLog();
169         candidateReplicatedLog.append(new SimpleReplicatedLogEntry(0, 2, new MockPayload("")));
170
171         member3Context.setReplicatedLog(candidateReplicatedLog);
172         member3Context.setCommitIndex(candidateReplicatedLog.lastIndex());
173         member3Context.setLastApplied(candidateReplicatedLog.lastIndex());
174         member3Context.getTermInformation().update(2, member1Context.getId());
175
176         // The member 3 Candidate will start a new term and send RequestVotes. However it will be
177         // partitioned from the cluster by having member 1 and 2 drop its RequestVote messages.
178
179         candidateElectionTerm = member3Context.getTermInformation().getCurrentTerm() + numCandidateElections;
180
181         member1Actor.dropMessagesToBehavior(RequestVote.class, numCandidateElections);
182
183         member2Actor.dropMessagesToBehavior(RequestVote.class, numCandidateElections);
184
185         member3Actor.self().tell(new SetBehavior(new Candidate(member3Context), member3Context),
186                 ActorRef.noSender());
187
188         // Send several additional ElectionTimeouts to Candidate member 3. Each ElectionTimeout will
189         // start a new term so Candidate member 3's current term will be greater than the leader's
190         // current term.
191
192         for (int i = 0; i < numCandidateElections - 1; i++) {
193             member3ActorRef.tell(ElectionTimeout.INSTANCE, ActorRef.noSender());
194         }
195
196         member1Actor.waitForExpectedMessages(RequestVote.class);
197         member2Actor.waitForExpectedMessages(RequestVote.class);
198
199         verifyBehaviorState("member 1", member1Actor, RaftState.Leader);
200         verifyBehaviorState("member 2", member2Actor, RaftState.Follower);
201         verifyBehaviorState("member 3", member3Actor, RaftState.Candidate);
202
203         assertEquals("member 1 election term", 3, member1Context.getTermInformation().getCurrentTerm());
204         assertEquals("member 2 election term", 3, member2Context.getTermInformation().getCurrentTerm());
205         assertEquals("member 3 election term", candidateElectionTerm,
206                 member3Context.getTermInformation().getCurrentTerm());
207
208         testLog.info("setupPartitionedCandidateMember3AndSendElectionTimeouts ending");
209     }
210
211     private void setupInitialMember1AndMember2Behaviors() {
212         testLog.info("setupInitialMember1AndMember2Behaviors starting");
213
214         // Initialize the ReplicatedLog and election term info for member 1 and 2. The current term
215         // will be 3 and the last term will be 2.
216
217         SimpleReplicatedLog replicatedLog = new SimpleReplicatedLog();
218         replicatedLog.append(new SimpleReplicatedLogEntry(0, 2, new MockPayload("")));
219         replicatedLog.append(new SimpleReplicatedLogEntry(1, 3, new MockPayload("")));
220
221         // Create member 2's behavior as Follower.
222
223         member2Context = newRaftActorContext("member2", member2ActorRef,
224                 ImmutableMap.<String,String>builder()
225                     .put("member1", member1ActorRef.path().toString())
226                     .put("member3", member3ActorRef.path().toString()).build());
227
228         DefaultConfigParamsImpl member2ConfigParams = newConfigParams();
229         member2Context.setConfigParams(member2ConfigParams);
230
231         member2Context.setReplicatedLog(replicatedLog);
232         member2Context.setCommitIndex(replicatedLog.lastIndex());
233         member2Context.setLastApplied(replicatedLog.lastIndex());
234         member2Context.getTermInformation().update(3, "member1");
235
236         member2Actor.self().tell(new SetBehavior(new Follower(member2Context), member2Context),
237                 ActorRef.noSender());
238
239         // Create member 1's behavior as Leader.
240
241         member1Context = newRaftActorContext("member1", member1ActorRef,
242                 ImmutableMap.<String,String>builder()
243                     .put("member2", member2ActorRef.path().toString())
244                     .put("member3", member3ActorRef.path().toString()).build());
245
246         DefaultConfigParamsImpl member1ConfigParams = newConfigParams();
247         member1Context.setConfigParams(member1ConfigParams);
248
249         member1Context.setReplicatedLog(replicatedLog);
250         member1Context.setCommitIndex(replicatedLog.lastIndex());
251         member1Context.setLastApplied(replicatedLog.lastIndex());
252         member1Context.getTermInformation().update(3, "member1");
253
254         initializeLeaderBehavior(member1Actor, member1Context, 1);
255
256         member2Actor.clear();
257         member3Actor.clear();
258
259         testLog.info("setupInitialMember1AndMember2Behaviors ending");
260     }
261 }