a7b6825c7db33b95ffd701beccaf24adf6865106
[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 akka.actor.ActorRef;
4 import akka.actor.Props;
5 import akka.testkit.JavaTestKit;
6 import org.junit.Test;
7 import org.opendaylight.controller.cluster.raft.AbstractActorTest;
8 import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
9 import org.opendaylight.controller.cluster.raft.RaftActorContext;
10 import org.opendaylight.controller.cluster.raft.RaftState;
11 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
12 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
13 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
14 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
15 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
16 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
17 import org.opendaylight.controller.cluster.raft.utils.DoNothingActor;
18
19 import java.util.ArrayList;
20 import java.util.List;
21
22 import static org.junit.Assert.assertEquals;
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);
76
77             RaftActorBehavior behavior = createBehavior(context);
78
79             // Send an unknown message so that the state of the RaftActor remains unchanged
80             RaftState expected = behavior.handleMessage(getRef(), "unknown");
81
82             RaftState raftState =
83                 behavior.handleMessage(getRef(), appendEntries);
84
85             assertEquals(expected, raftState);
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, "zero"));
124
125                 context.setReplicatedLog(log);
126
127                 List<ReplicatedLogEntry> entries = new ArrayList<>();
128                 entries.add(
129                     new MockRaftActorContext.MockReplicatedLogEntry(1, 0, "zero"));
130
131                 AppendEntries appendEntries =
132                     new AppendEntries(2, "leader-1", -1, 1, entries, 0);
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                 RaftState expected = behavior.handleMessage(getRef(), "unknown");
147
148                 RaftState raftState =
149                     behavior.handleMessage(getRef(), appendEntries);
150
151                 assertEquals(expected, raftState);
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 follower = createBehavior(
173                         createActorContext(behaviorActor));
174
175                     follower.handleMessage(getTestActor(),
176                         new RequestVote(1000, "test", 10000, 999));
177
178                     final Boolean out =
179                         new ExpectMsg<Boolean>(duration("1 seconds"),
180                             "RequestVoteReply") {
181                             // do not put code outside this method, will run afterwards
182                             protected Boolean match(Object in) {
183                                 if (in instanceof RequestVoteReply) {
184                                     RequestVoteReply reply =
185                                         (RequestVoteReply) in;
186                                     return reply.isVoteGranted();
187                                 } else {
188                                     throw noMatch();
189                                 }
190                             }
191                         }.get();
192
193                     assertEquals(true, out);
194                 }
195             };
196         }};
197     }
198
199     /**
200      * This test verifies that when a RaftActor receives a RequestVote message
201      * with a term that is greater than it's currentTerm but a less up-to-date
202      * log then the receiving RaftActor will not grant the vote to the sender
203      */
204     @Test
205     public void testHandleRequestVoteWhenSenderTermGreaterThanCurrentTermButSenderLogLessUptoDate() {
206         new JavaTestKit(getSystem()) {{
207
208             new Within(duration("1 seconds")) {
209                 protected void run() {
210
211                     RaftActorContext actorContext =
212                         createActorContext(behaviorActor);
213
214                     MockRaftActorContext.SimpleReplicatedLog
215                         log = new MockRaftActorContext.SimpleReplicatedLog();
216                     log.append(
217                         new MockRaftActorContext.MockReplicatedLogEntry(20000,
218                             1000000, ""));
219
220                     ((MockRaftActorContext) actorContext).setReplicatedLog(log);
221
222                     RaftActorBehavior follower = createBehavior(actorContext);
223
224                     follower.handleMessage(getTestActor(),
225                         new RequestVote(1000, "test", 10000, 999));
226
227                     final Boolean out =
228                         new ExpectMsg<Boolean>(duration("1 seconds"),
229                             "RequestVoteReply") {
230                             // do not put code outside this method, will run afterwards
231                             protected Boolean match(Object in) {
232                                 if (in instanceof RequestVoteReply) {
233                                     RequestVoteReply reply =
234                                         (RequestVoteReply) in;
235                                     return reply.isVoteGranted();
236                                 } else {
237                                     throw noMatch();
238                                 }
239                             }
240                         }.get();
241
242                     assertEquals(false, out);
243                 }
244             };
245         }};
246     }
247
248
249
250     /**
251      * This test verifies that the receiving RaftActor will not grant a vote
252      * to a sender if the sender's term is lesser than the currentTerm of the
253      * recipient RaftActor
254      */
255     @Test
256     public void testHandleRequestVoteWhenSenderTermLessThanCurrentTerm() {
257         new JavaTestKit(getSystem()) {{
258
259             new Within(duration("1 seconds")) {
260                 protected void run() {
261
262                     RaftActorContext context =
263                         createActorContext(behaviorActor);
264
265                     context.getTermInformation().update(1000, null);
266
267                     RaftActorBehavior follower = createBehavior(context);
268
269                     follower.handleMessage(getTestActor(),
270                         new RequestVote(999, "test", 10000, 999));
271
272                     final Boolean out =
273                         new ExpectMsg<Boolean>(duration("1 seconds"),
274                             "RequestVoteReply") {
275                             // do not put code outside this method, will run afterwards
276                             protected Boolean match(Object in) {
277                                 if (in instanceof RequestVoteReply) {
278                                     RequestVoteReply reply =
279                                         (RequestVoteReply) in;
280                                     return reply.isVoteGranted();
281                                 } else {
282                                     throw noMatch();
283                                 }
284                             }
285                         }.get();
286
287                     assertEquals(false, out);
288                 }
289             };
290         }};
291     }
292
293     protected void assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(
294         ActorRef actorRef, RaftRPC rpc) {
295
296         RaftActorContext actorContext = createActorContext();
297         setLastLogEntry(
298             (MockRaftActorContext) actorContext, 0, 0, "");
299
300         RaftState raftState = createBehavior(actorContext)
301             .handleMessage(actorRef, rpc);
302
303         assertEquals(RaftState.Follower, raftState);
304     }
305
306     protected MockRaftActorContext.SimpleReplicatedLog setLastLogEntry(
307         MockRaftActorContext actorContext, long term, long index, Object data) {
308         return setLastLogEntry(actorContext,
309             new MockRaftActorContext.MockReplicatedLogEntry(term, index, data));
310     }
311
312     protected MockRaftActorContext.SimpleReplicatedLog setLastLogEntry(
313         MockRaftActorContext actorContext, ReplicatedLogEntry logEntry) {
314         MockRaftActorContext.SimpleReplicatedLog
315             log = new MockRaftActorContext.SimpleReplicatedLog();
316         log.append(logEntry);
317         actorContext.setReplicatedLog(log);
318
319         return log;
320     }
321
322     protected abstract RaftActorBehavior createBehavior(
323         RaftActorContext actorContext);
324
325     protected RaftActorBehavior createBehavior() {
326         return createBehavior(createActorContext());
327     }
328
329     protected RaftActorContext createActorContext() {
330         return new MockRaftActorContext();
331     }
332
333     protected RaftActorContext createActorContext(ActorRef actor) {
334         return new MockRaftActorContext("test", getSystem(), actor);
335     }
336
337     protected AppendEntries createAppendEntriesWithNewerTerm() {
338         return new AppendEntries(100, "leader-1", 0, 0, null, 1);
339     }
340
341     protected AppendEntriesReply createAppendEntriesReplyWithNewerTerm() {
342         return new AppendEntriesReply("follower-1", 100, false, 100, 100);
343     }
344
345     protected RequestVote createRequestVoteWithNewerTerm() {
346         return new RequestVote(100, "candidate-1", 10, 100);
347     }
348
349     protected RequestVoteReply createRequestVoteReplyWithNewerTerm() {
350         return new RequestVoteReply(100, false);
351     }
352
353
354
355 }