X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;ds=sidebyside;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2FRaftActor.java;h=66a46ef3bde0ca8a8d1fa8fe1056ccb463a594d6;hb=f14033146e051aca1b51c791373f6e867af340b0;hp=27e8f34d327dd5ebe9e1b1ca8a9d8e6ea1b6ceba;hpb=c675288c3ebd727fc72013e899183995c3fa9be7;p=controller.git diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java index 27e8f34d32..66a46ef3bd 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java @@ -19,22 +19,26 @@ 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.common.base.Stopwatch; +import com.google.protobuf.ByteString; +import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries; 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 com.google.common.base.Optional; -import org.opendaylight.controller.cluster.raft.behaviors.Candidate; +import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat; import org.opendaylight.controller.cluster.raft.behaviors.Follower; -import org.opendaylight.controller.cluster.raft.behaviors.Leader; 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.client.messages.RemoveRaftPeer; +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.List; import java.util.Map; /** @@ -91,13 +95,20 @@ public abstract class RaftActor extends UntypedPersistentActor { * This context should NOT be passed directly to any other actor it is * only to be consumed by the RaftActorBehaviors */ - private RaftActorContext context; + private final RaftActorContext context; /** * The in-memory journal */ private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl(); + private CaptureSnapshot captureSnapshot = null; + + private volatile boolean hasSnapshotCaptureInitiated = false; + + private Stopwatch recoveryTimer; + + private int currentRecoveryBatchCount; public RaftActor(String id, Map peerAddresses) { this(id, peerAddresses, Optional.absent()); @@ -113,58 +124,175 @@ public abstract class RaftActor extends UntypedPersistentActor { LOG); } - @Override public void onReceiveRecover(Object message) { - if (message instanceof SnapshotOffer) { - LOG.debug("SnapshotOffer called.."); - SnapshotOffer offer = (SnapshotOffer) message; - Snapshot snapshot = (Snapshot) offer.snapshot(); - - // 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); - - LOG.debug("Applied snapshot to replicatedLog. " + - "snapshotIndex={}, snapshotTerm={}, journal-size={}", - replicatedLog.snapshotIndex, replicatedLog.snapshotTerm, - replicatedLog.size()); + private void initRecoveryTimer() { + if(recoveryTimer == null) { + recoveryTimer = new Stopwatch(); + recoveryTimer.start(); + } + } - // Apply the snapshot to the actors state - applySnapshot(snapshot.getState()); + @Override + public void preStart() throws Exception { + LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(), + context.getConfigParams().getJournalRecoveryLogBatchSize()); + super.preStart(); + } + @Override + public void onReceiveRecover(Object message) { + if (message instanceof SnapshotOffer) { + onRecoveredSnapshot((SnapshotOffer)message); } else if (message instanceof ReplicatedLogEntry) { - replicatedLog.append((ReplicatedLogEntry) message); + onRecoveredJournalLogEntry((ReplicatedLogEntry)message); + } else if (message instanceof ApplyLogEntries) { + onRecoveredApplyLogEntries((ApplyLogEntries)message); } else if (message instanceof DeleteEntries) { replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex()); } else if (message instanceof UpdateElectionTerm) { - context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(), ((UpdateElectionTerm) message).getVotedFor()); + context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(), + ((UpdateElectionTerm) message).getVotedFor()); } else if (message instanceof RecoveryCompleted) { - LOG.debug( - "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(); + onRecoveryCompletedMessage(); + } + } + + private void onRecoveredSnapshot(SnapshotOffer offer) { + if(LOG.isDebugEnabled()) { + LOG.debug("SnapshotOffer called.."); + } + + initRecoveryTimer(); + + Snapshot snapshot = (Snapshot) offer.snapshot(); + + // 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.setLastApplied(snapshot.getLastAppliedIndex()); + context.setCommitIndex(snapshot.getLastAppliedIndex()); + + Stopwatch timer = new Stopwatch(); + timer.start(); + + // Apply the snapshot to the actors state + applyRecoverySnapshot(ByteString.copyFrom(snapshot.getState())); + + timer.stop(); + LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" + + replicatedLog.size(), persistenceId(), timer.toString(), + replicatedLog.snapshotIndex, replicatedLog.snapshotTerm); + } + + private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) { + if(LOG.isDebugEnabled()) { + LOG.debug("Received ReplicatedLogEntry for recovery: {}", logEntry.getIndex()); + } + + replicatedLog.append(logEntry); + } + + private void onRecoveredApplyLogEntries(ApplyLogEntries ale) { + if(LOG.isDebugEnabled()) { + LOG.debug("Received ApplyLogEntries for recovery, applying to state: {} to {}", + context.getLastApplied() + 1, ale.getToIndex()); + } + + for (long i = context.getLastApplied() + 1; i <= ale.getToIndex(); i++) { + batchRecoveredLogEntry(replicatedLog.get(i)); } + + context.setLastApplied(ale.getToIndex()); + context.setCommitIndex(ale.getToIndex()); + } + + private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) { + initRecoveryTimer(); + + int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize(); + if(currentRecoveryBatchCount == 0) { + startLogRecoveryBatch(batchSize); + } + + appendRecoveredLogEntry(logEntry.getData()); + + if(++currentRecoveryBatchCount >= batchSize) { + endCurrentLogRecoveryBatch(); + } + } + + private void endCurrentLogRecoveryBatch() { + applyCurrentLogRecoveryBatch(); + currentRecoveryBatchCount = 0; + } + + private void onRecoveryCompletedMessage() { + if(currentRecoveryBatchCount > 0) { + endCurrentLogRecoveryBatch(); + } + + onRecoveryComplete(); + + String recoveryTime = ""; + if(recoveryTimer != null) { + recoveryTimer.stop(); + recoveryTime = " in " + recoveryTimer.toString(); + recoveryTimer = null; + } + + LOG.info( + "Recovery completed" + recoveryTime + " - Switching actor to Follower - " + + "Persistence Id = " + persistenceId() + + " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " + + "journal-size={}", + replicatedLog.lastIndex(), replicatedLog.snapshotIndex, + replicatedLog.snapshotTerm, replicatedLog.size()); + + currentBehavior = new Follower(context); + onStateChanged(); } @Override public void onReceiveCommand(Object message) { if (message instanceof ApplyState){ ApplyState applyState = (ApplyState) message; - LOG.debug("Applying state for log index {} data {}", - applyState.getReplicatedLogEntry().getIndex(), - applyState.getReplicatedLogEntry().getData()); + if(LOG.isDebugEnabled()) { + LOG.debug("Applying state for log index {} data {}", + applyState.getReplicatedLogEntry().getIndex(), + applyState.getReplicatedLogEntry().getData()); + } applyState(applyState.getClientActor(), applyState.getIdentifier(), applyState.getReplicatedLogEntry().getData()); + } else if (message instanceof ApplyLogEntries){ + ApplyLogEntries ale = (ApplyLogEntries) message; + if(LOG.isDebugEnabled()) { + LOG.debug("Persisting ApplyLogEntries with index={}", ale.getToIndex()); + } + persist(new ApplyLogEntries(ale.getToIndex()), new Procedure() { + @Override + public void apply(ApplyLogEntries param) throws Exception { + } + }); + } else if(message instanceof ApplySnapshot ) { - applySnapshot(((ApplySnapshot) message).getSnapshot()); + Snapshot snapshot = ((ApplySnapshot) message).getSnapshot(); + + if(LOG.isDebugEnabled()) { + 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( @@ -174,13 +302,26 @@ public abstract class RaftActor extends UntypedPersistentActor { } 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 + 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){ @@ -196,15 +337,36 @@ public abstract class RaftActor extends UntypedPersistentActor { RemoveRaftPeer rrp = (RemoveRaftPeer)message; context.removePeer(rrp.getName()); + } else if (message instanceof CaptureSnapshot) { + LOG.info("CaptureSnapshot received by actor"); + CaptureSnapshot cs = (CaptureSnapshot)message; + captureSnapshot = cs; + createSnapshot(); + + } else if (message instanceof CaptureSnapshotReply){ + LOG.info("CaptureSnapshotReply received by actor"); + CaptureSnapshotReply csr = (CaptureSnapshotReply) message; + + ByteString stateInBytes = csr.getSnapshot(); + LOG.info("CaptureSnapshotReply stateInBytes size:{}", stateInBytes.size()); + handleCaptureSnapshotReply(stateInBytes); + } else { + if (!(message instanceof AppendEntriesMessages.AppendEntries) + && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) { + if(LOG.isDebugEnabled()) { + LOG.debug("onReceiveCommand: message: {}", message.getClass()); + } + } - RaftState state = - currentBehavior.handleMessage(getSender(), message); RaftActorBehavior oldBehavior = currentBehavior; - currentBehavior = switchBehavior(state); + currentBehavior = currentBehavior.handleMessage(getSender(), message); + if(oldBehavior != currentBehavior){ onStateChanged(); } + + onLeaderChanged(oldBehavior.getLeaderId(), currentBehavior.getLeaderId()); } } @@ -234,7 +396,9 @@ public abstract class RaftActor extends UntypedPersistentActor { context.getReplicatedLog().lastIndex() + 1, context.getTermInformation().getCurrentTerm(), data); - LOG.debug("Persist data {}", replicatedLogEntry); + if(LOG.isDebugEnabled()) { + LOG.debug("Persist data {}", replicatedLogEntry); + } replicatedLog .appendAndPersist(clientActor, identifier, replicatedLogEntry); @@ -299,6 +463,10 @@ public abstract class RaftActor extends UntypedPersistentActor { return context.getLastApplied(); } + protected RaftActorContext getRaftActorContext() { + return context; + } + /** * setPeerAddress sets the address of a known peer at a later time. *

@@ -339,6 +507,38 @@ public abstract class RaftActor extends UntypedPersistentActor { protected abstract void applyState(ActorRef clientActor, String identifier, Object data); + /** + * This method is called during recovery at the start of a batch of state entries. Derived + * classes should perform any initialization needed to start a batch. + */ + protected abstract void startLogRecoveryBatch(int maxBatchSize); + + /** + * This method is called during recovery to append state data to the current batch. This method + * is called 1 or more times after {@link #startRecoveryStateBatch}. + * + * @param data the state data + */ + protected abstract void appendRecoveredLogEntry(Payload data); + + /** + * This method is called during recovery to reconstruct the state of the actor. + * + * @param snapshot A snapshot of the state of the actor + */ + protected abstract void applyRecoverySnapshot(ByteString snapshot); + + /** + * This method is called during recovery at the end of a batch to apply the current batched + * log entries. This method is called after {@link #appendRecoveryLogEntry}. + */ + protected abstract void applyCurrentLogRecoveryBatch(); + + /** + * This method is called when recovery is complete. + */ + protected abstract void onRecoveryComplete(); + /** * This method will be called by the RaftActor when a snapshot needs to be * created. The derived actor should respond with its current state. @@ -348,19 +548,16 @@ 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 - * reconstruct the state of the actor. - *

- * This method may also be called at any other point during normal + * This method can be called at any other point during normal * operations when the derived actor is out of sync with it's peers * and the only way to bring it in sync is by applying a snapshot * * @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 @@ -369,37 +566,7 @@ public abstract class RaftActor extends UntypedPersistentActor { */ protected abstract void onStateChanged(); - private RaftActorBehavior switchBehavior(RaftState state) { - if (currentBehavior != null) { - if (currentBehavior.state() == state) { - return currentBehavior; - } - LOG.info("Switching from state " + currentBehavior.state() + " to " - + state); - - try { - currentBehavior.close(); - } catch (Exception e) { - LOG.error(e, - "Failed to close behavior : " + currentBehavior.state()); - } - - } else { - LOG.info("Switching behavior to " + state); - } - RaftActorBehavior behavior = null; - if (state == RaftState.Candidate) { - behavior = new Candidate(context); - } else if (state == RaftState.Follower) { - behavior = new Follower(context); - } else { - behavior = new Leader(context); - } - - - - return behavior; - } + protected void onLeaderChanged(String oldLeader, String newLeader){}; private void trimPersistentData(long sequenceNumber) { // Trim akka snapshots @@ -421,17 +588,47 @@ public abstract class RaftActor extends UntypedPersistentActor { return null; } String peerAddress = context.getPeerAddress(leaderId); - LOG.debug("getLeaderAddress leaderId = " + leaderId + " peerAddress = " - + peerAddress); + if(LOG.isDebugEnabled()) { + LOG.debug("getLeaderAddress leaderId = {} peerAddress = {}", + leaderId, peerAddress); + } return peerAddress; } + private void handleCaptureSnapshotReply(ByteString stateInBytes) { + // create a snapshot object from the state provided and save it + // when snapshot is saved async, SaveSnapshotSuccess is raised. + + Snapshot sn = Snapshot.create(stateInBytes.toByteArray(), + context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1), + captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(), + captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm()); + + saveSnapshot(sn); + + LOG.info("Persisting of snapshot done:{}", sn.getLogMessage()); + + //be greedy and remove entries from in-mem journal which are in the snapshot + // and update snapshotIndex and snapshotTerm without waiting for the success, + + context.getReplicatedLog().snapshotPreCommit(stateInBytes, + captureSnapshot.getLastAppliedIndex(), + captureSnapshot.getLastAppliedTerm()); + + LOG.info("Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " + + "and term:{}", captureSnapshot.getLastAppliedIndex(), + captureSnapshot.getLastAppliedTerm()); + + captureSnapshot = null; + hasSnapshotCaptureInitiated = false; + } + private class ReplicatedLogImpl extends AbstractReplicatedLogImpl { public ReplicatedLogImpl(Snapshot snapshot) { - super(snapshot.getState(), + super(ByteString.copyFrom(snapshot.getState()), snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(), snapshot.getUnAppliedEntries()); } @@ -467,8 +664,11 @@ public abstract class RaftActor extends UntypedPersistentActor { public void appendAndPersist(final ActorRef clientActor, final String identifier, final ReplicatedLogEntry replicatedLogEntry) { - context.getLogger().debug( - "Append log entry and persist {} ", replicatedLogEntry); + + if(LOG.isDebugEnabled()) { + LOG.debug("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); @@ -479,9 +679,12 @@ public abstract class RaftActor extends UntypedPersistentActor { // of a single command. persist(replicatedLogEntry, new Procedure() { + @Override public void apply(ReplicatedLogEntry evt) throws Exception { - // FIXME : Tentatively create a snapshot every hundred thousand entries. To be tuned. - if (journal.size() > context.getConfigParams().getSnapshotBatchCount()) { + // 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; @@ -492,31 +695,19 @@ public abstract class RaftActor extends UntypedPersistentActor { lastAppliedTerm = lastAppliedEntry.getTerm(); } - 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); - - // create a snapshot object from the state provided and save it - // when snapshot is saved async, SaveSnapshotSuccess is raised. - Snapshot sn = Snapshot.create(createSnapshot(), - getFrom(context.getLastApplied() + 1), - lastIndex(), lastTerm(), lastAppliedIndex, - lastAppliedTerm); - saveSnapshot(sn); - - LOG.info("Persisting of snapshot done:{}", sn.getLogMessage()); - - //be greedy and remove entries from in-mem journal which are in the snapshot - // and update snapshotIndex and snapshotTerm without waiting for the success, - // TODO: damage-recovery to be done on failure - journal.subList(0, (int) (lastAppliedIndex - snapshotIndex)).clear(); - snapshotIndex = lastAppliedIndex; - snapshotTerm = lastAppliedTerm; - - LOG.info("Removed in-memory snapshotted entries, " + - "adjusted snaphsotIndex:{}" + - "and term:{}", snapshotIndex, lastAppliedTerm); + if(LOG.isDebugEnabled()) { + 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) { @@ -546,65 +737,6 @@ public abstract class RaftActor extends UntypedPersistentActor { } - private static class Snapshot implements Serializable { - private final Object state; - private final List unAppliedEntries; - private final long lastIndex; - private final long lastTerm; - private final long lastAppliedIndex; - private final long lastAppliedTerm; - - private Snapshot(Object state, - List 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 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 getUnAppliedEntries() { - return unAppliedEntries; - } - - public long getLastTerm() { - return lastTerm; - } - - public long getLastAppliedIndex() { - return lastAppliedIndex; - } - - public long getLastAppliedTerm() { - return lastAppliedTerm; - } - - public String getLogMessage() { - StringBuilder sb = new StringBuilder(); - return sb.append("Snapshot={") - .append("lastTerm:" + this.getLastTerm() + ", ") - .append("LastAppliedIndex:" + this.getLastAppliedIndex() + ", ") - .append("LastAppliedTerm:" + this.getLastAppliedTerm() + ", ") - .append("UnAppliedEntries size:" + this.getUnAppliedEntries().size() + "}") - .toString(); - - } - } - private class ElectionTermImpl implements ElectionTerm { /** * Identifier of the actor whose election term information this is @@ -612,17 +744,20 @@ public abstract class RaftActor extends UntypedPersistentActor { 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) { - LOG.debug("Set currentTerm={}, votedFor={}", currentTerm, votedFor); - + if(LOG.isDebugEnabled()) { + LOG.debug("Set currentTerm={}, votedFor={}", currentTerm, votedFor); + } this.currentTerm = currentTerm; this.votedFor = votedFor; }