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.japi.Procedure;
14 import akka.persistence.RecoveryCompleted;
15 import akka.persistence.SaveSnapshotFailure;
16 import akka.persistence.SaveSnapshotSuccess;
17 import akka.persistence.SnapshotOffer;
18 import akka.persistence.SnapshotSelectionCriteria;
19 import com.google.common.annotations.VisibleForTesting;
20 import com.google.common.base.Objects;
21 import com.google.common.base.Optional;
22 import com.google.common.base.Stopwatch;
23 import com.google.common.collect.ImmutableMap;
24 import com.google.common.collect.Lists;
25 import java.io.Serializable;
26 import java.util.Collection;
27 import java.util.List;
29 import java.util.concurrent.TimeUnit;
30 import org.apache.commons.lang3.time.DurationFormatUtils;
31 import org.opendaylight.controller.cluster.DataPersistenceProvider;
32 import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
33 import org.opendaylight.controller.cluster.NonPersistentDataProvider;
34 import org.opendaylight.controller.cluster.PersistentDataProvider;
35 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
36 import org.opendaylight.controller.cluster.notifications.LeaderStateChanged;
37 import org.opendaylight.controller.cluster.notifications.RoleChanged;
38 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
39 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
40 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
41 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
42 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
43 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
44 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
45 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
46 import org.opendaylight.controller.cluster.raft.behaviors.AbstractRaftActorBehavior;
47 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
48 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
49 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
50 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
51 import org.opendaylight.controller.cluster.raft.client.messages.FollowerInfo;
52 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
53 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
54 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
55 import org.slf4j.Logger;
56 import org.slf4j.LoggerFactory;
59 * RaftActor encapsulates a state machine that needs to be kept synchronized
60 * in a cluster. It implements the RAFT algorithm as described in the paper
61 * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
62 * In Search of an Understandable Consensus Algorithm</a>
64 * RaftActor has 3 states and each state has a certain behavior associated
65 * with it. A Raft actor can behave as,
68 * <li> A Follower (or) </li>
69 * <li> A Candidate </li>
73 * A RaftActor MUST be a Leader in order to accept requests from clients to
74 * change the state of it's encapsulated state machine. Once a RaftActor becomes
75 * a Leader it is also responsible for ensuring that all followers ultimately
76 * have the same log and therefore the same state machine as itself.
79 * The current behavior of a RaftActor determines how election for leadership
80 * is initiated and how peer RaftActors react to request for votes.
83 * Each RaftActor also needs to know the current election term. It uses this
84 * information for a couple of things. One is to simply figure out who it
85 * voted for in the last election. Another is to figure out if the message
86 * it received to update it's state is stale.
89 * The RaftActor uses akka-persistence to store it's replicated log.
90 * Furthermore through it's behaviors a Raft Actor determines
93 * <li> when a log entry should be persisted </li>
94 * <li> when a log entry should be applied to the state machine (and) </li>
95 * <li> when a snapshot should be saved </li>
98 public abstract class RaftActor extends AbstractUntypedPersistentActor {
100 private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis
102 private static final Procedure<ApplyJournalEntries> APPLY_JOURNAL_ENTRIES_PERSIST_CALLBACK =
103 new Procedure<ApplyJournalEntries>() {
105 public void apply(ApplyJournalEntries param) throws Exception {
108 private static final String COMMIT_SNAPSHOT = "commit_snapshot";
110 protected final Logger LOG = LoggerFactory.getLogger(getClass());
113 * The current state determines the current behavior of a RaftActor
114 * A Raft Actor always starts off in the Follower State
116 private RaftActorBehavior currentBehavior;
119 * This context should NOT be passed directly to any other actor it is
120 * only to be consumed by the RaftActorBehaviors
122 private final RaftActorContextImpl context;
124 private final DelegatingPersistentDataProvider delegatingPersistenceProvider = new DelegatingPersistentDataProvider(null);
126 private final Procedure<Void> createSnapshotProcedure = new CreateSnapshotProcedure();
129 * The in-memory journal
131 private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
133 private Stopwatch recoveryTimer;
135 private int currentRecoveryBatchCount;
137 private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder();
139 public RaftActor(String id, Map<String, String> peerAddresses) {
140 this(id, peerAddresses, Optional.<ConfigParams>absent());
143 public RaftActor(String id, Map<String, String> peerAddresses,
144 Optional<ConfigParams> configParams) {
146 context = new RaftActorContextImpl(this.getSelf(),
147 this.getContext(), id, new ElectionTermImpl(),
148 -1, -1, replicatedLog, peerAddresses,
149 (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
153 private void initRecoveryTimer() {
154 if(recoveryTimer == null) {
155 recoveryTimer = Stopwatch.createStarted();
160 public void preStart() throws Exception {
161 LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
162 context.getConfigParams().getJournalRecoveryLogBatchSize());
168 public void postStop() {
169 if(currentBehavior != null) {
171 currentBehavior.close();
172 } catch (Exception e) {
173 LOG.debug("{}: Error closing behavior {}", persistenceId(), currentBehavior.state());
181 public void handleRecover(Object message) {
182 if(persistence().isRecoveryApplicable()) {
183 if (message instanceof SnapshotOffer) {
184 onRecoveredSnapshot((SnapshotOffer) message);
185 } else if (message instanceof ReplicatedLogEntry) {
186 onRecoveredJournalLogEntry((ReplicatedLogEntry) message);
187 } else if (message instanceof ApplyLogEntries) {
188 // Handle this message for backwards compatibility with pre-Lithium versions.
189 onRecoveredApplyLogEntries(((ApplyLogEntries) message).getToIndex());
190 } else if (message instanceof ApplyJournalEntries) {
191 onRecoveredApplyLogEntries(((ApplyJournalEntries) message).getToIndex());
192 } else if (message instanceof DeleteEntries) {
193 replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex());
194 } else if (message instanceof UpdateElectionTerm) {
195 context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(),
196 ((UpdateElectionTerm) message).getVotedFor());
197 } else if (message instanceof RecoveryCompleted) {
198 onRecoveryCompletedMessage();
201 if (message instanceof RecoveryCompleted) {
202 // Delete all the messages from the akka journal so that we do not end up with consistency issues
203 // Note I am not using the dataPersistenceProvider and directly using the akka api here
204 deleteMessages(lastSequenceNr());
206 // Delete all the akka snapshots as they will not be needed
207 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
209 onRecoveryComplete();
211 initializeBehavior();
216 private void onRecoveredSnapshot(SnapshotOffer offer) {
217 if(LOG.isDebugEnabled()) {
218 LOG.debug("{}: SnapshotOffer called..", persistenceId());
223 Snapshot snapshot = (Snapshot) offer.snapshot();
225 // Create a replicated log with the snapshot information
226 // The replicated log can be used later on to retrieve this snapshot
227 // when we need to install it on a peer
228 replicatedLog = new ReplicatedLogImpl(snapshot);
230 context.setReplicatedLog(replicatedLog);
231 context.setLastApplied(snapshot.getLastAppliedIndex());
232 context.setCommitIndex(snapshot.getLastAppliedIndex());
234 Stopwatch timer = Stopwatch.createStarted();
236 // Apply the snapshot to the actors state
237 applyRecoverySnapshot(snapshot.getState());
240 LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
241 replicatedLog.size(), persistenceId(), timer.toString(),
242 replicatedLog.getSnapshotIndex(), replicatedLog.getSnapshotTerm());
245 private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
246 if(LOG.isDebugEnabled()) {
247 LOG.debug("{}: Received ReplicatedLogEntry for recovery: {}", persistenceId(), logEntry.getIndex());
250 replicatedLog.append(logEntry);
253 private void onRecoveredApplyLogEntries(long toIndex) {
254 if(LOG.isDebugEnabled()) {
255 LOG.debug("{}: Received ApplyLogEntries for recovery, applying to state: {} to {}",
256 persistenceId(), context.getLastApplied() + 1, toIndex);
259 for (long i = context.getLastApplied() + 1; i <= toIndex; i++) {
260 batchRecoveredLogEntry(replicatedLog.get(i));
263 context.setLastApplied(toIndex);
264 context.setCommitIndex(toIndex);
267 private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) {
270 int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize();
271 if(currentRecoveryBatchCount == 0) {
272 startLogRecoveryBatch(batchSize);
275 appendRecoveredLogEntry(logEntry.getData());
277 if(++currentRecoveryBatchCount >= batchSize) {
278 endCurrentLogRecoveryBatch();
282 private void endCurrentLogRecoveryBatch() {
283 applyCurrentLogRecoveryBatch();
284 currentRecoveryBatchCount = 0;
287 private void onRecoveryCompletedMessage() {
288 if(currentRecoveryBatchCount > 0) {
289 endCurrentLogRecoveryBatch();
292 onRecoveryComplete();
294 String recoveryTime = "";
295 if(recoveryTimer != null) {
296 recoveryTimer.stop();
297 recoveryTime = " in " + recoveryTimer.toString();
298 recoveryTimer = null;
302 "Recovery completed" + recoveryTime + " - Switching actor to Follower - " +
303 "Persistence Id = " + persistenceId() +
304 " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
306 replicatedLog.lastIndex(), replicatedLog.getSnapshotIndex(),
307 replicatedLog.getSnapshotTerm(), replicatedLog.size());
309 initializeBehavior();
312 protected void initializeBehavior(){
313 changeCurrentBehavior(new Follower(context));
316 protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
317 reusableBehaviorStateHolder.init(currentBehavior);
318 currentBehavior = newBehavior;
319 handleBehaviorChange(reusableBehaviorStateHolder, currentBehavior);
322 @Override public void handleCommand(Object message) {
323 if (message instanceof ApplyState){
324 ApplyState applyState = (ApplyState) message;
326 long elapsedTime = (System.nanoTime() - applyState.getStartTime());
327 if(elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS){
328 LOG.warn("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}",
329 TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState);
332 if(LOG.isDebugEnabled()) {
333 LOG.debug("{}: Applying state for log index {} data {}",
334 persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
335 applyState.getReplicatedLogEntry().getData());
338 applyState(applyState.getClientActor(), applyState.getIdentifier(),
339 applyState.getReplicatedLogEntry().getData());
341 } else if (message instanceof ApplyJournalEntries){
342 ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
343 if(LOG.isDebugEnabled()) {
344 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex());
347 persistence().persist(applyEntries, APPLY_JOURNAL_ENTRIES_PERSIST_CALLBACK);
349 } else if(message instanceof ApplySnapshot ) {
350 Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
352 if(LOG.isDebugEnabled()) {
353 LOG.debug("{}: ApplySnapshot called on Follower Actor " +
354 "snapshotIndex:{}, snapshotTerm:{}", persistenceId(), snapshot.getLastAppliedIndex(),
355 snapshot.getLastAppliedTerm()
359 applySnapshot(snapshot.getState());
361 //clears the followers log, sets the snapshot index to ensure adjusted-index works
362 replicatedLog = new ReplicatedLogImpl(snapshot);
363 context.setReplicatedLog(replicatedLog);
364 context.setLastApplied(snapshot.getLastAppliedIndex());
366 } else if (message instanceof FindLeader) {
368 new FindLeaderReply(getLeaderAddress()),
372 } else if (message instanceof SaveSnapshotSuccess) {
373 SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
374 LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId());
376 long sequenceNumber = success.metadata().sequenceNr();
378 commitSnapshot(sequenceNumber);
380 } else if (message instanceof SaveSnapshotFailure) {
381 SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
383 LOG.error("{}: SaveSnapshotFailure received for snapshot Cause:",
384 persistenceId(), saveSnapshotFailure.cause());
386 context.getSnapshotManager().rollback();
388 } else if (message instanceof CaptureSnapshot) {
389 LOG.debug("{}: CaptureSnapshot received by actor: {}", persistenceId(), message);
391 context.getSnapshotManager().create(createSnapshotProcedure);
393 } else if (message instanceof CaptureSnapshotReply) {
394 handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot());
395 } else if(message instanceof GetOnDemandRaftState) {
396 onGetOnDemandRaftStats();
397 } else if (message.equals(COMMIT_SNAPSHOT)) {
400 reusableBehaviorStateHolder.init(currentBehavior);
402 currentBehavior = currentBehavior.handleMessage(getSender(), message);
404 handleBehaviorChange(reusableBehaviorStateHolder, currentBehavior);
408 private void onGetOnDemandRaftStats() {
409 // Debugging message to retrieve raft stats.
411 OnDemandRaftState.Builder builder = OnDemandRaftState.builder()
412 .commitIndex(context.getCommitIndex())
413 .currentTerm(context.getTermInformation().getCurrentTerm())
414 .inMemoryJournalDataSize(replicatedLog.dataSize())
415 .inMemoryJournalLogSize(replicatedLog.size())
416 .isSnapshotCaptureInitiated(context.getSnapshotManager().isCapturing())
417 .lastApplied(context.getLastApplied())
418 .lastIndex(replicatedLog.lastIndex())
419 .lastTerm(replicatedLog.lastTerm())
420 .leader(getLeaderId())
421 .raftState(currentBehavior.state().toString())
422 .replicatedToAllIndex(currentBehavior.getReplicatedToAllIndex())
423 .snapshotIndex(replicatedLog.getSnapshotIndex())
424 .snapshotTerm(replicatedLog.getSnapshotTerm())
425 .votedFor(context.getTermInformation().getVotedFor())
426 .peerAddresses(ImmutableMap.copyOf(context.getPeerAddresses()));
428 ReplicatedLogEntry lastLogEntry = getLastLogEntry();
429 if (lastLogEntry != null) {
430 builder.lastLogIndex(lastLogEntry.getIndex());
431 builder.lastLogTerm(lastLogEntry.getTerm());
434 if(currentBehavior instanceof AbstractLeader) {
435 AbstractLeader leader = (AbstractLeader)currentBehavior;
436 Collection<String> followerIds = leader.getFollowerIds();
437 List<FollowerInfo> followerInfoList = Lists.newArrayListWithCapacity(followerIds.size());
438 for(String id: followerIds) {
439 final FollowerLogInformation info = leader.getFollower(id);
440 followerInfoList.add(new FollowerInfo(id, info.getNextIndex(), info.getMatchIndex(),
441 info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(info.timeSinceLastActivity())));
444 builder.followerInfoList(followerInfoList);
447 sender().tell(builder.build(), self());
451 private void handleBehaviorChange(BehaviorStateHolder oldBehaviorState, RaftActorBehavior currentBehavior) {
452 RaftActorBehavior oldBehavior = oldBehaviorState.getBehavior();
454 if (oldBehavior != currentBehavior){
458 String oldBehaviorLeaderId = oldBehavior == null ? null : oldBehaviorState.getLeaderId();
459 String oldBehaviorStateName = oldBehavior == null ? null : oldBehavior.state().name();
461 // it can happen that the state has not changed but the leader has changed.
462 Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
463 if(!Objects.equal(oldBehaviorLeaderId, currentBehavior.getLeaderId())) {
464 if(roleChangeNotifier.isPresent()) {
465 roleChangeNotifier.get().tell(new LeaderStateChanged(getId(), currentBehavior.getLeaderId()), getSelf());
468 onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
471 if (roleChangeNotifier.isPresent() &&
472 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
473 roleChangeNotifier.get().tell(new RoleChanged(getId(), oldBehaviorStateName ,
474 currentBehavior.state().name()), getSelf());
479 * When a derived RaftActor needs to persist something it must call
486 protected void persistData(final ActorRef clientActor, final String identifier,
487 final Payload data) {
489 ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
490 context.getReplicatedLog().lastIndex() + 1,
491 context.getTermInformation().getCurrentTerm(), data);
493 if(LOG.isDebugEnabled()) {
494 LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
497 final RaftActorContext raftContext = getRaftActorContext();
500 .appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
502 public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
504 // Increment the Commit Index and the Last Applied values
505 raftContext.setCommitIndex(replicatedLogEntry.getIndex());
506 raftContext.setLastApplied(replicatedLogEntry.getIndex());
508 // Apply the state immediately
509 applyState(clientActor, identifier, data);
511 // Send a ApplyJournalEntries message so that we write the fact that we applied
512 // the state to durable storage
513 self().tell(new ApplyJournalEntries(replicatedLogEntry.getIndex()), self());
515 context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior);
517 } else if (clientActor != null) {
518 // Send message for replication
519 currentBehavior.handleMessage(getSelf(),
520 new Replicate(clientActor, identifier,
528 protected String getId() {
529 return context.getId();
533 * Derived actors can call the isLeader method to check if the current
534 * RaftActor is the Leader or not
536 * @return true it this RaftActor is a Leader false otherwise
538 protected boolean isLeader() {
539 return context.getId().equals(currentBehavior.getLeaderId());
543 * Derived actor can call getLeader if they need a reference to the Leader.
544 * This would be useful for example in forwarding a request to an actor
545 * which is the leader
547 * @return A reference to the leader if known, null otherwise
549 protected ActorSelection getLeader(){
550 String leaderAddress = getLeaderAddress();
552 if(leaderAddress == null){
556 return context.actorSelection(leaderAddress);
561 * @return the current leader's id
563 protected String getLeaderId(){
564 return currentBehavior.getLeaderId();
567 protected RaftState getRaftState() {
568 return currentBehavior.state();
571 protected ReplicatedLogEntry getLastLogEntry() {
572 return replicatedLog.last();
575 protected Long getCurrentTerm(){
576 return context.getTermInformation().getCurrentTerm();
579 protected Long getCommitIndex(){
580 return context.getCommitIndex();
583 protected Long getLastApplied(){
584 return context.getLastApplied();
587 protected RaftActorContext getRaftActorContext() {
591 protected void updateConfigParams(ConfigParams configParams) {
592 context.setConfigParams(configParams);
595 public final DataPersistenceProvider persistence() {
596 return delegatingPersistenceProvider.getDelegate();
599 public void setPersistence(DataPersistenceProvider provider) {
600 delegatingPersistenceProvider.setDelegate(provider);
603 protected void setPersistence(boolean persistent) {
605 setPersistence(new PersistentDataProvider(this));
607 setPersistence(new NonPersistentDataProvider() {
609 * The way snapshotting works is,
611 * <li> RaftActor calls createSnapshot on the Shard
612 * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
613 * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save
614 * the snapshot. The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the
615 * RaftActor gets SaveSnapshot success it commits the snapshot to the in-memory journal. This
616 * commitSnapshot is mimicking what is done in SaveSnapshotSuccess.
620 public void saveSnapshot(Object o) {
621 // Make saving Snapshot successful
622 // Committing the snapshot here would end up calling commit in the creating state which would
623 // be a state violation. That's why now we send a message to commit the snapshot.
624 self().tell(COMMIT_SNAPSHOT, self());
631 * setPeerAddress sets the address of a known peer at a later time.
633 * This is to account for situations where a we know that a peer
634 * exists but we do not know an address up-front. This may also be used in
635 * situations where a known peer starts off in a different location and we
636 * need to change it's address
638 * Note that if the peerId does not match the list of peers passed to
639 * this actor during construction an IllegalStateException will be thrown.
644 protected void setPeerAddress(String peerId, String peerAddress){
645 context.setPeerAddress(peerId, peerAddress);
648 protected void commitSnapshot(long sequenceNumber) {
649 context.getSnapshotManager().commit(persistence(), sequenceNumber);
653 * The applyState method will be called by the RaftActor when some data
654 * needs to be applied to the actor's state
656 * @param clientActor A reference to the client who sent this message. This
657 * is the same reference that was passed to persistData
658 * by the derived actor. clientActor may be null when
659 * the RaftActor is behaving as a follower or during
661 * @param identifier The identifier of the persisted data. This is also
662 * the same identifier that was passed to persistData by
663 * the derived actor. identifier may be null when
664 * the RaftActor is behaving as a follower or during
666 * @param data A piece of data that was persisted by the persistData call.
667 * This should NEVER be null.
669 protected abstract void applyState(ActorRef clientActor, String identifier,
673 * This method is called during recovery at the start of a batch of state entries. Derived
674 * classes should perform any initialization needed to start a batch.
676 protected abstract void startLogRecoveryBatch(int maxBatchSize);
679 * This method is called during recovery to append state data to the current batch. This method
680 * is called 1 or more times after {@link #startLogRecoveryBatch}.
682 * @param data the state data
684 protected abstract void appendRecoveredLogEntry(Payload data);
687 * This method is called during recovery to reconstruct the state of the actor.
689 * @param snapshotBytes A snapshot of the state of the actor
691 protected abstract void applyRecoverySnapshot(byte[] snapshotBytes);
694 * This method is called during recovery at the end of a batch to apply the current batched
695 * log entries. This method is called after {@link #appendRecoveredLogEntry}.
697 protected abstract void applyCurrentLogRecoveryBatch();
700 * This method is called when recovery is complete.
702 protected abstract void onRecoveryComplete();
705 * This method will be called by the RaftActor when a snapshot needs to be
706 * created. The derived actor should respond with its current state.
708 * During recovery the state that is returned by the derived actor will
709 * be passed back to it by calling the applySnapshot method
711 * @return The current state of the actor
713 protected abstract void createSnapshot();
716 * This method can be called at any other point during normal
717 * operations when the derived actor is out of sync with it's peers
718 * and the only way to bring it in sync is by applying a snapshot
720 * @param snapshotBytes A snapshot of the state of the actor
722 protected abstract void applySnapshot(byte[] snapshotBytes);
725 * This method will be called by the RaftActor when the state of the
726 * RaftActor changes. The derived actor can then use methods like
727 * isLeader or getLeader to do something useful
729 protected abstract void onStateChanged();
732 * Notifier Actor for this RaftActor to notify when a role change happens
733 * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
735 protected abstract Optional<ActorRef> getRoleChangeNotifier();
737 protected void onLeaderChanged(String oldLeader, String newLeader){};
739 private String getLeaderAddress(){
741 return getSelf().path().toString();
743 String leaderId = currentBehavior.getLeaderId();
744 if (leaderId == null) {
747 String peerAddress = context.getPeerAddress(leaderId);
748 if(LOG.isDebugEnabled()) {
749 LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
750 persistenceId(), leaderId, peerAddress);
756 private void handleCaptureSnapshotReply(byte[] snapshotBytes) {
757 LOG.debug("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length);
759 context.getSnapshotManager().persist(persistence(), snapshotBytes, currentBehavior, getTotalMemory());
762 protected long getTotalMemory() {
763 return Runtime.getRuntime().totalMemory();
766 protected boolean hasFollowers(){
767 return getRaftActorContext().getPeerAddresses().keySet().size() > 0;
770 private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
771 private static final int DATA_SIZE_DIVIDER = 5;
772 private long dataSizeSinceLastSnapshot = 0L;
775 public ReplicatedLogImpl(Snapshot snapshot) {
776 super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
777 snapshot.getUnAppliedEntries());
780 public ReplicatedLogImpl() {
784 @Override public void removeFromAndPersist(long logEntryIndex) {
785 int adjustedIndex = adjustedIndex(logEntryIndex);
787 if (adjustedIndex < 0) {
791 // FIXME: Maybe this should be done after the command is saved
792 journal.subList(adjustedIndex , journal.size()).clear();
794 persistence().persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>() {
797 public void apply(DeleteEntries param)
799 //FIXME : Doing nothing for now
801 for (ReplicatedLogEntry entry : journal) {
802 dataSize += entry.size();
808 @Override public void appendAndPersist(
809 final ReplicatedLogEntry replicatedLogEntry) {
810 appendAndPersist(replicatedLogEntry, null);
813 public void appendAndPersist(
814 final ReplicatedLogEntry replicatedLogEntry,
815 final Procedure<ReplicatedLogEntry> callback) {
817 if(LOG.isDebugEnabled()) {
818 LOG.debug("{}: Append log entry and persist {} ", persistenceId(), replicatedLogEntry);
821 // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
822 journal.add(replicatedLogEntry);
824 // When persisting events with persist it is guaranteed that the
825 // persistent actor will not receive further commands between the
826 // persist call and the execution(s) of the associated event
827 // handler. This also holds for multiple persist calls in context
828 // of a single command.
829 persistence().persist(replicatedLogEntry,
830 new Procedure<ReplicatedLogEntry>() {
832 public void apply(ReplicatedLogEntry evt) throws Exception {
833 int logEntrySize = replicatedLogEntry.size();
835 dataSize += logEntrySize;
836 long dataSizeForCheck = dataSize;
838 dataSizeSinceLastSnapshot += logEntrySize;
840 if (!hasFollowers()) {
841 // When we do not have followers we do not maintain an in-memory log
842 // due to this the journalSize will never become anything close to the
843 // snapshot batch count. In fact will mostly be 1.
844 // Similarly since the journal's dataSize depends on the entries in the
845 // journal the journal's dataSize will never reach a value close to the
847 // By maintaining the dataSize outside the journal we are tracking essentially
848 // what we have written to the disk however since we no longer are in
849 // need of doing a snapshot just for the sake of freeing up memory we adjust
850 // the real size of data by the DATA_SIZE_DIVIDER so that we do not snapshot as often
851 // as if we were maintaining a real snapshot
852 dataSizeForCheck = dataSizeSinceLastSnapshot / DATA_SIZE_DIVIDER;
854 long journalSize = replicatedLogEntry.getIndex() + 1;
855 long dataThreshold = getTotalMemory() *
856 context.getConfigParams().getSnapshotDataThresholdPercentage() / 100;
858 if ((journalSize % context.getConfigParams().getSnapshotBatchCount() == 0
859 || dataSizeForCheck > dataThreshold)) {
861 boolean started = context.getSnapshotManager().capture(replicatedLogEntry,
862 currentBehavior.getReplicatedToAllIndex());
865 dataSizeSinceLastSnapshot = 0;
870 if (callback != null){
871 callback.apply(replicatedLogEntry);
880 static class DeleteEntries implements Serializable {
881 private static final long serialVersionUID = 1L;
882 private final int fromIndex;
884 public DeleteEntries(int fromIndex) {
885 this.fromIndex = fromIndex;
888 public int getFromIndex() {
894 private class ElectionTermImpl implements ElectionTerm {
896 * Identifier of the actor whose election term information this is
898 private long currentTerm = 0;
899 private String votedFor = null;
902 public long getCurrentTerm() {
907 public String getVotedFor() {
911 @Override public void update(long currentTerm, String votedFor) {
912 if(LOG.isDebugEnabled()) {
913 LOG.debug("{}: Set currentTerm={}, votedFor={}", persistenceId(), currentTerm, votedFor);
915 this.currentTerm = currentTerm;
916 this.votedFor = votedFor;
920 public void updateAndPersist(long currentTerm, String votedFor){
921 update(currentTerm, votedFor);
922 // FIXME : Maybe first persist then update the state
923 persistence().persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
925 @Override public void apply(UpdateElectionTerm param)
933 static class UpdateElectionTerm implements Serializable {
934 private static final long serialVersionUID = 1L;
935 private final long currentTerm;
936 private final String votedFor;
938 public UpdateElectionTerm(long currentTerm, String votedFor) {
939 this.currentTerm = currentTerm;
940 this.votedFor = votedFor;
943 public long getCurrentTerm() {
947 public String getVotedFor() {
952 private class CreateSnapshotProcedure implements Procedure<Void> {
955 public void apply(Void aVoid) throws Exception {
961 void setCurrentBehavior(AbstractRaftActorBehavior behavior) {
962 currentBehavior = behavior;
965 protected RaftActorBehavior getCurrentBehavior() {
966 return currentBehavior;
969 private static class BehaviorStateHolder {
970 private RaftActorBehavior behavior;
971 private String leaderId;
973 void init(RaftActorBehavior behavior) {
974 this.behavior = behavior;
975 this.leaderId = behavior != null ? behavior.getLeaderId() : null;
978 RaftActorBehavior getBehavior() {
982 String getLeaderId() {