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=df219436007be602f73946e178f8bf510577df84;hp=f56755b447d347981b687cf017eded1caa059ea6;hb=fe8352361d49c76a0ecc80162a2b8258d35198b5;hpb=b5e3b3f436f8534ddb6c7f326ccbef995b96ddc3 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 f56755b447..df21943600 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,6 +1,17 @@ +/* + * Copyright (c) 2014, 2015 Cisco Systems, Inc. and others. All rights reserved. + * + * This program and the accompanying materials are made available under the + * terms of the Eclipse Public License v1.0 which accompanies this distribution, + * and is available at http://www.eclipse.org/legal/epl-v10.html + */ + package org.opendaylight.controller.cluster.raft.behaviors; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; + import akka.actor.ActorRef; import akka.actor.Props; import akka.testkit.TestActorRef; @@ -19,18 +30,19 @@ 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.SerializationUtils; import org.opendaylight.controller.cluster.raft.TestActorFactory; 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.persisted.SimpleReplicatedLogEntry; +import org.opendaylight.controller.cluster.raft.policy.RaftPolicy; import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload; import org.opendaylight.controller.cluster.raft.utils.MessageCollectorActor; import org.slf4j.LoggerFactory; -public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { +public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { protected final TestActorFactory actorFactory = new TestActorFactory(getSystem()); @@ -41,7 +53,7 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { @After public void tearDown() throws Exception { - if(behavior != null) { + if (behavior != null) { behavior.close(); } @@ -51,12 +63,10 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { /** * 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 { - RaftActorContext actorContext = createActorContext(); + MockRaftActorContext actorContext = createActorContext(); assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(actorContext, behaviorActor, createAppendEntriesWithNewerTerm()); @@ -75,34 +85,34 @@ 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 + * change it's state and it responds back with a failure. */ @Test public void testHandleAppendEntriesSenderTermLessThanReceiverTerm() throws Exception { - MockRaftActorContext context = createActorContext(); + MockRaftActorContext context = createActorContext(); + short payloadVersion = 5; + context.setPayloadVersion(payloadVersion); - // First set the receivers term to a high number (1000) - context.getTermInformation().update(1000, "test"); + // 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, -1); + AppendEntries appendEntries = new AppendEntries(100, "leader-1", 0, 0, null, 101, -1, (short)4); - behavior = createBehavior(context); + behavior = createBehavior(context); - // Send an unknown message so that the state of the RaftActor remains unchanged - RaftActorBehavior expected = behavior.handleMessage(behaviorActor, "unknown"); + RaftState expected = behavior.state(); - RaftActorBehavior raftBehavior = behavior.handleMessage(behaviorActor, appendEntries); + RaftActorBehavior raftBehavior = behavior.handleMessage(behaviorActor, appendEntries); - assertEquals("Raft state", expected.state(), raftBehavior.state()); + assertEquals("Raft state", expected, raftBehavior.state()); - // Also expect an AppendEntriesReply to be sent where success is false + // Also expect an AppendEntriesReply to be sent where success is false - AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching( - behaviorActor, AppendEntriesReply.class); + AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching( + behaviorActor, AppendEntriesReply.class); - assertEquals("isSuccess", false, reply.isSuccess()); + assertEquals("isSuccess", false, reply.isSuccess()); + assertEquals("getPayloadVersion", payloadVersion, reply.getPayloadVersion()); } @@ -117,27 +127,22 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { setLastLogEntry(context, 2, 0, payload); List entries = new ArrayList<>(); - entries.add(new MockRaftActorContext.MockReplicatedLogEntry(2, 0, payload)); + entries.add(new SimpleReplicatedLogEntry(0, 2, payload)); - AppendEntries appendEntries = new AppendEntries(2, "leader-1", -1, -1, entries, 2, -1); + final AppendEntries appendEntries = new AppendEntries(2, "leader-1", -1, -1, entries, 2, -1, (short)0); behavior = createBehavior(context); - if (behavior instanceof Candidate) { - // 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"); - } + assertFalse("This test should be overridden when testing Candidate", behavior instanceof Candidate); + + RaftState expected = behavior.state(); - // Send an unknown message so that the state of the RaftActor remains unchanged - RaftActorBehavior expected = behavior.handleMessage(behaviorActor, "unknown"); + // Check that the behavior does not handle unknwon message + assertNull(behavior.handleMessage(behaviorActor, "unknown")); RaftActorBehavior raftBehavior = behavior.handleMessage(behaviorActor, appendEntries); - assertEquals("Raft state", expected.state(), raftBehavior.state()); + assertEquals("Raft state", expected, raftBehavior.state()); assertEquals("ReplicatedLog size", 1, context.getReplicatedLog().size()); @@ -174,7 +179,7 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { /** * 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 + * log then the receiving RaftActor will not grant the vote to the sender. */ @Test public void testHandleRequestVoteWhenSenderLogLessUptoDate() { @@ -202,11 +207,11 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { /** * 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 + * recipient RaftActor. */ @Test public void testHandleRequestVoteWhenSenderTermLessThanCurrentTerm() { - RaftActorContext context = createActorContext(); + MockRaftActorContext context = createActorContext(); context.getTermInformation().update(1000, null); @@ -250,7 +255,8 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { assertEquals(0, abstractBehavior.getReplicatedToAllIndex()); assertEquals(1, context.getReplicatedLog().size()); - //5 entries, lastApplied =2 and replicatedIndex = 3, but since we want to keep the lastapplied, indices 0 and 1 will only get purged + // 5 entries, lastApplied =2 and replicatedIndex = 3, but since we want to keep the lastapplied, indices 0 and + // 1 will only get purged context.setReplicatedLog(new MockRaftActorContext.MockReplicatedLogBuilder().createEntries(0, 5, 1).build()); context.setLastApplied(2); abstractBehavior.performSnapshotWithoutCapture(3); @@ -266,11 +272,11 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { } - protected void assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(RaftActorContext actorContext, + protected void assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(MockRaftActorContext actorContext, ActorRef actorRef, RaftRPC rpc) throws Exception { - Payload p = new MockRaftActorContext.MockPayload(""); - setLastLogEntry((MockRaftActorContext) actorContext, 1, 0, p); + Payload payload = new MockRaftActorContext.MockPayload(""); + setLastLogEntry(actorContext, 1, 0, payload); actorContext.getTermInformation().update(1, "test"); RaftActorBehavior origBehavior = createBehavior(actorContext); @@ -285,8 +291,7 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { protected MockRaftActorContext.SimpleReplicatedLog setLastLogEntry( MockRaftActorContext actorContext, long term, long index, Payload data) { - return setLastLogEntry(actorContext, - new MockRaftActorContext.MockReplicatedLogEntry(term, index, data)); + return setLastLogEntry(actorContext, new SimpleReplicatedLogEntry(index, term, data)); } protected MockRaftActorContext.SimpleReplicatedLog setLastLogEntry(MockRaftActorContext actorContext, @@ -298,8 +303,13 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { return log; } - protected abstract RaftActorBehavior createBehavior( - RaftActorContext actorContext); + protected abstract T createBehavior(RaftActorContext actorContext); + + protected final T createBehavior(MockRaftActorContext actorContext) { + T ret = createBehavior((RaftActorContext)actorContext); + actorContext.setCurrentBehavior(ret); + return ret; + } protected RaftActorBehavior createBehavior() { return createBehavior(createActorContext()); @@ -314,11 +324,11 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { } protected AppendEntries createAppendEntriesWithNewerTerm() { - return new AppendEntries(100, "leader-1", 0, 0, null, 1, -1); + return new AppendEntries(100, "leader-1", 0, 0, null, 1, -1, (short)0); } protected AppendEntriesReply createAppendEntriesReplyWithNewerTerm() { - return new AppendEntriesReply("follower-1", 100, false, 100, 100); + return new AppendEntriesReply("follower-1", 100, false, 100, 100, (short)0); } protected RequestVote createRequestVoteWithNewerTerm() { @@ -329,13 +339,9 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { return new RequestVoteReply(100, false); } - protected Object fromSerializableMessage(Object serializable){ - return SerializationUtils.fromSerializable(serializable); - } - protected ByteString toByteString(Map state) { ByteArrayOutputStream bos = new ByteArrayOutputStream(); - try(ObjectOutputStream oos = new ObjectOutputStream(bos)) { + try (ObjectOutputStream oos = new ObjectOutputStream(bos)) { oos.writeObject(state); return ByteString.copyFrom(bos.toByteArray()); } catch (IOException e) { @@ -346,4 +352,19 @@ public abstract class AbstractRaftActorBehaviorTest extends AbstractActorTest { protected void logStart(String name) { LoggerFactory.getLogger(LeaderTest.class).info("Starting " + name); } + + protected RaftPolicy createRaftPolicy(final boolean automaticElectionsEnabled, + final boolean applyModificationToStateBeforeConsensus) { + return new RaftPolicy() { + @Override + public boolean automaticElectionsEnabled() { + return automaticElectionsEnabled; + } + + @Override + public boolean applyModificationToStateBeforeConsensus() { + return applyModificationToStateBeforeConsensus; + } + }; + } }