Merge "BUG-2633 - Netconf northbound mapping."
[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 = (MockRaftActorContext)
69                 createActorContext();
70
71             // First set the receivers term to a high number (1000)
72             context.getTermInformation().update(1000, "test");
73
74             AppendEntries appendEntries =
75                 new AppendEntries(100, "leader-1", 0, 0, null, 101, -1);
76
77             RaftActorBehavior behavior = createBehavior(context);
78
79             // Send an unknown message so that the state of the RaftActor remains unchanged
80             RaftActorBehavior expected = behavior.handleMessage(getRef(), "unknown");
81
82             RaftActorBehavior raftBehavior =
83                 behavior.handleMessage(getRef(), appendEntries);
84
85             assertEquals(expected, raftBehavior);
86
87             // Also expect an AppendEntriesReply to be sent where success is false
88             final Boolean out = new ExpectMsg<Boolean>(duration("1 seconds"),
89                 "AppendEntriesReply") {
90                 // do not put code outside this method, will run afterwards
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 = (MockRaftActorContext)
114                     createActorContext();
115
116                 // First set the receivers term to lower number
117                 context.getTermInformation().update(2, "test");
118
119                 // Prepare the receivers log
120                 MockRaftActorContext.SimpleReplicatedLog log =
121                     new MockRaftActorContext.SimpleReplicatedLog();
122                 log.append(
123                     new MockRaftActorContext.MockReplicatedLogEntry(1, 0, new MockRaftActorContext.MockPayload("zero")));
124
125                 context.setReplicatedLog(log);
126
127                 List<ReplicatedLogEntry> entries = new ArrayList<>();
128                 entries.add(
129                     new MockRaftActorContext.MockReplicatedLogEntry(1, 0, new MockRaftActorContext.MockPayload("zero")));
130
131                 AppendEntries appendEntries =
132                     new AppendEntries(2, "leader-1", -1, 1, entries, 0, -1);
133
134                 RaftActorBehavior behavior = createBehavior(context);
135
136                 if (AbstractRaftActorBehaviorTest.this instanceof CandidateTest) {
137                     // Resetting the Candidates term to make sure it will match
138                     // the term sent by AppendEntries. If this was not done then
139                     // the test will fail because the Candidate will assume that
140                     // the message was sent to it from a lower term peer and will
141                     // thus respond with a failure
142                     context.getTermInformation().update(2, "test");
143                 }
144
145                 // Send an unknown message so that the state of the RaftActor remains unchanged
146                 RaftActorBehavior expected = behavior.handleMessage(getRef(), "unknown");
147
148                 RaftActorBehavior raftBehavior =
149                     behavior.handleMessage(getRef(), appendEntries);
150
151                 assertEquals(expected, raftBehavior);
152
153                 assertEquals(1, log.size());
154
155
156             }};
157     }
158
159     /**
160      * This test verifies that when a RequestVote is received by the RaftActor
161      * with a term which is greater than the RaftActors' currentTerm and the
162      * senders' log is more upto date than the receiver that the receiver grants
163      * the vote to the sender
164      */
165     @Test
166     public void testHandleRequestVoteWhenSenderTermGreaterThanCurrentTermAndSenderLogMoreUpToDate() {
167         new JavaTestKit(getSystem()) {{
168
169             new Within(duration("1 seconds")) {
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                                 protected Boolean match(Object in) {
187                                     if (in instanceof RequestVoteReply) {
188                                         RequestVoteReply reply =
189                                             (RequestVoteReply) in;
190                                         return reply.isVoteGranted();
191                                     } else {
192                                         throw noMatch();
193                                     }
194                                 }
195                             }.get();
196
197                         assertEquals(true, out);
198                     }
199                 }
200             };
201         }};
202     }
203
204     /**
205      * This test verifies that when a RaftActor receives a RequestVote message
206      * with a term that is greater than it's currentTerm but a less up-to-date
207      * log then the receiving RaftActor will not grant the vote to the sender
208      */
209     @Test
210     public void testHandleRequestVoteWhenSenderTermGreaterThanCurrentTermButSenderLogLessUptoDate() {
211         new JavaTestKit(getSystem()) {{
212
213             new Within(duration("1 seconds")) {
214                 protected void run() {
215
216                     RaftActorContext actorContext =
217                         createActorContext(behaviorActor);
218
219                     MockRaftActorContext.SimpleReplicatedLog
220                         log = new MockRaftActorContext.SimpleReplicatedLog();
221                     log.append(
222                         new MockRaftActorContext.MockReplicatedLogEntry(20000,
223                             1000000, new MockRaftActorContext.MockPayload("")));
224
225                     ((MockRaftActorContext) actorContext).setReplicatedLog(log);
226
227                     RaftActorBehavior behavior = createBehavior(actorContext);
228
229                     RaftActorBehavior raftBehavior = behavior.handleMessage(getTestActor(),
230                         new RequestVote(1000, "test", 10000, 999));
231
232                     if(!(behavior instanceof Follower)){
233                         assertTrue(raftBehavior instanceof Follower);
234                     } else {
235                         final Boolean out =
236                             new ExpectMsg<Boolean>(duration("1 seconds"),
237                                 "RequestVoteReply") {
238                                 // do not put code outside this method, will run afterwards
239                                 protected Boolean match(Object in) {
240                                     if (in instanceof RequestVoteReply) {
241                                         RequestVoteReply reply =
242                                             (RequestVoteReply) in;
243                                         return reply.isVoteGranted();
244                                     } else {
245                                         throw noMatch();
246                                     }
247                                 }
248                             }.get();
249
250                         assertEquals(false, out);
251                     }
252                 }
253             };
254         }};
255     }
256
257
258
259     /**
260      * This test verifies that the receiving RaftActor will not grant a vote
261      * to a sender if the sender's term is lesser than the currentTerm of the
262      * recipient RaftActor
263      */
264     @Test
265     public void testHandleRequestVoteWhenSenderTermLessThanCurrentTerm() {
266         new JavaTestKit(getSystem()) {{
267
268             new Within(duration("1 seconds")) {
269                 protected void run() {
270
271                     RaftActorContext context =
272                         createActorContext(behaviorActor);
273
274                     context.getTermInformation().update(1000, null);
275
276                     RaftActorBehavior follower = createBehavior(context);
277
278                     follower.handleMessage(getTestActor(),
279                         new RequestVote(999, "test", 10000, 999));
280
281                     final Boolean out =
282                         new ExpectMsg<Boolean>(duration("1 seconds"),
283                             "RequestVoteReply") {
284                             // do not put code outside this method, will run afterwards
285                             protected Boolean match(Object in) {
286                                 if (in instanceof RequestVoteReply) {
287                                     RequestVoteReply reply =
288                                         (RequestVoteReply) in;
289                                     return reply.isVoteGranted();
290                                 } else {
291                                     throw noMatch();
292                                 }
293                             }
294                         }.get();
295
296                     assertEquals(false, out);
297                 }
298             };
299         }};
300     }
301
302     @Test
303     public void testPerformSnapshot() {
304         MockRaftActorContext context = new MockRaftActorContext("test", getSystem(), behaviorActor);
305         AbstractRaftActorBehavior abstractBehavior =  (AbstractRaftActorBehavior) createBehavior(context);
306         if (abstractBehavior instanceof Candidate) {
307             return;
308         }
309
310         context.getTermInformation().update(1, "test");
311
312         //log has 1 entry with replicatedToAllIndex = 0, does not do anything, returns the
313         context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0, 1, 1).build());
314         context.setLastApplied(0);
315         abstractBehavior.performSnapshotWithoutCapture(0);
316         assertEquals(-1, abstractBehavior.getReplicatedToAllIndex());
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         abstractBehavior.performSnapshotWithoutCapture(0);
323         assertEquals(-1, abstractBehavior.getReplicatedToAllIndex());
324         assertEquals(2, context.getReplicatedLog().size());
325
326         //2 entries, lastApplied still 0, no purging.
327         context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0, 2, 1).build());
328         context.setLastApplied(1);
329         abstractBehavior.performSnapshotWithoutCapture(0);
330         assertEquals(0, abstractBehavior.getReplicatedToAllIndex());
331         assertEquals(1, context.getReplicatedLog().size());
332
333         //5 entries, lastApplied =2 and replicatedIndex = 3, but since we want to keep the lastapplied, indices 0 and 1 will only get purged
334         context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0, 5, 1).build());
335         context.setLastApplied(2);
336         abstractBehavior.performSnapshotWithoutCapture(3);
337         assertEquals(1, abstractBehavior.getReplicatedToAllIndex());
338         assertEquals(3, context.getReplicatedLog().size());
339
340         // scenario where Last applied > Replicated to all index (becoz of a slow follower)
341         context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0, 3, 1).build());
342         context.setLastApplied(2);
343         abstractBehavior.performSnapshotWithoutCapture(1);
344         assertEquals(1, abstractBehavior.getReplicatedToAllIndex());
345         assertEquals(1, context.getReplicatedLog().size());
346     }
347
348
349     protected void assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(
350         ActorRef actorRef, RaftRPC rpc) {
351
352         RaftActorContext actorContext = createActorContext();
353         Payload p = new MockRaftActorContext.MockPayload("");
354         setLastLogEntry(
355             (MockRaftActorContext) actorContext, 0, 0, p);
356
357         RaftActorBehavior raftBehavior = createBehavior(actorContext)
358             .handleMessage(actorRef, rpc);
359
360         assertTrue(raftBehavior instanceof Follower);
361     }
362
363     protected MockRaftActorContext.SimpleReplicatedLog setLastLogEntry(
364         MockRaftActorContext actorContext, long term, long index, Payload data) {
365         return setLastLogEntry(actorContext,
366             new MockRaftActorContext.MockReplicatedLogEntry(term, index, data));
367     }
368
369     protected MockRaftActorContext.SimpleReplicatedLog setLastLogEntry(
370         MockRaftActorContext actorContext, ReplicatedLogEntry logEntry) {
371         MockRaftActorContext.SimpleReplicatedLog
372             log = new MockRaftActorContext.SimpleReplicatedLog();
373         log.append(logEntry);
374         actorContext.setReplicatedLog(log);
375
376         return log;
377     }
378
379     protected abstract RaftActorBehavior createBehavior(
380         RaftActorContext actorContext);
381
382     protected RaftActorBehavior createBehavior() {
383         return createBehavior(createActorContext());
384     }
385
386     protected RaftActorContext createActorContext() {
387         return new MockRaftActorContext();
388     }
389
390     protected RaftActorContext createActorContext(ActorRef actor) {
391         return new MockRaftActorContext("test", getSystem(), actor);
392     }
393
394     protected AppendEntries createAppendEntriesWithNewerTerm() {
395         return new AppendEntries(100, "leader-1", 0, 0, null, 1, -1);
396     }
397
398     protected AppendEntriesReply createAppendEntriesReplyWithNewerTerm() {
399         return new AppendEntriesReply("follower-1", 100, false, 100, 100);
400     }
401
402     protected RequestVote createRequestVoteWithNewerTerm() {
403         return new RequestVote(100, "candidate-1", 10, 100);
404     }
405
406     protected RequestVoteReply createRequestVoteReplyWithNewerTerm() {
407         return new RequestVoteReply(100, false);
408     }
409
410     protected Object fromSerializableMessage(Object serializable){
411         return SerializationUtils.fromSerializable(serializable);
412     }
413 }