2 * Copyright (c) 2014 Cisco Systems, Inc. and others. All rights reserved.
4 * This program and the accompanying materials are made available under the
5 * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6 * and is available at http://www.eclipse.org/legal/epl-v10.html
9 package org.opendaylight.controller.cluster.raft;
11 import akka.actor.ActorRef;
12 import akka.actor.ActorSelection;
13 import akka.event.Logging;
14 import akka.event.LoggingAdapter;
15 import akka.japi.Procedure;
16 import akka.persistence.RecoveryCompleted;
17 import akka.persistence.SaveSnapshotFailure;
18 import akka.persistence.SaveSnapshotSuccess;
19 import akka.persistence.SnapshotOffer;
20 import akka.persistence.SnapshotSelectionCriteria;
21 import com.google.common.annotations.VisibleForTesting;
22 import com.google.common.base.Optional;
23 import com.google.common.base.Stopwatch;
24 import com.google.protobuf.ByteString;
25 import java.io.Serializable;
27 import org.opendaylight.controller.cluster.DataPersistenceProvider;
28 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
29 import org.opendaylight.controller.cluster.notifications.RoleChanged;
30 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
31 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
32 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
33 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
34 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
35 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
36 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
37 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
38 import org.opendaylight.controller.cluster.raft.behaviors.AbstractRaftActorBehavior;
39 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
40 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
41 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
42 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
43 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
44 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
45 import org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages;
48 * RaftActor encapsulates a state machine that needs to be kept synchronized
49 * in a cluster. It implements the RAFT algorithm as described in the paper
50 * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
51 * In Search of an Understandable Consensus Algorithm</a>
53 * RaftActor has 3 states and each state has a certain behavior associated
54 * with it. A Raft actor can behave as,
57 * <li> A Follower (or) </li>
58 * <li> A Candidate </li>
62 * A RaftActor MUST be a Leader in order to accept requests from clients to
63 * change the state of it's encapsulated state machine. Once a RaftActor becomes
64 * a Leader it is also responsible for ensuring that all followers ultimately
65 * have the same log and therefore the same state machine as itself.
68 * The current behavior of a RaftActor determines how election for leadership
69 * is initiated and how peer RaftActors react to request for votes.
72 * Each RaftActor also needs to know the current election term. It uses this
73 * information for a couple of things. One is to simply figure out who it
74 * voted for in the last election. Another is to figure out if the message
75 * it received to update it's state is stale.
78 * The RaftActor uses akka-persistence to store it's replicated log.
79 * Furthermore through it's behaviors a Raft Actor determines
82 * <li> when a log entry should be persisted </li>
83 * <li> when a log entry should be applied to the state machine (and) </li>
84 * <li> when a snapshot should be saved </li>
87 public abstract class RaftActor extends AbstractUntypedPersistentActor {
88 protected final LoggingAdapter LOG =
89 Logging.getLogger(getContext().system(), this);
92 * The current state determines the current behavior of a RaftActor
93 * A Raft Actor always starts off in the Follower State
95 private RaftActorBehavior currentBehavior;
98 * This context should NOT be passed directly to any other actor it is
99 * only to be consumed by the RaftActorBehaviors
101 private final RaftActorContext context;
104 * The in-memory journal
106 private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
108 private CaptureSnapshot captureSnapshot = null;
110 private volatile boolean hasSnapshotCaptureInitiated = false;
112 private Stopwatch recoveryTimer;
114 private int currentRecoveryBatchCount;
118 public RaftActor(String id, Map<String, String> peerAddresses) {
119 this(id, peerAddresses, Optional.<ConfigParams>absent());
122 public RaftActor(String id, Map<String, String> peerAddresses,
123 Optional<ConfigParams> configParams) {
125 context = new RaftActorContextImpl(this.getSelf(),
126 this.getContext(), id, new ElectionTermImpl(),
127 -1, -1, replicatedLog, peerAddresses,
128 (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
132 private void initRecoveryTimer() {
133 if(recoveryTimer == null) {
134 recoveryTimer = new Stopwatch();
135 recoveryTimer.start();
140 public void preStart() throws Exception {
141 LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
142 context.getConfigParams().getJournalRecoveryLogBatchSize());
148 public void handleRecover(Object message) {
149 if(persistence().isRecoveryApplicable()) {
150 if (message instanceof SnapshotOffer) {
151 onRecoveredSnapshot((SnapshotOffer) message);
152 } else if (message instanceof ReplicatedLogEntry) {
153 onRecoveredJournalLogEntry((ReplicatedLogEntry) message);
154 } else if (message instanceof ApplyLogEntries) {
155 onRecoveredApplyLogEntries((ApplyLogEntries) message);
156 } else if (message instanceof DeleteEntries) {
157 replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex());
158 } else if (message instanceof UpdateElectionTerm) {
159 context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(),
160 ((UpdateElectionTerm) message).getVotedFor());
161 } else if (message instanceof RecoveryCompleted) {
162 onRecoveryCompletedMessage();
165 if (message instanceof RecoveryCompleted) {
166 // Delete all the messages from the akka journal so that we do not end up with consistency issues
167 // Note I am not using the dataPersistenceProvider and directly using the akka api here
168 deleteMessages(lastSequenceNr());
170 // Delete all the akka snapshots as they will not be needed
171 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
173 onRecoveryComplete();
175 initializeBehavior();
180 private void onRecoveredSnapshot(SnapshotOffer offer) {
181 if(LOG.isDebugEnabled()) {
182 LOG.debug("{}: SnapshotOffer called..", persistenceId());
187 Snapshot snapshot = (Snapshot) offer.snapshot();
189 // Create a replicated log with the snapshot information
190 // The replicated log can be used later on to retrieve this snapshot
191 // when we need to install it on a peer
192 replicatedLog = new ReplicatedLogImpl(snapshot);
194 context.setReplicatedLog(replicatedLog);
195 context.setLastApplied(snapshot.getLastAppliedIndex());
196 context.setCommitIndex(snapshot.getLastAppliedIndex());
198 Stopwatch timer = new Stopwatch();
201 // Apply the snapshot to the actors state
202 applyRecoverySnapshot(snapshot.getState());
205 LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
206 replicatedLog.size(), persistenceId(), timer.toString(),
207 replicatedLog.getSnapshotIndex(), replicatedLog.getSnapshotTerm());
210 private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
211 if(LOG.isDebugEnabled()) {
212 LOG.debug("{}: Received ReplicatedLogEntry for recovery: {}", persistenceId(), logEntry.getIndex());
215 replicatedLog.append(logEntry);
218 private void onRecoveredApplyLogEntries(ApplyLogEntries ale) {
219 if(LOG.isDebugEnabled()) {
220 LOG.debug("{}: Received ApplyLogEntries for recovery, applying to state: {} to {}",
221 persistenceId(), context.getLastApplied() + 1, ale.getToIndex());
224 for (long i = context.getLastApplied() + 1; i <= ale.getToIndex(); i++) {
225 batchRecoveredLogEntry(replicatedLog.get(i));
228 context.setLastApplied(ale.getToIndex());
229 context.setCommitIndex(ale.getToIndex());
232 private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) {
235 int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize();
236 if(currentRecoveryBatchCount == 0) {
237 startLogRecoveryBatch(batchSize);
240 appendRecoveredLogEntry(logEntry.getData());
242 if(++currentRecoveryBatchCount >= batchSize) {
243 endCurrentLogRecoveryBatch();
247 private void endCurrentLogRecoveryBatch() {
248 applyCurrentLogRecoveryBatch();
249 currentRecoveryBatchCount = 0;
252 private void onRecoveryCompletedMessage() {
253 if(currentRecoveryBatchCount > 0) {
254 endCurrentLogRecoveryBatch();
257 onRecoveryComplete();
259 String recoveryTime = "";
260 if(recoveryTimer != null) {
261 recoveryTimer.stop();
262 recoveryTime = " in " + recoveryTimer.toString();
263 recoveryTimer = null;
267 "Recovery completed" + recoveryTime + " - Switching actor to Follower - " +
268 "Persistence Id = " + persistenceId() +
269 " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
271 replicatedLog.lastIndex(), replicatedLog.getSnapshotIndex(),
272 replicatedLog.getSnapshotTerm(), replicatedLog.size());
274 initializeBehavior();
277 protected void initializeBehavior(){
278 changeCurrentBehavior(new Follower(context));
281 protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
282 RaftActorBehavior oldBehavior = currentBehavior;
283 currentBehavior = newBehavior;
284 handleBehaviorChange(oldBehavior, currentBehavior);
287 @Override public void handleCommand(Object message) {
288 if (message instanceof ApplyState){
289 ApplyState applyState = (ApplyState) message;
291 if(LOG.isDebugEnabled()) {
292 LOG.debug("{}: Applying state for log index {} data {}",
293 persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
294 applyState.getReplicatedLogEntry().getData());
297 applyState(applyState.getClientActor(), applyState.getIdentifier(),
298 applyState.getReplicatedLogEntry().getData());
300 } else if (message instanceof ApplyLogEntries){
301 ApplyLogEntries ale = (ApplyLogEntries) message;
302 if(LOG.isDebugEnabled()) {
303 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), ale.getToIndex());
305 persistence().persist(new ApplyLogEntries(ale.getToIndex()), new Procedure<ApplyLogEntries>() {
307 public void apply(ApplyLogEntries param) throws Exception {
311 } else if(message instanceof ApplySnapshot ) {
312 Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
314 if(LOG.isDebugEnabled()) {
315 LOG.debug("{}: ApplySnapshot called on Follower Actor " +
316 "snapshotIndex:{}, snapshotTerm:{}", persistenceId(), snapshot.getLastAppliedIndex(),
317 snapshot.getLastAppliedTerm()
321 applySnapshot(snapshot.getState());
323 //clears the followers log, sets the snapshot index to ensure adjusted-index works
324 replicatedLog = new ReplicatedLogImpl(snapshot);
325 context.setReplicatedLog(replicatedLog);
326 context.setLastApplied(snapshot.getLastAppliedIndex());
328 } else if (message instanceof FindLeader) {
330 new FindLeaderReply(getLeaderAddress()),
334 } else if (message instanceof SaveSnapshotSuccess) {
335 SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
336 LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId());
338 long sequenceNumber = success.metadata().sequenceNr();
340 commitSnapshot(sequenceNumber);
342 } else if (message instanceof SaveSnapshotFailure) {
343 SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
345 LOG.error(saveSnapshotFailure.cause(), "{}: SaveSnapshotFailure received for snapshot Cause:",
348 context.getReplicatedLog().snapshotRollback();
350 LOG.info("{}: Replicated Log rollbacked. Snapshot will be attempted in the next cycle." +
351 "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
352 context.getReplicatedLog().getSnapshotIndex(),
353 context.getReplicatedLog().getSnapshotTerm(),
354 context.getReplicatedLog().size());
356 } else if (message instanceof CaptureSnapshot) {
357 LOG.info("{}: CaptureSnapshot received by actor", persistenceId());
359 if(captureSnapshot == null) {
360 captureSnapshot = (CaptureSnapshot)message;
364 } else if (message instanceof CaptureSnapshotReply){
365 handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot());
368 if (!(message instanceof AppendEntriesMessages.AppendEntries)
369 && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) {
370 if(LOG.isDebugEnabled()) {
371 LOG.debug("{}: onReceiveCommand: message: {}", persistenceId(), message.getClass());
375 RaftActorBehavior oldBehavior = currentBehavior;
376 currentBehavior = currentBehavior.handleMessage(getSender(), message);
378 handleBehaviorChange(oldBehavior, currentBehavior);
382 private void handleBehaviorChange(RaftActorBehavior oldBehavior, RaftActorBehavior currentBehavior) {
383 if (oldBehavior != currentBehavior){
387 String oldBehaviorLeaderId = oldBehavior == null? null : oldBehavior.getLeaderId();
388 String oldBehaviorState = oldBehavior == null? null : oldBehavior.state().name();
390 // it can happen that the state has not changed but the leader has changed.
391 onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
393 if (getRoleChangeNotifier().isPresent() &&
394 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
395 getRoleChangeNotifier().get().tell(
396 new RoleChanged(getId(), oldBehaviorState , currentBehavior.state().name()),
402 * When a derived RaftActor needs to persist something it must call
409 protected void persistData(final ActorRef clientActor, final String identifier,
410 final Payload data) {
412 ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
413 context.getReplicatedLog().lastIndex() + 1,
414 context.getTermInformation().getCurrentTerm(), data);
416 if(LOG.isDebugEnabled()) {
417 LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
420 final RaftActorContext raftContext = getRaftActorContext();
423 .appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
425 public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
427 // Increment the Commit Index and the Last Applied values
428 raftContext.setCommitIndex(replicatedLogEntry.getIndex());
429 raftContext.setLastApplied(replicatedLogEntry.getIndex());
431 // Apply the state immediately
432 applyState(clientActor, identifier, data);
434 // Send a ApplyLogEntries message so that we write the fact that we applied
435 // the state to durable storage
436 self().tell(new ApplyLogEntries((int) replicatedLogEntry.getIndex()), self());
438 // Check if the "real" snapshot capture has been initiated. If no then do the fake snapshot
439 if(!hasSnapshotCaptureInitiated){
440 raftContext.getReplicatedLog().snapshotPreCommit(raftContext.getLastApplied(),
441 raftContext.getTermInformation().getCurrentTerm());
442 raftContext.getReplicatedLog().snapshotCommit();
444 LOG.debug("{}: Skipping fake snapshotting for {} because real snapshotting is in progress",
445 persistenceId(), getId());
447 } else if (clientActor != null) {
448 // Send message for replication
449 currentBehavior.handleMessage(getSelf(),
450 new Replicate(clientActor, identifier,
458 protected String getId() {
459 return context.getId();
463 * Derived actors can call the isLeader method to check if the current
464 * RaftActor is the Leader or not
466 * @return true it this RaftActor is a Leader false otherwise
468 protected boolean isLeader() {
469 return context.getId().equals(currentBehavior.getLeaderId());
473 * Derived actor can call getLeader if they need a reference to the Leader.
474 * This would be useful for example in forwarding a request to an actor
475 * which is the leader
477 * @return A reference to the leader if known, null otherwise
479 protected ActorSelection getLeader(){
480 String leaderAddress = getLeaderAddress();
482 if(leaderAddress == null){
486 return context.actorSelection(leaderAddress);
491 * @return the current leader's id
493 protected String getLeaderId(){
494 return currentBehavior.getLeaderId();
497 protected RaftState getRaftState() {
498 return currentBehavior.state();
501 protected ReplicatedLogEntry getLastLogEntry() {
502 return replicatedLog.last();
505 protected Long getCurrentTerm(){
506 return context.getTermInformation().getCurrentTerm();
509 protected Long getCommitIndex(){
510 return context.getCommitIndex();
513 protected Long getLastApplied(){
514 return context.getLastApplied();
517 protected RaftActorContext getRaftActorContext() {
522 * setPeerAddress sets the address of a known peer at a later time.
524 * This is to account for situations where a we know that a peer
525 * exists but we do not know an address up-front. This may also be used in
526 * situations where a known peer starts off in a different location and we
527 * need to change it's address
529 * Note that if the peerId does not match the list of peers passed to
530 * this actor during construction an IllegalStateException will be thrown.
535 protected void setPeerAddress(String peerId, String peerAddress){
536 context.setPeerAddress(peerId, peerAddress);
539 protected void commitSnapshot(long sequenceNumber) {
540 context.getReplicatedLog().snapshotCommit();
542 // TODO: Not sure if we want to be this aggressive with trimming stuff
543 trimPersistentData(sequenceNumber);
547 * The applyState method will be called by the RaftActor when some data
548 * needs to be applied to the actor's state
550 * @param clientActor A reference to the client who sent this message. This
551 * is the same reference that was passed to persistData
552 * by the derived actor. clientActor may be null when
553 * the RaftActor is behaving as a follower or during
555 * @param identifier The identifier of the persisted data. This is also
556 * the same identifier that was passed to persistData by
557 * the derived actor. identifier may be null when
558 * the RaftActor is behaving as a follower or during
560 * @param data A piece of data that was persisted by the persistData call.
561 * This should NEVER be null.
563 protected abstract void applyState(ActorRef clientActor, String identifier,
567 * This method is called during recovery at the start of a batch of state entries. Derived
568 * classes should perform any initialization needed to start a batch.
570 protected abstract void startLogRecoveryBatch(int maxBatchSize);
573 * This method is called during recovery to append state data to the current batch. This method
574 * is called 1 or more times after {@link #startLogRecoveryBatch}.
576 * @param data the state data
578 protected abstract void appendRecoveredLogEntry(Payload data);
581 * This method is called during recovery to reconstruct the state of the actor.
583 * @param snapshot A snapshot of the state of the actor
585 protected abstract void applyRecoverySnapshot(byte[] snapshotBytes);
588 * This method is called during recovery at the end of a batch to apply the current batched
589 * log entries. This method is called after {@link #appendRecoveredLogEntry}.
591 protected abstract void applyCurrentLogRecoveryBatch();
594 * This method is called when recovery is complete.
596 protected abstract void onRecoveryComplete();
599 * This method will be called by the RaftActor when a snapshot needs to be
600 * created. The derived actor should respond with its current state.
602 * During recovery the state that is returned by the derived actor will
603 * be passed back to it by calling the applySnapshot method
605 * @return The current state of the actor
607 protected abstract void createSnapshot();
610 * This method can be called at any other point during normal
611 * operations when the derived actor is out of sync with it's peers
612 * and the only way to bring it in sync is by applying a snapshot
614 * @param snapshotBytes A snapshot of the state of the actor
616 protected abstract void applySnapshot(byte[] snapshotBytes);
619 * This method will be called by the RaftActor when the state of the
620 * RaftActor changes. The derived actor can then use methods like
621 * isLeader or getLeader to do something useful
623 protected abstract void onStateChanged();
625 protected abstract DataPersistenceProvider persistence();
628 * Notifier Actor for this RaftActor to notify when a role change happens
629 * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
631 protected abstract Optional<ActorRef> getRoleChangeNotifier();
633 protected void onLeaderChanged(String oldLeader, String newLeader){};
635 private void trimPersistentData(long sequenceNumber) {
636 // Trim akka snapshots
637 // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied
638 // For now guessing that it is ANDed.
639 persistence().deleteSnapshots(new SnapshotSelectionCriteria(
640 sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
643 persistence().deleteMessages(sequenceNumber);
646 private String getLeaderAddress(){
648 return getSelf().path().toString();
650 String leaderId = currentBehavior.getLeaderId();
651 if (leaderId == null) {
654 String peerAddress = context.getPeerAddress(leaderId);
655 if(LOG.isDebugEnabled()) {
656 LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
657 persistenceId(), leaderId, peerAddress);
663 private void handleCaptureSnapshotReply(byte[] snapshotBytes) {
664 LOG.info("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length);
666 // create a snapshot object from the state provided and save it
667 // when snapshot is saved async, SaveSnapshotSuccess is raised.
669 Snapshot sn = Snapshot.create(snapshotBytes,
670 context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1),
671 captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
672 captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
674 persistence().saveSnapshot(sn);
676 LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), sn.getLogMessage());
678 //be greedy and remove entries from in-mem journal which are in the snapshot
679 // and update snapshotIndex and snapshotTerm without waiting for the success,
681 context.getReplicatedLog().snapshotPreCommit(
682 captureSnapshot.getLastAppliedIndex(),
683 captureSnapshot.getLastAppliedTerm());
685 LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " +
686 "and term:{}", persistenceId(), captureSnapshot.getLastAppliedIndex(),
687 captureSnapshot.getLastAppliedTerm());
689 if (isLeader() && captureSnapshot.isInstallSnapshotInitiated()) {
690 // this would be call straight to the leader and won't initiate in serialization
691 currentBehavior.handleMessage(getSelf(), new SendInstallSnapshot(
692 ByteString.copyFrom(snapshotBytes)));
695 captureSnapshot = null;
696 hasSnapshotCaptureInitiated = false;
699 protected boolean hasFollowers(){
700 return getRaftActorContext().getPeerAddresses().keySet().size() > 0;
703 private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
705 private static final int DATA_SIZE_DIVIDER = 5;
706 private long dataSizeSinceLastSnapshot = 0;
708 public ReplicatedLogImpl(Snapshot snapshot) {
709 super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
710 snapshot.getUnAppliedEntries());
713 public ReplicatedLogImpl() {
717 @Override public void removeFromAndPersist(long logEntryIndex) {
718 int adjustedIndex = adjustedIndex(logEntryIndex);
720 if (adjustedIndex < 0) {
724 // FIXME: Maybe this should be done after the command is saved
725 journal.subList(adjustedIndex , journal.size()).clear();
727 persistence().persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>(){
729 @Override public void apply(DeleteEntries param)
731 //FIXME : Doing nothing for now
733 for(ReplicatedLogEntry entry : journal){
734 dataSize += entry.size();
740 @Override public void appendAndPersist(
741 final ReplicatedLogEntry replicatedLogEntry) {
742 appendAndPersist(replicatedLogEntry, null);
746 public int dataSize() {
750 public void appendAndPersist(
751 final ReplicatedLogEntry replicatedLogEntry,
752 final Procedure<ReplicatedLogEntry> callback) {
754 if(LOG.isDebugEnabled()) {
755 LOG.debug("{}: Append log entry and persist {} ", persistenceId(), replicatedLogEntry);
758 // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
759 journal.add(replicatedLogEntry);
761 // When persisting events with persist it is guaranteed that the
762 // persistent actor will not receive further commands between the
763 // persist call and the execution(s) of the associated event
764 // handler. This also holds for multiple persist calls in context
765 // of a single command.
766 persistence().persist(replicatedLogEntry,
767 new Procedure<ReplicatedLogEntry>() {
769 public void apply(ReplicatedLogEntry evt) throws Exception {
770 int logEntrySize = replicatedLogEntry.size();
772 dataSize += logEntrySize;
773 long dataSizeForCheck = dataSize;
775 dataSizeSinceLastSnapshot += logEntrySize;
776 long journalSize = lastIndex()+1;
778 if(!hasFollowers()) {
779 // When we do not have followers we do not maintain an in-memory log
780 // due to this the journalSize will never become anything close to the
781 // snapshot batch count. In fact will mostly be 1.
782 // Similarly since the journal's dataSize depends on the entries in the
783 // journal the journal's dataSize will never reach a value close to the
785 // By maintaining the dataSize outside the journal we are tracking essentially
786 // what we have written to the disk however since we no longer are in
787 // need of doing a snapshot just for the sake of freeing up memory we adjust
788 // the real size of data by the DATA_SIZE_DIVIDER so that we do not snapshot as often
789 // as if we were maintaining a real snapshot
790 dataSizeForCheck = dataSizeSinceLastSnapshot / DATA_SIZE_DIVIDER;
793 long dataThreshold = Runtime.getRuntime().totalMemory() *
794 getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100;
796 // when a snaphsot is being taken, captureSnapshot != null
797 if (hasSnapshotCaptureInitiated == false &&
798 ( journalSize % context.getConfigParams().getSnapshotBatchCount() == 0 ||
799 dataSizeForCheck > dataThreshold)) {
801 dataSizeSinceLastSnapshot = 0;
803 LOG.info("{}: Initiating Snapshot Capture..", persistenceId());
804 long lastAppliedIndex = -1;
805 long lastAppliedTerm = -1;
807 ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied());
808 if (!hasFollowers()) {
809 lastAppliedIndex = replicatedLogEntry.getIndex();
810 lastAppliedTerm = replicatedLogEntry.getTerm();
811 } else if (lastAppliedEntry != null) {
812 lastAppliedIndex = lastAppliedEntry.getIndex();
813 lastAppliedTerm = lastAppliedEntry.getTerm();
816 if(LOG.isDebugEnabled()) {
817 LOG.debug("{}: Snapshot Capture logSize: {}", persistenceId(), journal.size());
818 LOG.debug("{}: Snapshot Capture lastApplied:{} ",
819 persistenceId(), context.getLastApplied());
820 LOG.debug("{}: Snapshot Capture lastAppliedIndex:{}", persistenceId(),
822 LOG.debug("{}: Snapshot Capture lastAppliedTerm:{}", persistenceId(),
826 // send a CaptureSnapshot to self to make the expensive operation async.
827 getSelf().tell(new CaptureSnapshot(
828 lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm),
830 hasSnapshotCaptureInitiated = true;
832 if(callback != null){
833 callback.apply(replicatedLogEntry);
842 static class DeleteEntries implements Serializable {
843 private static final long serialVersionUID = 1L;
844 private final int fromIndex;
846 public DeleteEntries(int fromIndex) {
847 this.fromIndex = fromIndex;
850 public int getFromIndex() {
856 private class ElectionTermImpl implements ElectionTerm {
858 * Identifier of the actor whose election term information this is
860 private long currentTerm = 0;
861 private String votedFor = null;
864 public long getCurrentTerm() {
869 public String getVotedFor() {
873 @Override public void update(long currentTerm, String votedFor) {
874 if(LOG.isDebugEnabled()) {
875 LOG.debug("{}: Set currentTerm={}, votedFor={}", persistenceId(), currentTerm, votedFor);
877 this.currentTerm = currentTerm;
878 this.votedFor = votedFor;
882 public void updateAndPersist(long currentTerm, String votedFor){
883 update(currentTerm, votedFor);
884 // FIXME : Maybe first persist then update the state
885 persistence().persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
887 @Override public void apply(UpdateElectionTerm param)
895 static class UpdateElectionTerm implements Serializable {
896 private static final long serialVersionUID = 1L;
897 private final long currentTerm;
898 private final String votedFor;
900 public UpdateElectionTerm(long currentTerm, String votedFor) {
901 this.currentTerm = currentTerm;
902 this.votedFor = votedFor;
905 public long getCurrentTerm() {
909 public String getVotedFor() {
914 protected class NonPersistentRaftDataProvider extends NonPersistentDataProvider {
916 public NonPersistentRaftDataProvider(){
921 * The way snapshotting works is,
923 * <li> RaftActor calls createSnapshot on the Shard
924 * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
925 * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save the snapshot.
926 * The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the RaftActor gets SaveSnapshot
927 * success it commits the snapshot to the in-memory journal. This commitSnapshot is mimicking what is done
928 * in SaveSnapshotSuccess.
933 public void saveSnapshot(Object o) {
934 // Make saving Snapshot successful
940 void setCurrentBehavior(AbstractRaftActorBehavior behavior) {
941 currentBehavior = behavior;
944 protected RaftActorBehavior getCurrentBehavior() {
945 return currentBehavior;