X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Ftest%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2FReplicationAndSnapshotsWithLaggingFollowerIntegrationTest.java;h=d6a53a0aeeb6e84b86d457b3ca667dbaee111101;hb=f33beecf2a10955a9219757529ba3017079816cc;hp=4211119e12e998a956e54feeecbaf323d96e2d8d;hpb=e1eca73a5ae2ffae8dd78c6fe5281cd2f45d5ef3;p=controller.git 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 4211119e12..d6a53a0aee 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 @@ -20,7 +20,8 @@ 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.eclipse.jdt.annotation.Nullable; import org.junit.Assert; import org.junit.Test; import org.opendaylight.controller.cluster.raft.MockRaftActorContext.MockPayload; @@ -36,6 +37,8 @@ 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; @@ -92,12 +95,22 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A testLog.info("Leader created and elected"); } + private void setupFollower2() { + follower2Actor = newTestRaftActor(follower2Id, ImmutableMap.of(leaderId, testActorPath(leaderId), + follower1Id, testActorPath(follower1Id)), newFollowerConfigParams()); + + follower2Context = follower2Actor.underlyingActor().getRaftActorContext(); + follower2 = follower2Actor.underlyingActor().getCurrentBehavior(); + + follower2CollectorActor = follower2Actor.underlyingActor().collectorActor(); + } + /** * Send 2 payload instances with follower 2 lagging then resume the follower and verifies it gets * caught up via AppendEntries. */ @Test - public void testReplicationsWithLaggingFollowerCaughtUpViaAppendEntries() throws Exception { + public void testReplicationsWithLaggingFollowerCaughtUpViaAppendEntries() { testLog.info("testReplicationsWithLaggingFollowerCaughtUpViaAppendEntries starting: sending 2 new payloads"); setup(); @@ -168,7 +181,7 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A * sent by the leader. */ @Test - public void testLeaderSnapshotWithLaggingFollowerCaughtUpViaAppendEntries() throws Exception { + public void testLeaderSnapshotWithLaggingFollowerCaughtUpViaAppendEntries() { testLog.info("testLeaderSnapshotWithLaggingFollowerCaughtUpViaAppendEntries starting"); setup(); @@ -287,7 +300,7 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A * installed by the leader. */ @Test - public void testLeaderSnapshotWithLaggingFollowerCaughtUpViaInstallSnapshot() throws Exception { + public void testLeaderSnapshotWithLaggingFollowerCaughtUpViaInstallSnapshot() { testLog.info("testLeaderSnapshotWithLaggingFollowerCaughtUpViaInstallSnapshot starting"); setup(); @@ -297,6 +310,10 @@ 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. final MockPayload payload2 = sendPayloadData(leaderActor, "two"); final MockPayload payload3 = sendPayloadData(leaderActor, "three"); @@ -317,10 +334,6 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A testLog.info("testLeaderSnapshotWithLaggingFollowerCaughtUpViaInstallSnapshot: " + "sending 1 more payload to trigger second snapshot"); - // 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 to trigger a second leader snapshot. MockPayload payload7 = sendPayloadData(leaderActor, "seven"); @@ -380,6 +393,80 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A testLog.info("testLeaderSnapshotWithLaggingFollowerCaughtUpViaInstallSnapshot complete"); } + /** + * Tests whether the leader reattempts to send a snapshot when a follower crashes before replying with + * InstallSnapshotReply after the last chunk has been sent. + */ + @Test + public void testLeaderInstallsSnapshotWithRestartedFollowerDuringSnapshotInstallation() throws Exception { + testLog.info("testLeaderInstallsSnapshotWithRestartedFollowerDuringSnapshotInstallation starting"); + + setup(); + + sendInitialPayloadsReplicatedToAllFollowers("zero", "one"); + + // Configure follower 2 to drop messages and lag. + follower2Actor.stop(); + + // 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. + 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); + + // Verify the leader got consensus and applies each log entry even though follower 2 didn't respond. + List applyStates = MessageCollectorActor.expectMatching(leaderCollectorActor, ApplyState.class, 5); + verifyApplyState(applyStates.get(0), leaderCollectorActor, payload2.toString(), currentTerm, 2, payload2); + verifyApplyState(applyStates.get(2), leaderCollectorActor, payload4.toString(), currentTerm, 4, payload4); + verifyApplyState(applyStates.get(4), leaderCollectorActor, payload6.toString(), currentTerm, 6, payload6); + + MessageCollectorActor.clearMessages(leaderCollectorActor); + + testLog.info("testLeaderInstallsSnapshotWithRestartedFollowerDuringSnapshotInstallation: " + + "sending 1 more payload to trigger second snapshot"); + + // Send another payload to trigger a second leader snapshot. + MockPayload payload7 = sendPayloadData(leaderActor, "seven"); + + MessageCollectorActor.expectFirstMatching(leaderCollectorActor, SaveSnapshotSuccess.class); + + + ApplyState applyState = MessageCollectorActor.expectFirstMatching(leaderCollectorActor, ApplyState.class); + verifyApplyState(applyState, leaderCollectorActor, payload7.toString(), currentTerm, 7, payload7); + + // Verify follower 1 applies each log entry. + applyStates = MessageCollectorActor.expectMatching(follower1CollectorActor, ApplyState.class, 6); + verifyApplyState(applyStates.get(0), null, null, currentTerm, 2, payload2); + verifyApplyState(applyStates.get(2), null, null, currentTerm, 4, payload4); + verifyApplyState(applyStates.get(5), null, null, currentTerm, 7, payload7); + + leaderActor.underlyingActor() + .startDropMessages(InstallSnapshotReply.class, reply -> reply.getChunkIndex() == 5); + + setupFollower2(); + + MessageCollectorActor.expectMatching(follower2CollectorActor, InstallSnapshot.class, 5); + + follower2Actor.stop(); + + // need to get rid of persistence for follower2 + InMemorySnapshotStore.clearSnapshotsFor(follower2Id); + + leaderActor.underlyingActor().stopDropMessages(InstallSnapshotReply.class); + + MessageCollectorActor.clearMessages(follower2CollectorActor); + setupFollower2(); + + MessageCollectorActor.expectMatching(follower2CollectorActor, SaveSnapshotSuccess.class, 1); + } + /** * Send payloads with follower 2 lagging with the last payload having a large enough size to trigger a * leader snapshot such that the leader trims its log from the last applied index.. Follower 2's log will @@ -387,7 +474,7 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A * by the leader. */ @Test - public void testLeaderSnapshotTriggeredByMemoryThresholdExceededWithLaggingFollower() throws Exception { + public void testLeaderSnapshotTriggeredByMemoryThresholdExceededWithLaggingFollower() { testLog.info("testLeaderSnapshotTriggeredByMemoryThresholdExceededWithLaggingFollower starting"); snapshotBatchCount = 5; @@ -402,6 +489,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); @@ -462,16 +553,12 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A 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); @@ -483,7 +570,7 @@ 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. */ - private void verifyNoSubsequentSnapshotAfterMemoryThresholdExceededSnapshot() throws Exception { + private void verifyNoSubsequentSnapshotAfterMemoryThresholdExceededSnapshot() { ApplyState applyState; CaptureSnapshot captureSnapshot; @@ -527,8 +614,8 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A * Resume the lagging follower 2 and verify it receives an install snapshot from the leader. */ private void verifyInstallSnapshotToLaggingFollower(long lastAppliedIndex, - @Nullable ServerConfigurationPayload expServerConfig) throws Exception { - testLog.info("testInstallSnapshotToLaggingFollower starting"); + @Nullable ServerConfigurationPayload expServerConfig) { + testLog.info("verifyInstallSnapshotToLaggingFollower starting"); MessageCollectorActor.clearMessages(leaderCollectorActor); @@ -552,7 +639,7 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A List unAppliedEntry = persistedSnapshot.getUnAppliedEntries(); assertEquals("Persisted Snapshot getUnAppliedEntries size", 0, unAppliedEntry.size()); - int snapshotSize = persistedSnapshot.getState().length; + int snapshotSize = SerializationUtils.serialize(persistedSnapshot.getState()).length; final int expTotalChunks = snapshotSize / SNAPSHOT_CHUNK_SIZE + (snapshotSize % SNAPSHOT_CHUNK_SIZE > 0 ? 1 : 0); @@ -614,15 +701,16 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A 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. */ - private void verifyReplicationsAndSnapshotWithNoLaggingAfterInstallSnapshot() throws Exception { - testLog.info("testReplicationsAndSnapshotAfterInstallSnapshot starting: replicatedToAllIndex: {}", + private long verifyReplicationsAndSnapshotWithNoLaggingAfterInstallSnapshot() { + testLog.info( + "verifyReplicationsAndSnapshotWithNoLaggingAfterInstallSnapshot starting: replicatedToAllIndex: {}", leader.getReplicatedToAllIndex()); // Send another payload - a snapshot should occur. @@ -637,10 +725,21 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A // 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"); @@ -666,8 +765,8 @@ 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 = @@ -701,11 +800,12 @@ 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; } /** @@ -713,7 +813,8 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A */ private void verifyLeaderRecoveryAfterReinstatement(long lastIndex, long snapshotIndex, long firstJournalEntryIndex) { - testLog.info("testLeaderReinstatement starting"); + testLog.info("verifyLeaderRecoveryAfterReinstatement starting: lastIndex: {}, snapshotIndex: {}, " + + "firstJournalEntryIndex: {}", lastIndex, snapshotIndex, firstJournalEntryIndex); killActor(leaderActor); @@ -741,7 +842,7 @@ public class ReplicationAndSnapshotsWithLaggingFollowerIntegrationTest extends A assertEquals("Leader applied state", expSnapshotState, testRaftActor.getState()); - testLog.info("testLeaderReinstatement ending"); + testLog.info("verifyLeaderRecoveryAfterReinstatement ending"); } private void sendInitialPayloadsReplicatedToAllFollowers(String... data) {