Remove MockReplicatedLogEntry
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / test / java / org / opendaylight / controller / cluster / raft / behaviors / FollowerTest.java
index df468cf24c59f1ae2e991fdac02342301744bfa2..9d64b140fb7d9db2c191eb12ffc847deb5273561 100644 (file)
@@ -15,11 +15,10 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
+
 import akka.actor.ActorRef;
 import akka.actor.Props;
 import akka.testkit.TestActorRef;
@@ -53,6 +52,7 @@ import org.opendaylight.controller.cluster.raft.messages.RequestVote;
 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
 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.utils.MessageCollectorActor;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -71,7 +71,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     @Override
     @After
     public void tearDown() throws Exception {
-        if(follower != null) {
+        if (follower != null) {
             follower.close();
         }
 
@@ -89,14 +89,14 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     }
 
     @Override
-    protected  MockRaftActorContext createActorContext(ActorRef actorRef){
+    protected  MockRaftActorContext createActorContext(ActorRef actorRef) {
         MockRaftActorContext context = new MockRaftActorContext("follower", getSystem(), actorRef);
         context.setPayloadVersion(payloadVersion );
         return context;
     }
 
     @Test
-    public void testThatAnElectionTimeoutIsTriggered(){
+    public void testThatAnElectionTimeoutIsTriggered() {
         MockRaftActorContext actorContext = createActorContext();
         follower = new Follower(actorContext);
 
@@ -123,15 +123,15 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         logStart("testHandleElectionTimeoutWhenLeaderMessageReceived");
 
         MockRaftActorContext context = createActorContext();
-        ((DefaultConfigParamsImpl) context.getConfigParams()).
-                setHeartBeatInterval(new FiniteDuration(100, TimeUnit.MILLISECONDS));
+        ((DefaultConfigParamsImpl) context.getConfigParams())
+                .setHeartBeatInterval(new FiniteDuration(100, TimeUnit.MILLISECONDS));
         ((DefaultConfigParamsImpl) context.getConfigParams()).setElectionTimeoutFactor(4);
 
         follower = new Follower(context);
         context.setCurrentBehavior(follower);
 
-        Uninterruptibles.sleepUninterruptibly(context.getConfigParams().
-                getElectionTimeOutInterval().toMillis() - 100, TimeUnit.MILLISECONDS);
+        Uninterruptibles.sleepUninterruptibly(context.getConfigParams()
+                .getElectionTimeOutInterval().toMillis() - 100, TimeUnit.MILLISECONDS);
         follower.handleMessage(leaderActor, new AppendEntries(1, "leader", -1, -1, Collections.emptyList(),
                 -1, -1, (short) 1));
 
@@ -139,8 +139,8 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         RaftActorBehavior raftBehavior = follower.handleMessage(leaderActor, ElectionTimeout.INSTANCE);
         assertTrue(raftBehavior instanceof Follower);
 
-        Uninterruptibles.sleepUninterruptibly(context.getConfigParams().
-                getElectionTimeOutInterval().toMillis() - 150, TimeUnit.MILLISECONDS);
+        Uninterruptibles.sleepUninterruptibly(context.getConfigParams()
+                .getElectionTimeOutInterval().toMillis() - 150, TimeUnit.MILLISECONDS);
         follower.handleMessage(leaderActor, new AppendEntries(1, "leader", -1, -1, Collections.emptyList(),
                 -1, -1, (short) 1));
 
@@ -169,7 +169,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     }
 
     @Test
-    public void testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForIsNotTheSameAsCandidateId(){
+    public void testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForIsNotTheSameAsCandidateId() {
         logStart("testHandleRequestVoteWhenSenderTermEqualToCurrentTermAndVotedForIsNotTheSameAsCandidateId");
 
         MockRaftActorContext context = createActorContext();
@@ -207,7 +207,8 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         follower = createBehavior(context);
         follower.handleMessage(leaderActor, appendEntries);
 
-        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor,
+                FollowerInitialSyncUpStatus.class);
         AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, AppendEntriesReply.class);
 
         assertFalse(syncStatus.isInitialSyncDone());
@@ -229,7 +230,8 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         follower = createBehavior(context);
         follower.handleMessage(leaderActor, appendEntries);
 
-        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor,
+                FollowerInitialSyncUpStatus.class);
         AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, AppendEntriesReply.class);
 
         assertFalse(syncStatus.isInitialSyncDone());
@@ -237,7 +239,8 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     }
 
     @Test
-    public void testHandleFirstAppendEntriesWithPrevIndexMinusOneAndReplicatedToAllIndexPresentInLog() throws Exception {
+    public void testHandleFirstAppendEntriesWithPrevIndexMinusOneAndReplicatedToAllIndexPresentInLog()
+            throws Exception {
         logStart("testHandleFirstAppendEntries");
 
         MockRaftActorContext context = createActorContext();
@@ -254,7 +257,8 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         follower = createBehavior(context);
         follower.handleMessage(leaderActor, appendEntries);
 
-        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor,
+                FollowerInitialSyncUpStatus.class);
         AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, AppendEntriesReply.class);
 
         assertFalse(syncStatus.isInitialSyncDone());
@@ -262,7 +266,8 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     }
 
     @Test
-    public void testHandleFirstAppendEntriesWithPrevIndexMinusOneAndReplicatedToAllIndexPresentInSnapshot() throws Exception {
+    public void testHandleFirstAppendEntriesWithPrevIndexMinusOneAndReplicatedToAllIndexPresentInSnapshot()
+            throws Exception {
         logStart("testHandleFirstAppendEntries");
 
         MockRaftActorContext context = createActorContext();
@@ -278,7 +283,8 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         follower = createBehavior(context);
         follower.handleMessage(leaderActor, appendEntries);
 
-        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor,
+                FollowerInitialSyncUpStatus.class);
         AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, AppendEntriesReply.class);
 
         assertFalse(syncStatus.isInitialSyncDone());
@@ -286,8 +292,10 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     }
 
     @Test
-    public void testHandleFirstAppendEntriesWithPrevIndexMinusOneAndReplicatedToAllIndexPresentInSnapshotButCalculatedPreviousEntryMissing() throws Exception {
-        logStart("testHandleFirstAppendEntries");
+    public void testFirstAppendEntriesWithNoPrevIndexAndReplicatedToAllPresentInSnapshotButCalculatedPrevEntryMissing()
+            throws Exception {
+        logStart(
+               "testFirstAppendEntriesWithNoPrevIndexAndReplicatedToAllPresentInSnapshotButCalculatedPrevEntryMissing");
 
         MockRaftActorContext context = createActorContext();
         context.getReplicatedLog().clear(0,2);
@@ -302,7 +310,8 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         follower = createBehavior(context);
         follower.handleMessage(leaderActor, appendEntries);
 
-        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor,
+                FollowerInitialSyncUpStatus.class);
         AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, AppendEntriesReply.class);
 
         assertFalse(syncStatus.isInitialSyncDone());
@@ -324,7 +333,8 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         follower = createBehavior(context);
         follower.handleMessage(leaderActor, appendEntries);
 
-        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor,
+                FollowerInitialSyncUpStatus.class);
 
         assertFalse(syncStatus.isInitialSyncDone());
 
@@ -374,7 +384,8 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         follower = createBehavior(context);
         follower.handleMessage(leaderActor, appendEntries);
 
-        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor,
+                FollowerInitialSyncUpStatus.class);
 
         assertFalse(syncStatus.isInitialSyncDone());
 
@@ -415,7 +426,8 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         follower = createBehavior(context);
         follower.handleMessage(leaderActor, appendEntries);
 
-        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor, FollowerInitialSyncUpStatus.class);
+        FollowerInitialSyncUpStatus syncStatus = MessageCollectorActor.expectFirstMatching(followerActor,
+                FollowerInitialSyncUpStatus.class);
 
         assertFalse(syncStatus.isInitialSyncDone());
 
@@ -465,8 +477,6 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
      * with a commitIndex that is greater than what has been applied to the
      * state machine of the RaftActor, the RaftActor applies the state and
      * sets it current applied state to the commitIndex of the sender.
-     *
-     * @throws Exception
      */
     @Test
     public void testHandleAppendEntriesWithNewerCommitIndex() throws Exception {
@@ -495,8 +505,6 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
      * This test verifies that when an AppendEntries is received a specific prevLogTerm
      * which does not match the term that is in RaftActors log entry at prevLogIndex
      * then the RaftActor does not change it's state and it returns a failure.
-     *
-     * @throws Exception
      */
     @Test
     public void testHandleAppendEntriesSenderPrevLogTermNotSameAsReceiverPrevLogTerm() {
@@ -527,9 +535,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
      * This test verifies that when a new AppendEntries message is received with
      * new entries and the logs of the sender and receiver match that the new
      * entries get added to the log and the log is incremented by the number of
-     * entries received in appendEntries
-     *
-     * @throws Exception
+     * entries received in appendEntries.
      */
     @Test
     public void testHandleAppendEntriesAddNewEntries() {
@@ -581,7 +587,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
      * This test verifies that when a new AppendEntries message is received with
      * new entries and the logs of the sender and receiver are out-of-sync that
      * the log is first corrected by removing the out of sync entries from the
-     * log and then adding in the new entries sent with the AppendEntries message
+     * log and then adding in the new entries sent with the AppendEntries message.
      */
     @Test
     public void testHandleAppendEntriesCorrectReceiverLogEntries() {
@@ -673,10 +679,10 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     }
 
     @Test
-    public void testHandleAppendEntriesPreviousLogEntryMissing(){
+    public void testHandleAppendEntriesPreviousLogEntryMissing() {
         logStart("testHandleAppendEntriesPreviousLogEntryMissing");
 
-        MockRaftActorContext context = createActorContext();
+        final MockRaftActorContext context = createActorContext();
 
         // Prepare the receivers log
         MockRaftActorContext.SimpleReplicatedLog log = new MockRaftActorContext.SimpleReplicatedLog();
@@ -743,7 +749,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     }
 
     @Test
-    public void testHandleAppendEntriesAfterInstallingSnapshot(){
+    public void testHandleAppendEntriesAfterInstallingSnapshot() {
         logStart("testHandleAppendAfterInstallingSnapshot");
 
         MockRaftActorContext context = createActorContext();
@@ -776,8 +782,6 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     /**
      * This test verifies that when InstallSnapshot is received by
      * the follower its applied correctly.
-     *
-     * @throws Exception
      */
     @Test
     public void testHandleInstallSnapshot() throws Exception {
@@ -792,12 +796,12 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         int offset = 0;
         int snapshotLength = bsSnapshot.size();
         int chunkSize = 50;
-        int totalChunks = (snapshotLength / chunkSize) + ((snapshotLength % chunkSize) > 0 ? 1 : 0);
+        int totalChunks = snapshotLength / chunkSize + (snapshotLength % chunkSize > 0 ? 1 : 0);
         int lastIncludedIndex = 1;
         int chunkIndex = 1;
         InstallSnapshot lastInstallSnapshot = null;
 
-        for(int i = 0; i < totalChunks; i++) {
+        for (int i = 0; i < totalChunks; i++) {
             byte[] chunkData = getNextChunk(bsSnapshot, offset, chunkSize);
             lastInstallSnapshot = new InstallSnapshot(1, "leader", lastIncludedIndex, 1,
                     chunkData, chunkIndex, totalChunks);
@@ -827,7 +831,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         assertEquals("InstallSnapshotReply count", totalChunks, replies.size());
 
         chunkIndex = 1;
-        for(InstallSnapshotReply reply: replies) {
+        for (InstallSnapshotReply reply: replies) {
             assertEquals("getChunkIndex", chunkIndex++, reply.getChunkIndex());
             assertEquals("getTerm", 1, reply.getTerm());
             assertEquals("isSuccess", true, reply.isSuccess());
@@ -841,8 +845,6 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     /**
      * Verify that when an AppendEntries is sent to a follower during a snapshot install
      * the Follower short-circuits the processing of the AppendEntries message.
-     *
-     * @throws Exception
      */
     @Test
     public void testReceivingAppendEntriesDuringInstallSnapshot() throws Exception {
@@ -855,7 +857,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         ByteString bsSnapshot  = createSnapshot();
         int snapshotLength = bsSnapshot.size();
         int chunkSize = 50;
-        int totalChunks = (snapshotLength / chunkSize) + ((snapshotLength % chunkSize) > 0 ? 1 : 0);
+        int totalChunks = snapshotLength / chunkSize + (snapshotLength % chunkSize > 0 ? 1 : 0);
         int lastIncludedIndex = 1;
 
         // Check that snapshot installation is not in progress
@@ -873,19 +875,61 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         assertNotNull(follower.getSnapshotTracker());
 
         // Send an append entry
-        AppendEntries appendEntries = mock(AppendEntries.class);
-        doReturn(context.getTermInformation().getCurrentTerm()).when(appendEntries).getTerm();
+        AppendEntries appendEntries = new AppendEntries(1, "leader", 1, 1,
+                Arrays.asList(newReplicatedLogEntry(2, 1, "3")), 2, -1, (short)1);
 
         follower.handleMessage(leaderActor, appendEntries);
 
         AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, AppendEntriesReply.class);
-        assertEquals(context.getReplicatedLog().lastIndex(), reply.getLogLastIndex());
-        assertEquals(context.getReplicatedLog().lastTerm(), reply.getLogLastTerm());
-        assertEquals(context.getTermInformation().getCurrentTerm(), reply.getTerm());
+        assertEquals("isSuccess", true, reply.isSuccess());
+        assertEquals("getLogLastIndex", context.getReplicatedLog().lastIndex(), reply.getLogLastIndex());
+        assertEquals("getLogLastTerm", context.getReplicatedLog().lastTerm(), reply.getLogLastTerm());
+        assertEquals("getTerm", context.getTermInformation().getCurrentTerm(), reply.getTerm());
 
-        // We should not hit the code that needs to look at prevLogIndex because we are short circuiting
-        verify(appendEntries, never()).getPrevLogIndex();
+        assertNotNull(follower.getSnapshotTracker());
+    }
+
+    @Test
+    public void testReceivingAppendEntriesDuringInstallSnapshotFromDifferentLeader() throws Exception {
+        logStart("testReceivingAppendEntriesDuringInstallSnapshotFromDifferentLeader");
 
+        MockRaftActorContext context = createActorContext();
+
+        follower = createBehavior(context);
+
+        ByteString bsSnapshot  = createSnapshot();
+        int snapshotLength = bsSnapshot.size();
+        int chunkSize = 50;
+        int totalChunks = snapshotLength / chunkSize + (snapshotLength % chunkSize > 0 ? 1 : 0);
+        int lastIncludedIndex = 1;
+
+        // Check that snapshot installation is not in progress
+        assertNull(follower.getSnapshotTracker());
+
+        // Make sure that we have more than 1 chunk to send
+        assertTrue(totalChunks > 1);
+
+        // Send an install snapshot with the first chunk to start the process of installing a snapshot
+        byte[] chunkData = getNextChunk(bsSnapshot, 0, chunkSize);
+        follower.handleMessage(leaderActor, new InstallSnapshot(1, "leader", lastIncludedIndex, 1,
+                chunkData, 1, totalChunks));
+
+        // Check if snapshot installation is in progress now
+        assertNotNull(follower.getSnapshotTracker());
+
+        // Send appendEntries with a new term and leader.
+        AppendEntries appendEntries = new AppendEntries(2, "new-leader", 1, 1,
+                Arrays.asList(newReplicatedLogEntry(2, 2, "3")), 2, -1, (short)1);
+
+        follower.handleMessage(leaderActor, appendEntries);
+
+        AppendEntriesReply reply = MessageCollectorActor.expectFirstMatching(leaderActor, AppendEntriesReply.class);
+        assertEquals("isSuccess", true, reply.isSuccess());
+        assertEquals("getLogLastIndex", 2, reply.getLogLastIndex());
+        assertEquals("getLogLastTerm", 2, reply.getLogLastTerm());
+        assertEquals("getTerm", 2, reply.getTerm());
+
+        assertNull(follower.getSnapshotTracker());
     }
 
     @Test
@@ -901,12 +945,12 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
         int offset = 0;
         int snapshotLength = bsSnapshot.size();
         int chunkSize = 50;
-        int totalChunks = (snapshotLength / chunkSize) + ((snapshotLength % chunkSize) > 0 ? 1 : 0);
+        int totalChunks = snapshotLength / chunkSize + (snapshotLength % chunkSize > 0 ? 1 : 0);
         int lastIncludedIndex = 1;
         int chunkIndex = 1;
         InstallSnapshot lastInstallSnapshot = null;
 
-        for(int i = 0; i < totalChunks; i++) {
+        for (int i = 0; i < totalChunks; i++) {
             byte[] chunkData = getNextChunk(bsSnapshot, offset, chunkSize);
             lastInstallSnapshot = new InstallSnapshot(1, "leader", lastIncludedIndex, 1,
                     chunkData, chunkIndex, totalChunks);
@@ -967,7 +1011,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     }
 
     @Test
-    public void testFollowerSchedulesElectionTimeoutImmediatelyWhenItHasNoPeers(){
+    public void testFollowerSchedulesElectionTimeoutImmediatelyWhenItHasNoPeers() {
         MockRaftActorContext context = createActorContext();
 
         Stopwatch stopwatch = Stopwatch.createStarted();
@@ -985,9 +1029,9 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     }
 
     @Test
-    public void testFollowerSchedulesElectionIfAutomaticElectionsAreDisabled(){
+    public void testFollowerSchedulesElectionIfAutomaticElectionsAreDisabled() {
         MockRaftActorContext context = createActorContext();
-        context.setConfigParams(new DefaultConfigParamsImpl(){
+        context.setConfigParams(new DefaultConfigParamsImpl() {
             @Override
             public FiniteDuration getElectionTimeOutInterval() {
                 return FiniteDuration.apply(100, TimeUnit.MILLISECONDS);
@@ -1004,7 +1048,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     }
 
     @Test
-    public void testFollowerSchedulesElectionIfNonVoting(){
+    public void testFollowerSchedulesElectionIfNonVoting() {
         MockRaftActorContext context = createActorContext();
         context.updatePeerIds(new ServerConfigurationPayload(Arrays.asList(new ServerInfo(context.getId(), false))));
         ((DefaultConfigParamsImpl)context.getConfigParams()).setHeartBeatInterval(
@@ -1021,7 +1065,7 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     }
 
     @Test
-    public void testElectionScheduledWhenAnyRaftRPCReceived(){
+    public void testElectionScheduledWhenAnyRaftRPCReceived() {
         MockRaftActorContext context = createActorContext();
         follower = createBehavior(context);
         follower.handleMessage(leaderActor, new RaftRPC() {
@@ -1036,21 +1080,21 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
     }
 
     @Test
-    public void testElectionNotScheduledWhenNonRaftRPCMessageReceived(){
+    public void testElectionNotScheduledWhenNonRaftRPCMessageReceived() {
         MockRaftActorContext context = createActorContext();
         follower = createBehavior(context);
         follower.handleMessage(leaderActor, "non-raft-rpc");
         verify(follower, never()).scheduleElection(any(FiniteDuration.class));
     }
 
-    public byte[] getNextChunk (ByteString bs, int offset, int chunkSize){
+    public byte[] getNextChunk(ByteString bs, int offset, int chunkSize) {
         int snapshotLength = bs.size();
         int start = offset;
         int size = chunkSize;
         if (chunkSize > snapshotLength) {
             size = snapshotLength;
         } else {
-            if ((start + chunkSize) > snapshotLength) {
+            if (start + chunkSize > snapshotLength) {
                 size = snapshotLength - start;
             }
         }
@@ -1083,11 +1127,11 @@ public class FollowerTest extends AbstractRaftActorBehaviorTest<Follower> {
 
 
     private static ReplicatedLogEntry newReplicatedLogEntry(long term, long index, String data) {
-        return new MockRaftActorContext.MockReplicatedLogEntry(term, index,
+        return new SimpleReplicatedLogEntry(index, term,
                 new MockRaftActorContext.MockPayload(data));
     }
 
-    private ByteString createSnapshot(){
+    private ByteString createSnapshot() {
         HashMap<String, String> followerSnapshot = new HashMap<>();
         followerSnapshot.put("1", "A");
         followerSnapshot.put("2", "B");