47ccfb7ed96cc5a9577ae6557f81ef1f4c7a798c
[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.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;
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.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;
57
58 /**
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>
63  * <p/>
64  * RaftActor has 3 states and each state has a certain behavior associated
65  * with it. A Raft actor can behave as,
66  * <ul>
67  * <li> A Leader </li>
68  * <li> A Follower (or) </li>
69  * <li> A Candidate </li>
70  * </ul>
71  * <p/>
72  * <p/>
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.
77  * <p/>
78  * <p/>
79  * The current behavior of a RaftActor determines how election for leadership
80  * is initiated and how peer RaftActors react to request for votes.
81  * <p/>
82  * <p/>
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.
87  * <p/>
88  * <p/>
89  * The RaftActor uses akka-persistence to store it's replicated log.
90  * Furthermore through it's behaviors a Raft Actor determines
91  * <p/>
92  * <ul>
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>
96  * </ul>
97  */
98 public abstract class RaftActor extends AbstractUntypedPersistentActor {
99
100     private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis
101
102     private static final Procedure<ApplyJournalEntries> APPLY_JOURNAL_ENTRIES_PERSIST_CALLBACK =
103             new Procedure<ApplyJournalEntries>() {
104                 @Override
105                 public void apply(ApplyJournalEntries param) throws Exception {
106                 }
107             };
108     private static final String COMMIT_SNAPSHOT = "commit_snapshot";
109
110     protected final Logger LOG = LoggerFactory.getLogger(getClass());
111
112     /**
113      * The current state determines the current behavior of a RaftActor
114      * A Raft Actor always starts off in the Follower State
115      */
116     private RaftActorBehavior currentBehavior;
117
118     /**
119      * This context should NOT be passed directly to any other actor it is
120      * only to be consumed by the RaftActorBehaviors
121      */
122     private final RaftActorContextImpl context;
123
124     private final DelegatingPersistentDataProvider delegatingPersistenceProvider = new DelegatingPersistentDataProvider(null);
125
126     private final Procedure<Void> createSnapshotProcedure = new CreateSnapshotProcedure();
127
128     /**
129      * The in-memory journal
130      */
131     private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
132
133     private Stopwatch recoveryTimer;
134
135     private int currentRecoveryBatchCount;
136
137     private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder();
138
139     public RaftActor(String id, Map<String, String> peerAddresses) {
140         this(id, peerAddresses, Optional.<ConfigParams>absent());
141     }
142
143     public RaftActor(String id, Map<String, String> peerAddresses,
144          Optional<ConfigParams> configParams) {
145
146         context = new RaftActorContextImpl(this.getSelf(),
147             this.getContext(), id, new ElectionTermImpl(),
148             -1, -1, replicatedLog, peerAddresses,
149             (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
150             LOG);
151     }
152
153     private void initRecoveryTimer() {
154         if(recoveryTimer == null) {
155             recoveryTimer = Stopwatch.createStarted();
156         }
157     }
158
159     @Override
160     public void preStart() throws Exception {
161         LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
162                 context.getConfigParams().getJournalRecoveryLogBatchSize());
163
164         super.preStart();
165     }
166
167     @Override
168     public void postStop() {
169         if(currentBehavior != null) {
170             try {
171                 currentBehavior.close();
172             } catch (Exception e) {
173                 LOG.debug("{}: Error closing behavior {}", persistenceId(), currentBehavior.state());
174             }
175         }
176
177         super.postStop();
178     }
179
180     @Override
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();
199             }
200         } else {
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());
205
206                 // Delete all the akka snapshots as they will not be needed
207                 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
208
209                 onRecoveryComplete();
210
211                 initializeBehavior();
212             }
213         }
214     }
215
216     private void onRecoveredSnapshot(SnapshotOffer offer) {
217         if(LOG.isDebugEnabled()) {
218             LOG.debug("{}: SnapshotOffer called..", persistenceId());
219         }
220
221         initRecoveryTimer();
222
223         Snapshot snapshot = (Snapshot) offer.snapshot();
224
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);
229
230         context.setReplicatedLog(replicatedLog);
231         context.setLastApplied(snapshot.getLastAppliedIndex());
232         context.setCommitIndex(snapshot.getLastAppliedIndex());
233
234         Stopwatch timer = Stopwatch.createStarted();
235
236         // Apply the snapshot to the actors state
237         applyRecoverySnapshot(snapshot.getState());
238
239         timer.stop();
240         LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
241                 replicatedLog.size(), persistenceId(), timer.toString(),
242                 replicatedLog.getSnapshotIndex(), replicatedLog.getSnapshotTerm());
243     }
244
245     private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
246         if(LOG.isDebugEnabled()) {
247             LOG.debug("{}: Received ReplicatedLogEntry for recovery: {}", persistenceId(), logEntry.getIndex());
248         }
249
250         replicatedLog.append(logEntry);
251     }
252
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);
257         }
258
259         for (long i = context.getLastApplied() + 1; i <= toIndex; i++) {
260             batchRecoveredLogEntry(replicatedLog.get(i));
261         }
262
263         context.setLastApplied(toIndex);
264         context.setCommitIndex(toIndex);
265     }
266
267     private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) {
268         initRecoveryTimer();
269
270         int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize();
271         if(currentRecoveryBatchCount == 0) {
272             startLogRecoveryBatch(batchSize);
273         }
274
275         appendRecoveredLogEntry(logEntry.getData());
276
277         if(++currentRecoveryBatchCount >= batchSize) {
278             endCurrentLogRecoveryBatch();
279         }
280     }
281
282     private void endCurrentLogRecoveryBatch() {
283         applyCurrentLogRecoveryBatch();
284         currentRecoveryBatchCount = 0;
285     }
286
287     private void onRecoveryCompletedMessage() {
288         if(currentRecoveryBatchCount > 0) {
289             endCurrentLogRecoveryBatch();
290         }
291
292         onRecoveryComplete();
293
294         String recoveryTime = "";
295         if(recoveryTimer != null) {
296             recoveryTimer.stop();
297             recoveryTime = " in " + recoveryTimer.toString();
298             recoveryTimer = null;
299         }
300
301         LOG.info(
302             "Recovery completed" + recoveryTime + " - Switching actor to Follower - " +
303                 "Persistence Id =  " + persistenceId() +
304                 " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
305                 "journal-size={}",
306             replicatedLog.lastIndex(), replicatedLog.getSnapshotIndex(),
307             replicatedLog.getSnapshotTerm(), replicatedLog.size());
308
309         initializeBehavior();
310     }
311
312     protected void initializeBehavior(){
313         changeCurrentBehavior(new Follower(context));
314     }
315
316     protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
317         reusableBehaviorStateHolder.init(currentBehavior);
318         currentBehavior = newBehavior;
319         handleBehaviorChange(reusableBehaviorStateHolder, currentBehavior);
320     }
321
322     @Override public void handleCommand(Object message) {
323         if (message instanceof ApplyState){
324             ApplyState applyState = (ApplyState) message;
325
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);
330             }
331
332             if(LOG.isDebugEnabled()) {
333                 LOG.debug("{}: Applying state for log index {} data {}",
334                     persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
335                     applyState.getReplicatedLogEntry().getData());
336             }
337
338             applyState(applyState.getClientActor(), applyState.getIdentifier(),
339                 applyState.getReplicatedLogEntry().getData());
340
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());
345             }
346
347             persistence().persist(applyEntries, APPLY_JOURNAL_ENTRIES_PERSIST_CALLBACK);
348
349         } else if(message instanceof ApplySnapshot ) {
350             Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
351
352             if(LOG.isDebugEnabled()) {
353                 LOG.debug("{}: ApplySnapshot called on Follower Actor " +
354                         "snapshotIndex:{}, snapshotTerm:{}", persistenceId(), snapshot.getLastAppliedIndex(),
355                     snapshot.getLastAppliedTerm()
356                 );
357             }
358
359             applySnapshot(snapshot.getState());
360
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());
365
366         } else if (message instanceof FindLeader) {
367             getSender().tell(
368                 new FindLeaderReply(getLeaderAddress()),
369                 getSelf()
370             );
371
372         } else if (message instanceof SaveSnapshotSuccess) {
373             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
374             LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId());
375
376             long sequenceNumber = success.metadata().sequenceNr();
377
378             commitSnapshot(sequenceNumber);
379
380         } else if (message instanceof SaveSnapshotFailure) {
381             SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
382
383             LOG.error("{}: SaveSnapshotFailure received for snapshot Cause:",
384                     persistenceId(), saveSnapshotFailure.cause());
385
386             context.getSnapshotManager().rollback();
387
388         } else if (message instanceof CaptureSnapshot) {
389             LOG.debug("{}: CaptureSnapshot received by actor: {}", persistenceId(), message);
390
391             context.getSnapshotManager().create(createSnapshotProcedure);
392
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)) {
398             commitSnapshot(-1);
399         } else {
400             reusableBehaviorStateHolder.init(currentBehavior);
401
402             currentBehavior = currentBehavior.handleMessage(getSender(), message);
403
404             handleBehaviorChange(reusableBehaviorStateHolder, currentBehavior);
405         }
406     }
407
408     private void onGetOnDemandRaftStats() {
409         // Debugging message to retrieve raft stats.
410
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()));
427
428         ReplicatedLogEntry lastLogEntry = getLastLogEntry();
429         if (lastLogEntry != null) {
430             builder.lastLogIndex(lastLogEntry.getIndex());
431             builder.lastLogTerm(lastLogEntry.getTerm());
432         }
433
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())));
442             }
443
444             builder.followerInfoList(followerInfoList);
445         }
446
447         sender().tell(builder.build(), self());
448
449     }
450
451     private void handleBehaviorChange(BehaviorStateHolder oldBehaviorState, RaftActorBehavior currentBehavior) {
452         RaftActorBehavior oldBehavior = oldBehaviorState.getBehavior();
453
454         if (oldBehavior != currentBehavior){
455             onStateChanged();
456         }
457
458         String oldBehaviorLeaderId = oldBehavior == null ? null : oldBehaviorState.getLeaderId();
459         String oldBehaviorStateName = oldBehavior == null ? null : oldBehavior.state().name();
460
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());
466             }
467
468             onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
469         }
470
471         if (roleChangeNotifier.isPresent() &&
472                 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
473             roleChangeNotifier.get().tell(new RoleChanged(getId(), oldBehaviorStateName ,
474                     currentBehavior.state().name()), getSelf());
475         }
476     }
477
478     /**
479      * When a derived RaftActor needs to persist something it must call
480      * persistData.
481      *
482      * @param clientActor
483      * @param identifier
484      * @param data
485      */
486     protected void persistData(final ActorRef clientActor, final String identifier,
487         final Payload data) {
488
489         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
490             context.getReplicatedLog().lastIndex() + 1,
491             context.getTermInformation().getCurrentTerm(), data);
492
493         if(LOG.isDebugEnabled()) {
494             LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
495         }
496
497         final RaftActorContext raftContext = getRaftActorContext();
498
499         replicatedLog
500                 .appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
501                     @Override
502                     public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
503                         if(!hasFollowers()){
504                             // Increment the Commit Index and the Last Applied values
505                             raftContext.setCommitIndex(replicatedLogEntry.getIndex());
506                             raftContext.setLastApplied(replicatedLogEntry.getIndex());
507
508                             // Apply the state immediately
509                             applyState(clientActor, identifier, data);
510
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());
514
515                             context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior);
516
517                         } else if (clientActor != null) {
518                             // Send message for replication
519                             currentBehavior.handleMessage(getSelf(),
520                                     new Replicate(clientActor, identifier,
521                                             replicatedLogEntry)
522                             );
523                         }
524
525                     }
526                 });    }
527
528     protected String getId() {
529         return context.getId();
530     }
531
532     /**
533      * Derived actors can call the isLeader method to check if the current
534      * RaftActor is the Leader or not
535      *
536      * @return true it this RaftActor is a Leader false otherwise
537      */
538     protected boolean isLeader() {
539         return context.getId().equals(currentBehavior.getLeaderId());
540     }
541
542     /**
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
546      *
547      * @return A reference to the leader if known, null otherwise
548      */
549     protected ActorSelection getLeader(){
550         String leaderAddress = getLeaderAddress();
551
552         if(leaderAddress == null){
553             return null;
554         }
555
556         return context.actorSelection(leaderAddress);
557     }
558
559     /**
560      *
561      * @return the current leader's id
562      */
563     protected String getLeaderId(){
564         return currentBehavior.getLeaderId();
565     }
566
567     protected RaftState getRaftState() {
568         return currentBehavior.state();
569     }
570
571     protected ReplicatedLogEntry getLastLogEntry() {
572         return replicatedLog.last();
573     }
574
575     protected Long getCurrentTerm(){
576         return context.getTermInformation().getCurrentTerm();
577     }
578
579     protected Long getCommitIndex(){
580         return context.getCommitIndex();
581     }
582
583     protected Long getLastApplied(){
584         return context.getLastApplied();
585     }
586
587     protected RaftActorContext getRaftActorContext() {
588         return context;
589     }
590
591     protected void updateConfigParams(ConfigParams configParams) {
592         context.setConfigParams(configParams);
593     }
594
595     public final DataPersistenceProvider persistence() {
596         return delegatingPersistenceProvider.getDelegate();
597     }
598
599     public void setPersistence(DataPersistenceProvider provider) {
600         delegatingPersistenceProvider.setDelegate(provider);
601     }
602
603     protected void setPersistence(boolean persistent) {
604         if(persistent) {
605             setPersistence(new PersistentDataProvider(this));
606         } else {
607             setPersistence(new NonPersistentDataProvider() {
608                 /**
609                  * The way snapshotting works is,
610                  * <ol>
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.
617                  * </ol>
618                  */
619                 @Override
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());
625                 }
626             });
627         }
628     }
629
630     /**
631      * setPeerAddress sets the address of a known peer at a later time.
632      * <p>
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
637      * <p>
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.
640      *
641      * @param peerId
642      * @param peerAddress
643      */
644     protected void setPeerAddress(String peerId, String peerAddress){
645         context.setPeerAddress(peerId, peerAddress);
646     }
647
648     protected void commitSnapshot(long sequenceNumber) {
649         context.getSnapshotManager().commit(persistence(), sequenceNumber);
650     }
651
652     /**
653      * The applyState method will be called by the RaftActor when some data
654      * needs to be applied to the actor's state
655      *
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
660      *                    recovery.
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
665      *                    recovery
666      * @param data        A piece of data that was persisted by the persistData call.
667      *                    This should NEVER be null.
668      */
669     protected abstract void applyState(ActorRef clientActor, String identifier,
670         Object data);
671
672     /**
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.
675      */
676     protected abstract void startLogRecoveryBatch(int maxBatchSize);
677
678     /**
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}.
681      *
682      * @param data the state data
683      */
684     protected abstract void appendRecoveredLogEntry(Payload data);
685
686     /**
687      * This method is called during recovery to reconstruct the state of the actor.
688      *
689      * @param snapshotBytes A snapshot of the state of the actor
690      */
691     protected abstract void applyRecoverySnapshot(byte[] snapshotBytes);
692
693     /**
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}.
696      */
697     protected abstract void applyCurrentLogRecoveryBatch();
698
699     /**
700      * This method is called when recovery is complete.
701      */
702     protected abstract void onRecoveryComplete();
703
704     /**
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.
707      * <p/>
708      * During recovery the state that is returned by the derived actor will
709      * be passed back to it by calling the applySnapshot  method
710      *
711      * @return The current state of the actor
712      */
713     protected abstract void createSnapshot();
714
715     /**
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
719      *
720      * @param snapshotBytes A snapshot of the state of the actor
721      */
722     protected abstract void applySnapshot(byte[] snapshotBytes);
723
724     /**
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
728      */
729     protected abstract void onStateChanged();
730
731     /**
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.
734      */
735     protected abstract Optional<ActorRef> getRoleChangeNotifier();
736
737     protected void onLeaderChanged(String oldLeader, String newLeader){};
738
739     private String getLeaderAddress(){
740         if(isLeader()){
741             return getSelf().path().toString();
742         }
743         String leaderId = currentBehavior.getLeaderId();
744         if (leaderId == null) {
745             return null;
746         }
747         String peerAddress = context.getPeerAddress(leaderId);
748         if(LOG.isDebugEnabled()) {
749             LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
750                     persistenceId(), leaderId, peerAddress);
751         }
752
753         return peerAddress;
754     }
755
756     private void handleCaptureSnapshotReply(byte[] snapshotBytes) {
757         LOG.debug("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length);
758
759         context.getSnapshotManager().persist(persistence(), snapshotBytes, currentBehavior, getTotalMemory());
760     }
761
762     protected long getTotalMemory() {
763         return Runtime.getRuntime().totalMemory();
764     }
765
766     protected boolean hasFollowers(){
767         return getRaftActorContext().getPeerAddresses().keySet().size() > 0;
768     }
769
770     private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
771         private static final int DATA_SIZE_DIVIDER = 5;
772         private long dataSizeSinceLastSnapshot = 0L;
773
774
775         public ReplicatedLogImpl(Snapshot snapshot) {
776             super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
777                 snapshot.getUnAppliedEntries());
778         }
779
780         public ReplicatedLogImpl() {
781             super();
782         }
783
784         @Override public void removeFromAndPersist(long logEntryIndex) {
785             int adjustedIndex = adjustedIndex(logEntryIndex);
786
787             if (adjustedIndex < 0) {
788                 return;
789             }
790
791             // FIXME: Maybe this should be done after the command is saved
792             journal.subList(adjustedIndex , journal.size()).clear();
793
794             persistence().persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>() {
795
796                 @Override
797                 public void apply(DeleteEntries param)
798                         throws Exception {
799                     //FIXME : Doing nothing for now
800                     dataSize = 0;
801                     for (ReplicatedLogEntry entry : journal) {
802                         dataSize += entry.size();
803                     }
804                 }
805             });
806         }
807
808         @Override public void appendAndPersist(
809             final ReplicatedLogEntry replicatedLogEntry) {
810             appendAndPersist(replicatedLogEntry, null);
811         }
812
813         public void appendAndPersist(
814             final ReplicatedLogEntry replicatedLogEntry,
815             final Procedure<ReplicatedLogEntry> callback)  {
816
817             if(LOG.isDebugEnabled()) {
818                 LOG.debug("{}: Append log entry and persist {} ", persistenceId(), replicatedLogEntry);
819             }
820
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);
823
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>() {
831                     @Override
832                     public void apply(ReplicatedLogEntry evt) throws Exception {
833                         int logEntrySize = replicatedLogEntry.size();
834
835                         dataSize += logEntrySize;
836                         long dataSizeForCheck = dataSize;
837
838                         dataSizeSinceLastSnapshot += logEntrySize;
839
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
846                             // memory threshold.
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;
853                         }
854                         long journalSize = replicatedLogEntry.getIndex() + 1;
855                         long dataThreshold = getTotalMemory() *
856                                 context.getConfigParams().getSnapshotDataThresholdPercentage() / 100;
857
858                         if ((journalSize % context.getConfigParams().getSnapshotBatchCount() == 0
859                                 || dataSizeForCheck > dataThreshold)) {
860
861                             boolean started = context.getSnapshotManager().capture(replicatedLogEntry,
862                                     currentBehavior.getReplicatedToAllIndex());
863
864                             if(started){
865                                 dataSizeSinceLastSnapshot = 0;
866                             }
867
868                         }
869
870                         if (callback != null){
871                             callback.apply(replicatedLogEntry);
872                         }
873                     }
874                 }
875             );
876         }
877
878     }
879
880     static class DeleteEntries implements Serializable {
881         private static final long serialVersionUID = 1L;
882         private final int fromIndex;
883
884         public DeleteEntries(int fromIndex) {
885             this.fromIndex = fromIndex;
886         }
887
888         public int getFromIndex() {
889             return fromIndex;
890         }
891     }
892
893
894     private class ElectionTermImpl implements ElectionTerm {
895         /**
896          * Identifier of the actor whose election term information this is
897          */
898         private long currentTerm = 0;
899         private String votedFor = null;
900
901         @Override
902         public long getCurrentTerm() {
903             return currentTerm;
904         }
905
906         @Override
907         public String getVotedFor() {
908             return votedFor;
909         }
910
911         @Override public void update(long currentTerm, String votedFor) {
912             if(LOG.isDebugEnabled()) {
913                 LOG.debug("{}: Set currentTerm={}, votedFor={}", persistenceId(), currentTerm, votedFor);
914             }
915             this.currentTerm = currentTerm;
916             this.votedFor = votedFor;
917         }
918
919         @Override
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>(){
924
925                 @Override public void apply(UpdateElectionTerm param)
926                     throws Exception {
927
928                 }
929             });
930         }
931     }
932
933     static class UpdateElectionTerm implements Serializable {
934         private static final long serialVersionUID = 1L;
935         private final long currentTerm;
936         private final String votedFor;
937
938         public UpdateElectionTerm(long currentTerm, String votedFor) {
939             this.currentTerm = currentTerm;
940             this.votedFor = votedFor;
941         }
942
943         public long getCurrentTerm() {
944             return currentTerm;
945         }
946
947         public String getVotedFor() {
948             return votedFor;
949         }
950     }
951
952     private class CreateSnapshotProcedure implements Procedure<Void> {
953
954         @Override
955         public void apply(Void aVoid) throws Exception {
956             createSnapshot();
957         }
958     }
959
960     @VisibleForTesting
961     void setCurrentBehavior(AbstractRaftActorBehavior behavior) {
962         currentBehavior = behavior;
963     }
964
965     protected RaftActorBehavior getCurrentBehavior() {
966         return currentBehavior;
967     }
968
969     private static class BehaviorStateHolder {
970         private RaftActorBehavior behavior;
971         private String leaderId;
972
973         void init(RaftActorBehavior behavior) {
974             this.behavior = behavior;
975             this.leaderId = behavior != null ? behavior.getLeaderId() : null;
976         }
977
978         RaftActorBehavior getBehavior() {
979             return behavior;
980         }
981
982         String getLeaderId() {
983             return leaderId;
984         }
985     }
986 }