Add ServerConfigPayload to InstallSnapshot message
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest.java
1 /*
2  * Copyright (c) 2015 Brocade Communications Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.raft;
9
10 import static org.junit.Assert.assertEquals;
11 import static org.junit.Assert.assertNotNull;
12 import akka.actor.ActorRef;
13 import akka.persistence.SaveSnapshotSuccess;
14 import com.google.common.collect.ImmutableMap;
15 import java.util.Arrays;
16 import java.util.HashSet;
17 import java.util.List;
18 import java.util.Map;
19 import java.util.Set;
20 import javax.annotation.Nullable;
21 import org.junit.Assert;
22 import org.junit.Test;
23 import org.opendaylight.controller.cluster.raft.MockRaftActorContext.MockPayload;
24 import org.opendaylight.controller.cluster.raft.ServerConfigurationPayload.ServerInfo;
25 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
26 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
27 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
28 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
29 import org.opendaylight.controller.cluster.raft.base.messages.UpdateElectionTerm;
30 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
31 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
32 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
33 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
34 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
35 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
36 import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal;
37 import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore;
38 import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor;
39
40 /**
41  * Tests replication and snapshots end-to-end using real RaftActors and behavior communication with a
42  * lagging follower.
43  *
44  * @author Thomas Pantelis
45  */
46 public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends AbstractRaftActorIntegrationTest {
47
48     private void setup() {
49         leaderId = factory.generateActorId("leader");
50         follower1Id = factory.generateActorId("follower");
51         follower2Id = factory.generateActorId("follower");
52
53         // Setup the persistent journal for the leader - just an election term and no journal/snapshots.
54         InMemoryJournal.addEntry(leaderId, 1, new UpdateElectionTerm(initialTerm, leaderId));
55
56         // Create the leader and 2 follower actors.
57         follower1Actor = newTestRaftActor(follower1Id, ImmutableMap.of(leaderId, testActorPath(leaderId),
58                 follower2Id, testActorPath(follower2Id)), newFollowerConfigParams());
59
60         follower2Actor = newTestRaftActor(follower2Id, ImmutableMap.of(leaderId, testActorPath(leaderId),
61                 follower1Id, testActorPath(follower1Id)), newFollowerConfigParams());
62
63         Map<String, String> peerAddresses = ImmutableMap.<String, String>builder().
64                 put(follower1Id, follower1Actor.path().toString()).
65                 put(follower2Id, follower2Actor.path().toString()).build();
66
67         leaderConfigParams = newLeaderConfigParams();
68         leaderActor = newTestRaftActor(leaderId, peerAddresses, leaderConfigParams);
69
70         waitUntilLeader(leaderActor);
71
72         leaderContext = leaderActor.underlyingActor().getRaftActorContext();
73         leader = leaderActor.underlyingActor().getCurrentBehavior();
74
75         follower1Context = follower1Actor.underlyingActor().getRaftActorContext();
76         follower1 = follower1Actor.underlyingActor().getCurrentBehavior();
77
78         follower2Context = follower2Actor.underlyingActor().getRaftActorContext();
79         follower2 = follower2Actor.underlyingActor().getCurrentBehavior();
80
81         currentTerm = leaderContext.getTermInformation().getCurrentTerm();
82         assertEquals("Current term > " + initialTerm, true, currentTerm > initialTerm);
83
84         leaderCollectorActor = leaderActor.underlyingActor().collectorActor();
85         follower1CollectorActor = follower1Actor.underlyingActor().collectorActor();
86         follower2CollectorActor = follower2Actor.underlyingActor().collectorActor();
87
88         testLog.info("Leader created and elected");
89     }
90
91     /**
92      * Send 2 payload instances with follower 2 lagging then resume the follower and verifies it gets
93      * caught up via AppendEntries.
94      */
95     @Test
96     public void testReplicationsWithLaggingFollowerCaughtUpViaAppendEntries() throws Exception {
97         testLog.info("testReplicationsWithLaggingFollowerCaughtUpViaAppendEntries starting: sending 2 new payloads");
98
99         setup();
100
101         // Simulate lagging by dropping AppendEntries messages in follower 2.
102         follower2Actor.underlyingActor().startDropMessages(AppendEntries.class);
103
104         // Send the payloads.
105         MockPayload payload0 = sendPayloadData(leaderActor, "zero");
106         MockPayload payload1 = sendPayloadData(leaderActor, "one");
107
108         // Verify the leader got consensus and applies each log entry even though follower 2 didn't respond.
109         List<ApplyState> applyStates = MessageCollectorActor.expectMatching(leaderCollectorActor, ApplyState.class, 2);
110         verifyApplyState(applyStates.get(0), leaderCollectorActor, payload0.toString(), currentTerm, 0, payload0);
111         verifyApplyState(applyStates.get(1), leaderCollectorActor, payload1.toString(), currentTerm, 1, payload1);
112
113         // Verify follower 1 applies each log entry.
114         applyStates = MessageCollectorActor.expectMatching(follower1CollectorActor, ApplyState.class, 2);
115         verifyApplyState(applyStates.get(0), null, null, currentTerm, 0, payload0);
116         verifyApplyState(applyStates.get(1), null, null, currentTerm, 1, payload1);
117
118         // Ensure there's at least 1 more heartbeat.
119         MessageCollectorActor.clearMessages(leaderCollectorActor);
120         MessageCollectorActor.expectFirstMatching(leaderCollectorActor, AppendEntriesReply.class);
121
122         // The leader should not have performed fake snapshots to trim the log because the entries have not
123         // been replicated to follower 2.
124         assertEquals("Leader snapshot term", -1, leaderContext.getReplicatedLog().getSnapshotTerm());
125         assertEquals("Leader snapshot index", -1, leaderContext.getReplicatedLog().getSnapshotIndex());
126         assertEquals("Leader journal log size", 2, leaderContext.getReplicatedLog().size());
127         assertEquals("Leader journal last index", 1, leaderContext.getReplicatedLog().lastIndex());
128         assertEquals("Leader commit index", 1, leaderContext.getCommitIndex());
129         assertEquals("Leader last applied", 1, leaderContext.getLastApplied());
130         assertEquals("Leader replicatedToAllIndex", -1, leader.getReplicatedToAllIndex());
131
132         testLog.info("testReplicationsWithLaggingFollowerCaughtUpViaAppendEntries: new entries applied - resuming follower {}", follower2Id);
133
134         // Now stop dropping AppendEntries in follower 2.
135         follower2Actor.underlyingActor().stopDropMessages(AppendEntries.class);
136
137         // Verify follower 2 applies each log entry.
138         applyStates = MessageCollectorActor.expectMatching(follower2CollectorActor, ApplyState.class, 2);
139         verifyApplyState(applyStates.get(0), null, null, currentTerm, 0, payload0);
140         verifyApplyState(applyStates.get(1), null, null, currentTerm, 1, payload1);
141
142         // Ensure there's at least 1 more heartbeat.
143         MessageCollectorActor.clearMessages(leaderCollectorActor);
144         MessageCollectorActor.expectFirstMatching(leaderCollectorActor, AppendEntriesReply.class);
145
146         // The leader should now have performed fake snapshots to trim the log.
147         verifyLeadersTrimmedLog(1);
148
149         // Even though follower 2 lagged behind, the leader should not have tried to install a snapshot
150         // to catch it up because no snapshotting was done so the follower's next index was present in the log.
151         InstallSnapshot installSnapshot = MessageCollectorActor.getFirstMatching(follower2CollectorActor,
152                 InstallSnapshot.class);
153         Assert.assertNull("Follower 2 received unexpected InstallSnapshot", installSnapshot);
154
155         testLog.info("testReplicationsWithLaggingFollowerCaughtUpViaAppendEntries complete");
156     }
157
158     /**
159      * Send payloads to trigger a leader snapshot due to snapshotBatchCount reached with follower 2
160      * lagging but not enough for the leader to trim its log from the last applied index. Follower 2's log
161      * will be behind by several entries and, when it is resumed, it should be caught up via AppendEntries
162      * sent by the leader.
163      *
164      * @throws Exception
165      */
166     @Test
167     public void testLeaderSnapshotWithLaggingFollowerCaughtUpViaAppendEntries() throws Exception {
168         testLog.info("testLeaderSnapshotWithLaggingFollowerCaughtUpViaAppendEntries starting");
169
170         setup();
171
172         sendInitialPayloadsReplicatedToAllFollowers("zero", "one");
173
174         // Configure follower 2 to drop messages and lag.
175         follower2Actor.underlyingActor().startDropMessages(AppendEntries.class);
176
177         // Send the first payload and verify it gets applied by the leader and follower 1.
178         MockPayload payload2 = sendPayloadData(leaderActor, "two");
179
180         ApplyState applyState = MessageCollectorActor.expectFirstMatching(leaderCollectorActor, ApplyState.class);
181         verifyApplyState(applyState, leaderCollectorActor, payload2.toString(), currentTerm, 2, payload2);
182
183         applyState = MessageCollectorActor.expectFirstMatching(follower1CollectorActor, ApplyState.class);
184         verifyApplyState(applyState, null, null, currentTerm, 2, payload2);
185
186         expSnapshotState.add(payload2);
187
188         MessageCollectorActor.clearMessages(leaderCollectorActor);
189         MessageCollectorActor.clearMessages(follower1CollectorActor);
190
191         // Send another payload - this should cause a snapshot due to snapshotBatchCount reached.
192         MockPayload payload3 = sendPayloadData(leaderActor, "three");
193
194         MessageCollectorActor.expectFirstMatching(leaderCollectorActor, SaveSnapshotSuccess.class);
195
196         testLog.info("testLeaderSnapshotWithLaggingFollowerCaughtUpViaAppendEntries: sending 2 more payloads");
197
198         // Send 2 more payloads - not enough to trigger another snapshot.
199         MockPayload payload4 = sendPayloadData(leaderActor, "four");
200         MockPayload payload5 = sendPayloadData(leaderActor, "five");
201
202         // Verify the leader got consensus and applies each log entry even though follower 2 didn't respond.
203         List<ApplyState> applyStates = MessageCollectorActor.expectMatching(leaderCollectorActor, ApplyState.class, 3);
204         verifyApplyState(applyStates.get(0), leaderCollectorActor, payload3.toString(), currentTerm, 3, payload3);
205         verifyApplyState(applyStates.get(1), leaderCollectorActor, payload4.toString(), currentTerm, 4, payload4);
206         verifyApplyState(applyStates.get(2), leaderCollectorActor, payload5.toString(), currentTerm, 5, payload5);
207
208         // Verify follower 1 applies each log entry.
209         applyStates = MessageCollectorActor.expectMatching(follower1CollectorActor, ApplyState.class, 3);
210         verifyApplyState(applyStates.get(0), null, null, currentTerm, 3, payload3);
211         verifyApplyState(applyStates.get(1), null, null, currentTerm, 4, payload4);
212         verifyApplyState(applyStates.get(2), null, null, currentTerm, 5, payload5);
213
214         // The snapshot should have caused the leader to advanced the snapshot index to the
215         // last previously applied index (1) that was replicated to all followers at the time of capture.
216         // Note: since the log size (3) did not exceed the snapshot batch count (4), the leader should not
217         // have trimmed the log to the last index actually applied (5).
218         assertEquals("Leader snapshot term", currentTerm, leaderContext.getReplicatedLog().getSnapshotTerm());
219         assertEquals("Leader snapshot index", 1, leaderContext.getReplicatedLog().getSnapshotIndex());
220         assertEquals("Leader journal log size", 4, leaderContext.getReplicatedLog().size());
221         assertEquals("Leader journal last index", 5, leaderContext.getReplicatedLog().lastIndex());
222         assertEquals("Leader commit index", 5, leaderContext.getCommitIndex());
223         assertEquals("Leader last applied", 5, leaderContext.getLastApplied());
224         assertEquals("Leader replicatedToAllIndex", 1, leader.getReplicatedToAllIndex());
225
226         // Now stop dropping AppendEntries in follower 2.
227         follower2Actor.underlyingActor().stopDropMessages(AppendEntries.class);
228
229         // Verify follower 2 applies each log entry. The leader should not install a snapshot b/c
230         // follower 2's next index (3) is still present in the log.
231         applyStates = MessageCollectorActor.expectMatching(follower2CollectorActor, ApplyState.class, 4);
232         verifyApplyState(applyStates.get(0), null, null, currentTerm, 2, payload2);
233         verifyApplyState(applyStates.get(1), null, null, currentTerm, 3, payload3);
234         verifyApplyState(applyStates.get(2), null, null, currentTerm, 4, payload4);
235         verifyApplyState(applyStates.get(3), null, null, currentTerm, 5, payload5);
236
237         // Verify the leader did not try to install a snapshot to catch up follower 2.
238         InstallSnapshot installSnapshot = MessageCollectorActor.getFirstMatching(follower2CollectorActor, InstallSnapshot.class);
239         Assert.assertNull("Follower 2 received unexpected InstallSnapshot", installSnapshot);
240
241         // Ensure there's at least 1 more heartbeat.
242         MessageCollectorActor.clearMessages(leaderCollectorActor);
243         MessageCollectorActor.expectFirstMatching(leaderCollectorActor, AppendEntriesReply.class);
244
245         // The leader should now have performed fake snapshots to advance the snapshot index and to trim
246         // the log. In addition replicatedToAllIndex should've advanced.
247         verifyLeadersTrimmedLog(5);
248
249         // Verify the leader's persisted snapshot.
250         List<Snapshot> persistedSnapshots = InMemorySnapshotStore.getSnapshots(leaderId, Snapshot.class);
251         assertEquals("Persisted snapshots size", 1, persistedSnapshots.size());
252         verifySnapshot("Persisted", persistedSnapshots.get(0), currentTerm, 2, currentTerm, 3);
253         List<ReplicatedLogEntry> unAppliedEntry = persistedSnapshots.get(0).getUnAppliedEntries();
254         assertEquals("Persisted Snapshot getUnAppliedEntries size", 1, unAppliedEntry.size());
255         verifyReplicatedLogEntry(unAppliedEntry.get(0), currentTerm, 3, payload3);
256
257         // Verify follower 1's log and snapshot indexes.
258         MessageCollectorActor.clearMessages(follower1CollectorActor);
259         MessageCollectorActor.expectFirstMatching(follower1CollectorActor, AppendEntries.class);
260         verifyFollowersTrimmedLog(1, follower1Actor, 5);
261
262         // Verify follower 2's log and snapshot indexes.
263         MessageCollectorActor.clearMessages(follower2CollectorActor);
264         MessageCollectorActor.expectFirstMatching(follower2CollectorActor, AppendEntries.class);
265         verifyFollowersTrimmedLog(2, follower2Actor, 5);
266
267         MessageCollectorActor.clearMessages(leaderCollectorActor);
268         MessageCollectorActor.clearMessages(follower1CollectorActor);
269         MessageCollectorActor.clearMessages(follower2CollectorActor);
270
271         expSnapshotState.add(payload3);
272         expSnapshotState.add(payload4);
273         expSnapshotState.add(payload5);
274
275         testLog.info("testLeaderSnapshotWithLaggingFollowerCaughtUpViaAppendEntries complete");
276     }
277
278     /**
279      * Send payloads to trigger a leader snapshot due to snapshotBatchCount reached with follower 2
280      * lagging where the leader trims its log from the last applied index. Follower 2's log
281      * will be behind by several entries and, when it is resumed, it should be caught up via a snapshot
282      * installed by the leader.
283      *
284      * @throws Exception
285      */
286     @Test
287     public void testLeaderSnapshotWithLaggingFollowerCaughtUpViaInstallSnapshot() throws Exception {
288         testLog.info("testLeaderSnapshotWithLaggingFollowerCaughtUpViaInstallSnapshot starting");
289
290         setup();
291
292         sendInitialPayloadsReplicatedToAllFollowers("zero", "one");
293
294         // Configure follower 2 to drop messages and lag.
295         follower2Actor.underlyingActor().startDropMessages(AppendEntries.class);
296
297         // Send 5 payloads - the second should cause a leader snapshot.
298         MockPayload payload2 = sendPayloadData(leaderActor, "two");
299         MockPayload payload3 = sendPayloadData(leaderActor, "three");
300         MockPayload payload4 = sendPayloadData(leaderActor, "four");
301         MockPayload payload5 = sendPayloadData(leaderActor, "five");
302         MockPayload payload6 = sendPayloadData(leaderActor, "six");
303
304         MessageCollectorActor.expectFirstMatching(leaderCollectorActor, SaveSnapshotSuccess.class);
305
306         // Verify the leader got consensus and applies each log entry even though follower 2 didn't respond.
307         List<ApplyState> applyStates = MessageCollectorActor.expectMatching(leaderCollectorActor, ApplyState.class, 5);
308         verifyApplyState(applyStates.get(0), leaderCollectorActor, payload2.toString(), currentTerm, 2, payload2);
309         verifyApplyState(applyStates.get(2), leaderCollectorActor, payload4.toString(), currentTerm, 4, payload4);
310         verifyApplyState(applyStates.get(4), leaderCollectorActor, payload6.toString(), currentTerm, 6, payload6);
311
312         MessageCollectorActor.clearMessages(leaderCollectorActor);
313
314         testLog.info("testLeaderSnapshotWithLaggingFollowerCaughtUpViaAppendEntries: sending 1 more payload to trigger second snapshot");
315
316         // Send another payload to trigger a second leader snapshot.
317         MockPayload payload7 = sendPayloadData(leaderActor, "seven");
318
319         MessageCollectorActor.expectFirstMatching(leaderCollectorActor, SaveSnapshotSuccess.class);
320
321         ApplyState applyState = MessageCollectorActor.expectFirstMatching(leaderCollectorActor, ApplyState.class);
322         verifyApplyState(applyState, leaderCollectorActor, payload7.toString(), currentTerm, 7, payload7);
323
324         // Verify follower 1 applies each log entry.
325         applyStates = MessageCollectorActor.expectMatching(follower1CollectorActor, ApplyState.class, 6);
326         verifyApplyState(applyStates.get(0), null, null, currentTerm, 2, payload2);
327         verifyApplyState(applyStates.get(2), null, null, currentTerm, 4, payload4);
328         verifyApplyState(applyStates.get(5), null, null, currentTerm, 7, payload7);
329
330         // The snapshot should have caused the leader to advanced the snapshot index to the leader's last
331         // applied index (6) since the log size should have exceed the snapshot batch count (4).
332         // replicatedToAllIndex should remain at 1 since follower 2 is lagging.
333         verifyLeadersTrimmedLog(7, 1);
334
335         expSnapshotState.add(payload2);
336         expSnapshotState.add(payload3);
337         expSnapshotState.add(payload4);
338         expSnapshotState.add(payload5);
339         expSnapshotState.add(payload6);
340
341         MessageCollectorActor.clearMessages(leaderCollectorActor);
342         MessageCollectorActor.clearMessages(follower1CollectorActor);
343
344         // Send a server config change to test that the install snapshot includes the server config.
345
346         ServerConfigurationPayload serverConfig = new ServerConfigurationPayload(Arrays.asList(
347                 new ServerInfo(leaderId, true),
348                 new ServerInfo(follower1Id, false),
349                 new ServerInfo(follower2Id, false)));
350         leaderContext.updatePeerIds(serverConfig);
351         ((AbstractLeader)leader).updateMinReplicaCount();
352         leaderActor.tell(serverConfig, ActorRef.noSender());
353
354         applyState = MessageCollectorActor.expectFirstMatching(leaderCollectorActor, ApplyState.class);
355         verifyApplyState(applyState, leaderCollectorActor, "serverConfig", currentTerm, 8, serverConfig);
356
357         applyState = MessageCollectorActor.expectFirstMatching(follower1CollectorActor, ApplyState.class);
358         verifyApplyState(applyState, null, null, currentTerm, 8, serverConfig);
359
360         // Verify the leader's persisted snapshot.
361         List<Snapshot> persistedSnapshots = InMemorySnapshotStore.getSnapshots(leaderId, Snapshot.class);
362         assertEquals("Persisted snapshots size", 1, persistedSnapshots.size());
363         verifySnapshot("Persisted", persistedSnapshots.get(0), currentTerm, 6, currentTerm, 7);
364         List<ReplicatedLogEntry> unAppliedEntry = persistedSnapshots.get(0).getUnAppliedEntries();
365         assertEquals("Persisted Snapshot getUnAppliedEntries size", 1, unAppliedEntry.size());
366         verifyReplicatedLogEntry(unAppliedEntry.get(0), currentTerm, 7, payload7);
367
368         expSnapshotState.add(payload7);
369
370         verifyInstallSnapshotToLaggingFollower(8, serverConfig);
371
372         testLog.info("testLeaderSnapshotWithLaggingFollowerCaughtUpViaInstallSnapshot complete");
373     }
374
375     /**
376      * Send payloads with follower 2 lagging with the last payload having a large enough size to trigger a
377      * leader snapshot such that the leader trims its log from the last applied index.. Follower 2's log will
378      * be behind by several entries and, when it is resumed, it should be caught up via a snapshot installed
379      * by the leader.
380      *
381      * @throws Exception
382      */
383     @Test
384     public void testLeaderSnapshotTriggeredByMemoryThresholdExceededWithLaggingFollower() throws Exception {
385         testLog.info("testLeaderSnapshotTriggeredByMemoryThresholdExceededWithLaggingFollower starting");
386
387         snapshotBatchCount = 5;
388         setup();
389
390         sendInitialPayloadsReplicatedToAllFollowers("zero");
391
392         leaderActor.underlyingActor().setMockTotalMemory(1000);
393
394         // We'll expect a ReplicatedLogImplEntry message and an ApplyJournalEntries message added to the journal.
395         InMemoryJournal.addWriteMessagesCompleteLatch(leaderId, 2);
396
397         follower2Actor.underlyingActor().startDropMessages(AppendEntries.class);
398
399         // Send a payload with a large relative size but not enough to trigger a snapshot.
400         MockPayload payload1 = sendPayloadData(leaderActor, "one", 500);
401
402         // Verify the leader got consensus and applies the first log entry even though follower 2 didn't respond.
403         List<ApplyState> applyStates = MessageCollectorActor.expectMatching(leaderCollectorActor, ApplyState.class, 1);
404         verifyApplyState(applyStates.get(0), leaderCollectorActor, payload1.toString(), currentTerm, 1, payload1);
405
406         // Wait for all the ReplicatedLogImplEntry and ApplyJournalEntries messages to be added to the journal
407         // before the snapshot so the snapshot sequence # will be higher to ensure the snapshot gets
408         // purged from the snapshot store after subsequent snapshots.
409         InMemoryJournal.waitForWriteMessagesComplete(leaderId);
410
411         // Verify a snapshot is not triggered.
412         CaptureSnapshot captureSnapshot = MessageCollectorActor.getFirstMatching(leaderCollectorActor, CaptureSnapshot.class);
413         Assert.assertNull("Leader received unexpected CaptureSnapshot", captureSnapshot);
414
415         expSnapshotState.add(payload1);
416
417         // Send another payload with a large enough relative size in combination with the last payload
418         // that exceeds the memory threshold (70% * 1000 = 700) - this should do a snapshot.
419         MockPayload payload2 = sendPayloadData(leaderActor, "two", 201);
420
421         // Verify the leader applies the last log entry.
422         applyStates = MessageCollectorActor.expectMatching(leaderCollectorActor, ApplyState.class, 2);
423         verifyApplyState(applyStates.get(1), leaderCollectorActor, payload2.toString(), currentTerm, 2, payload2);
424
425         // Verify follower 1 applies each log entry.
426         applyStates = MessageCollectorActor.expectMatching(follower1CollectorActor, ApplyState.class, 2);
427         verifyApplyState(applyStates.get(0), null, null, currentTerm, 1, payload1);
428         verifyApplyState(applyStates.get(1), null, null, currentTerm, 2, payload2);
429
430         // A snapshot should've occurred - wait for it to complete.
431         MessageCollectorActor.expectFirstMatching(leaderCollectorActor, SaveSnapshotSuccess.class);
432
433         // Because the snapshot was triggered by exceeding the memory threshold the leader should've advanced
434         // the snapshot index to the last applied index and trimmed the log even though the entries weren't
435         // replicated to all followers.
436         verifyLeadersTrimmedLog(2, 0);
437
438         // Verify the leader's persisted snapshot.
439         List<Snapshot> persistedSnapshots = InMemorySnapshotStore.getSnapshots(leaderId, Snapshot.class);
440         assertEquals("Persisted snapshots size", 1, persistedSnapshots.size());
441         verifySnapshot("Persisted", persistedSnapshots.get(0), currentTerm, 1, currentTerm, 2);
442         List<ReplicatedLogEntry> unAppliedEntry = persistedSnapshots.get(0).getUnAppliedEntries();
443         assertEquals("Persisted Snapshot getUnAppliedEntries size", 1, unAppliedEntry.size());
444         verifyReplicatedLogEntry(unAppliedEntry.get(0), currentTerm, 2, payload2);
445
446         expSnapshotState.add(payload2);
447
448         verifyInstallSnapshotToLaggingFollower(2L, null);
449
450         // Sends a payload with index 3.
451         verifyNoSubsequentSnapshotAfterMemoryThresholdExceededSnapshot();
452
453         // Sends 3 payloads with indexes 4, 5 and 6.
454         verifyReplicationsAndSnapshotWithNoLaggingAfterInstallSnapshot();
455
456         // Recover the leader from persistence and verify.
457         long leadersLastIndexOnRecovery = 6;
458
459         // The leader's last snapshot was triggered by index 4 so the last applied index in the snapshot was 3.
460         long leadersSnapshotIndexOnRecovery = 3;
461
462         // The recovered journal should have 3 entries starting at index 4.
463         long leadersFirstJournalEntryIndexOnRecovery = 4;
464
465         verifyLeaderRecoveryAfterReinstatement(leadersLastIndexOnRecovery, leadersSnapshotIndexOnRecovery,
466                 leadersFirstJournalEntryIndexOnRecovery);
467
468         testLog.info("testLeaderSnapshotTriggeredByMemoryThresholdExceeded ending");
469     }
470
471     /**
472      * Send another payload to verify another snapshot is not done since the last snapshot trimmed the
473      * first log entry so the memory threshold should not be exceeded.
474      *
475      * @throws Exception
476      */
477     private void verifyNoSubsequentSnapshotAfterMemoryThresholdExceededSnapshot() throws Exception {
478         ApplyState applyState;
479         CaptureSnapshot captureSnapshot;
480
481         MockPayload payload3 = sendPayloadData(leaderActor, "three");
482
483         // Verify the leader applies the state.
484         applyState = MessageCollectorActor.expectFirstMatching(leaderCollectorActor, ApplyState.class);
485         verifyApplyState(applyState, leaderCollectorActor, payload3.toString(), currentTerm, 3, payload3);
486
487         captureSnapshot = MessageCollectorActor.getFirstMatching(leaderCollectorActor, CaptureSnapshot.class);
488         Assert.assertNull("Leader received unexpected CaptureSnapshot", captureSnapshot);
489
490         // Verify the follower 1 applies the state.
491         applyState = MessageCollectorActor.expectFirstMatching(follower1CollectorActor, ApplyState.class);
492         verifyApplyState(applyState, null, null, currentTerm, 3, payload3);
493
494         // Verify the follower 2 applies the state.
495         applyState = MessageCollectorActor.expectFirstMatching(follower2CollectorActor, ApplyState.class);
496         verifyApplyState(applyState, null, null, currentTerm, 3, payload3);
497
498         // Verify the leader's state.
499         verifyLeadersTrimmedLog(3);
500
501         // Verify follower 1's state.
502         verifyFollowersTrimmedLog(1, follower1Actor, 3);
503
504         // Verify follower 2's state.
505         verifyFollowersTrimmedLog(2, follower2Actor, 3);
506
507         // Revert back to JVM total memory.
508         leaderActor.underlyingActor().setMockTotalMemory(0);
509
510         MessageCollectorActor.clearMessages(leaderCollectorActor);
511         MessageCollectorActor.clearMessages(follower1CollectorActor);
512         MessageCollectorActor.clearMessages(follower2CollectorActor);
513
514         expSnapshotState.add(payload3);
515     }
516
517     /**
518      * Resume the lagging follower 2 and verify it receives an install snapshot from the leader.
519      *
520      * @throws Exception
521      */
522     private void verifyInstallSnapshotToLaggingFollower(long lastAppliedIndex,
523             @Nullable ServerConfigurationPayload expServerConfig) throws Exception {
524         List<Snapshot> persistedSnapshots;
525         List<ReplicatedLogEntry> unAppliedEntry;
526         ApplySnapshot applySnapshot;
527         InstallSnapshot installSnapshot;
528
529         testLog.info("testInstallSnapshotToLaggingFollower starting");
530
531         MessageCollectorActor.clearMessages(leaderCollectorActor);
532
533         // Now stop dropping AppendEntries in follower 2.
534         follower2Actor.underlyingActor().stopDropMessages(AppendEntries.class);
535
536
537         MessageCollectorActor.expectFirstMatching(leaderCollectorActor, SaveSnapshotSuccess.class);
538
539         // Verify the leader's persisted snapshot. The previous snapshot (currently) won't be deleted from
540         // the snapshot store because the second snapshot was initiated by the follower install snapshot and
541         // not because the batch count was reached so the persisted journal sequence number wasn't advanced
542         // far enough to cause the previous snapshot to be deleted. This is because
543         // RaftActor#trimPersistentData subtracts the snapshotBatchCount from the snapshot's sequence number.
544         // This is OK - the next snapshot should delete it. In production, even if the system restarted
545         // before another snapshot, they would both get applied which wouldn't hurt anything.
546         persistedSnapshots = InMemorySnapshotStore.getSnapshots(leaderId, Snapshot.class);
547         Assert.assertTrue("Expected at least 1 persisted snapshots", persistedSnapshots.size() > 0);
548         Snapshot persistedSnapshot = persistedSnapshots.get(persistedSnapshots.size() - 1);
549         verifySnapshot("Persisted", persistedSnapshot, currentTerm, lastAppliedIndex, currentTerm, lastAppliedIndex);
550         unAppliedEntry = persistedSnapshot.getUnAppliedEntries();
551         assertEquals("Persisted Snapshot getUnAppliedEntries size", 0, unAppliedEntry.size());
552
553         int snapshotSize = persistedSnapshot.getState().length;
554         int expTotalChunks = (snapshotSize / SNAPSHOT_CHUNK_SIZE) + ((snapshotSize % SNAPSHOT_CHUNK_SIZE) > 0 ? 1 : 0);
555
556         installSnapshot = MessageCollectorActor.expectFirstMatching(follower2CollectorActor, InstallSnapshot.class);
557         assertEquals("InstallSnapshot getTerm", currentTerm, installSnapshot.getTerm());
558         assertEquals("InstallSnapshot getLeaderId", leaderId, installSnapshot.getLeaderId());
559         assertEquals("InstallSnapshot getChunkIndex", 1, installSnapshot.getChunkIndex());
560         assertEquals("InstallSnapshot getTotalChunks", expTotalChunks, installSnapshot.getTotalChunks());
561         assertEquals("InstallSnapshot getLastIncludedTerm", currentTerm, installSnapshot.getLastIncludedTerm());
562         assertEquals("InstallSnapshot getLastIncludedIndex", lastAppliedIndex, installSnapshot.getLastIncludedIndex());
563         //assertArrayEquals("InstallSnapshot getData", snapshot, installSnapshot.getData().toByteArray());
564
565         List<InstallSnapshotReply> installSnapshotReplies = MessageCollectorActor.expectMatching(
566                 leaderCollectorActor, InstallSnapshotReply.class, expTotalChunks);
567         int index = 1;
568         for(InstallSnapshotReply installSnapshotReply: installSnapshotReplies) {
569             assertEquals("InstallSnapshotReply getTerm", currentTerm, installSnapshotReply.getTerm());
570             assertEquals("InstallSnapshotReply getChunkIndex", index++, installSnapshotReply.getChunkIndex());
571             assertEquals("InstallSnapshotReply getFollowerId", follower2Id, installSnapshotReply.getFollowerId());
572             assertEquals("InstallSnapshotReply isSuccess", true, installSnapshotReply.isSuccess());
573         }
574
575         // Verify follower 2 applies the snapshot.
576         applySnapshot = MessageCollectorActor.expectFirstMatching(follower2CollectorActor, ApplySnapshot.class);
577         verifySnapshot("Follower 2", applySnapshot.getSnapshot(), currentTerm, lastAppliedIndex, currentTerm, lastAppliedIndex);
578         assertEquals("Persisted Snapshot getUnAppliedEntries size", 0, applySnapshot.getSnapshot().getUnAppliedEntries().size());
579
580         // Wait for the snapshot to complete.
581         MessageCollectorActor.expectFirstMatching(leaderCollectorActor, SaveSnapshotSuccess.class);
582
583         // Ensure there's at least 1 more heartbeat.
584         MessageCollectorActor.clearMessages(leaderCollectorActor);
585         MessageCollectorActor.expectFirstMatching(leaderCollectorActor, AppendEntriesReply.class);
586
587         // The leader should now have performed fake snapshots to advance the snapshot index and to trim
588         // the log. In addition replicatedToAllIndex should've advanced.
589         verifyLeadersTrimmedLog(lastAppliedIndex);
590
591         if(expServerConfig != null) {
592             Set<ServerInfo> expServerInfo = new HashSet<>(expServerConfig.getServerConfig());
593             assertEquals("Leader snapshot server config", expServerInfo,
594                     new HashSet<>(persistedSnapshot.getServerConfiguration().getServerConfig()));
595
596             assertEquals("Follower 2 snapshot server config", expServerInfo,
597                     new HashSet<>(applySnapshot.getSnapshot().getServerConfiguration().getServerConfig()));
598
599             ServerConfigurationPayload follower2ServerConfig = follower2Context.getPeerServerInfo(true);
600             assertNotNull("Follower 2 server config is null", follower2ServerConfig);
601
602             assertEquals("Follower 2 server config", expServerInfo,
603                     new HashSet<>(follower2ServerConfig.getServerConfig()));
604         }
605
606         MessageCollectorActor.clearMessages(leaderCollectorActor);
607         MessageCollectorActor.clearMessages(follower1CollectorActor);
608         MessageCollectorActor.clearMessages(follower2CollectorActor);
609
610         testLog.info("testInstallSnapshotToLaggingFollower complete");
611     }
612
613     /**
614      * Do another round of payloads and snapshot to verify replicatedToAllIndex gets back on track and
615      * snapshots works as expected after doing a follower snapshot. In this step we don't lag a follower.
616      *
617      * @throws Exception
618      */
619     private void verifyReplicationsAndSnapshotWithNoLaggingAfterInstallSnapshot() throws Exception {
620         List<ApplyState> applyStates;
621         ApplyState applyState;
622
623         testLog.info("testReplicationsAndSnapshotAfterInstallSnapshot starting: replicatedToAllIndex: {}",
624                 leader.getReplicatedToAllIndex());
625
626         // Send another payload - a snapshot should occur.
627         MockPayload payload4 = sendPayloadData(leaderActor, "four");
628
629         // Wait for the snapshot to complete.
630         MessageCollectorActor.expectFirstMatching(leaderCollectorActor, SaveSnapshotSuccess.class);
631
632         applyState = MessageCollectorActor.expectFirstMatching(leaderCollectorActor, ApplyState.class);
633         verifyApplyState(applyState, leaderCollectorActor, payload4.toString(), currentTerm, 4, payload4);
634
635         // Verify the leader's last persisted snapshot (previous ones may not be purged yet).
636         List<Snapshot> persistedSnapshots = InMemorySnapshotStore.getSnapshots(leaderId, Snapshot.class);
637         Snapshot persistedSnapshot = persistedSnapshots.get(persistedSnapshots.size() - 1);
638         verifySnapshot("Persisted", persistedSnapshot, currentTerm, 3, currentTerm, 4);
639         List<ReplicatedLogEntry> unAppliedEntry = persistedSnapshot.getUnAppliedEntries();
640         assertEquals("Persisted Snapshot getUnAppliedEntries size", 1, unAppliedEntry.size());
641         verifyReplicatedLogEntry(unAppliedEntry.get(0), currentTerm, 4, payload4);
642
643         // Send a couple more payloads.
644         MockPayload payload5 = sendPayloadData(leaderActor, "five");
645         MockPayload payload6 = sendPayloadData(leaderActor, "six");
646
647         // Verify the leader applies the 2 log entries.
648         applyStates = MessageCollectorActor.expectMatching(leaderCollectorActor, ApplyState.class, 3);
649         verifyApplyState(applyStates.get(1), leaderCollectorActor, payload5.toString(), currentTerm, 5, payload5);
650         verifyApplyState(applyStates.get(2), leaderCollectorActor, payload6.toString(), currentTerm, 6, payload6);
651
652         // Verify the leader applies a log entry for at least the last entry index.
653         verifyApplyJournalEntries(leaderCollectorActor, 6);
654
655         // Ensure there's at least 1 more heartbeat to trim the log.
656         MessageCollectorActor.clearMessages(leaderCollectorActor);
657         MessageCollectorActor.expectFirstMatching(leaderCollectorActor, AppendEntriesReply.class);
658
659         // Verify the leader's final state.
660         verifyLeadersTrimmedLog(6);
661
662         InMemoryJournal.dumpJournal(leaderId);
663
664         // Verify the leaders's persisted journal log - it should only contain the last 2 ReplicatedLogEntries
665         // added after the snapshot as the persisted journal should've been purged to the snapshot
666         // sequence number.
667         verifyPersistedJournal(leaderId, Arrays.asList(new ReplicatedLogImplEntry(5, currentTerm, payload5),
668                 new ReplicatedLogImplEntry(6, currentTerm, payload6)));
669
670         // Verify the leaders's persisted journal contains an ApplyJournalEntries for at least the last entry index.
671         List<ApplyJournalEntries> persistedApplyJournalEntries = InMemoryJournal.get(leaderId, ApplyJournalEntries.class);
672         boolean found = false;
673         for(ApplyJournalEntries entry: persistedApplyJournalEntries) {
674             if(entry.getToIndex() == 6) {
675                 found = true;
676                 break;
677             }
678         }
679
680         Assert.assertTrue(String.format("ApplyJournalEntries with index %d not found in leader's persisted journal", 6), found);
681
682         // Verify follower 1 applies the 3 log entries.
683         applyStates = MessageCollectorActor.expectMatching(follower1CollectorActor, ApplyState.class, 3);
684         verifyApplyState(applyStates.get(0), null, null, currentTerm, 4, payload4);
685         verifyApplyState(applyStates.get(1), null, null, currentTerm, 5, payload5);
686         verifyApplyState(applyStates.get(2), null, null, currentTerm, 6, payload6);
687
688         // Verify follower 1's log state.
689         verifyFollowersTrimmedLog(1, follower1Actor, 6);
690
691         // Verify follower 2 applies the 3 log entries.
692         applyStates = MessageCollectorActor.expectMatching(follower2CollectorActor, ApplyState.class, 3);
693         verifyApplyState(applyStates.get(0), null, null, currentTerm, 4, payload4);
694         verifyApplyState(applyStates.get(1), null, null, currentTerm, 5, payload5);
695         verifyApplyState(applyStates.get(2), null, null, currentTerm, 6, payload6);
696
697         // Verify follower 2's log state.
698         verifyFollowersTrimmedLog(2, follower2Actor, 6);
699
700         expSnapshotState.add(payload4);
701         expSnapshotState.add(payload5);
702         expSnapshotState.add(payload6);
703
704         testLog.info("testReplicationsAndSnapshotAfterInstallSnapshot ending");
705     }
706
707     /**
708      * Kill the leader actor, reinstate it and verify the recovered journal.
709      */
710     private void verifyLeaderRecoveryAfterReinstatement(long lastIndex, long snapshotIndex, long firstJournalEntryIndex) {
711         testLog.info("testLeaderReinstatement starting");
712
713         killActor(leaderActor);
714
715         leaderActor = newTestRaftActor(leaderId, peerAddresses, leaderConfigParams);
716         TestRaftActor testRaftActor = leaderActor.underlyingActor();
717
718         testRaftActor.startDropMessages(RequestVoteReply.class);
719
720         leaderContext = testRaftActor.getRaftActorContext();
721
722         testRaftActor.waitForRecoveryComplete();
723
724         int logSize = (int) (expSnapshotState.size() - firstJournalEntryIndex);
725         assertEquals("Leader snapshot term", currentTerm, leaderContext.getReplicatedLog().getSnapshotTerm());
726         assertEquals("Leader snapshot index", snapshotIndex, leaderContext.getReplicatedLog().getSnapshotIndex());
727         assertEquals("Leader journal log size", logSize, leaderContext.getReplicatedLog().size());
728         assertEquals("Leader journal last index", lastIndex, leaderContext.getReplicatedLog().lastIndex());
729         assertEquals("Leader commit index", lastIndex, leaderContext.getCommitIndex());
730         assertEquals("Leader last applied", lastIndex, leaderContext.getLastApplied());
731
732         for(long i = firstJournalEntryIndex; i < expSnapshotState.size(); i++) {
733             verifyReplicatedLogEntry(leaderContext.getReplicatedLog().get(i), currentTerm, i,
734                     expSnapshotState.get((int) i));
735         }
736
737         assertEquals("Leader applied state", expSnapshotState, testRaftActor.getState());
738
739         testLog.info("testLeaderReinstatement ending");
740     }
741
742     private void sendInitialPayloadsReplicatedToAllFollowers(String... data) {
743
744         // Send the payloads.
745         for(String d: data) {
746             expSnapshotState.add(sendPayloadData(leaderActor, d));
747         }
748
749         int nEntries = data.length;
750
751         // Verify the leader got consensus and applies each log entry even though follower 2 didn't respond.
752         List<ApplyState> applyStates = MessageCollectorActor.expectMatching(leaderCollectorActor, ApplyState.class, nEntries);
753         for(int i = 0; i < expSnapshotState.size(); i++) {
754             MockPayload payload = expSnapshotState.get(i);
755             verifyApplyState(applyStates.get(i), leaderCollectorActor, payload.toString(), currentTerm, i, payload);
756         }
757
758         // Verify follower 1 applies each log entry.
759         applyStates = MessageCollectorActor.expectMatching(follower1CollectorActor, ApplyState.class, nEntries);
760         for(int i = 0; i < expSnapshotState.size(); i++) {
761             MockPayload payload = expSnapshotState.get(i);
762             verifyApplyState(applyStates.get(i), null, null, currentTerm, i, payload);
763         }
764
765         // Verify follower 2 applies each log entry.
766         applyStates = MessageCollectorActor.expectMatching(follower2CollectorActor, ApplyState.class, nEntries);
767         for(int i = 0; i < expSnapshotState.size(); i++) {
768             MockPayload payload = expSnapshotState.get(i);
769             verifyApplyState(applyStates.get(i), null, null, currentTerm, i, payload);
770         }
771
772         // Ensure there's at least 1 more heartbeat.
773         MessageCollectorActor.clearMessages(leaderCollectorActor);
774         MessageCollectorActor.expectFirstMatching(leaderCollectorActor, AppendEntriesReply.class);
775
776         // The leader should have performed fake snapshots to trim the log to the last index replicated to
777         // all followers.
778         verifyLeadersTrimmedLog(nEntries - 1);
779
780         MessageCollectorActor.clearMessages(leaderCollectorActor);
781         MessageCollectorActor.clearMessages(follower1CollectorActor);
782         MessageCollectorActor.clearMessages(follower2CollectorActor);
783     }
784 }