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