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