Refactor ReplicatedLogImpl to separate class
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActor.java
index b74259d4851153659df0c2866f6323b9234eff06..1c30fe23175b5af62ffe808d9428c3361688a4b0 100644 (file)
@@ -29,6 +29,9 @@ import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import org.apache.commons.lang3.time.DurationFormatUtils;
 import org.opendaylight.controller.cluster.DataPersistenceProvider;
+import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
+import org.opendaylight.controller.cluster.NonPersistentDataProvider;
+import org.opendaylight.controller.cluster.PersistentDataProvider;
 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
 import org.opendaylight.controller.cluster.notifications.LeaderStateChanged;
 import org.opendaylight.controller.cluster.notifications.RoleChanged;
@@ -40,7 +43,7 @@ import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
-import org.opendaylight.controller.cluster.raft.behaviors.AbstractRaftActorBehavior;
+import org.opendaylight.controller.cluster.raft.behaviors.DelegatingRaftActorBehavior;
 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
@@ -96,12 +99,6 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
 
     private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis
 
-    private static final Procedure<ApplyJournalEntries> APPLY_JOURNAL_ENTRIES_PERSIST_CALLBACK =
-            new Procedure<ApplyJournalEntries>() {
-                @Override
-                public void apply(ApplyJournalEntries param) throws Exception {
-                }
-            };
     private static final String COMMIT_SNAPSHOT = "commit_snapshot";
 
     protected final Logger LOG = LoggerFactory.getLogger(getClass());
@@ -110,7 +107,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
      * The current state determines the current behavior of a RaftActor
      * A Raft Actor always starts off in the Follower State
      */
-    private RaftActorBehavior currentBehavior;
+    private final DelegatingRaftActorBehavior currentBehavior = new DelegatingRaftActorBehavior();
 
     /**
      * This context should NOT be passed directly to any other actor it is
@@ -118,12 +115,9 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
      */
     private final RaftActorContextImpl context;
 
-    private final Procedure<Void> createSnapshotProcedure = new CreateSnapshotProcedure();
+    private final DelegatingPersistentDataProvider delegatingPersistenceProvider = new DelegatingPersistentDataProvider(null);
 
-    /**
-     * The in-memory journal
-     */
-    private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
+    private final Procedure<Void> createSnapshotProcedure = new CreateSnapshotProcedure();
 
     private Stopwatch recoveryTimer;
 
@@ -139,10 +133,11 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
          Optional<ConfigParams> configParams) {
 
         context = new RaftActorContextImpl(this.getSelf(),
-            this.getContext(), id, new ElectionTermImpl(),
-            -1, -1, replicatedLog, peerAddresses,
-            (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
-            LOG);
+            this.getContext(), id, new ElectionTermImpl(delegatingPersistenceProvider, id, LOG),
+            -1, -1, peerAddresses,
+            (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()), LOG);
+
+        context.setReplicatedLog(ReplicatedLogImpl.newInstance(context, delegatingPersistenceProvider, currentBehavior));
     }
 
     private void initRecoveryTimer() {
@@ -185,7 +180,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
             } else if (message instanceof ApplyJournalEntries) {
                 onRecoveredApplyLogEntries(((ApplyJournalEntries) message).getToIndex());
             } else if (message instanceof DeleteEntries) {
-                replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex());
+                replicatedLog().removeFrom(((DeleteEntries) message).getFromIndex());
             } else if (message instanceof UpdateElectionTerm) {
                 context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(),
                         ((UpdateElectionTerm) message).getVotedFor());
@@ -220,9 +215,9 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         // Create a replicated log with the snapshot information
         // The replicated log can be used later on to retrieve this snapshot
         // when we need to install it on a peer
-        replicatedLog = new ReplicatedLogImpl(snapshot);
 
-        context.setReplicatedLog(replicatedLog);
+        context.setReplicatedLog(ReplicatedLogImpl.newInstance(snapshot, context, delegatingPersistenceProvider,
+                currentBehavior));
         context.setLastApplied(snapshot.getLastAppliedIndex());
         context.setCommitIndex(snapshot.getLastAppliedIndex());
 
@@ -233,8 +228,8 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
 
         timer.stop();
         LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
-                replicatedLog.size(), persistenceId(), timer.toString(),
-                replicatedLog.getSnapshotIndex(), replicatedLog.getSnapshotTerm());
+                replicatedLog().size(), persistenceId(), timer.toString(),
+                replicatedLog().getSnapshotIndex(), replicatedLog().getSnapshotTerm());
     }
 
     private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
@@ -242,7 +237,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
             LOG.debug("{}: Received ReplicatedLogEntry for recovery: {}", persistenceId(), logEntry.getIndex());
         }
 
-        replicatedLog.append(logEntry);
+        replicatedLog().append(logEntry);
     }
 
     private void onRecoveredApplyLogEntries(long toIndex) {
@@ -252,7 +247,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         }
 
         for (long i = context.getLastApplied() + 1; i <= toIndex; i++) {
-            batchRecoveredLogEntry(replicatedLog.get(i));
+            batchRecoveredLogEntry(replicatedLog().get(i));
         }
 
         context.setLastApplied(toIndex);
@@ -298,8 +293,8 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
                 "Persistence Id =  " + persistenceId() +
                 " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
                 "journal-size={}",
-            replicatedLog.lastIndex(), replicatedLog.getSnapshotIndex(),
-            replicatedLog.getSnapshotTerm(), replicatedLog.size());
+            replicatedLog().lastIndex(), replicatedLog().getSnapshotIndex(),
+            replicatedLog().getSnapshotTerm(), replicatedLog().size());
 
         initializeBehavior();
     }
@@ -309,9 +304,9 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
     }
 
     protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
-        reusableBehaviorStateHolder.init(currentBehavior);
-        currentBehavior = newBehavior;
-        handleBehaviorChange(reusableBehaviorStateHolder, currentBehavior);
+        reusableBehaviorStateHolder.init(getCurrentBehavior());
+        setCurrentBehavior(newBehavior);
+        handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
     }
 
     @Override public void handleCommand(Object message) {
@@ -339,7 +334,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
                 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex());
             }
 
-            persistence().persist(applyEntries, APPLY_JOURNAL_ENTRIES_PERSIST_CALLBACK);
+            persistence().persist(applyEntries, NoopProcedure.instance());
 
         } else if(message instanceof ApplySnapshot ) {
             Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
@@ -354,8 +349,8 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
             applySnapshot(snapshot.getState());
 
             //clears the followers log, sets the snapshot index to ensure adjusted-index works
-            replicatedLog = new ReplicatedLogImpl(snapshot);
-            context.setReplicatedLog(replicatedLog);
+            context.setReplicatedLog(ReplicatedLogImpl.newInstance(snapshot, context, delegatingPersistenceProvider,
+                    currentBehavior));
             context.setLastApplied(snapshot.getLastAppliedIndex());
 
         } else if (message instanceof FindLeader) {
@@ -392,11 +387,11 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         } else if (message.equals(COMMIT_SNAPSHOT)) {
             commitSnapshot(-1);
         } else {
-            reusableBehaviorStateHolder.init(currentBehavior);
+            reusableBehaviorStateHolder.init(getCurrentBehavior());
 
-            currentBehavior = currentBehavior.handleMessage(getSender(), message);
+            setCurrentBehavior(currentBehavior.handleMessage(getSender(), message));
 
-            handleBehaviorChange(reusableBehaviorStateHolder, currentBehavior);
+            handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
         }
     }
 
@@ -406,17 +401,17 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         OnDemandRaftState.Builder builder = OnDemandRaftState.builder()
                 .commitIndex(context.getCommitIndex())
                 .currentTerm(context.getTermInformation().getCurrentTerm())
-                .inMemoryJournalDataSize(replicatedLog.dataSize())
-                .inMemoryJournalLogSize(replicatedLog.size())
+                .inMemoryJournalDataSize(replicatedLog().dataSize())
+                .inMemoryJournalLogSize(replicatedLog().size())
                 .isSnapshotCaptureInitiated(context.getSnapshotManager().isCapturing())
                 .lastApplied(context.getLastApplied())
-                .lastIndex(replicatedLog.lastIndex())
-                .lastTerm(replicatedLog.lastTerm())
+                .lastIndex(replicatedLog().lastIndex())
+                .lastTerm(replicatedLog().lastTerm())
                 .leader(getLeaderId())
                 .raftState(currentBehavior.state().toString())
                 .replicatedToAllIndex(currentBehavior.getReplicatedToAllIndex())
-                .snapshotIndex(replicatedLog.getSnapshotIndex())
-                .snapshotTerm(replicatedLog.getSnapshotTerm())
+                .snapshotIndex(replicatedLog().getSnapshotIndex())
+                .snapshotTerm(replicatedLog().getSnapshotTerm())
                 .votedFor(context.getTermInformation().getVotedFor())
                 .peerAddresses(ImmutableMap.copyOf(context.getPeerAddresses()));
 
@@ -426,8 +421,8 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
             builder.lastLogTerm(lastLogEntry.getTerm());
         }
 
-        if(currentBehavior instanceof AbstractLeader) {
-            AbstractLeader leader = (AbstractLeader)currentBehavior;
+        if(getCurrentBehavior() instanceof AbstractLeader) {
+            AbstractLeader leader = (AbstractLeader)getCurrentBehavior();
             Collection<String> followerIds = leader.getFollowerIds();
             List<FollowerInfo> followerInfoList = Lists.newArrayListWithCapacity(followerIds.size());
             for(String id: followerIds) {
@@ -491,39 +486,49 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
 
         final RaftActorContext raftContext = getRaftActorContext();
 
-        replicatedLog
-                .appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
-                    @Override
-                    public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
-                        if(!hasFollowers()){
-                            // Increment the Commit Index and the Last Applied values
-                            raftContext.setCommitIndex(replicatedLogEntry.getIndex());
-                            raftContext.setLastApplied(replicatedLogEntry.getIndex());
+        replicatedLog().appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
+            @Override
+            public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
+                if(!hasFollowers()){
+                    // Increment the Commit Index and the Last Applied values
+                    raftContext.setCommitIndex(replicatedLogEntry.getIndex());
+                    raftContext.setLastApplied(replicatedLogEntry.getIndex());
 
-                            // Apply the state immediately
-                            applyState(clientActor, identifier, data);
+                    // Apply the state immediately
+                    applyState(clientActor, identifier, data);
 
-                            // Send a ApplyJournalEntries message so that we write the fact that we applied
-                            // the state to durable storage
-                            self().tell(new ApplyJournalEntries(replicatedLogEntry.getIndex()), self());
+                    // Send a ApplyJournalEntries message so that we write the fact that we applied
+                    // the state to durable storage
+                    self().tell(new ApplyJournalEntries(replicatedLogEntry.getIndex()), self());
 
-                            context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior);
+                    context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior);
 
-                        } else if (clientActor != null) {
-                            // Send message for replication
-                            currentBehavior.handleMessage(getSelf(),
-                                    new Replicate(clientActor, identifier,
-                                            replicatedLogEntry)
-                            );
-                        }
+                } else if (clientActor != null) {
+                    // Send message for replication
+                    currentBehavior.handleMessage(getSelf(),
+                            new Replicate(clientActor, identifier, replicatedLogEntry));
+                }
+            }
+        });
+    }
 
-                    }
-                });    }
+    private ReplicatedLog replicatedLog() {
+        return context.getReplicatedLog();
+    }
 
     protected String getId() {
         return context.getId();
     }
 
+    @VisibleForTesting
+    void setCurrentBehavior(RaftActorBehavior behavior) {
+        currentBehavior.setDelegate(behavior);
+    }
+
+    protected RaftActorBehavior getCurrentBehavior() {
+        return currentBehavior.getDelegate();
+    }
+
     /**
      * Derived actors can call the isLeader method to check if the current
      * RaftActor is the Leader or not
@@ -564,7 +569,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
     }
 
     protected ReplicatedLogEntry getLastLogEntry() {
-        return replicatedLog.last();
+        return replicatedLog().last();
     }
 
     protected Long getCurrentTerm(){
@@ -587,6 +592,41 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         context.setConfigParams(configParams);
     }
 
+    public final DataPersistenceProvider persistence() {
+        return delegatingPersistenceProvider.getDelegate();
+    }
+
+    public void setPersistence(DataPersistenceProvider provider) {
+        delegatingPersistenceProvider.setDelegate(provider);
+    }
+
+    protected void setPersistence(boolean persistent) {
+        if(persistent) {
+            setPersistence(new PersistentDataProvider(this));
+        } else {
+            setPersistence(new NonPersistentDataProvider() {
+                /**
+                 * The way snapshotting works is,
+                 * <ol>
+                 * <li> RaftActor calls createSnapshot on the Shard
+                 * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
+                 * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save
+                 * the snapshot. The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the
+                 * RaftActor gets SaveSnapshot success it commits the snapshot to the in-memory journal. This
+                 * commitSnapshot is mimicking what is done in SaveSnapshotSuccess.
+                 * </ol>
+                 */
+                @Override
+                public void saveSnapshot(Object o) {
+                    // Make saving Snapshot successful
+                    // Committing the snapshot here would end up calling commit in the creating state which would
+                    // be a state violation. That's why now we send a message to commit the snapshot.
+                    self().tell(COMMIT_SNAPSHOT, self());
+                }
+            });
+        }
+    }
+
     /**
      * setPeerAddress sets the address of a known peer at a later time.
      * <p>
@@ -688,8 +728,6 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
      */
     protected abstract void onStateChanged();
 
-    protected abstract DataPersistenceProvider persistence();
-
     /**
      * Notifier Actor for this RaftActor to notify when a role change happens
      * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
@@ -718,125 +756,11 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
     private void handleCaptureSnapshotReply(byte[] snapshotBytes) {
         LOG.debug("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length);
 
-        context.getSnapshotManager().persist(persistence(), snapshotBytes, currentBehavior, getTotalMemory());
-    }
-
-    protected long getTotalMemory() {
-        return Runtime.getRuntime().totalMemory();
+        context.getSnapshotManager().persist(persistence(), snapshotBytes, currentBehavior, context.getTotalMemory());
     }
 
     protected boolean hasFollowers(){
-        return getRaftActorContext().getPeerAddresses().keySet().size() > 0;
-    }
-
-    private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
-        private static final int DATA_SIZE_DIVIDER = 5;
-        private long dataSizeSinceLastSnapshot = 0L;
-
-
-        public ReplicatedLogImpl(Snapshot snapshot) {
-            super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
-                snapshot.getUnAppliedEntries());
-        }
-
-        public ReplicatedLogImpl() {
-            super();
-        }
-
-        @Override public void removeFromAndPersist(long logEntryIndex) {
-            int adjustedIndex = adjustedIndex(logEntryIndex);
-
-            if (adjustedIndex < 0) {
-                return;
-            }
-
-            // FIXME: Maybe this should be done after the command is saved
-            journal.subList(adjustedIndex , journal.size()).clear();
-
-            persistence().persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>() {
-
-                @Override
-                public void apply(DeleteEntries param)
-                        throws Exception {
-                    //FIXME : Doing nothing for now
-                    dataSize = 0;
-                    for (ReplicatedLogEntry entry : journal) {
-                        dataSize += entry.size();
-                    }
-                }
-            });
-        }
-
-        @Override public void appendAndPersist(
-            final ReplicatedLogEntry replicatedLogEntry) {
-            appendAndPersist(replicatedLogEntry, null);
-        }
-
-        public void appendAndPersist(
-            final ReplicatedLogEntry replicatedLogEntry,
-            final Procedure<ReplicatedLogEntry> callback)  {
-
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("{}: Append log entry and persist {} ", persistenceId(), replicatedLogEntry);
-            }
-
-            // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
-            journal.add(replicatedLogEntry);
-
-            // When persisting events with persist it is guaranteed that the
-            // persistent actor will not receive further commands between the
-            // persist call and the execution(s) of the associated event
-            // handler. This also holds for multiple persist calls in context
-            // of a single command.
-            persistence().persist(replicatedLogEntry,
-                new Procedure<ReplicatedLogEntry>() {
-                    @Override
-                    public void apply(ReplicatedLogEntry evt) throws Exception {
-                        int logEntrySize = replicatedLogEntry.size();
-
-                        dataSize += logEntrySize;
-                        long dataSizeForCheck = dataSize;
-
-                        dataSizeSinceLastSnapshot += logEntrySize;
-
-                        if (!hasFollowers()) {
-                            // When we do not have followers we do not maintain an in-memory log
-                            // due to this the journalSize will never become anything close to the
-                            // snapshot batch count. In fact will mostly be 1.
-                            // Similarly since the journal's dataSize depends on the entries in the
-                            // journal the journal's dataSize will never reach a value close to the
-                            // memory threshold.
-                            // By maintaining the dataSize outside the journal we are tracking essentially
-                            // what we have written to the disk however since we no longer are in
-                            // need of doing a snapshot just for the sake of freeing up memory we adjust
-                            // the real size of data by the DATA_SIZE_DIVIDER so that we do not snapshot as often
-                            // as if we were maintaining a real snapshot
-                            dataSizeForCheck = dataSizeSinceLastSnapshot / DATA_SIZE_DIVIDER;
-                        }
-                        long journalSize = replicatedLogEntry.getIndex() + 1;
-                        long dataThreshold = getTotalMemory() *
-                                context.getConfigParams().getSnapshotDataThresholdPercentage() / 100;
-
-                        if ((journalSize % context.getConfigParams().getSnapshotBatchCount() == 0
-                                || dataSizeForCheck > dataThreshold)) {
-
-                            boolean started = context.getSnapshotManager().capture(replicatedLogEntry,
-                                    currentBehavior.getReplicatedToAllIndex());
-
-                            if(started){
-                                dataSizeSinceLastSnapshot = 0;
-                            }
-
-                        }
-
-                        if (callback != null){
-                            callback.apply(replicatedLogEntry);
-                        }
-                    }
-                }
-            );
-        }
-
+        return getRaftActorContext().hasFollowers();
     }
 
     static class DeleteEntries implements Serializable {
@@ -852,46 +776,6 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         }
     }
 
-
-    private class ElectionTermImpl implements ElectionTerm {
-        /**
-         * Identifier of the actor whose election term information this is
-         */
-        private long currentTerm = 0;
-        private String votedFor = null;
-
-        @Override
-        public long getCurrentTerm() {
-            return currentTerm;
-        }
-
-        @Override
-        public String getVotedFor() {
-            return votedFor;
-        }
-
-        @Override public void update(long currentTerm, String votedFor) {
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("{}: Set currentTerm={}, votedFor={}", persistenceId(), currentTerm, votedFor);
-            }
-            this.currentTerm = currentTerm;
-            this.votedFor = votedFor;
-        }
-
-        @Override
-        public void updateAndPersist(long currentTerm, String votedFor){
-            update(currentTerm, votedFor);
-            // FIXME : Maybe first persist then update the state
-            persistence().persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
-
-                @Override public void apply(UpdateElectionTerm param)
-                    throws Exception {
-
-                }
-            });
-        }
-    }
-
     static class UpdateElectionTerm implements Serializable {
         private static final long serialVersionUID = 1L;
         private final long currentTerm;
@@ -911,34 +795,6 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         }
     }
 
-    protected class NonPersistentRaftDataProvider extends NonPersistentDataProvider {
-
-        public NonPersistentRaftDataProvider(){
-
-        }
-
-        /**
-         * The way snapshotting works is,
-         * <ol>
-         * <li> RaftActor calls createSnapshot on the Shard
-         * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
-         * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save the snapshot.
-         * The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the RaftActor gets SaveSnapshot
-         * success it commits the snapshot to the in-memory journal. This commitSnapshot is mimicking what is done
-         * in SaveSnapshotSuccess.
-         * </ol>
-         * @param o
-         */
-        @Override
-        public void saveSnapshot(Object o) {
-            // Make saving Snapshot successful
-            // Committing the snapshot here would end up calling commit in the creating state which would
-            // be a state violation. That's why now we send a message to commit the snapshot.
-            self().tell(COMMIT_SNAPSHOT, self());
-        }
-    }
-
-
     private class CreateSnapshotProcedure implements Procedure<Void> {
 
         @Override
@@ -947,15 +803,6 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         }
     }
 
-    @VisibleForTesting
-    void setCurrentBehavior(AbstractRaftActorBehavior behavior) {
-        currentBehavior = behavior;
-    }
-
-    protected RaftActorBehavior getCurrentBehavior() {
-        return currentBehavior;
-    }
-
     private static class BehaviorStateHolder {
         private RaftActorBehavior behavior;
         private String leaderId;