Refactor LeaderTest
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractRaftActorBehaviorTest.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.JavaTestKit;
8 import java.util.ArrayList;
9 import java.util.List;
10 import org.junit.Test;
11 import org.opendaylight.controller.cluster.raft.AbstractActorTest;
12 import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
13 import org.opendaylight.controller.cluster.raft.RaftActorContext;
14 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
15 import org.opendaylight.controller.cluster.raft.SerializationUtils;
16 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
17 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
18 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
19 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
20 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
21 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
22 import org.opendaylight.controller.cluster.raft.utils.DoNothingActor;
23
24 public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest {
25
26     private final ActorRef behaviorActor = getSystem().actorOf(Props.create(
27         DoNothingActor.class));
28
29     /**
30      * This test checks that when a new Raft RPC message is received with a newer
31      * term the RaftActor gets into the Follower state.
32      *
33      * @throws Exception
34      */
35     @Test
36     public void testHandleRaftRPCWithNewerTerm() throws Exception {
37         new JavaTestKit(getSystem()) {{
38
39             assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(getTestActor(),
40                 createAppendEntriesWithNewerTerm());
41
42             assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(getTestActor(),
43                 createAppendEntriesReplyWithNewerTerm());
44
45             assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(getTestActor(),
46                 createRequestVoteWithNewerTerm());
47
48             assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(getTestActor(),
49                 createRequestVoteReplyWithNewerTerm());
50
51
52         }};
53     }
54
55
56     /**
57      * This test verifies that when an AppendEntries is received with a term that
58      * is less that the currentTerm of the RaftActor then the RaftActor does not
59      * change it's state and it responds back with a failure
60      *
61      * @throws Exception
62      */
63     @Test
64     public void testHandleAppendEntriesSenderTermLessThanReceiverTerm()
65         throws Exception {
66         new JavaTestKit(getSystem()) {{
67
68             MockRaftActorContext context = createActorContext();
69
70             // First set the receivers term to a high number (1000)
71             context.getTermInformation().update(1000, "test");
72
73             AppendEntries appendEntries =
74                 new AppendEntries(100, "leader-1", 0, 0, null, 101, -1);
75
76             RaftActorBehavior behavior = createBehavior(context);
77
78             // Send an unknown message so that the state of the RaftActor remains unchanged
79             RaftActorBehavior expected = behavior.handleMessage(getRef(), "unknown");
80
81             RaftActorBehavior raftBehavior =
82                 behavior.handleMessage(getRef(), appendEntries);
83
84             assertEquals(expected, raftBehavior);
85
86             // Also expect an AppendEntriesReply to be sent where success is false
87             final Boolean out = new ExpectMsg<Boolean>(duration("1 seconds"),
88                 "AppendEntriesReply") {
89                 // do not put code outside this method, will run afterwards
90                 @Override
91                 protected Boolean match(Object in) {
92                     if (in instanceof AppendEntriesReply) {
93                         AppendEntriesReply reply = (AppendEntriesReply) in;
94                         return reply.isSuccess();
95                     } else {
96                         throw noMatch();
97                     }
98                 }
99             }.get();
100
101             assertEquals(false, out);
102
103
104         }};
105     }
106
107
108     @Test
109     public void testHandleAppendEntriesAddSameEntryToLog(){
110         new JavaTestKit(getSystem()) {
111             {
112
113                 MockRaftActorContext context = createActorContext();
114
115                 // First set the receivers term to lower number
116                 context.getTermInformation().update(2, "test");
117
118                 // Prepare the receivers log
119                 MockRaftActorContext.SimpleReplicatedLog log =
120                     new MockRaftActorContext.SimpleReplicatedLog();
121                 log.append(
122                     new MockRaftActorContext.MockReplicatedLogEntry(1, 0, new MockRaftActorContext.MockPayload("zero")));
123
124                 context.setReplicatedLog(log);
125
126                 List<ReplicatedLogEntry> entries = new ArrayList<>();
127                 entries.add(
128                     new MockRaftActorContext.MockReplicatedLogEntry(1, 0, new MockRaftActorContext.MockPayload("zero")));
129
130                 AppendEntries appendEntries =
131                     new AppendEntries(2, "leader-1", -1, 1, entries, 0, -1);
132
133                 RaftActorBehavior behavior = createBehavior(context);
134
135                 if (AbstractRaftActorBehaviorTest.this instanceof CandidateTest) {
136                     // Resetting the Candidates term to make sure it will match
137                     // the term sent by AppendEntries. If this was not done then
138                     // the test will fail because the Candidate will assume that
139                     // the message was sent to it from a lower term peer and will
140                     // thus respond with a failure
141                     context.getTermInformation().update(2, "test");
142                 }
143
144                 // Send an unknown message so that the state of the RaftActor remains unchanged
145                 RaftActorBehavior expected = behavior.handleMessage(getRef(), "unknown");
146
147                 RaftActorBehavior raftBehavior =
148                     behavior.handleMessage(getRef(), appendEntries);
149
150                 assertEquals(expected, raftBehavior);
151
152                 assertEquals(1, log.size());
153
154
155             }};
156     }
157
158     /**
159      * This test verifies that when a RequestVote is received by the RaftActor
160      * with a term which is greater than the RaftActors' currentTerm and the
161      * senders' log is more upto date than the receiver that the receiver grants
162      * the vote to the sender
163      */
164     @Test
165     public void testHandleRequestVoteWhenSenderTermGreaterThanCurrentTermAndSenderLogMoreUpToDate() {
166         new JavaTestKit(getSystem()) {{
167
168             new Within(duration("1 seconds")) {
169                 @Override
170                 protected void run() {
171
172                     RaftActorBehavior behavior = createBehavior(
173                         createActorContext(behaviorActor));
174
175                     RaftActorBehavior raftBehavior = behavior.handleMessage(getTestActor(),
176                         new RequestVote(1000, "test", 10000, 999));
177
178                     if(!(behavior instanceof Follower)){
179                         assertTrue(raftBehavior instanceof Follower);
180                     } else {
181
182                         final Boolean out =
183                             new ExpectMsg<Boolean>(duration("1 seconds"),
184                                 "RequestVoteReply") {
185                                 // do not put code outside this method, will run afterwards
186                                 @Override
187                                 protected Boolean match(Object in) {
188                                     if (in instanceof RequestVoteReply) {
189                                         RequestVoteReply reply =
190                                             (RequestVoteReply) in;
191                                         return reply.isVoteGranted();
192                                     } else {
193                                         throw noMatch();
194                                     }
195                                 }
196                             }.get();
197
198                         assertEquals(true, out);
199                     }
200                 }
201             };
202         }};
203     }
204
205     /**
206      * This test verifies that when a RaftActor receives a RequestVote message
207      * with a term that is greater than it's currentTerm but a less up-to-date
208      * log then the receiving RaftActor will not grant the vote to the sender
209      */
210     @Test
211     public void testHandleRequestVoteWhenSenderTermGreaterThanCurrentTermButSenderLogLessUptoDate() {
212         new JavaTestKit(getSystem()) {{
213
214             new Within(duration("1 seconds")) {
215                 @Override
216                 protected void run() {
217
218                     RaftActorContext actorContext =
219                         createActorContext(behaviorActor);
220
221                     MockRaftActorContext.SimpleReplicatedLog
222                         log = new MockRaftActorContext.SimpleReplicatedLog();
223                     log.append(
224                         new MockRaftActorContext.MockReplicatedLogEntry(20000,
225                             1000000, new MockRaftActorContext.MockPayload("")));
226
227                     ((MockRaftActorContext) actorContext).setReplicatedLog(log);
228
229                     RaftActorBehavior behavior = createBehavior(actorContext);
230
231                     RaftActorBehavior raftBehavior = behavior.handleMessage(getTestActor(),
232                         new RequestVote(1000, "test", 10000, 999));
233
234                     if(!(behavior instanceof Follower)){
235                         assertTrue(raftBehavior instanceof Follower);
236                     } else {
237                         final Boolean out =
238                             new ExpectMsg<Boolean>(duration("1 seconds"),
239                                 "RequestVoteReply") {
240                                 // do not put code outside this method, will run afterwards
241                                 @Override
242                                 protected Boolean match(Object in) {
243                                     if (in instanceof RequestVoteReply) {
244                                         RequestVoteReply reply =
245                                             (RequestVoteReply) in;
246                                         return reply.isVoteGranted();
247                                     } else {
248                                         throw noMatch();
249                                     }
250                                 }
251                             }.get();
252
253                         assertEquals(false, out);
254                     }
255                 }
256             };
257         }};
258     }
259
260
261
262     /**
263      * This test verifies that the receiving RaftActor will not grant a vote
264      * to a sender if the sender's term is lesser than the currentTerm of the
265      * recipient RaftActor
266      */
267     @Test
268     public void testHandleRequestVoteWhenSenderTermLessThanCurrentTerm() {
269         new JavaTestKit(getSystem()) {{
270
271             new Within(duration("1 seconds")) {
272                 @Override
273                 protected void run() {
274
275                     RaftActorContext context =
276                         createActorContext(behaviorActor);
277
278                     context.getTermInformation().update(1000, null);
279
280                     RaftActorBehavior follower = createBehavior(context);
281
282                     follower.handleMessage(getTestActor(),
283                         new RequestVote(999, "test", 10000, 999));
284
285                     final Boolean out =
286                         new ExpectMsg<Boolean>(duration("1 seconds"),
287                             "RequestVoteReply") {
288                             // do not put code outside this method, will run afterwards
289                             @Override
290                             protected Boolean match(Object in) {
291                                 if (in instanceof RequestVoteReply) {
292                                     RequestVoteReply reply =
293                                         (RequestVoteReply) in;
294                                     return reply.isVoteGranted();
295                                 } else {
296                                     throw noMatch();
297                                 }
298                             }
299                         }.get();
300
301                     assertEquals(false, out);
302                 }
303             };
304         }};
305     }
306
307     @Test
308     public void testFakeSnapshots() {
309         MockRaftActorContext context = new MockRaftActorContext("test", getSystem(), behaviorActor);
310         AbstractRaftActorBehavior behavior = new Leader(context);
311         context.getTermInformation().update(1, "leader");
312
313         //entry with 1 index=0 entry with replicatedToAllIndex = 0, does not do anything, returns the
314         context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0, 1, 1).build());
315         context.setLastApplied(0);
316         assertEquals(-1, behavior.fakeSnapshot(0, -1));
317         assertEquals(1, context.getReplicatedLog().size());
318
319         //2 entries, lastApplied still 0, no purging.
320         context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0,2,1).build());
321         context.setLastApplied(0);
322         assertEquals(-1, behavior.fakeSnapshot(0, -1));
323         assertEquals(2, context.getReplicatedLog().size());
324
325         //2 entries, lastApplied still 0, no purging.
326         context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0,2,1).build());
327         context.setLastApplied(1);
328         assertEquals(0, behavior.fakeSnapshot(0, -1));
329         assertEquals(1, context.getReplicatedLog().size());
330
331         //5 entries, lastApplied =2 and replicatedIndex = 3, but since we want to keep the lastapplied, indices 0 and 1 will only get purged
332         context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0,5,1).build());
333         context.setLastApplied(2);
334         assertEquals(1, behavior.fakeSnapshot(3, 1));
335         assertEquals(3, context.getReplicatedLog().size());
336
337
338     }
339
340     protected void assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(
341         ActorRef actorRef, RaftRPC rpc) {
342
343         RaftActorContext actorContext = createActorContext();
344         Payload p = new MockRaftActorContext.MockPayload("");
345         setLastLogEntry(
346             (MockRaftActorContext) actorContext, 0, 0, p);
347
348         RaftActorBehavior raftBehavior = createBehavior(actorContext)
349             .handleMessage(actorRef, rpc);
350
351         assertTrue(raftBehavior instanceof Follower);
352     }
353
354     protected MockRaftActorContext.SimpleReplicatedLog setLastLogEntry(
355         MockRaftActorContext actorContext, long term, long index, Payload data) {
356         return setLastLogEntry(actorContext,
357             new MockRaftActorContext.MockReplicatedLogEntry(term, index, data));
358     }
359
360     protected MockRaftActorContext.SimpleReplicatedLog setLastLogEntry(
361         MockRaftActorContext actorContext, ReplicatedLogEntry logEntry) {
362         MockRaftActorContext.SimpleReplicatedLog
363             log = new MockRaftActorContext.SimpleReplicatedLog();
364         log.append(logEntry);
365         actorContext.setReplicatedLog(log);
366
367         return log;
368     }
369
370     protected abstract RaftActorBehavior createBehavior(
371         RaftActorContext actorContext);
372
373     protected RaftActorBehavior createBehavior() {
374         return createBehavior(createActorContext());
375     }
376
377     protected MockRaftActorContext createActorContext() {
378         return new MockRaftActorContext();
379     }
380
381     protected MockRaftActorContext createActorContext(ActorRef actor) {
382         return new MockRaftActorContext("test", getSystem(), actor);
383     }
384
385     protected AppendEntries createAppendEntriesWithNewerTerm() {
386         return new AppendEntries(100, "leader-1", 0, 0, null, 1, -1);
387     }
388
389     protected AppendEntriesReply createAppendEntriesReplyWithNewerTerm() {
390         return new AppendEntriesReply("follower-1", 100, false, 100, 100);
391     }
392
393     protected RequestVote createRequestVoteWithNewerTerm() {
394         return new RequestVote(100, "candidate-1", 10, 100);
395     }
396
397     protected RequestVoteReply createRequestVoteReplyWithNewerTerm() {
398         return new RequestVoteReply(100, false);
399     }
400
401     protected Object fromSerializableMessage(Object serializable){
402         return SerializationUtils.fromSerializable(serializable);
403     }
404 }