Merge "Add more info to ShardStats JXM bean"
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActor.java
1 /*
2  * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
3  *
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
7  */
8
9 package org.opendaylight.controller.cluster.raft;
10
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.Optional;
21 import com.google.common.base.Stopwatch;
22 import com.google.common.collect.ImmutableMap;
23 import com.google.common.collect.Lists;
24 import com.google.protobuf.ByteString;
25 import java.io.Serializable;
26 import java.util.Collection;
27 import java.util.List;
28 import java.util.Map;
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.common.actor.AbstractUntypedPersistentActor;
33 import org.opendaylight.controller.cluster.notifications.RoleChanged;
34 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
35 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
36 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
37 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
38 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
39 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
40 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
41 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
42 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
43 import org.opendaylight.controller.cluster.raft.behaviors.AbstractRaftActorBehavior;
44 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
45 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
46 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
47 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
48 import org.opendaylight.controller.cluster.raft.client.messages.FollowerInfo;
49 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
50 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
51 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
52 import org.slf4j.Logger;
53 import org.slf4j.LoggerFactory;
54
55 /**
56  * RaftActor encapsulates a state machine that needs to be kept synchronized
57  * in a cluster. It implements the RAFT algorithm as described in the paper
58  * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
59  * In Search of an Understandable Consensus Algorithm</a>
60  * <p/>
61  * RaftActor has 3 states and each state has a certain behavior associated
62  * with it. A Raft actor can behave as,
63  * <ul>
64  * <li> A Leader </li>
65  * <li> A Follower (or) </li>
66  * <li> A Candidate </li>
67  * </ul>
68  * <p/>
69  * <p/>
70  * A RaftActor MUST be a Leader in order to accept requests from clients to
71  * change the state of it's encapsulated state machine. Once a RaftActor becomes
72  * a Leader it is also responsible for ensuring that all followers ultimately
73  * have the same log and therefore the same state machine as itself.
74  * <p/>
75  * <p/>
76  * The current behavior of a RaftActor determines how election for leadership
77  * is initiated and how peer RaftActors react to request for votes.
78  * <p/>
79  * <p/>
80  * Each RaftActor also needs to know the current election term. It uses this
81  * information for a couple of things. One is to simply figure out who it
82  * voted for in the last election. Another is to figure out if the message
83  * it received to update it's state is stale.
84  * <p/>
85  * <p/>
86  * The RaftActor uses akka-persistence to store it's replicated log.
87  * Furthermore through it's behaviors a Raft Actor determines
88  * <p/>
89  * <ul>
90  * <li> when a log entry should be persisted </li>
91  * <li> when a log entry should be applied to the state machine (and) </li>
92  * <li> when a snapshot should be saved </li>
93  * </ul>
94  */
95 public abstract class RaftActor extends AbstractUntypedPersistentActor {
96
97     private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis
98
99     private static final Procedure<ApplyJournalEntries> APPLY_JOURNAL_ENTRIES_PERSIST_CALLBACK =
100             new Procedure<ApplyJournalEntries>() {
101                 @Override
102                 public void apply(ApplyJournalEntries param) throws Exception {
103                 }
104             };
105
106     protected final Logger LOG = LoggerFactory.getLogger(getClass());
107
108     /**
109      * The current state determines the current behavior of a RaftActor
110      * A Raft Actor always starts off in the Follower State
111      */
112     private RaftActorBehavior currentBehavior;
113
114     /**
115      * This context should NOT be passed directly to any other actor it is
116      * only to be consumed by the RaftActorBehaviors
117      */
118     private final RaftActorContextImpl context;
119
120     /**
121      * The in-memory journal
122      */
123     private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
124
125     private CaptureSnapshot captureSnapshot = null;
126
127     private Stopwatch recoveryTimer;
128
129     private int currentRecoveryBatchCount;
130
131     public RaftActor(String id, Map<String, String> peerAddresses) {
132         this(id, peerAddresses, Optional.<ConfigParams>absent());
133     }
134
135     public RaftActor(String id, Map<String, String> peerAddresses,
136          Optional<ConfigParams> configParams) {
137
138         context = new RaftActorContextImpl(this.getSelf(),
139             this.getContext(), id, new ElectionTermImpl(),
140             -1, -1, replicatedLog, peerAddresses,
141             (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
142             LOG);
143     }
144
145     private void initRecoveryTimer() {
146         if(recoveryTimer == null) {
147             recoveryTimer = Stopwatch.createStarted();
148         }
149     }
150
151     @Override
152     public void preStart() throws Exception {
153         LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
154                 context.getConfigParams().getJournalRecoveryLogBatchSize());
155
156         super.preStart();
157     }
158
159     @Override
160     public void postStop() {
161         if(currentBehavior != null) {
162             try {
163                 currentBehavior.close();
164             } catch (Exception e) {
165                 LOG.debug("{}: Error closing behavior {}", persistenceId(), currentBehavior.state());
166             }
167         }
168
169         super.postStop();
170     }
171
172     @Override
173     public void handleRecover(Object message) {
174         if(persistence().isRecoveryApplicable()) {
175             if (message instanceof SnapshotOffer) {
176                 onRecoveredSnapshot((SnapshotOffer) message);
177             } else if (message instanceof ReplicatedLogEntry) {
178                 onRecoveredJournalLogEntry((ReplicatedLogEntry) message);
179             } else if (message instanceof ApplyLogEntries) {
180                 // Handle this message for backwards compatibility with pre-Lithium versions.
181                 onRecoveredApplyLogEntries(((ApplyLogEntries) message).getToIndex());
182             } else if (message instanceof ApplyJournalEntries) {
183                 onRecoveredApplyLogEntries(((ApplyJournalEntries) message).getToIndex());
184             } else if (message instanceof DeleteEntries) {
185                 replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex());
186             } else if (message instanceof UpdateElectionTerm) {
187                 context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(),
188                         ((UpdateElectionTerm) message).getVotedFor());
189             } else if (message instanceof RecoveryCompleted) {
190                 onRecoveryCompletedMessage();
191             }
192         } else {
193             if (message instanceof RecoveryCompleted) {
194                 // Delete all the messages from the akka journal so that we do not end up with consistency issues
195                 // Note I am not using the dataPersistenceProvider and directly using the akka api here
196                 deleteMessages(lastSequenceNr());
197
198                 // Delete all the akka snapshots as they will not be needed
199                 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
200
201                 onRecoveryComplete();
202
203                 initializeBehavior();
204             }
205         }
206     }
207
208     private void onRecoveredSnapshot(SnapshotOffer offer) {
209         if(LOG.isDebugEnabled()) {
210             LOG.debug("{}: SnapshotOffer called..", persistenceId());
211         }
212
213         initRecoveryTimer();
214
215         Snapshot snapshot = (Snapshot) offer.snapshot();
216
217         // Create a replicated log with the snapshot information
218         // The replicated log can be used later on to retrieve this snapshot
219         // when we need to install it on a peer
220         replicatedLog = new ReplicatedLogImpl(snapshot);
221
222         context.setReplicatedLog(replicatedLog);
223         context.setLastApplied(snapshot.getLastAppliedIndex());
224         context.setCommitIndex(snapshot.getLastAppliedIndex());
225
226         Stopwatch timer = Stopwatch.createStarted();
227
228         // Apply the snapshot to the actors state
229         applyRecoverySnapshot(snapshot.getState());
230
231         timer.stop();
232         LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
233                 replicatedLog.size(), persistenceId(), timer.toString(),
234                 replicatedLog.getSnapshotIndex(), replicatedLog.getSnapshotTerm());
235     }
236
237     private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
238         if(LOG.isDebugEnabled()) {
239             LOG.debug("{}: Received ReplicatedLogEntry for recovery: {}", persistenceId(), logEntry.getIndex());
240         }
241
242         replicatedLog.append(logEntry);
243     }
244
245     private void onRecoveredApplyLogEntries(long toIndex) {
246         if(LOG.isDebugEnabled()) {
247             LOG.debug("{}: Received ApplyLogEntries for recovery, applying to state: {} to {}",
248                     persistenceId(), context.getLastApplied() + 1, toIndex);
249         }
250
251         for (long i = context.getLastApplied() + 1; i <= toIndex; i++) {
252             batchRecoveredLogEntry(replicatedLog.get(i));
253         }
254
255         context.setLastApplied(toIndex);
256         context.setCommitIndex(toIndex);
257     }
258
259     private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) {
260         initRecoveryTimer();
261
262         int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize();
263         if(currentRecoveryBatchCount == 0) {
264             startLogRecoveryBatch(batchSize);
265         }
266
267         appendRecoveredLogEntry(logEntry.getData());
268
269         if(++currentRecoveryBatchCount >= batchSize) {
270             endCurrentLogRecoveryBatch();
271         }
272     }
273
274     private void endCurrentLogRecoveryBatch() {
275         applyCurrentLogRecoveryBatch();
276         currentRecoveryBatchCount = 0;
277     }
278
279     private void onRecoveryCompletedMessage() {
280         if(currentRecoveryBatchCount > 0) {
281             endCurrentLogRecoveryBatch();
282         }
283
284         onRecoveryComplete();
285
286         String recoveryTime = "";
287         if(recoveryTimer != null) {
288             recoveryTimer.stop();
289             recoveryTime = " in " + recoveryTimer.toString();
290             recoveryTimer = null;
291         }
292
293         LOG.info(
294             "Recovery completed" + recoveryTime + " - Switching actor to Follower - " +
295                 "Persistence Id =  " + persistenceId() +
296                 " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
297                 "journal-size={}",
298             replicatedLog.lastIndex(), replicatedLog.getSnapshotIndex(),
299             replicatedLog.getSnapshotTerm(), replicatedLog.size());
300
301         initializeBehavior();
302     }
303
304     protected void initializeBehavior(){
305         changeCurrentBehavior(new Follower(context));
306     }
307
308     protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
309         RaftActorBehavior oldBehavior = currentBehavior;
310         currentBehavior = newBehavior;
311         handleBehaviorChange(oldBehavior, currentBehavior);
312     }
313
314     @Override public void handleCommand(Object message) {
315         if (message instanceof ApplyState){
316             ApplyState applyState = (ApplyState) message;
317
318             long elapsedTime = (System.nanoTime() - applyState.getStartTime());
319             if(elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS){
320                 LOG.warn("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}",
321                         TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState);
322             }
323
324             if(LOG.isDebugEnabled()) {
325                 LOG.debug("{}: Applying state for log index {} data {}",
326                     persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
327                     applyState.getReplicatedLogEntry().getData());
328             }
329
330             applyState(applyState.getClientActor(), applyState.getIdentifier(),
331                 applyState.getReplicatedLogEntry().getData());
332
333         } else if (message instanceof ApplyJournalEntries){
334             ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
335             if(LOG.isDebugEnabled()) {
336                 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex());
337             }
338
339             persistence().persist(applyEntries, APPLY_JOURNAL_ENTRIES_PERSIST_CALLBACK);
340
341         } else if(message instanceof ApplySnapshot ) {
342             Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
343
344             if(LOG.isDebugEnabled()) {
345                 LOG.debug("{}: ApplySnapshot called on Follower Actor " +
346                         "snapshotIndex:{}, snapshotTerm:{}", persistenceId(), snapshot.getLastAppliedIndex(),
347                     snapshot.getLastAppliedTerm()
348                 );
349             }
350
351             applySnapshot(snapshot.getState());
352
353             //clears the followers log, sets the snapshot index to ensure adjusted-index works
354             replicatedLog = new ReplicatedLogImpl(snapshot);
355             context.setReplicatedLog(replicatedLog);
356             context.setLastApplied(snapshot.getLastAppliedIndex());
357
358         } else if (message instanceof FindLeader) {
359             getSender().tell(
360                 new FindLeaderReply(getLeaderAddress()),
361                 getSelf()
362             );
363
364         } else if (message instanceof SaveSnapshotSuccess) {
365             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
366             LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId());
367
368             long sequenceNumber = success.metadata().sequenceNr();
369
370             commitSnapshot(sequenceNumber);
371
372         } else if (message instanceof SaveSnapshotFailure) {
373             SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
374
375             LOG.error("{}: SaveSnapshotFailure received for snapshot Cause:",
376                     persistenceId(), saveSnapshotFailure.cause());
377
378             context.getReplicatedLog().snapshotRollback();
379
380             LOG.info("{}: Replicated Log rollbacked. Snapshot will be attempted in the next cycle." +
381                 "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
382                 context.getReplicatedLog().getSnapshotIndex(),
383                 context.getReplicatedLog().getSnapshotTerm(),
384                 context.getReplicatedLog().size());
385
386         } else if (message instanceof CaptureSnapshot) {
387             LOG.debug("{}: CaptureSnapshot received by actor: {}", persistenceId(), message);
388
389             if(captureSnapshot == null) {
390                 captureSnapshot = (CaptureSnapshot)message;
391                 createSnapshot();
392             }
393
394         } else if (message instanceof CaptureSnapshotReply){
395             handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot());
396         } else if(message instanceof GetOnDemandRaftState) {
397             onGetOnDemandRaftStats();
398         } else {
399             RaftActorBehavior oldBehavior = currentBehavior;
400             currentBehavior = currentBehavior.handleMessage(getSender(), message);
401
402             handleBehaviorChange(oldBehavior, currentBehavior);
403         }
404     }
405
406     private void onGetOnDemandRaftStats() {
407         // Debugging message to retrieve raft stats.
408
409         OnDemandRaftState.Builder builder = OnDemandRaftState.builder()
410                 .commitIndex(context.getCommitIndex())
411                 .currentTerm(context.getTermInformation().getCurrentTerm())
412                 .inMemoryJournalDataSize(replicatedLog.dataSize())
413                 .inMemoryJournalLogSize(replicatedLog.size())
414                 .isSnapshotCaptureInitiated(context.isSnapshotCaptureInitiated())
415                 .lastApplied(context.getLastApplied())
416                 .lastIndex(replicatedLog.lastIndex())
417                 .lastTerm(replicatedLog.lastTerm())
418                 .leader(getLeaderId())
419                 .raftState(currentBehavior.state().toString())
420                 .replicatedToAllIndex(currentBehavior.getReplicatedToAllIndex())
421                 .snapshotIndex(replicatedLog.getSnapshotIndex())
422                 .snapshotTerm(replicatedLog.getSnapshotTerm())
423                 .votedFor(context.getTermInformation().getVotedFor())
424                 .peerAddresses(ImmutableMap.copyOf(context.getPeerAddresses()));
425
426         ReplicatedLogEntry lastLogEntry = getLastLogEntry();
427         if (lastLogEntry != null) {
428             builder.lastLogIndex(lastLogEntry.getIndex());
429             builder.lastLogTerm(lastLogEntry.getTerm());
430         }
431
432         if(currentBehavior instanceof AbstractLeader) {
433             AbstractLeader leader = (AbstractLeader)currentBehavior;
434             Collection<String> followerIds = leader.getFollowerIds();
435             List<FollowerInfo> followerInfoList = Lists.newArrayListWithCapacity(followerIds.size());
436             for(String id: followerIds) {
437                 final FollowerLogInformation info = leader.getFollower(id);
438                 followerInfoList.add(new FollowerInfo(id, info.getNextIndex(), info.getMatchIndex(),
439                         info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(info.timeSinceLastActivity())));
440             }
441
442             builder.followerInfoList(followerInfoList);
443         }
444
445         sender().tell(builder.build(), self());
446
447     }
448
449     private void handleBehaviorChange(RaftActorBehavior oldBehavior, RaftActorBehavior currentBehavior) {
450         if (oldBehavior != currentBehavior){
451             onStateChanged();
452         }
453
454         String oldBehaviorLeaderId = oldBehavior == null? null : oldBehavior.getLeaderId();
455         String oldBehaviorState = oldBehavior == null? null : oldBehavior.state().name();
456
457         // it can happen that the state has not changed but the leader has changed.
458         onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
459
460         if (getRoleChangeNotifier().isPresent() &&
461                 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
462             getRoleChangeNotifier().get().tell(
463                     new RoleChanged(getId(), oldBehaviorState , currentBehavior.state().name()),
464                     getSelf());
465         }
466     }
467
468     /**
469      * When a derived RaftActor needs to persist something it must call
470      * persistData.
471      *
472      * @param clientActor
473      * @param identifier
474      * @param data
475      */
476     protected void persistData(final ActorRef clientActor, final String identifier,
477         final Payload data) {
478
479         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
480             context.getReplicatedLog().lastIndex() + 1,
481             context.getTermInformation().getCurrentTerm(), data);
482
483         if(LOG.isDebugEnabled()) {
484             LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
485         }
486
487         final RaftActorContext raftContext = getRaftActorContext();
488
489         replicatedLog
490                 .appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
491                     @Override
492                     public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
493                         if(!hasFollowers()){
494                             // Increment the Commit Index and the Last Applied values
495                             raftContext.setCommitIndex(replicatedLogEntry.getIndex());
496                             raftContext.setLastApplied(replicatedLogEntry.getIndex());
497
498                             // Apply the state immediately
499                             applyState(clientActor, identifier, data);
500
501                             // Send a ApplyJournalEntries message so that we write the fact that we applied
502                             // the state to durable storage
503                             self().tell(new ApplyJournalEntries(replicatedLogEntry.getIndex()), self());
504
505                             // Check if the "real" snapshot capture has been initiated. If no then do the fake snapshot
506                             if(!context.isSnapshotCaptureInitiated()){
507                                 raftContext.getReplicatedLog().snapshotPreCommit(raftContext.getLastApplied(),
508                                         raftContext.getTermInformation().getCurrentTerm());
509                                 raftContext.getReplicatedLog().snapshotCommit();
510                             } else {
511                                 LOG.debug("{}: Skipping fake snapshotting for {} because real snapshotting is in progress",
512                                         persistenceId(), getId());
513                             }
514                         } else if (clientActor != null) {
515                             // Send message for replication
516                             currentBehavior.handleMessage(getSelf(),
517                                     new Replicate(clientActor, identifier,
518                                             replicatedLogEntry)
519                             );
520                         }
521
522                     }
523                 });    }
524
525     protected String getId() {
526         return context.getId();
527     }
528
529     /**
530      * Derived actors can call the isLeader method to check if the current
531      * RaftActor is the Leader or not
532      *
533      * @return true it this RaftActor is a Leader false otherwise
534      */
535     protected boolean isLeader() {
536         return context.getId().equals(currentBehavior.getLeaderId());
537     }
538
539     /**
540      * Derived actor can call getLeader if they need a reference to the Leader.
541      * This would be useful for example in forwarding a request to an actor
542      * which is the leader
543      *
544      * @return A reference to the leader if known, null otherwise
545      */
546     protected ActorSelection getLeader(){
547         String leaderAddress = getLeaderAddress();
548
549         if(leaderAddress == null){
550             return null;
551         }
552
553         return context.actorSelection(leaderAddress);
554     }
555
556     /**
557      *
558      * @return the current leader's id
559      */
560     protected String getLeaderId(){
561         return currentBehavior.getLeaderId();
562     }
563
564     protected RaftState getRaftState() {
565         return currentBehavior.state();
566     }
567
568     protected ReplicatedLogEntry getLastLogEntry() {
569         return replicatedLog.last();
570     }
571
572     protected Long getCurrentTerm(){
573         return context.getTermInformation().getCurrentTerm();
574     }
575
576     protected Long getCommitIndex(){
577         return context.getCommitIndex();
578     }
579
580     protected Long getLastApplied(){
581         return context.getLastApplied();
582     }
583
584     protected RaftActorContext getRaftActorContext() {
585         return context;
586     }
587
588     protected void updateConfigParams(ConfigParams configParams) {
589         context.setConfigParams(configParams);
590     }
591
592     /**
593      * setPeerAddress sets the address of a known peer at a later time.
594      * <p>
595      * This is to account for situations where a we know that a peer
596      * exists but we do not know an address up-front. This may also be used in
597      * situations where a known peer starts off in a different location and we
598      * need to change it's address
599      * <p>
600      * Note that if the peerId does not match the list of peers passed to
601      * this actor during construction an IllegalStateException will be thrown.
602      *
603      * @param peerId
604      * @param peerAddress
605      */
606     protected void setPeerAddress(String peerId, String peerAddress){
607         context.setPeerAddress(peerId, peerAddress);
608     }
609
610     protected void commitSnapshot(long sequenceNumber) {
611         context.getReplicatedLog().snapshotCommit();
612
613         // TODO: Not sure if we want to be this aggressive with trimming stuff
614         trimPersistentData(sequenceNumber);
615     }
616
617     /**
618      * The applyState method will be called by the RaftActor when some data
619      * needs to be applied to the actor's state
620      *
621      * @param clientActor A reference to the client who sent this message. This
622      *                    is the same reference that was passed to persistData
623      *                    by the derived actor. clientActor may be null when
624      *                    the RaftActor is behaving as a follower or during
625      *                    recovery.
626      * @param identifier  The identifier of the persisted data. This is also
627      *                    the same identifier that was passed to persistData by
628      *                    the derived actor. identifier may be null when
629      *                    the RaftActor is behaving as a follower or during
630      *                    recovery
631      * @param data        A piece of data that was persisted by the persistData call.
632      *                    This should NEVER be null.
633      */
634     protected abstract void applyState(ActorRef clientActor, String identifier,
635         Object data);
636
637     /**
638      * This method is called during recovery at the start of a batch of state entries. Derived
639      * classes should perform any initialization needed to start a batch.
640      */
641     protected abstract void startLogRecoveryBatch(int maxBatchSize);
642
643     /**
644      * This method is called during recovery to append state data to the current batch. This method
645      * is called 1 or more times after {@link #startLogRecoveryBatch}.
646      *
647      * @param data the state data
648      */
649     protected abstract void appendRecoveredLogEntry(Payload data);
650
651     /**
652      * This method is called during recovery to reconstruct the state of the actor.
653      *
654      * @param snapshotBytes A snapshot of the state of the actor
655      */
656     protected abstract void applyRecoverySnapshot(byte[] snapshotBytes);
657
658     /**
659      * This method is called during recovery at the end of a batch to apply the current batched
660      * log entries. This method is called after {@link #appendRecoveredLogEntry}.
661      */
662     protected abstract void applyCurrentLogRecoveryBatch();
663
664     /**
665      * This method is called when recovery is complete.
666      */
667     protected abstract void onRecoveryComplete();
668
669     /**
670      * This method will be called by the RaftActor when a snapshot needs to be
671      * created. The derived actor should respond with its current state.
672      * <p/>
673      * During recovery the state that is returned by the derived actor will
674      * be passed back to it by calling the applySnapshot  method
675      *
676      * @return The current state of the actor
677      */
678     protected abstract void createSnapshot();
679
680     /**
681      * This method can be called at any other point during normal
682      * operations when the derived actor is out of sync with it's peers
683      * and the only way to bring it in sync is by applying a snapshot
684      *
685      * @param snapshotBytes A snapshot of the state of the actor
686      */
687     protected abstract void applySnapshot(byte[] snapshotBytes);
688
689     /**
690      * This method will be called by the RaftActor when the state of the
691      * RaftActor changes. The derived actor can then use methods like
692      * isLeader or getLeader to do something useful
693      */
694     protected abstract void onStateChanged();
695
696     protected abstract DataPersistenceProvider persistence();
697
698     /**
699      * Notifier Actor for this RaftActor to notify when a role change happens
700      * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
701      */
702     protected abstract Optional<ActorRef> getRoleChangeNotifier();
703
704     protected void onLeaderChanged(String oldLeader, String newLeader){};
705
706     private void trimPersistentData(long sequenceNumber) {
707         // Trim akka snapshots
708         // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied
709         // For now guessing that it is ANDed.
710         persistence().deleteSnapshots(new SnapshotSelectionCriteria(
711             sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
712
713         // Trim akka journal
714         persistence().deleteMessages(sequenceNumber);
715     }
716
717     private String getLeaderAddress(){
718         if(isLeader()){
719             return getSelf().path().toString();
720         }
721         String leaderId = currentBehavior.getLeaderId();
722         if (leaderId == null) {
723             return null;
724         }
725         String peerAddress = context.getPeerAddress(leaderId);
726         if(LOG.isDebugEnabled()) {
727             LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
728                     persistenceId(), leaderId, peerAddress);
729         }
730
731         return peerAddress;
732     }
733
734     private void handleCaptureSnapshotReply(byte[] snapshotBytes) {
735         LOG.debug("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length);
736
737         // create a snapshot object from the state provided and save it
738         // when snapshot is saved async, SaveSnapshotSuccess is raised.
739
740         Snapshot sn = Snapshot.create(snapshotBytes,
741             context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1),
742             captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
743             captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
744
745         persistence().saveSnapshot(sn);
746
747         LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), sn.getLogMessage());
748
749         long dataThreshold = getTotalMemory() *
750                 getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100;
751         if (context.getReplicatedLog().dataSize() > dataThreshold) {
752
753             if(LOG.isDebugEnabled()) {
754                 LOG.debug("{}: dataSize {} exceeds dataThreshold {} - doing snapshotPreCommit with index {}",
755                         persistenceId(), context.getReplicatedLog().dataSize(), dataThreshold,
756                         captureSnapshot.getLastAppliedIndex());
757             }
758
759             // if memory is less, clear the log based on lastApplied.
760             // this could/should only happen if one of the followers is down
761             // as normally we keep removing from the log when its replicated to all.
762             context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getLastAppliedIndex(),
763                     captureSnapshot.getLastAppliedTerm());
764
765             // Don't reset replicatedToAllIndex to -1 as this may prevent us from trimming the log after an
766             // install snapshot to a follower.
767             if(captureSnapshot.getReplicatedToAllIndex() >= 0) {
768                 getCurrentBehavior().setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
769             }
770         } else if(captureSnapshot.getReplicatedToAllIndex() != -1){
771             // clear the log based on replicatedToAllIndex
772             context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getReplicatedToAllIndex(),
773                     captureSnapshot.getReplicatedToAllTerm());
774
775             getCurrentBehavior().setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
776         } else {
777             // The replicatedToAllIndex was not found in the log
778             // This means that replicatedToAllIndex never moved beyond -1 or that it is already in the snapshot.
779             // In this scenario we may need to save the snapshot to the akka persistence
780             // snapshot for recovery but we do not need to do the replicated log trimming.
781             context.getReplicatedLog().snapshotPreCommit(replicatedLog.getSnapshotIndex(),
782                     replicatedLog.getSnapshotTerm());
783         }
784
785
786         LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex: {} " +
787             "and term: {}", persistenceId(), replicatedLog.getSnapshotIndex(),
788             replicatedLog.getSnapshotTerm());
789
790         if (isLeader() && captureSnapshot.isInstallSnapshotInitiated()) {
791             // this would be call straight to the leader and won't initiate in serialization
792             currentBehavior.handleMessage(getSelf(), new SendInstallSnapshot(
793                     ByteString.copyFrom(snapshotBytes)));
794         }
795
796         captureSnapshot = null;
797         context.setSnapshotCaptureInitiated(false);
798     }
799
800     protected long getTotalMemory() {
801         return Runtime.getRuntime().totalMemory();
802     }
803
804     protected boolean hasFollowers(){
805         return getRaftActorContext().getPeerAddresses().keySet().size() > 0;
806     }
807
808     private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
809
810         private static final int DATA_SIZE_DIVIDER = 5;
811         private long dataSizeSinceLastSnapshot = 0;
812
813         public ReplicatedLogImpl(Snapshot snapshot) {
814             super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
815                 snapshot.getUnAppliedEntries());
816         }
817
818         public ReplicatedLogImpl() {
819             super();
820         }
821
822         @Override public void removeFromAndPersist(long logEntryIndex) {
823             int adjustedIndex = adjustedIndex(logEntryIndex);
824
825             if (adjustedIndex < 0) {
826                 return;
827             }
828
829             // FIXME: Maybe this should be done after the command is saved
830             journal.subList(adjustedIndex , journal.size()).clear();
831
832             persistence().persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>() {
833
834                 @Override
835                 public void apply(DeleteEntries param)
836                         throws Exception {
837                     //FIXME : Doing nothing for now
838                     dataSize = 0;
839                     for (ReplicatedLogEntry entry : journal) {
840                         dataSize += entry.size();
841                     }
842                 }
843             });
844         }
845
846         @Override public void appendAndPersist(
847             final ReplicatedLogEntry replicatedLogEntry) {
848             appendAndPersist(replicatedLogEntry, null);
849         }
850
851         public void appendAndPersist(
852             final ReplicatedLogEntry replicatedLogEntry,
853             final Procedure<ReplicatedLogEntry> callback)  {
854
855             if(LOG.isDebugEnabled()) {
856                 LOG.debug("{}: Append log entry and persist {} ", persistenceId(), replicatedLogEntry);
857             }
858
859             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
860             journal.add(replicatedLogEntry);
861
862             // When persisting events with persist it is guaranteed that the
863             // persistent actor will not receive further commands between the
864             // persist call and the execution(s) of the associated event
865             // handler. This also holds for multiple persist calls in context
866             // of a single command.
867             persistence().persist(replicatedLogEntry,
868                 new Procedure<ReplicatedLogEntry>() {
869                     @Override
870                     public void apply(ReplicatedLogEntry evt) throws Exception {
871                         int logEntrySize = replicatedLogEntry.size();
872
873                         dataSize += logEntrySize;
874                         long dataSizeForCheck = dataSize;
875
876                         dataSizeSinceLastSnapshot += logEntrySize;
877                         long journalSize = lastIndex() + 1;
878
879                         if(!hasFollowers()) {
880                             // When we do not have followers we do not maintain an in-memory log
881                             // due to this the journalSize will never become anything close to the
882                             // snapshot batch count. In fact will mostly be 1.
883                             // Similarly since the journal's dataSize depends on the entries in the
884                             // journal the journal's dataSize will never reach a value close to the
885                             // memory threshold.
886                             // By maintaining the dataSize outside the journal we are tracking essentially
887                             // what we have written to the disk however since we no longer are in
888                             // need of doing a snapshot just for the sake of freeing up memory we adjust
889                             // the real size of data by the DATA_SIZE_DIVIDER so that we do not snapshot as often
890                             // as if we were maintaining a real snapshot
891                             dataSizeForCheck = dataSizeSinceLastSnapshot / DATA_SIZE_DIVIDER;
892                         }
893
894                         long dataThreshold = getTotalMemory() *
895                                 getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100;
896
897                         // when a snaphsot is being taken, captureSnapshot != null
898                         if (!context.isSnapshotCaptureInitiated() &&
899                                 ( journalSize % context.getConfigParams().getSnapshotBatchCount() == 0 ||
900                                         dataSizeForCheck > dataThreshold)) {
901
902                             dataSizeSinceLastSnapshot = 0;
903
904                             LOG.info("{}: Initiating Snapshot Capture, journalSize = {}, dataSizeForCheck = {}," +
905                                 " dataThreshold = {}", persistenceId(), journalSize, dataSizeForCheck, dataThreshold);
906
907                             long lastAppliedIndex = -1;
908                             long lastAppliedTerm = -1;
909
910                             ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied());
911                             if (!hasFollowers()) {
912                                 lastAppliedIndex = replicatedLogEntry.getIndex();
913                                 lastAppliedTerm = replicatedLogEntry.getTerm();
914                             } else if (lastAppliedEntry != null) {
915                                 lastAppliedIndex = lastAppliedEntry.getIndex();
916                                 lastAppliedTerm = lastAppliedEntry.getTerm();
917                             }
918
919                             if(LOG.isDebugEnabled()) {
920                                 LOG.debug("{}: Snapshot Capture logSize: {}", persistenceId(), journal.size());
921                                 LOG.debug("{}: Snapshot Capture lastApplied:{} ",
922                                         persistenceId(), context.getLastApplied());
923                                 LOG.debug("{}: Snapshot Capture lastAppliedIndex:{}", persistenceId(),
924                                         lastAppliedIndex);
925                                 LOG.debug("{}: Snapshot Capture lastAppliedTerm:{}", persistenceId(),
926                                         lastAppliedTerm);
927                             }
928
929                             // send a CaptureSnapshot to self to make the expensive operation async.
930                             long replicatedToAllIndex = getCurrentBehavior().getReplicatedToAllIndex();
931                             ReplicatedLogEntry replicatedToAllEntry = context.getReplicatedLog().get(replicatedToAllIndex);
932                             getSelf().tell(new CaptureSnapshot(lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm,
933                                 (replicatedToAllEntry != null ? replicatedToAllEntry.getIndex() : -1),
934                                 (replicatedToAllEntry != null ? replicatedToAllEntry.getTerm() : -1)),
935                                 null);
936                             context.setSnapshotCaptureInitiated(true);
937                         }
938                         if (callback != null){
939                             callback.apply(replicatedLogEntry);
940                         }
941                     }
942                 }
943             );
944         }
945
946     }
947
948     static class DeleteEntries implements Serializable {
949         private static final long serialVersionUID = 1L;
950         private final int fromIndex;
951
952         public DeleteEntries(int fromIndex) {
953             this.fromIndex = fromIndex;
954         }
955
956         public int getFromIndex() {
957             return fromIndex;
958         }
959     }
960
961
962     private class ElectionTermImpl implements ElectionTerm {
963         /**
964          * Identifier of the actor whose election term information this is
965          */
966         private long currentTerm = 0;
967         private String votedFor = null;
968
969         @Override
970         public long getCurrentTerm() {
971             return currentTerm;
972         }
973
974         @Override
975         public String getVotedFor() {
976             return votedFor;
977         }
978
979         @Override public void update(long currentTerm, String votedFor) {
980             if(LOG.isDebugEnabled()) {
981                 LOG.debug("{}: Set currentTerm={}, votedFor={}", persistenceId(), currentTerm, votedFor);
982             }
983             this.currentTerm = currentTerm;
984             this.votedFor = votedFor;
985         }
986
987         @Override
988         public void updateAndPersist(long currentTerm, String votedFor){
989             update(currentTerm, votedFor);
990             // FIXME : Maybe first persist then update the state
991             persistence().persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
992
993                 @Override public void apply(UpdateElectionTerm param)
994                     throws Exception {
995
996                 }
997             });
998         }
999     }
1000
1001     static class UpdateElectionTerm implements Serializable {
1002         private static final long serialVersionUID = 1L;
1003         private final long currentTerm;
1004         private final String votedFor;
1005
1006         public UpdateElectionTerm(long currentTerm, String votedFor) {
1007             this.currentTerm = currentTerm;
1008             this.votedFor = votedFor;
1009         }
1010
1011         public long getCurrentTerm() {
1012             return currentTerm;
1013         }
1014
1015         public String getVotedFor() {
1016             return votedFor;
1017         }
1018     }
1019
1020     protected class NonPersistentRaftDataProvider extends NonPersistentDataProvider {
1021
1022         public NonPersistentRaftDataProvider(){
1023
1024         }
1025
1026         /**
1027          * The way snapshotting works is,
1028          * <ol>
1029          * <li> RaftActor calls createSnapshot on the Shard
1030          * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
1031          * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save the snapshot.
1032          * The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the RaftActor gets SaveSnapshot
1033          * success it commits the snapshot to the in-memory journal. This commitSnapshot is mimicking what is done
1034          * in SaveSnapshotSuccess.
1035          * </ol>
1036          * @param o
1037          */
1038         @Override
1039         public void saveSnapshot(Object o) {
1040             // Make saving Snapshot successful
1041             commitSnapshot(-1L);
1042         }
1043     }
1044
1045     @VisibleForTesting
1046     void setCurrentBehavior(AbstractRaftActorBehavior behavior) {
1047         currentBehavior = behavior;
1048     }
1049
1050     protected RaftActorBehavior getCurrentBehavior() {
1051         return currentBehavior;
1052     }
1053
1054 }