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%2Fbehaviors%2FLeaderTest.java;h=151015e97ec785277beaf51b0bdb4fe0681f6582;hb=3c298061e7c22390eaf790aae977decfe0f94ad1;hp=6b534deb1f5db6e8f7b773ddd2004f6b0f44ee71;hpb=274cee29fd987e5218f00a0352a1506adc2736ed;p=controller.git diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java index 6b534deb1f..151015e97e 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/LeaderTest.java @@ -1,10 +1,14 @@ package org.opendaylight.controller.cluster.raft.behaviors; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; import akka.actor.ActorRef; import akka.actor.PoisonPill; import akka.actor.Props; import akka.actor.Terminated; import akka.testkit.JavaTestKit; +import akka.testkit.TestActorRef; import com.google.common.base.Optional; import com.google.common.util.concurrent.Uninterruptibles; import com.google.protobuf.ByteString; @@ -42,9 +46,6 @@ import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor; import org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages; import org.opendaylight.controller.protobuff.messages.cluster.raft.InstallSnapshotMessages; import scala.concurrent.duration.FiniteDuration; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; public class LeaderTest extends AbstractRaftActorBehaviorTest { @@ -66,12 +67,12 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { }}; } - @Test public void testThatLeaderSendsAHeartbeatMessageToAllFollowers() { new JavaTestKit(getSystem()) {{ new Within(duration("1 seconds")) { + @Override protected void run() { ActorRef followerActor = getTestActor(); @@ -91,6 +92,7 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { final String out = new ExpectMsg(duration("1 seconds"), "match hint") { // do not put code outside this method, will run afterwards + @Override protected String match(Object in) { Object msg = fromSerializableMessage(in); if (msg instanceof AppendEntries) { @@ -116,6 +118,7 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { new JavaTestKit(getSystem()) {{ new Within(duration("1 seconds")) { + @Override protected void run() { ActorRef followerActor = getTestActor(); @@ -126,7 +129,7 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { Map peerAddresses = new HashMap<>(); peerAddresses.put(followerActor.path().toString(), - followerActor.path().toString()); + followerActor.path().toString()); actorContext.setPeerAddresses(peerAddresses); @@ -144,6 +147,7 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { final String out = new ExpectMsg(duration("1 seconds"), "match hint") { // do not put code outside this method, will run afterwards + @Override protected String match(Object in) { Object msg = fromSerializableMessage(in); if (msg instanceof AppendEntries) { @@ -168,6 +172,7 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { new JavaTestKit(getSystem()) {{ new Within(duration("1 seconds")) { + @Override protected void run() { ActorRef raftActor = getTestActor(); @@ -194,6 +199,7 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { new ExpectMsg(duration("1 seconds"), "match hint") { // do not put code outside this method, will run afterwards + @Override protected String match(Object in) { if (in instanceof ApplyState) { if (((ApplyState) in).getIdentifier().equals("state-id")) { @@ -481,6 +487,7 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { final String out = new ExpectMsg(duration("1 seconds"), "match hint") { // do not put code outside this method, will run afterwards + @Override protected String match(Object in) { if (in instanceof InstallSnapshotMessages.InstallSnapshot) { InstallSnapshot is = (InstallSnapshot) @@ -561,16 +568,166 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { assertTrue(raftBehavior instanceof Leader); - assertEquals(leader.mapFollowerToSnapshot.size(), 0); - assertEquals(leader.followerToLog.size(), 1); - assertNotNull(leader.followerToLog.get(followerActor.path().toString())); - FollowerLogInformation fli = leader.followerToLog.get(followerActor.path().toString()); - assertEquals(snapshotIndex, fli.getMatchIndex().get()); - assertEquals(snapshotIndex, fli.getMatchIndex().get()); - assertEquals(snapshotIndex + 1, fli.getNextIndex().get()); + assertEquals(0, leader.followerSnapshotSize()); + assertEquals(1, leader.followerLogSize()); + assertNotNull(leader.getFollower(followerActor.path().toString())); + FollowerLogInformation fli = leader.getFollower(followerActor.path().toString()); + assertEquals(snapshotIndex, fli.getMatchIndex()); + assertEquals(snapshotIndex, fli.getMatchIndex()); + assertEquals(snapshotIndex + 1, fli.getNextIndex()); + }}; + } + + @Test + public void testHandleInstallSnapshotReplyWithInvalidChunkIndex() throws Exception { + new JavaTestKit(getSystem()) {{ + + TestActorRef followerActor = + TestActorRef.create(getSystem(), Props.create(MessageCollectorActor.class), "follower"); + + Map peerAddresses = new HashMap<>(); + peerAddresses.put(followerActor.path().toString(), + followerActor.path().toString()); + + final int followersLastIndex = 2; + final int snapshotIndex = 3; + final int snapshotTerm = 1; + final int currentTerm = 2; + + MockRaftActorContext actorContext = + (MockRaftActorContext) createActorContext(); + + actorContext.setConfigParams(new DefaultConfigParamsImpl(){ + @Override + public int getSnapshotChunkSize() { + return 50; + } + }); + actorContext.setPeerAddresses(peerAddresses); + actorContext.setCommitIndex(followersLastIndex); + + MockLeader leader = new MockLeader(actorContext); + + Map leadersSnapshot = new HashMap<>(); + leadersSnapshot.put("1", "A"); + leadersSnapshot.put("2", "B"); + leadersSnapshot.put("3", "C"); + + // set the snapshot variables in replicatedlog + actorContext.getReplicatedLog().setSnapshotIndex(snapshotIndex); + actorContext.getReplicatedLog().setSnapshotTerm(snapshotTerm); + actorContext.getTermInformation().update(currentTerm, leaderActor.path().toString()); + + ByteString bs = toByteString(leadersSnapshot); + leader.setSnapshot(Optional.of(bs)); + + leader.handleMessage(leaderActor, new SendInstallSnapshot(bs)); + + Object o = MessageCollectorActor.getAllMessages(followerActor).get(0); + + assertTrue(o instanceof InstallSnapshotMessages.InstallSnapshot); + + InstallSnapshotMessages.InstallSnapshot installSnapshot = (InstallSnapshotMessages.InstallSnapshot) o; + + assertEquals(1, installSnapshot.getChunkIndex()); + assertEquals(3, installSnapshot.getTotalChunks()); + + + leader.handleMessage(followerActor, new InstallSnapshotReply(actorContext.getTermInformation().getCurrentTerm(), followerActor.path().toString(), -1, false)); + + leader.handleMessage(leaderActor, new SendHeartBeat()); + + o = MessageCollectorActor.getAllMessages(followerActor).get(1); + + assertTrue(o instanceof InstallSnapshotMessages.InstallSnapshot); + + installSnapshot = (InstallSnapshotMessages.InstallSnapshot) o; + + assertEquals(1, installSnapshot.getChunkIndex()); + assertEquals(3, installSnapshot.getTotalChunks()); + + followerActor.tell(PoisonPill.getInstance(), getRef()); }}; } + @Test + public void testHandleSnapshotSendsPreviousChunksHashCodeWhenSendingNextChunk() throws Exception { + new JavaTestKit(getSystem()) { + { + + TestActorRef followerActor = + TestActorRef.create(getSystem(), Props.create(MessageCollectorActor.class), "follower"); + + Map peerAddresses = new HashMap<>(); + peerAddresses.put(followerActor.path().toString(), + followerActor.path().toString()); + + final int followersLastIndex = 2; + final int snapshotIndex = 3; + final int snapshotTerm = 1; + final int currentTerm = 2; + + MockRaftActorContext actorContext = + (MockRaftActorContext) createActorContext(); + + actorContext.setConfigParams(new DefaultConfigParamsImpl() { + @Override + public int getSnapshotChunkSize() { + return 50; + } + }); + actorContext.setPeerAddresses(peerAddresses); + actorContext.setCommitIndex(followersLastIndex); + + MockLeader leader = new MockLeader(actorContext); + + Map leadersSnapshot = new HashMap<>(); + leadersSnapshot.put("1", "A"); + leadersSnapshot.put("2", "B"); + leadersSnapshot.put("3", "C"); + + // set the snapshot variables in replicatedlog + actorContext.getReplicatedLog().setSnapshotIndex(snapshotIndex); + actorContext.getReplicatedLog().setSnapshotTerm(snapshotTerm); + actorContext.getTermInformation().update(currentTerm, leaderActor.path().toString()); + + ByteString bs = toByteString(leadersSnapshot); + leader.setSnapshot(Optional.of(bs)); + + leader.handleMessage(leaderActor, new SendInstallSnapshot(bs)); + + Object o = MessageCollectorActor.getAllMessages(followerActor).get(0); + + assertTrue(o instanceof InstallSnapshotMessages.InstallSnapshot); + + InstallSnapshotMessages.InstallSnapshot installSnapshot = (InstallSnapshotMessages.InstallSnapshot) o; + + assertEquals(1, installSnapshot.getChunkIndex()); + assertEquals(3, installSnapshot.getTotalChunks()); + assertEquals(AbstractLeader.INITIAL_LAST_CHUNK_HASH_CODE, installSnapshot.getLastChunkHashCode()); + + int hashCode = installSnapshot.getData().hashCode(); + + leader.handleMessage(followerActor, new InstallSnapshotReply(installSnapshot.getTerm(),followerActor.path().toString(),1,true )); + + leader.handleMessage(leaderActor, new SendHeartBeat()); + + Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS); + + o = MessageCollectorActor.getAllMessages(followerActor).get(1); + + assertTrue(o instanceof InstallSnapshotMessages.InstallSnapshot); + + installSnapshot = (InstallSnapshotMessages.InstallSnapshot) o; + + assertEquals(2, installSnapshot.getChunkIndex()); + assertEquals(3, installSnapshot.getTotalChunks()); + assertEquals(hashCode, installSnapshot.getLastChunkHashCode()); + + followerActor.tell(PoisonPill.getInstance(), getRef()); + }}; + } + @Test public void testFollowerToSnapshotLogic() { @@ -628,6 +785,7 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { return createActorContext(leaderActor); } + @Override protected RaftActorContext createActorContext(ActorRef actorRef) { return new MockRaftActorContext("test", getSystem(), actorRef); } @@ -994,10 +1152,7 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { final Terminated termMsg1 = probe.expectMsgClass(Terminated.class); assertEquals(termMsg1.getActor(), followerActor1); - //sleep enough for all the follower stopwatches to lapse - Uninterruptibles.sleepUninterruptibly(leaderActorContext.getConfigParams(). - getElectionTimeOutInterval().toMillis(), TimeUnit.MILLISECONDS); - + leader.markFollowerInActive("follower-1"); leader.markFollowerActive("follower-2"); behavior = leader.handleMessage(leaderActor, new IsolatedLeaderCheck()); Assert.assertTrue("Behavior not instance of Leader when majority of followers are active", @@ -1010,10 +1165,7 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { final Terminated termMsg2 = probe.expectMsgClass(Terminated.class); assertEquals(termMsg2.getActor(), followerActor2); - //sleep enough for the remaining the follower stopwatches to lapse - Uninterruptibles.sleepUninterruptibly(leaderActorContext.getConfigParams(). - getElectionTimeOutInterval().toMillis(), TimeUnit.MILLISECONDS); - + leader.markFollowerInActive("follower-2"); behavior = leader.handleMessage(leaderActor, new IsolatedLeaderCheck()); Assert.assertTrue("Behavior not instance of IsolatedLeader when majority followers are inactive", behavior instanceof IsolatedLeader); @@ -1035,8 +1187,7 @@ public class LeaderTest extends AbstractRaftActorBehaviorTest { public void createFollowerToSnapshot(String followerId, ByteString bs ) { fts = new FollowerToSnapshot(bs); - mapFollowerToSnapshot.put(followerId, fts); - + setFollowerSnapshot(followerId, fts); } }