Merge "BUG 1623 - Clustering : Parsing Error thrown on startup"
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActor.java
index 0ff2341c708431d8da540d625638c9d70bb3dd11..8135d837d3ad86563ad0246022941bfc6134b59d 100644 (file)
@@ -19,6 +19,14 @@ import akka.persistence.SaveSnapshotSuccess;
 import akka.persistence.SnapshotOffer;
 import akka.persistence.SnapshotSelectionCriteria;
 import akka.persistence.UntypedPersistentActor;
+import com.google.common.base.Optional;
+import com.google.protobuf.ByteString;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
+import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
+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.base.messages.SendHeartBeat;
 import org.opendaylight.controller.cluster.raft.behaviors.Candidate;
 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
 import org.opendaylight.controller.cluster.raft.behaviors.Leader;
@@ -26,14 +34,12 @@ import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
 import org.opendaylight.controller.cluster.raft.client.messages.AddRaftPeer;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
-import org.opendaylight.controller.cluster.raft.internal.messages.ApplySnapshot;
 import org.opendaylight.controller.cluster.raft.client.messages.RemoveRaftPeer;
-import org.opendaylight.controller.cluster.raft.internal.messages.ApplyState;
-import org.opendaylight.controller.cluster.raft.internal.messages.Replicate;
+import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
+import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
+import org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages;
 
 import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Map;
 
 /**
@@ -97,16 +103,27 @@ public abstract class RaftActor extends UntypedPersistentActor {
      */
     private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
 
+    private CaptureSnapshot captureSnapshot = null;
+
+    private volatile boolean hasSnapshotCaptureInitiated = false;
 
     public RaftActor(String id, Map<String, String> peerAddresses) {
+        this(id, peerAddresses, Optional.<ConfigParams>absent());
+    }
+
+    public RaftActor(String id, Map<String, String> peerAddresses,
+         Optional<ConfigParams> configParams) {
+
         context = new RaftActorContextImpl(this.getSelf(),
-            this.getContext(),
-            id, new ElectionTermImpl(),
-            -1, -1, replicatedLog, peerAddresses, LOG);
+            this.getContext(), id, new ElectionTermImpl(),
+            -1, -1, replicatedLog, peerAddresses,
+            (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
+            LOG);
     }
 
     @Override public void onReceiveRecover(Object message) {
         if (message instanceof SnapshotOffer) {
+            LOG.debug("SnapshotOffer called..");
             SnapshotOffer offer = (SnapshotOffer) message;
             Snapshot snapshot = (Snapshot) offer.snapshot();
 
@@ -115,8 +132,16 @@ public abstract class RaftActor extends UntypedPersistentActor {
             // when we need to install it on a peer
             replicatedLog = new ReplicatedLogImpl(snapshot);
 
+            context.setReplicatedLog(replicatedLog);
+            context.setLastApplied(snapshot.getLastAppliedIndex());
+
+            LOG.debug("Applied snapshot to replicatedLog. " +
+                "snapshotIndex={}, snapshotTerm={}, journal-size={}",
+                replicatedLog.snapshotIndex, replicatedLog.snapshotTerm,
+                replicatedLog.size());
+
             // Apply the snapshot to the actors state
-            applySnapshot(snapshot.getState());
+            applySnapshot(ByteString.copyFrom(snapshot.getState()));
 
         } else if (message instanceof ReplicatedLogEntry) {
             replicatedLog.append((ReplicatedLogEntry) message);
@@ -126,8 +151,13 @@ public abstract class RaftActor extends UntypedPersistentActor {
             context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(), ((UpdateElectionTerm) message).getVotedFor());
         } else if (message instanceof RecoveryCompleted) {
             LOG.debug(
-                "Last index in log : " + replicatedLog.lastIndex());
+                "RecoveryCompleted - Switching actor to Follower - " +
+                    "Last index in log:{}, snapshotIndex={}, snapshotTerm={}, " +
+                    "journal-size={}",
+                replicatedLog.lastIndex(), replicatedLog.snapshotIndex,
+                replicatedLog.snapshotTerm, replicatedLog.size());
             currentBehavior = switchBehavior(RaftState.Follower);
+            onStateChanged();
         }
     }
 
@@ -141,24 +171,48 @@ public abstract class RaftActor extends UntypedPersistentActor {
 
             applyState(applyState.getClientActor(), applyState.getIdentifier(),
                 applyState.getReplicatedLogEntry().getData());
+
         } else if(message instanceof ApplySnapshot ) {
-            applySnapshot(((ApplySnapshot) message).getSnapshot());
+            Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
+
+            LOG.debug("ApplySnapshot called on Follower Actor " +
+                "snapshotIndex:{}, snapshotTerm:{}", snapshot.getLastAppliedIndex(),
+                snapshot.getLastAppliedTerm());
+            applySnapshot(ByteString.copyFrom(snapshot.getState()));
+
+            //clears the followers log, sets the snapshot index to ensure adjusted-index works
+            replicatedLog = new ReplicatedLogImpl(snapshot);
+            context.setReplicatedLog(replicatedLog);
+            context.setLastApplied(snapshot.getLastAppliedIndex());
+
         } else if (message instanceof FindLeader) {
             getSender().tell(
-                new FindLeaderReply(
-                    context.getPeerAddress(currentBehavior.getLeaderId())),
+                new FindLeaderReply(getLeaderAddress()),
                 getSelf()
             );
+
         } else if (message instanceof SaveSnapshotSuccess) {
             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
+            LOG.info("SaveSnapshotSuccess received for snapshot");
+
+            context.getReplicatedLog().snapshotCommit();
 
             // TODO: Not sure if we want to be this aggressive with trimming stuff
             trimPersistentData(success.metadata().sequenceNr());
 
         } else if (message instanceof SaveSnapshotFailure) {
+            SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
+
+            LOG.info("saveSnapshotFailure.metadata():{}", saveSnapshotFailure.metadata().toString());
+            LOG.error(saveSnapshotFailure.cause(), "SaveSnapshotFailure received for snapshot Cause:");
 
-            // TODO: Handle failure in saving the snapshot
-            // Maybe do retries on failure
+            context.getReplicatedLog().snapshotRollback();
+
+            LOG.info("Replicated Log rollbacked. Snapshot will be attempted in the next cycle." +
+                "snapshotIndex:{}, snapshotTerm:{}, log-size:{}",
+                context.getReplicatedLog().getSnapshotIndex(),
+                context.getReplicatedLog().getSnapshotTerm(),
+                context.getReplicatedLog().size());
 
         } else if (message instanceof AddRaftPeer){
 
@@ -174,14 +228,45 @@ public abstract class RaftActor extends UntypedPersistentActor {
             RemoveRaftPeer rrp = (RemoveRaftPeer)message;
             context.removePeer(rrp.getName());
 
+        } else if (message instanceof CaptureSnapshot) {
+            LOG.debug("CaptureSnapshot received by actor");
+            CaptureSnapshot cs = (CaptureSnapshot)message;
+            captureSnapshot = cs;
+            createSnapshot();
+
+        } else if (message instanceof CaptureSnapshotReply){
+            LOG.debug("CaptureSnapshotReply received by actor");
+            CaptureSnapshotReply csr = (CaptureSnapshotReply) message;
+
+            ByteString stateInBytes = csr.getSnapshot();
+            LOG.debug("CaptureSnapshotReply stateInBytes size:{}", stateInBytes.size());
+            handleCaptureSnapshotReply(stateInBytes);
+
         } else {
+            if (!(message instanceof AppendEntriesMessages.AppendEntries)
+                && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) {
+                LOG.debug("onReceiveCommand: message:" + message.getClass());
+            }
 
             RaftState state =
                 currentBehavior.handleMessage(getSender(), message);
+            RaftActorBehavior oldBehavior = currentBehavior;
             currentBehavior = switchBehavior(state);
+            if(oldBehavior != currentBehavior){
+                onStateChanged();
+            }
         }
     }
 
+    public java.util.Set<String> getPeers() {
+        return context.getPeerAddresses().keySet();
+    }
+
+    protected String getReplicatedLogState() {
+        return "snapshotIndex=" + context.getReplicatedLog().getSnapshotIndex()
+            + ", snapshotTerm=" + context.getReplicatedLog().getSnapshotTerm()
+            + ", im-mem journal size=" + context.getReplicatedLog().size();
+    }
 
 
     /**
@@ -193,7 +278,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
      * @param data
      */
     protected void persistData(ActorRef clientActor, String identifier,
-        Object data) {
+        Payload data) {
 
         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
             context.getReplicatedLog().lastIndex() + 1,
@@ -227,20 +312,61 @@ public abstract class RaftActor extends UntypedPersistentActor {
      * @return A reference to the leader if known, null otherwise
      */
     protected ActorSelection getLeader(){
-        String leaderId = currentBehavior.getLeaderId();
-        if (leaderId == null) {
+        String leaderAddress = getLeaderAddress();
+
+        if(leaderAddress == null){
             return null;
         }
-        String peerAddress = context.getPeerAddress(leaderId);
-        LOG.debug("getLeader leaderId = " + leaderId + " peerAddress = "
-            + peerAddress);
-        return context.actorSelection(peerAddress);
+
+        return context.actorSelection(leaderAddress);
+    }
+
+    /**
+     *
+     * @return the current leader's id
+     */
+    protected String getLeaderId(){
+        return currentBehavior.getLeaderId();
     }
 
     protected RaftState getRaftState() {
         return currentBehavior.state();
     }
 
+    protected ReplicatedLogEntry getLastLogEntry() {
+        return replicatedLog.last();
+    }
+
+    protected Long getCurrentTerm(){
+        return context.getTermInformation().getCurrentTerm();
+    }
+
+    protected Long getCommitIndex(){
+        return context.getCommitIndex();
+    }
+
+    protected Long getLastApplied(){
+        return context.getLastApplied();
+    }
+
+    /**
+     * setPeerAddress sets the address of a known peer at a later time.
+     * <p>
+     * This is to account for situations where a we know that a peer
+     * exists but we do not know an address up-front. This may also be used in
+     * situations where a known peer starts off in a different location and we
+     * need to change it's address
+     * <p>
+     * Note that if the peerId does not match the list of peers passed to
+     * this actor during construction an IllegalStateException will be thrown.
+     *
+     * @param peerId
+     * @param peerAddress
+     */
+    protected void setPeerAddress(String peerId, String peerAddress){
+        context.setPeerAddress(peerId, peerAddress);
+    }
+
 
 
     /**
@@ -272,7 +398,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
      *
      * @return The current state of the actor
      */
-    protected abstract Object createSnapshot();
+    protected abstract void createSnapshot();
 
     /**
      * This method will be called by the RaftActor during recovery to
@@ -284,7 +410,14 @@ public abstract class RaftActor extends UntypedPersistentActor {
      *
      * @param snapshot A snapshot of the state of the actor
      */
-    protected abstract void applySnapshot(Object snapshot);
+    protected abstract void applySnapshot(ByteString snapshot);
+
+    /**
+     * This method will be called by the RaftActor when the state of the
+     * RaftActor changes. The derived actor can then use methods like
+     * isLeader or getLeader to do something useful
+     */
+    protected abstract void onStateChanged();
 
     private RaftActorBehavior switchBehavior(RaftState state) {
         if (currentBehavior != null) {
@@ -312,89 +445,83 @@ public abstract class RaftActor extends UntypedPersistentActor {
         } else {
             behavior = new Leader(context);
         }
+
+
+
         return behavior;
     }
 
     private void trimPersistentData(long sequenceNumber) {
-        // Trim snapshots
+        // Trim akka snapshots
         // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied
         // For now guessing that it is ANDed.
         deleteSnapshots(new SnapshotSelectionCriteria(
-            sequenceNumber - 100000, 43200000));
+            sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
 
-        // Trim journal
+        // Trim akka journal
         deleteMessages(sequenceNumber);
     }
 
-
-    private class ReplicatedLogImpl implements ReplicatedLog {
-        private final List<ReplicatedLogEntry> journal;
-        private final Object snapshot;
-        private long snapshotIndex = -1;
-        private long snapshotTerm = -1;
-
-        public ReplicatedLogImpl(Snapshot snapshot) {
-            this.snapshot = snapshot.getState();
-            this.snapshotIndex = snapshot.getLastAppliedIndex();
-            this.snapshotTerm = snapshot.getLastAppliedTerm();
-
-            this.journal = new ArrayList<>(snapshot.getUnAppliedEntries());
+    private String getLeaderAddress(){
+        if(isLeader()){
+            return getSelf().path().toString();
         }
-
-        public ReplicatedLogImpl() {
-            this.snapshot = null;
-            this.journal = new ArrayList<>();
+        String leaderId = currentBehavior.getLeaderId();
+        if (leaderId == null) {
+            return null;
         }
+        String peerAddress = context.getPeerAddress(leaderId);
+        LOG.debug("getLeaderAddress leaderId = " + leaderId + " peerAddress = "
+            + peerAddress);
 
-        @Override public ReplicatedLogEntry get(long index) {
-            int adjustedIndex = adjustedIndex(index);
+        return peerAddress;
+    }
 
-            if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
-                return null;
-            }
+    private void handleCaptureSnapshotReply(ByteString stateInBytes) {
+        // create a snapshot object from the state provided and save it
+        // when snapshot is saved async, SaveSnapshotSuccess is raised.
 
-            return journal.get(adjustedIndex);
-        }
+        Snapshot sn = Snapshot.create(stateInBytes.toByteArray(),
+            context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1),
+            captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
+            captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
 
-        @Override public ReplicatedLogEntry last() {
-            if (journal.size() == 0) {
-                return null;
-            }
-            return get(journal.size() - 1);
-        }
+        saveSnapshot(sn);
 
-        @Override public long lastIndex() {
-            if (journal.size() == 0) {
-                return -1;
-            }
+        LOG.info("Persisting of snapshot done:{}", sn.getLogMessage());
 
-            return last().getIndex();
-        }
+        //be greedy and remove entries from in-mem journal which are in the snapshot
+        // and update snapshotIndex and snapshotTerm without waiting for the success,
 
-        @Override public long lastTerm() {
-            if (journal.size() == 0) {
-                return -1;
-            }
+        context.getReplicatedLog().snapshotPreCommit(stateInBytes,
+            captureSnapshot.getLastAppliedIndex(),
+            captureSnapshot.getLastAppliedTerm());
 
-            return last().getTerm();
-        }
+        LOG.info("Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " +
+            "and term:{}", captureSnapshot.getLastAppliedIndex(),
+            captureSnapshot.getLastAppliedTerm());
 
+        captureSnapshot = null;
+        hasSnapshotCaptureInitiated = false;
+    }
 
-        @Override public void removeFrom(long index) {
-            int adjustedIndex = adjustedIndex(index);
 
-            if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
-                return;
-            }
+    private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
 
-            journal.subList(adjustedIndex , journal.size()).clear();
+        public ReplicatedLogImpl(Snapshot snapshot) {
+            super(ByteString.copyFrom(snapshot.getState()),
+                snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
+                snapshot.getUnAppliedEntries());
         }
 
+        public ReplicatedLogImpl() {
+            super();
+        }
 
-        @Override public void removeFromAndPersist(long index) {
-            int adjustedIndex = adjustedIndex(index);
+        @Override public void removeFromAndPersist(long logEntryIndex) {
+            int adjustedIndex = adjustedIndex(logEntryIndex);
 
-            if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
+            if (adjustedIndex < 0) {
                 return;
             }
 
@@ -408,29 +535,6 @@ public abstract class RaftActor extends UntypedPersistentActor {
                     //FIXME : Doing nothing for now
                 }
             });
-
-
-        }
-
-        @Override public void append(
-            final ReplicatedLogEntry replicatedLogEntry) {
-            journal.add(replicatedLogEntry);
-        }
-
-        @Override public List<ReplicatedLogEntry> getFrom(long index) {
-            int adjustedIndex = adjustedIndex(index);
-
-            List<ReplicatedLogEntry> entries = new ArrayList<>(100);
-            if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
-                return entries;
-            }
-
-
-            for (int i = adjustedIndex;
-                 i < journal.size(); i++) {
-                entries.add(journal.get(i));
-            }
-            return entries;
         }
 
         @Override public void appendAndPersist(
@@ -442,7 +546,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
             final String identifier,
             final ReplicatedLogEntry replicatedLogEntry) {
             context.getLogger().debug(
-                "Append log entry and persist " + replicatedLogEntry);
+                "Append log entry and persist {} ", 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);
 
@@ -454,21 +558,30 @@ public abstract class RaftActor extends UntypedPersistentActor {
             persist(replicatedLogEntry,
                 new Procedure<ReplicatedLogEntry>() {
                     public void apply(ReplicatedLogEntry evt) throws Exception {
-                        // FIXME : Tentatively create a snapshot every hundred thousand entries. To be tuned.
-                        if (size() > 100000) {
-                            ReplicatedLogEntry lastAppliedEntry =
-                                get(context.getLastApplied());
+                        // when a snaphsot is being taken, captureSnapshot != null
+                        if (hasSnapshotCaptureInitiated == false &&
+                            journal.size() % context.getConfigParams().getSnapshotBatchCount() == 0) {
+
+                            LOG.info("Initiating Snapshot Capture..");
                             long lastAppliedIndex = -1;
                             long lastAppliedTerm = -1;
+
+                            ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied());
                             if (lastAppliedEntry != null) {
                                 lastAppliedIndex = lastAppliedEntry.getIndex();
                                 lastAppliedTerm = lastAppliedEntry.getTerm();
                             }
 
-                            saveSnapshot(Snapshot.create(createSnapshot(),
-                                getFrom(context.getLastApplied() + 1),
-                                lastIndex(), lastTerm(), lastAppliedIndex,
-                                lastAppliedTerm));
+                            LOG.debug("Snapshot Capture logSize: {}", journal.size());
+                            LOG.debug("Snapshot Capture lastApplied:{} ", context.getLastApplied());
+                            LOG.debug("Snapshot Capture lastAppliedIndex:{}", lastAppliedIndex);
+                            LOG.debug("Snapshot Capture lastAppliedTerm:{}", lastAppliedTerm);
+
+                            // send a CaptureSnapshot to self to make the expensive operation async.
+                            getSelf().tell(new CaptureSnapshot(
+                                lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm),
+                                null);
+                            hasSnapshotCaptureInitiated = true;
                         }
                         // Send message for replication
                         if (clientActor != null) {
@@ -482,76 +595,6 @@ public abstract class RaftActor extends UntypedPersistentActor {
             );
         }
 
-        @Override public long size() {
-            return journal.size() + snapshotIndex + 1;
-        }
-
-        @Override public boolean isPresent(long index) {
-            int adjustedIndex = adjustedIndex(index);
-
-            if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
-                return false;
-            }
-            return true;
-        }
-
-        @Override public boolean isInSnapshot(long index) {
-            return index <= snapshotIndex;
-        }
-
-        @Override public Object getSnapshot() {
-            return snapshot;
-        }
-
-        @Override public long getSnapshotIndex() {
-            return snapshotIndex;
-        }
-
-        @Override public long getSnapshotTerm() {
-            return snapshotTerm;
-        }
-
-        private int adjustedIndex(long index) {
-            if(snapshotIndex < 0){
-                return (int) index;
-            }
-            return (int) (index - snapshotIndex);
-        }
-    }
-
-
-    private static class ReplicatedLogImplEntry implements ReplicatedLogEntry,
-        Serializable {
-
-        private final long index;
-        private final long term;
-        private final Object payload;
-
-        public ReplicatedLogImplEntry(long index, long term, Object payload) {
-
-            this.index = index;
-            this.term = term;
-            this.payload = payload;
-        }
-
-        @Override public Object getData() {
-            return payload;
-        }
-
-        @Override public long getTerm() {
-            return term;
-        }
-
-        @Override public long getIndex() {
-            return index;
-        }
-
-        @Override public String toString() {
-            return "Entry{" +
-                "index=" + index +
-                ", term=" + term +
-                '}';
-        }
     }
 
     private static class DeleteEntries implements Serializable {
@@ -568,54 +611,6 @@ public abstract class RaftActor extends UntypedPersistentActor {
     }
 
 
-    private static class Snapshot implements Serializable {
-        private final Object state;
-        private final List<ReplicatedLogEntry> unAppliedEntries;
-        private final long lastIndex;
-        private final long lastTerm;
-        private final long lastAppliedIndex;
-        private final long lastAppliedTerm;
-
-        private Snapshot(Object state,
-            List<ReplicatedLogEntry> unAppliedEntries, long lastIndex,
-            long lastTerm, long lastAppliedIndex, long lastAppliedTerm) {
-            this.state = state;
-            this.unAppliedEntries = unAppliedEntries;
-            this.lastIndex = lastIndex;
-            this.lastTerm = lastTerm;
-            this.lastAppliedIndex = lastAppliedIndex;
-            this.lastAppliedTerm = lastAppliedTerm;
-        }
-
-
-        public static Snapshot create(Object state,
-            List<ReplicatedLogEntry> entries, long lastIndex, long lastTerm,
-            long lastAppliedIndex, long lastAppliedTerm) {
-            return new Snapshot(state, entries, lastIndex, lastTerm,
-                lastAppliedIndex, lastAppliedTerm);
-        }
-
-        public Object getState() {
-            return state;
-        }
-
-        public List<ReplicatedLogEntry> getUnAppliedEntries() {
-            return unAppliedEntries;
-        }
-
-        public long getLastTerm() {
-            return lastTerm;
-        }
-
-        public long getLastAppliedIndex() {
-            return lastAppliedIndex;
-        }
-
-        public long getLastAppliedTerm() {
-            return lastAppliedTerm;
-        }
-    }
-
     private class ElectionTermImpl implements ElectionTerm {
         /**
          * Identifier of the actor whose election term information this is
@@ -632,7 +627,7 @@ public abstract class RaftActor extends UntypedPersistentActor {
         }
 
         @Override public void update(long currentTerm, String votedFor) {
-            LOG.info("Set currentTerm={}, votedFor={}", currentTerm, votedFor);
+            LOG.debug("Set currentTerm={}, votedFor={}", currentTerm, votedFor);
 
             this.currentTerm = currentTerm;
             this.votedFor = votedFor;