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