X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Ftest%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2FReplicationAndSnapshotsWithLaggingFollowerIntegrationTest.java;h=0dbcb1fbd0943ec0e26328910f97060826ce13a1;hp=03387abcd5cca234649b9f58796a86992e9b8b00;hb=dac16f0d464eff3325b3800a803e81b303964e4b;hpb=252ba03242407ee584c38fafdbfa1c322e66151d diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest.java index 03387abcd5..0dbcb1fbd0 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest.java @@ -8,24 +8,38 @@ package org.opendaylight.controller.cluster.raft; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import akka.actor.ActorRef; import akka.persistence.SaveSnapshotSuccess; import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.Uninterruptibles; import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import javax.annotation.Nullable; +import org.apache.commons.lang3.SerializationUtils; import org.junit.Assert; import org.junit.Test; import org.opendaylight.controller.cluster.raft.MockRaftActorContext.MockPayload; -import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries; import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot; import org.opendaylight.controller.cluster.raft.base.messages.ApplyState; import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot; -import org.opendaylight.controller.cluster.raft.base.messages.UpdateElectionTerm; +import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader; import org.opendaylight.controller.cluster.raft.messages.AppendEntries; import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply; import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot; import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply; import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; +import org.opendaylight.controller.cluster.raft.persisted.ApplyJournalEntries; +import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload; +import org.opendaylight.controller.cluster.raft.persisted.ServerInfo; +import org.opendaylight.controller.cluster.raft.persisted.SimpleReplicatedLogEntry; +import org.opendaylight.controller.cluster.raft.persisted.Snapshot; +import org.opendaylight.controller.cluster.raft.persisted.UpdateElectionTerm; import org.opendaylight.controller.cluster.raft.utils.InMemoryJournal; import org.opendaylight.controller.cluster.raft.utils.InMemorySnapshotStore; import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor; @@ -53,12 +67,12 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A follower2Actor = newTestRaftActor(follower2Id, ImmutableMap.of(leaderId, testActorPath(leaderId), follower1Id, testActorPath(follower1Id)), newFollowerConfigParams()); - Map peerAddresses = ImmutableMap.builder(). - put(follower1Id, follower1Actor.path().toString()). - put(follower2Id, follower2Actor.path().toString()).build(); + Map leaderPeerAddresses = ImmutableMap.builder() + .put(follower1Id, follower1Actor.path().toString()) + .put(follower2Id, follower2Actor.path().toString()).build(); leaderConfigParams = newLeaderConfigParams(); - leaderActor = newTestRaftActor(leaderId, peerAddresses, leaderConfigParams); + leaderActor = newTestRaftActor(leaderId, leaderPeerAddresses, leaderConfigParams); waitUntilLeader(leaderActor); @@ -122,7 +136,9 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A assertEquals("Leader last applied", 1, leaderContext.getLastApplied()); assertEquals("Leader replicatedToAllIndex", -1, leader.getReplicatedToAllIndex()); - testLog.info("testReplicationsWithLaggingFollowerCaughtUpViaAppendEntries: new entries applied - resuming follower {}", follower2Id); + testLog.info( + "testReplicationsWithLaggingFollowerCaughtUpViaAppendEntries: new entries applied - resuming follower {}", + follower2Id); // Now stop dropping AppendEntries in follower 2. follower2Actor.underlyingActor().stopDropMessages(AppendEntries.class); @@ -153,8 +169,6 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A * lagging but not enough for the leader to trim its log from the last applied index. Follower 2's log * will be behind by several entries and, when it is resumed, it should be caught up via AppendEntries * sent by the leader. - * - * @throws Exception */ @Test public void testLeaderSnapshotWithLaggingFollowerCaughtUpViaAppendEntries() throws Exception { @@ -228,7 +242,8 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A verifyApplyState(applyStates.get(3), null, null, currentTerm, 5, payload5); // Verify the leader did not try to install a snapshot to catch up follower 2. - InstallSnapshot installSnapshot = MessageCollectorActor.getFirstMatching(follower2CollectorActor, InstallSnapshot.class); + InstallSnapshot installSnapshot = MessageCollectorActor.getFirstMatching(follower2CollectorActor, + InstallSnapshot.class); Assert.assertNull("Follower 2 received unexpected InstallSnapshot", installSnapshot); // Ensure there's at least 1 more heartbeat. @@ -273,8 +288,6 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A * lagging where the leader trims its log from the last applied index. Follower 2's log * will be behind by several entries and, when it is resumed, it should be caught up via a snapshot * installed by the leader. - * - * @throws Exception */ @Test public void testLeaderSnapshotWithLaggingFollowerCaughtUpViaInstallSnapshot() throws Exception { @@ -287,12 +300,16 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A // Configure follower 2 to drop messages and lag. follower2Actor.underlyingActor().startDropMessages(AppendEntries.class); + // Sleep for at least the election timeout interval so follower 2 is deemed inactive by the leader. + Uninterruptibles.sleepUninterruptibly(leaderConfigParams.getElectionTimeOutInterval().toMillis() + 5, + TimeUnit.MILLISECONDS); + // Send 5 payloads - the second should cause a leader snapshot. - MockPayload payload2 = sendPayloadData(leaderActor, "two"); - MockPayload payload3 = sendPayloadData(leaderActor, "three"); - MockPayload payload4 = sendPayloadData(leaderActor, "four"); - MockPayload payload5 = sendPayloadData(leaderActor, "five"); - MockPayload payload6 = sendPayloadData(leaderActor, "six"); + final MockPayload payload2 = sendPayloadData(leaderActor, "two"); + final MockPayload payload3 = sendPayloadData(leaderActor, "three"); + final MockPayload payload4 = sendPayloadData(leaderActor, "four"); + final MockPayload payload5 = sendPayloadData(leaderActor, "five"); + final MockPayload payload6 = sendPayloadData(leaderActor, "six"); MessageCollectorActor.expectFirstMatching(leaderCollectorActor, SaveSnapshotSuccess.class); @@ -304,7 +321,8 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A MessageCollectorActor.clearMessages(leaderCollectorActor); - testLog.info("testLeaderSnapshotWithLaggingFollowerCaughtUpViaAppendEntries: sending 1 more payload to trigger second snapshot"); + testLog.info("testLeaderSnapshotWithLaggingFollowerCaughtUpViaInstallSnapshot: " + + "sending 1 more payload to trigger second snapshot"); // Send another payload to trigger a second leader snapshot. MockPayload payload7 = sendPayloadData(leaderActor, "seven"); @@ -331,6 +349,25 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A expSnapshotState.add(payload5); expSnapshotState.add(payload6); + MessageCollectorActor.clearMessages(leaderCollectorActor); + MessageCollectorActor.clearMessages(follower1CollectorActor); + + // Send a server config change to test that the install snapshot includes the server config. + + ServerConfigurationPayload serverConfig = new ServerConfigurationPayload(Arrays.asList( + new ServerInfo(leaderId, true), + new ServerInfo(follower1Id, false), + new ServerInfo(follower2Id, false))); + leaderContext.updatePeerIds(serverConfig); + ((AbstractLeader)leader).updateMinReplicaCount(); + leaderActor.tell(serverConfig, ActorRef.noSender()); + + applyState = MessageCollectorActor.expectFirstMatching(leaderCollectorActor, ApplyState.class); + verifyApplyState(applyState, leaderCollectorActor, "serverConfig", currentTerm, 8, serverConfig); + + applyState = MessageCollectorActor.expectFirstMatching(follower1CollectorActor, ApplyState.class); + verifyApplyState(applyState, null, null, currentTerm, 8, serverConfig); + // Verify the leader's persisted snapshot. List persistedSnapshots = InMemorySnapshotStore.getSnapshots(leaderId, Snapshot.class); assertEquals("Persisted snapshots size", 1, persistedSnapshots.size()); @@ -341,7 +378,7 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A expSnapshotState.add(payload7); - verifyInstallSnapshotToLaggingFollower(7); + verifyInstallSnapshotToLaggingFollower(8, serverConfig); testLog.info("testLeaderSnapshotWithLaggingFollowerCaughtUpViaInstallSnapshot complete"); } @@ -351,8 +388,6 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A * leader snapshot such that the leader trims its log from the last applied index.. Follower 2's log will * be behind by several entries and, when it is resumed, it should be caught up via a snapshot installed * by the leader. - * - * @throws Exception */ @Test public void testLeaderSnapshotTriggeredByMemoryThresholdExceededWithLaggingFollower() throws Exception { @@ -370,6 +405,10 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A follower2Actor.underlyingActor().startDropMessages(AppendEntries.class); + // Sleep for at least the election timeout interval so follower 2 is deemed inactive by the leader. + Uninterruptibles.sleepUninterruptibly(leaderConfigParams.getElectionTimeOutInterval().toMillis() + 5, + TimeUnit.MILLISECONDS); + // Send a payload with a large relative size but not enough to trigger a snapshot. MockPayload payload1 = sendPayloadData(leaderActor, "one", 500); @@ -383,11 +422,16 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A InMemoryJournal.waitForWriteMessagesComplete(leaderId); // Verify a snapshot is not triggered. - CaptureSnapshot captureSnapshot = MessageCollectorActor.getFirstMatching(leaderCollectorActor, CaptureSnapshot.class); + CaptureSnapshot captureSnapshot = MessageCollectorActor.getFirstMatching(leaderCollectorActor, + CaptureSnapshot.class); Assert.assertNull("Leader received unexpected CaptureSnapshot", captureSnapshot); expSnapshotState.add(payload1); + // Sleep for at least the election timeout interval so follower 2 is deemed inactive by the leader. + Uninterruptibles.sleepUninterruptibly(leaderConfigParams.getElectionTimeOutInterval().toMillis() + 5, + TimeUnit.MILLISECONDS); + // Send another payload with a large enough relative size in combination with the last payload // that exceeds the memory threshold (70% * 1000 = 700) - this should do a snapshot. MockPayload payload2 = sendPayloadData(leaderActor, "two", 201); @@ -419,22 +463,18 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A expSnapshotState.add(payload2); - verifyInstallSnapshotToLaggingFollower(2L); + verifyInstallSnapshotToLaggingFollower(2L, null); // Sends a payload with index 3. verifyNoSubsequentSnapshotAfterMemoryThresholdExceededSnapshot(); // Sends 3 payloads with indexes 4, 5 and 6. - verifyReplicationsAndSnapshotWithNoLaggingAfterInstallSnapshot(); + long leadersSnapshotIndexOnRecovery = verifyReplicationsAndSnapshotWithNoLaggingAfterInstallSnapshot(); // Recover the leader from persistence and verify. long leadersLastIndexOnRecovery = 6; - // The leader's last snapshot was triggered by index 4 so the last applied index in the snapshot was 3. - long leadersSnapshotIndexOnRecovery = 3; - - // The recovered journal should have 3 entries starting at index 4. - long leadersFirstJournalEntryIndexOnRecovery = 4; + long leadersFirstJournalEntryIndexOnRecovery = leadersSnapshotIndexOnRecovery + 1; verifyLeaderRecoveryAfterReinstatement(leadersLastIndexOnRecovery, leadersSnapshotIndexOnRecovery, leadersFirstJournalEntryIndexOnRecovery); @@ -445,8 +485,6 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A /** * Send another payload to verify another snapshot is not done since the last snapshot trimmed the * first log entry so the memory threshold should not be exceeded. - * - * @throws Exception */ private void verifyNoSubsequentSnapshotAfterMemoryThresholdExceededSnapshot() throws Exception { ApplyState applyState; @@ -490,16 +528,10 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A /** * Resume the lagging follower 2 and verify it receives an install snapshot from the leader. - * - * @throws Exception */ - private void verifyInstallSnapshotToLaggingFollower(long lastAppliedIndex) throws Exception { - List persistedSnapshots; - List unAppliedEntry; - ApplySnapshot applySnapshot; - InstallSnapshot installSnapshot; - - testLog.info("testInstallSnapshotToLaggingFollower starting"); + private void verifyInstallSnapshotToLaggingFollower(long lastAppliedIndex, + @Nullable ServerConfigurationPayload expServerConfig) throws Exception { + testLog.info("verifyInstallSnapshotToLaggingFollower starting"); MessageCollectorActor.clearMessages(leaderCollectorActor); @@ -516,17 +548,19 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A // RaftActor#trimPersistentData subtracts the snapshotBatchCount from the snapshot's sequence number. // This is OK - the next snapshot should delete it. In production, even if the system restarted // before another snapshot, they would both get applied which wouldn't hurt anything. - persistedSnapshots = InMemorySnapshotStore.getSnapshots(leaderId, Snapshot.class); + List persistedSnapshots = InMemorySnapshotStore.getSnapshots(leaderId, Snapshot.class); Assert.assertTrue("Expected at least 1 persisted snapshots", persistedSnapshots.size() > 0); Snapshot persistedSnapshot = persistedSnapshots.get(persistedSnapshots.size() - 1); verifySnapshot("Persisted", persistedSnapshot, currentTerm, lastAppliedIndex, currentTerm, lastAppliedIndex); - unAppliedEntry = persistedSnapshot.getUnAppliedEntries(); + List unAppliedEntry = persistedSnapshot.getUnAppliedEntries(); assertEquals("Persisted Snapshot getUnAppliedEntries size", 0, unAppliedEntry.size()); - int snapshotSize = persistedSnapshot.getState().length; - int expTotalChunks = (snapshotSize / SNAPSHOT_CHUNK_SIZE) + ((snapshotSize % SNAPSHOT_CHUNK_SIZE) > 0 ? 1 : 0); + int snapshotSize = SerializationUtils.serialize(persistedSnapshot.getState()).length; + final int expTotalChunks = snapshotSize / SNAPSHOT_CHUNK_SIZE + + (snapshotSize % SNAPSHOT_CHUNK_SIZE > 0 ? 1 : 0); - installSnapshot = MessageCollectorActor.expectFirstMatching(follower2CollectorActor, InstallSnapshot.class); + InstallSnapshot installSnapshot = MessageCollectorActor.expectFirstMatching(follower2CollectorActor, + InstallSnapshot.class); assertEquals("InstallSnapshot getTerm", currentTerm, installSnapshot.getTerm()); assertEquals("InstallSnapshot getLeaderId", leaderId, installSnapshot.getLeaderId()); assertEquals("InstallSnapshot getChunkIndex", 1, installSnapshot.getChunkIndex()); @@ -538,7 +572,7 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A List installSnapshotReplies = MessageCollectorActor.expectMatching( leaderCollectorActor, InstallSnapshotReply.class, expTotalChunks); int index = 1; - for(InstallSnapshotReply installSnapshotReply: installSnapshotReplies) { + for (InstallSnapshotReply installSnapshotReply: installSnapshotReplies) { assertEquals("InstallSnapshotReply getTerm", currentTerm, installSnapshotReply.getTerm()); assertEquals("InstallSnapshotReply getChunkIndex", index++, installSnapshotReply.getChunkIndex()); assertEquals("InstallSnapshotReply getFollowerId", follower2Id, installSnapshotReply.getFollowerId()); @@ -546,9 +580,12 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A } // Verify follower 2 applies the snapshot. - applySnapshot = MessageCollectorActor.expectFirstMatching(follower2CollectorActor, ApplySnapshot.class); - verifySnapshot("Follower 2", applySnapshot.getSnapshot(), currentTerm, lastAppliedIndex, currentTerm, lastAppliedIndex); - assertEquals("Persisted Snapshot getUnAppliedEntries size", 0, applySnapshot.getSnapshot().getUnAppliedEntries().size()); + ApplySnapshot applySnapshot = MessageCollectorActor.expectFirstMatching(follower2CollectorActor, + ApplySnapshot.class); + verifySnapshot("Follower 2", applySnapshot.getSnapshot(), currentTerm, lastAppliedIndex, currentTerm, + lastAppliedIndex); + assertEquals("Persisted Snapshot getUnAppliedEntries size", 0, + applySnapshot.getSnapshot().getUnAppliedEntries().size()); // Wait for the snapshot to complete. MessageCollectorActor.expectFirstMatching(leaderCollectorActor, SaveSnapshotSuccess.class); @@ -561,24 +598,35 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A // the log. In addition replicatedToAllIndex should've advanced. verifyLeadersTrimmedLog(lastAppliedIndex); + if (expServerConfig != null) { + Set expServerInfo = new HashSet<>(expServerConfig.getServerConfig()); + assertEquals("Leader snapshot server config", expServerInfo, + new HashSet<>(persistedSnapshot.getServerConfiguration().getServerConfig())); + + assertEquals("Follower 2 snapshot server config", expServerInfo, + new HashSet<>(applySnapshot.getSnapshot().getServerConfiguration().getServerConfig())); + + ServerConfigurationPayload follower2ServerConfig = follower2Context.getPeerServerInfo(true); + assertNotNull("Follower 2 server config is null", follower2ServerConfig); + + assertEquals("Follower 2 server config", expServerInfo, + new HashSet<>(follower2ServerConfig.getServerConfig())); + } + MessageCollectorActor.clearMessages(leaderCollectorActor); MessageCollectorActor.clearMessages(follower1CollectorActor); MessageCollectorActor.clearMessages(follower2CollectorActor); - testLog.info("testInstallSnapshotToLaggingFollower complete"); + testLog.info("verifyInstallSnapshotToLaggingFollower complete"); } /** * Do another round of payloads and snapshot to verify replicatedToAllIndex gets back on track and * snapshots works as expected after doing a follower snapshot. In this step we don't lag a follower. - * - * @throws Exception */ - private void verifyReplicationsAndSnapshotWithNoLaggingAfterInstallSnapshot() throws Exception { - List applyStates; - ApplyState applyState; - - testLog.info("testReplicationsAndSnapshotAfterInstallSnapshot starting: replicatedToAllIndex: {}", + private long verifyReplicationsAndSnapshotWithNoLaggingAfterInstallSnapshot() throws Exception { + testLog.info( + "verifyReplicationsAndSnapshotWithNoLaggingAfterInstallSnapshot starting: replicatedToAllIndex: {}", leader.getReplicatedToAllIndex()); // Send another payload - a snapshot should occur. @@ -587,23 +635,34 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A // Wait for the snapshot to complete. MessageCollectorActor.expectFirstMatching(leaderCollectorActor, SaveSnapshotSuccess.class); - applyState = MessageCollectorActor.expectFirstMatching(leaderCollectorActor, ApplyState.class); + ApplyState applyState = MessageCollectorActor.expectFirstMatching(leaderCollectorActor, ApplyState.class); verifyApplyState(applyState, leaderCollectorActor, payload4.toString(), currentTerm, 4, payload4); // Verify the leader's last persisted snapshot (previous ones may not be purged yet). List persistedSnapshots = InMemorySnapshotStore.getSnapshots(leaderId, Snapshot.class); Snapshot persistedSnapshot = persistedSnapshots.get(persistedSnapshots.size() - 1); - verifySnapshot("Persisted", persistedSnapshot, currentTerm, 3, currentTerm, 4); + // The last (fourth) payload may or may not have been applied when the snapshot is captured depending on the + // timing when the async persistence completes. List unAppliedEntry = persistedSnapshot.getUnAppliedEntries(); - assertEquals("Persisted Snapshot getUnAppliedEntries size", 1, unAppliedEntry.size()); - verifyReplicatedLogEntry(unAppliedEntry.get(0), currentTerm, 4, payload4); + long leadersSnapshotIndex; + if (unAppliedEntry.isEmpty()) { + leadersSnapshotIndex = 4; + expSnapshotState.add(payload4); + verifySnapshot("Persisted", persistedSnapshot, currentTerm, 4, currentTerm, 4); + } else { + leadersSnapshotIndex = 3; + verifySnapshot("Persisted", persistedSnapshot, currentTerm, 3, currentTerm, 4); + assertEquals("Persisted Snapshot getUnAppliedEntries size", 1, unAppliedEntry.size()); + verifyReplicatedLogEntry(unAppliedEntry.get(0), currentTerm, 4, payload4); + expSnapshotState.add(payload4); + } // Send a couple more payloads. MockPayload payload5 = sendPayloadData(leaderActor, "five"); MockPayload payload6 = sendPayloadData(leaderActor, "six"); // Verify the leader applies the 2 log entries. - applyStates = MessageCollectorActor.expectMatching(leaderCollectorActor, ApplyState.class, 3); + List applyStates = MessageCollectorActor.expectMatching(leaderCollectorActor, ApplyState.class, 3); verifyApplyState(applyStates.get(1), leaderCollectorActor, payload5.toString(), currentTerm, 5, payload5); verifyApplyState(applyStates.get(2), leaderCollectorActor, payload6.toString(), currentTerm, 6, payload6); @@ -622,20 +681,22 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A // Verify the leaders's persisted journal log - it should only contain the last 2 ReplicatedLogEntries // added after the snapshot as the persisted journal should've been purged to the snapshot // sequence number. - verifyPersistedJournal(leaderId, Arrays.asList(new ReplicatedLogImplEntry(5, currentTerm, payload5), - new ReplicatedLogImplEntry(6, currentTerm, payload6))); + verifyPersistedJournal(leaderId, Arrays.asList(new SimpleReplicatedLogEntry(5, currentTerm, payload5), + new SimpleReplicatedLogEntry(6, currentTerm, payload6))); // Verify the leaders's persisted journal contains an ApplyJournalEntries for at least the last entry index. - List persistedApplyJournalEntries = InMemoryJournal.get(leaderId, ApplyJournalEntries.class); + List persistedApplyJournalEntries = + InMemoryJournal.get(leaderId, ApplyJournalEntries.class); boolean found = false; - for(ApplyJournalEntries entry: persistedApplyJournalEntries) { - if(entry.getToIndex() == 6) { + for (ApplyJournalEntries entry: persistedApplyJournalEntries) { + if (entry.getToIndex() == 6) { found = true; break; } } - Assert.assertTrue(String.format("ApplyJournalEntries with index %d not found in leader's persisted journal", 6), found); + Assert.assertTrue(String.format("ApplyJournalEntries with index %d not found in leader's persisted journal", 6), + found); // Verify follower 1 applies the 3 log entries. applyStates = MessageCollectorActor.expectMatching(follower1CollectorActor, ApplyState.class, 3); @@ -655,18 +716,21 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A // Verify follower 2's log state. verifyFollowersTrimmedLog(2, follower2Actor, 6); - expSnapshotState.add(payload4); expSnapshotState.add(payload5); expSnapshotState.add(payload6); - testLog.info("testReplicationsAndSnapshotAfterInstallSnapshot ending"); + testLog.info("verifyReplicationsAndSnapshotWithNoLaggingAfterInstallSnapshot ending"); + + return leadersSnapshotIndex; } /** * Kill the leader actor, reinstate it and verify the recovered journal. */ - private void verifyLeaderRecoveryAfterReinstatement(long lastIndex, long snapshotIndex, long firstJournalEntryIndex) { - testLog.info("testLeaderReinstatement starting"); + private void verifyLeaderRecoveryAfterReinstatement(long lastIndex, long snapshotIndex, + long firstJournalEntryIndex) { + testLog.info("verifyLeaderRecoveryAfterReinstatement starting: lastIndex: {}, snapshotIndex: {}, " + + "firstJournalEntryIndex: {}", lastIndex, snapshotIndex, firstJournalEntryIndex); killActor(leaderActor); @@ -687,42 +751,43 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A assertEquals("Leader commit index", lastIndex, leaderContext.getCommitIndex()); assertEquals("Leader last applied", lastIndex, leaderContext.getLastApplied()); - for(long i = firstJournalEntryIndex; i < expSnapshotState.size(); i++) { + for (long i = firstJournalEntryIndex; i < expSnapshotState.size(); i++) { verifyReplicatedLogEntry(leaderContext.getReplicatedLog().get(i), currentTerm, i, expSnapshotState.get((int) i)); } assertEquals("Leader applied state", expSnapshotState, testRaftActor.getState()); - testLog.info("testLeaderReinstatement ending"); + testLog.info("verifyLeaderRecoveryAfterReinstatement ending"); } private void sendInitialPayloadsReplicatedToAllFollowers(String... data) { // Send the payloads. - for(String d: data) { + for (String d: data) { expSnapshotState.add(sendPayloadData(leaderActor, d)); } - int nEntries = data.length; + int numEntries = data.length; // Verify the leader got consensus and applies each log entry even though follower 2 didn't respond. - List applyStates = MessageCollectorActor.expectMatching(leaderCollectorActor, ApplyState.class, nEntries); - for(int i = 0; i < expSnapshotState.size(); i++) { + List applyStates = MessageCollectorActor.expectMatching(leaderCollectorActor, + ApplyState.class, numEntries); + for (int i = 0; i < expSnapshotState.size(); i++) { MockPayload payload = expSnapshotState.get(i); verifyApplyState(applyStates.get(i), leaderCollectorActor, payload.toString(), currentTerm, i, payload); } // Verify follower 1 applies each log entry. - applyStates = MessageCollectorActor.expectMatching(follower1CollectorActor, ApplyState.class, nEntries); - for(int i = 0; i < expSnapshotState.size(); i++) { + applyStates = MessageCollectorActor.expectMatching(follower1CollectorActor, ApplyState.class, numEntries); + for (int i = 0; i < expSnapshotState.size(); i++) { MockPayload payload = expSnapshotState.get(i); verifyApplyState(applyStates.get(i), null, null, currentTerm, i, payload); } // Verify follower 2 applies each log entry. - applyStates = MessageCollectorActor.expectMatching(follower2CollectorActor, ApplyState.class, nEntries); - for(int i = 0; i < expSnapshotState.size(); i++) { + applyStates = MessageCollectorActor.expectMatching(follower2CollectorActor, ApplyState.class, numEntries); + for (int i = 0; i < expSnapshotState.size(); i++) { MockPayload payload = expSnapshotState.get(i); verifyApplyState(applyStates.get(i), null, null, currentTerm, i, payload); } @@ -733,7 +798,7 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A // The leader should have performed fake snapshots to trim the log to the last index replicated to // all followers. - verifyLeadersTrimmedLog(nEntries - 1); + verifyLeadersTrimmedLog(numEntries - 1); MessageCollectorActor.clearMessages(leaderCollectorActor); MessageCollectorActor.clearMessages(follower1CollectorActor);