Merge "Do not override artifact versions"
[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 akka.actor.ActorRef;
5 import akka.actor.Props;
6 import akka.testkit.TestActorRef;
7 import com.google.protobuf.ByteString;
8 import java.io.ByteArrayOutputStream;
9 import java.io.IOException;
10 import java.io.ObjectOutputStream;
11 import java.util.ArrayList;
12 import java.util.List;
13 import java.util.Map;
14 import org.junit.After;
15 import org.junit.Assert;
16 import org.junit.Test;
17 import org.opendaylight.controller.cluster.raft.AbstractActorTest;
18 import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
19 import org.opendaylight.controller.cluster.raft.RaftActorContext;
20 import org.opendaylight.controller.cluster.raft.RaftState;
21 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
22 import org.opendaylight.controller.cluster.raft.SerializationUtils;
23 import org.opendaylight.controller.cluster.raft.TestActorFactory;
24 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
25 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
26 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
27 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
28 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
29 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
30 import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
31 import org.slf4j.LoggerFactory;
32
33 public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest {
34
35     protected final TestActorFactory actorFactory = new TestActorFactory(getSystem());
36
37     private final TestActorRef<MessageCollectorActor> behaviorActor = actorFactory.createTestActor(
38             Props.create(MessageCollectorActor.class), actorFactory.generateActorId("behavior"));
39
40     RaftActorBehavior behavior;
41
42     @After
43     public void tearDown() throws Exception {
44         if(behavior != null) {
45             behavior.close();
46         }
47
48         actorFactory.close();
49     }
50
51     /**
52      * This test checks that when a new Raft RPC message is received with a newer
53      * term the RaftActor gets into the Follower state.
54      *
55      * @throws Exception
56      */
57     @Test
58     public void testHandleRaftRPCWithNewerTerm() throws Exception {
59         RaftActorContext actorContext = createActorContext();
60
61         assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(actorContext, behaviorActor,
62                 createAppendEntriesWithNewerTerm());
63
64         assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(actorContext, behaviorActor,
65                 createAppendEntriesReplyWithNewerTerm());
66
67         assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(actorContext, behaviorActor,
68                 createRequestVoteWithNewerTerm());
69
70         assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(actorContext, behaviorActor,
71                 createRequestVoteReplyWithNewerTerm());
72     }
73
74
75     /**
76      * This test verifies that when an AppendEntries is received with a term that
77      * is less that the currentTerm of the RaftActor then the RaftActor does not
78      * change it's state and it responds back with a failure
79      *
80      * @throws Exception
81      */
82     @Test
83     public void testHandleAppendEntriesSenderTermLessThanReceiverTerm() throws Exception {
84             MockRaftActorContext context = createActorContext();
85
86             // First set the receivers term to a high number (1000)
87             context.getTermInformation().update(1000, "test");
88
89             AppendEntries appendEntries = new AppendEntries(100, "leader-1", 0, 0, null, 101, -1);
90
91             behavior = createBehavior(context);
92
93             // Send an unknown message so that the state of the RaftActor remains unchanged
94             RaftActorBehavior expected = behavior.handleMessage(behaviorActor, "unknown");
95
96             RaftActorBehavior raftBehavior = behavior.handleMessage(behaviorActor, appendEntries);
97
98             assertEquals("Raft state", expected.state(), raftBehavior.state());
99
100             // Also expect an AppendEntriesReply to be sent where success is false
101
102             AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(
103                     behaviorActor, AppendEntriesReply.class);
104
105             assertEquals("isSuccess", false, reply.isSuccess());
106     }
107
108
109     @Test
110     public void testHandleAppendEntriesAddSameEntryToLog() throws Exception {
111         MockRaftActorContext context = createActorContext();
112
113         context.getTermInformation().update(2, "test");
114
115         // Prepare the receivers log
116         MockRaftActorContext.MockPayload payload = new MockRaftActorContext.MockPayload("zero");
117         setLastLogEntry(context, 2, 0, payload);
118
119         List<ReplicatedLogEntry> entries = new ArrayList<>();
120         entries.add(new MockRaftActorContext.MockReplicatedLogEntry(2, 0, payload));
121
122         AppendEntries appendEntries = new AppendEntries(2, "leader-1", -1, -1, entries, 2, -1);
123
124         behavior = createBehavior(context);
125
126         if (behavior instanceof Candidate) {
127             // Resetting the Candidates term to make sure it will match
128             // the term sent by AppendEntries. If this was not done then
129             // the test will fail because the Candidate will assume that
130             // the message was sent to it from a lower term peer and will
131             // thus respond with a failure
132             context.getTermInformation().update(2, "test");
133         }
134
135         // Send an unknown message so that the state of the RaftActor remains unchanged
136         RaftActorBehavior expected = behavior.handleMessage(behaviorActor, "unknown");
137
138         RaftActorBehavior raftBehavior = behavior.handleMessage(behaviorActor, appendEntries);
139
140         assertEquals("Raft state", expected.state(), raftBehavior.state());
141
142         assertEquals("ReplicatedLog size", 1, context.getReplicatedLog().size());
143
144         handleAppendEntriesAddSameEntryToLogReply(behaviorActor);
145     }
146
147     protected void handleAppendEntriesAddSameEntryToLogReply(TestActorRef<MessageCollectorActor> replyActor)
148             throws Exception {
149         AppendEntriesReply reply = MessageCollectorActor.getFirstMatching(replyActor, AppendEntriesReply.class);
150         Assert.assertNull("Expected no AppendEntriesReply", reply);
151     }
152
153     /**
154      * This test verifies that when a RequestVote is received by the RaftActor
155      * with the senders' log is more up to date than the receiver that the receiver grants
156      * the vote to the sender.
157      */
158     @Test
159     public void testHandleRequestVoteWhenSenderLogMoreUpToDate() {
160         MockRaftActorContext context = createActorContext();
161
162         behavior = createBehavior(context);
163
164         context.getTermInformation().update(1, "test");
165
166         behavior.handleMessage(behaviorActor, new RequestVote(context.getTermInformation().getCurrentTerm(),
167                 "test", 10000, 999));
168
169         RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(behaviorActor,
170                 RequestVoteReply.class);
171         assertEquals("isVoteGranted", true, reply.isVoteGranted());
172     }
173
174     /**
175      * This test verifies that when a RaftActor receives a RequestVote message
176      * with a term that is greater than it's currentTerm but a less up-to-date
177      * log then the receiving RaftActor will not grant the vote to the sender
178      */
179     @Test
180     public void testHandleRequestVoteWhenSenderLogLessUptoDate() {
181         MockRaftActorContext context = createActorContext();
182
183         behavior = createBehavior(context);
184
185         context.getTermInformation().update(1, "test");
186
187         int index = 2000;
188         setLastLogEntry(context, context.getTermInformation().getCurrentTerm(), index,
189                 new MockRaftActorContext.MockPayload(""));
190
191         behavior.handleMessage(behaviorActor, new RequestVote(
192                 context.getTermInformation().getCurrentTerm(), "test",
193                 index - 1, context.getTermInformation().getCurrentTerm()));
194
195         RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(behaviorActor,
196                 RequestVoteReply.class);
197         assertEquals("isVoteGranted", false, reply.isVoteGranted());
198     }
199
200
201
202     /**
203      * This test verifies that the receiving RaftActor will not grant a vote
204      * to a sender if the sender's term is lesser than the currentTerm of the
205      * recipient RaftActor
206      */
207     @Test
208     public void testHandleRequestVoteWhenSenderTermLessThanCurrentTerm() {
209         RaftActorContext context = createActorContext();
210
211         context.getTermInformation().update(1000, null);
212
213         behavior = createBehavior(context);
214
215         behavior.handleMessage(behaviorActor, new RequestVote(999, "test", 10000, 999));
216
217         RequestVoteReply reply = MessageCollectorActor.expectFirstMatching(behaviorActor,
218                 RequestVoteReply.class);
219         assertEquals("isVoteGranted", false, reply.isVoteGranted());
220     }
221
222     @Test
223     public void testPerformSnapshot() {
224         MockRaftActorContext context = new MockRaftActorContext("test", getSystem(), behaviorActor);
225         AbstractRaftActorBehavior abstractBehavior =  (AbstractRaftActorBehavior) createBehavior(context);
226         if (abstractBehavior instanceof Candidate) {
227             return;
228         }
229
230         context.getTermInformation().update(1, "test");
231
232         //log has 1 entry with replicatedToAllIndex = 0, does not do anything, returns the
233         context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0, 1, 1).build());
234         context.setLastApplied(0);
235         abstractBehavior.performSnapshotWithoutCapture(0);
236         assertEquals(-1, abstractBehavior.getReplicatedToAllIndex());
237         assertEquals(1, context.getReplicatedLog().size());
238
239         //2 entries, lastApplied still 0, no purging.
240         context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0, 2, 1).build());
241         context.setLastApplied(0);
242         abstractBehavior.performSnapshotWithoutCapture(0);
243         assertEquals(-1, abstractBehavior.getReplicatedToAllIndex());
244         assertEquals(2, context.getReplicatedLog().size());
245
246         //2 entries, lastApplied still 0, no purging.
247         context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0, 2, 1).build());
248         context.setLastApplied(1);
249         abstractBehavior.performSnapshotWithoutCapture(0);
250         assertEquals(0, abstractBehavior.getReplicatedToAllIndex());
251         assertEquals(1, context.getReplicatedLog().size());
252
253         //5 entries, lastApplied =2 and replicatedIndex = 3, but since we want to keep the lastapplied, indices 0 and 1 will only get purged
254         context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0, 5, 1).build());
255         context.setLastApplied(2);
256         abstractBehavior.performSnapshotWithoutCapture(3);
257         assertEquals(1, abstractBehavior.getReplicatedToAllIndex());
258         assertEquals(3, context.getReplicatedLog().size());
259
260         // scenario where Last applied > Replicated to all index (becoz of a slow follower)
261         context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0, 3, 1).build());
262         context.setLastApplied(2);
263         abstractBehavior.performSnapshotWithoutCapture(1);
264         assertEquals(1, abstractBehavior.getReplicatedToAllIndex());
265         assertEquals(1, context.getReplicatedLog().size());
266     }
267
268
269     protected void assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(RaftActorContext actorContext,
270             ActorRef actorRef, RaftRPC rpc) throws Exception {
271
272         Payload p = new MockRaftActorContext.MockPayload("");
273         setLastLogEntry((MockRaftActorContext) actorContext, 1, 0, p);
274         actorContext.getTermInformation().update(1, "test");
275
276         RaftActorBehavior origBehavior = createBehavior(actorContext);
277         RaftActorBehavior raftBehavior = origBehavior.handleMessage(actorRef, rpc);
278
279         assertEquals("New raft state", RaftState.Follower, raftBehavior.state());
280         assertEquals("New election term", rpc.getTerm(), actorContext.getTermInformation().getCurrentTerm());
281
282         origBehavior.close();
283         raftBehavior.close();
284     }
285
286     protected MockRaftActorContext.SimpleReplicatedLog setLastLogEntry(
287         MockRaftActorContext actorContext, long term, long index, Payload data) {
288         return setLastLogEntry(actorContext,
289             new MockRaftActorContext.MockReplicatedLogEntry(term, index, data));
290     }
291
292     protected MockRaftActorContext.SimpleReplicatedLog setLastLogEntry(MockRaftActorContext actorContext,
293             ReplicatedLogEntry logEntry) {
294         MockRaftActorContext.SimpleReplicatedLog log = new MockRaftActorContext.SimpleReplicatedLog();
295         log.append(logEntry);
296         actorContext.setReplicatedLog(log);
297
298         return log;
299     }
300
301     protected abstract RaftActorBehavior createBehavior(
302         RaftActorContext actorContext);
303
304     protected RaftActorBehavior createBehavior() {
305         return createBehavior(createActorContext());
306     }
307
308     protected MockRaftActorContext createActorContext() {
309         return new MockRaftActorContext();
310     }
311
312     protected MockRaftActorContext createActorContext(ActorRef actor) {
313         return new MockRaftActorContext("test", getSystem(), actor);
314     }
315
316     protected AppendEntries createAppendEntriesWithNewerTerm() {
317         return new AppendEntries(100, "leader-1", 0, 0, null, 1, -1);
318     }
319
320     protected AppendEntriesReply createAppendEntriesReplyWithNewerTerm() {
321         return new AppendEntriesReply("follower-1", 100, false, 100, 100);
322     }
323
324     protected RequestVote createRequestVoteWithNewerTerm() {
325         return new RequestVote(100, "candidate-1", 10, 100);
326     }
327
328     protected RequestVoteReply createRequestVoteReplyWithNewerTerm() {
329         return new RequestVoteReply(100, false);
330     }
331
332     protected Object fromSerializableMessage(Object serializable){
333         return SerializationUtils.fromSerializable(serializable);
334     }
335
336     protected ByteString toByteString(Map<String, String> state) {
337         ByteArrayOutputStream bos = new ByteArrayOutputStream();
338         try(ObjectOutputStream oos = new ObjectOutputStream(bos)) {
339             oos.writeObject(state);
340             return ByteString.copyFrom(bos.toByteArray());
341         } catch (IOException e) {
342             throw new AssertionError("IOException occurred converting Map to Bytestring", e);
343         }
344     }
345
346     protected void logStart(String name) {
347         LoggerFactory.getLogger(LeaderTest.class).info("Starting " + name);
348     }
349 }