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%2Fbehaviors%2FAbstractRaftActorBehaviorTest.java;h=1a37b921e3ff500e00b6b5b0c3854960f12b706f;hp=e6bf26cdcdd9d8beefbcd1557c096ec24f312736;hb=00570ac6512cdeea8d3c1da85e4a2e118a6f925c;hpb=45f2bf8ff9daf30ef7fbe4525366ea92a95fa213 diff --git a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehaviorTest.java b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehaviorTest.java index e6bf26cdcd..1a37b921e3 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehaviorTest.java +++ b/opendaylight/md-sal/sal-akka-raft/src/test/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractRaftActorBehaviorTest.java @@ -1,25 +1,39 @@ package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; +import akka.actor.Props; import akka.testkit.JavaTestKit; import org.junit.Test; import org.opendaylight.controller.cluster.raft.AbstractActorTest; import org.opendaylight.controller.cluster.raft.MockRaftActorContext; import org.opendaylight.controller.cluster.raft.RaftActorContext; import org.opendaylight.controller.cluster.raft.RaftState; +import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry; import org.opendaylight.controller.cluster.raft.messages.AppendEntries; import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply; import org.opendaylight.controller.cluster.raft.messages.RaftRPC; import org.opendaylight.controller.cluster.raft.messages.RequestVote; import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; +import org.opendaylight.controller.cluster.raft.utils.DoNothingActor; -import java.util.concurrent.atomic.AtomicLong; +import java.util.ArrayList; +import java.util.List; import static org.junit.Assert.assertEquals; -public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest{ - @Test - public void testHandlingOfRaftRPCWithNewerTerm() throws Exception { +public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { + + private final ActorRef behaviorActor = getSystem().actorOf(Props.create( + DoNothingActor.class)); + + /** + * This test checks that when a new Raft RPC message is received with a newer + * term the RaftActor gets into the Follower state. + * + * @throws Exception + */ + @Test + public void testHandleRaftRPCWithNewerTerm() throws Exception { new JavaTestKit(getSystem()) {{ assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(getTestActor(), @@ -38,58 +52,313 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest{ }}; } + + /** + * This test verifies that when an AppendEntries is received with a term that + * is less that the currentTerm of the RaftActor then the RaftActor does not + * change it's state and it responds back with a failure + * + * @throws Exception + */ @Test - public void testHandlingOfAppendEntriesWithNewerCommitIndex() throws Exception{ + public void testHandleAppendEntriesSenderTermLessThanReceiverTerm() + throws Exception { new JavaTestKit(getSystem()) {{ - RaftActorContext context = + MockRaftActorContext context = (MockRaftActorContext) createActorContext(); - ((MockRaftActorContext) context).setLastApplied(new AtomicLong(100)); + // First set the receivers term to a high number (1000) + context.getTermInformation().update(1000, "test"); AppendEntries appendEntries = new AppendEntries(100, "leader-1", 0, 0, null, 101); + RaftActorBehavior behavior = createBehavior(context); + + // Send an unknown message so that the state of the RaftActor remains unchanged + RaftState expected = behavior.handleMessage(getRef(), "unknown"); + RaftState raftState = - createBehavior(context).handleMessage(getRef(), appendEntries); + behavior.handleMessage(getRef(), appendEntries); + + assertEquals(expected, raftState); + + // Also expect an AppendEntriesReply to be sent where success is false + final Boolean out = new ExpectMsg(duration("1 seconds"), + "AppendEntriesReply") { + // do not put code outside this method, will run afterwards + protected Boolean match(Object in) { + if (in instanceof AppendEntriesReply) { + AppendEntriesReply reply = (AppendEntriesReply) in; + return reply.isSuccess(); + } else { + throw noMatch(); + } + } + }.get(); + + assertEquals(false, out); + + + }}; + } + + + @Test + public void testHandleAppendEntriesAddSameEntryToLog(){ + new JavaTestKit(getSystem()) { + { + + MockRaftActorContext context = (MockRaftActorContext) + createActorContext(); + + // First set the receivers term to lower number + context.getTermInformation().update(2, "test"); + + // Prepare the receivers log + MockRaftActorContext.SimpleReplicatedLog log = + new MockRaftActorContext.SimpleReplicatedLog(); + log.append( + new MockRaftActorContext.MockReplicatedLogEntry(1, 0, "zero")); + + context.setReplicatedLog(log); + + List entries = new ArrayList<>(); + entries.add( + new MockRaftActorContext.MockReplicatedLogEntry(1, 0, "zero")); + + AppendEntries appendEntries = + new AppendEntries(2, "leader-1", -1, 1, entries, 0); + + RaftActorBehavior behavior = createBehavior(context); + + if (AbstractRaftActorBehaviorTest.this instanceof CandidateTest) { + // Resetting the Candidates term to make sure it will match + // the term sent by AppendEntries. If this was not done then + // the test will fail because the Candidate will assume that + // the message was sent to it from a lower term peer and will + // thus respond with a failure + context.getTermInformation().update(2, "test"); + } + + // Send an unknown message so that the state of the RaftActor remains unchanged + RaftState expected = behavior.handleMessage(getRef(), "unknown"); + + RaftState raftState = + behavior.handleMessage(getRef(), appendEntries); + + assertEquals(expected, raftState); + + assertEquals(1, log.size()); + + + }}; + } + + /** + * This test verifies that when a RequestVote is received by the RaftActor + * with a term which is greater than the RaftActors' currentTerm and the + * senders' log is more upto date than the receiver that the receiver grants + * the vote to the sender + */ + @Test + public void testHandleRequestVoteWhenSenderTermGreaterThanCurrentTermAndSenderLogMoreUpToDate() { + new JavaTestKit(getSystem()) {{ + + new Within(duration("1 seconds")) { + protected void run() { + + RaftActorBehavior behavior = createBehavior( + createActorContext(behaviorActor)); + + RaftState raftState = behavior.handleMessage(getTestActor(), + new RequestVote(1000, "test", 10000, 999)); + + if(behavior.state() != RaftState.Follower){ + assertEquals(RaftState.Follower, raftState); + } else { - assertEquals(new AtomicLong(101).get(), context.getLastApplied().get()); + final Boolean out = + new ExpectMsg(duration("1 seconds"), + "RequestVoteReply") { + // do not put code outside this method, will run afterwards + protected Boolean match(Object in) { + if (in instanceof RequestVoteReply) { + RequestVoteReply reply = + (RequestVoteReply) in; + return reply.isVoteGranted(); + } else { + throw noMatch(); + } + } + }.get(); + assertEquals(true, out); + } + } + }; + }}; + } + + /** + * This test verifies that when a RaftActor receives a RequestVote message + * with a term that is greater than it's currentTerm but a less up-to-date + * log then the receiving RaftActor will not grant the vote to the sender + */ + @Test + public void testHandleRequestVoteWhenSenderTermGreaterThanCurrentTermButSenderLogLessUptoDate() { + new JavaTestKit(getSystem()) {{ + + new Within(duration("1 seconds")) { + protected void run() { + + RaftActorContext actorContext = + createActorContext(behaviorActor); + + MockRaftActorContext.SimpleReplicatedLog + log = new MockRaftActorContext.SimpleReplicatedLog(); + log.append( + new MockRaftActorContext.MockReplicatedLogEntry(20000, + 1000000, "")); + + ((MockRaftActorContext) actorContext).setReplicatedLog(log); + + RaftActorBehavior behavior = createBehavior(actorContext); + + RaftState raftState = behavior.handleMessage(getTestActor(), + new RequestVote(1000, "test", 10000, 999)); + + if(behavior.state() != RaftState.Follower){ + assertEquals(RaftState.Follower, raftState); + } else { + final Boolean out = + new ExpectMsg(duration("1 seconds"), + "RequestVoteReply") { + // do not put code outside this method, will run afterwards + protected Boolean match(Object in) { + if (in instanceof RequestVoteReply) { + RequestVoteReply reply = + (RequestVoteReply) in; + return reply.isVoteGranted(); + } else { + throw noMatch(); + } + } + }.get(); + + assertEquals(false, out); + } + } + }; + }}; + } + + + + /** + * This test verifies that the receiving RaftActor will not grant a vote + * to a sender if the sender's term is lesser than the currentTerm of the + * recipient RaftActor + */ + @Test + public void testHandleRequestVoteWhenSenderTermLessThanCurrentTerm() { + new JavaTestKit(getSystem()) {{ + + new Within(duration("1 seconds")) { + protected void run() { + + RaftActorContext context = + createActorContext(behaviorActor); + + context.getTermInformation().update(1000, null); + + RaftActorBehavior follower = createBehavior(context); + + follower.handleMessage(getTestActor(), + new RequestVote(999, "test", 10000, 999)); + + final Boolean out = + new ExpectMsg(duration("1 seconds"), + "RequestVoteReply") { + // do not put code outside this method, will run afterwards + protected Boolean match(Object in) { + if (in instanceof RequestVoteReply) { + RequestVoteReply reply = + (RequestVoteReply) in; + return reply.isVoteGranted(); + } else { + throw noMatch(); + } + } + }.get(); + + assertEquals(false, out); + } + }; }}; } protected void assertStateChangesToFollowerWhenRaftRPCHasNewerTerm( - ActorRef actorRef, RaftRPC rpc){ - RaftState raftState = createBehavior() + ActorRef actorRef, RaftRPC rpc) { + + RaftActorContext actorContext = createActorContext(); + setLastLogEntry( + (MockRaftActorContext) actorContext, 0, 0, ""); + + RaftState raftState = createBehavior(actorContext) .handleMessage(actorRef, rpc); assertEquals(RaftState.Follower, raftState); } - protected abstract RaftActorBehavior createBehavior(RaftActorContext actorContext); + protected MockRaftActorContext.SimpleReplicatedLog setLastLogEntry( + MockRaftActorContext actorContext, long term, long index, Object data) { + return setLastLogEntry(actorContext, + new MockRaftActorContext.MockReplicatedLogEntry(term, index, data)); + } + + protected MockRaftActorContext.SimpleReplicatedLog setLastLogEntry( + MockRaftActorContext actorContext, ReplicatedLogEntry logEntry) { + MockRaftActorContext.SimpleReplicatedLog + log = new MockRaftActorContext.SimpleReplicatedLog(); + log.append(logEntry); + actorContext.setReplicatedLog(log); + + return log; + } + + protected abstract RaftActorBehavior createBehavior( + RaftActorContext actorContext); - protected RaftActorBehavior createBehavior(){ + protected RaftActorBehavior createBehavior() { return createBehavior(createActorContext()); } - protected RaftActorContext createActorContext(){ + protected RaftActorContext createActorContext() { return new MockRaftActorContext(); } - protected AppendEntries createAppendEntriesWithNewerTerm(){ + protected RaftActorContext createActorContext(ActorRef actor) { + return new MockRaftActorContext("test", getSystem(), actor); + } + + protected AppendEntries createAppendEntriesWithNewerTerm() { return new AppendEntries(100, "leader-1", 0, 0, null, 1); } - protected AppendEntriesReply createAppendEntriesReplyWithNewerTerm(){ - return new AppendEntriesReply(100, false); + protected AppendEntriesReply createAppendEntriesReplyWithNewerTerm() { + return new AppendEntriesReply("follower-1", 100, false, 100, 100); } - protected RequestVote createRequestVoteWithNewerTerm(){ + protected RequestVote createRequestVoteWithNewerTerm() { return new RequestVote(100, "candidate-1", 10, 100); } - protected RequestVoteReply createRequestVoteReplyWithNewerTerm(){ + protected RequestVoteReply createRequestVoteReplyWithNewerTerm() { return new RequestVoteReply(100, false); } + + }