Remove MockReplicatedLogEntry
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractRaftActorBehaviorTest.java
index 30ca63b0634a74c7ecbe0887ba0e8452f72b6176..df219436007be602f73946e178f8bf510577df84 100644 (file)
@@ -10,6 +10,8 @@ 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;
@@ -28,13 +30,13 @@ 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;
@@ -51,7 +53,7 @@ public abstract class AbstractRaftActorBehaviorTest<T extends RaftActorBehavior>
 
     @After
     public void tearDown() throws Exception {
-        if(behavior != null) {
+        if (behavior != null) {
             behavior.close();
         }
 
@@ -61,8 +63,6 @@ public abstract class AbstractRaftActorBehaviorTest<T extends RaftActorBehavior>
     /**
      * 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 {
@@ -85,9 +85,7 @@ public abstract class AbstractRaftActorBehaviorTest<T extends RaftActorBehavior>
     /**
      * 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 {
@@ -102,12 +100,11 @@ public abstract class AbstractRaftActorBehaviorTest<T extends RaftActorBehavior>
 
         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);
 
-        assertEquals("Raft state", expected.state(), raftBehavior.state());
+        assertEquals("Raft state", expected, raftBehavior.state());
 
         // Also expect an AppendEntriesReply to be sent where success is false
 
@@ -130,20 +127,22 @@ public abstract class AbstractRaftActorBehaviorTest<T extends RaftActorBehavior>
         setLastLogEntry(context, 2, 0, payload);
 
         List<ReplicatedLogEntry> 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, (short)0);
+        final AppendEntries appendEntries = new AppendEntries(2, "leader-1", -1, -1, entries, 2, -1, (short)0);
 
         behavior = createBehavior(context);
 
         assertFalse("This test should be overridden when testing Candidate", behavior instanceof Candidate);
 
-        // Send an unknown message so that the state of the RaftActor remains unchanged
-        RaftActorBehavior expected = behavior.handleMessage(behaviorActor, "unknown");
+        RaftState expected = behavior.state();
+
+        // 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());
 
@@ -180,7 +179,7 @@ public abstract class AbstractRaftActorBehaviorTest<T extends RaftActorBehavior>
     /**
      * 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() {
@@ -208,7 +207,7 @@ public abstract class AbstractRaftActorBehaviorTest<T extends RaftActorBehavior>
     /**
      * 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() {
@@ -256,7 +255,8 @@ public abstract class AbstractRaftActorBehaviorTest<T extends RaftActorBehavior>
         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);
@@ -275,8 +275,8 @@ public abstract class AbstractRaftActorBehaviorTest<T extends RaftActorBehavior>
     protected void assertStateChangesToFollowerWhenRaftRPCHasNewerTerm(MockRaftActorContext actorContext,
             ActorRef actorRef, RaftRPC rpc) throws Exception {
 
-        Payload p = new MockRaftActorContext.MockPayload("");
-        setLastLogEntry(actorContext, 1, 0, p);
+        Payload payload = new MockRaftActorContext.MockPayload("");
+        setLastLogEntry(actorContext, 1, 0, payload);
         actorContext.getTermInformation().update(1, "test");
 
         RaftActorBehavior origBehavior = createBehavior(actorContext);
@@ -291,8 +291,7 @@ public abstract class AbstractRaftActorBehaviorTest<T extends RaftActorBehavior>
 
     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,
@@ -340,13 +339,9 @@ public abstract class AbstractRaftActorBehaviorTest<T extends RaftActorBehavior>
         return new RequestVoteReply(100, false);
     }
 
-    protected Object fromSerializableMessage(Object serializable){
-        return SerializationUtils.fromSerializable(serializable);
-    }
-
     protected ByteString toByteString(Map<String, String> 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) {
@@ -359,7 +354,7 @@ public abstract class AbstractRaftActorBehaviorTest<T extends RaftActorBehavior>
     }
 
     protected RaftPolicy createRaftPolicy(final boolean automaticElectionsEnabled,
-                                          final boolean applyModificationToStateBeforeConsensus){
+                                          final boolean applyModificationToStateBeforeConsensus) {
         return new RaftPolicy() {
             @Override
             public boolean automaticElectionsEnabled() {