2480a0db8f734bc6d1d0801b1b57e92e73769189
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / Follower.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.behaviors;
10
11 import akka.actor.ActorRef;
12 import akka.japi.Procedure;
13 import com.google.common.annotations.VisibleForTesting;
14 import com.google.common.base.Preconditions;
15 import java.util.ArrayList;
16 import org.opendaylight.controller.cluster.raft.RaftActorContext;
17 import org.opendaylight.controller.cluster.raft.RaftState;
18 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
19 import org.opendaylight.controller.cluster.raft.ServerConfigurationPayload;
20 import org.opendaylight.controller.cluster.raft.Snapshot;
21 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
22 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
23 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
24 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
25 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
26 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
27 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
28 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
29 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
30
31 /**
32  * The behavior of a RaftActor in the Follower state
33  * <p/>
34  * <ul>
35  * <li> Respond to RPCs from candidates and leaders
36  * <li> If election timeout elapses without receiving AppendEntries
37  * RPC from current leader or granting vote to candidate:
38  * convert to candidate
39  * </ul>
40  */
41 public class Follower extends AbstractRaftActorBehavior {
42     private static final int SYNC_THRESHOLD = 10;
43
44     private final SyncStatusTracker initialSyncStatusTracker;
45
46     private final Procedure<ReplicatedLogEntry> appendAndPersistCallback = new Procedure<ReplicatedLogEntry>() {
47         @Override
48         public void apply(ReplicatedLogEntry logEntry) {
49             context.getReplicatedLog().captureSnapshotIfReady(logEntry);
50         }
51     };
52
53     private SnapshotTracker snapshotTracker = null;
54     private String leaderId;
55     private short leaderPayloadVersion;
56
57     public Follower(RaftActorContext context) {
58         this(context, null, (short)-1);
59     }
60
61     public Follower(RaftActorContext context, String initialLeaderId, short initialLeaderPayloadVersion) {
62         super(context, RaftState.Follower);
63         this.leaderId = initialLeaderId;
64         this.leaderPayloadVersion = initialLeaderPayloadVersion;
65
66         initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD);
67
68         if(canStartElection()) {
69             if (context.getPeerIds().isEmpty() && getLeaderId() == null) {
70                 actor().tell(ElectionTimeout.INSTANCE, actor());
71             } else {
72                 scheduleElection(electionDuration());
73             }
74         }
75     }
76
77     @Override
78     public final String getLeaderId() {
79         return leaderId;
80     }
81
82     @VisibleForTesting
83     protected final void setLeaderId(final String leaderId) {
84         this.leaderId = Preconditions.checkNotNull(leaderId);
85     }
86
87     @Override
88     public short getLeaderPayloadVersion() {
89         return leaderPayloadVersion;
90     }
91
92     @VisibleForTesting
93     protected final void setLeaderPayloadVersion(short leaderPayloadVersion) {
94         this.leaderPayloadVersion = leaderPayloadVersion;
95     }
96
97     private boolean isLogEntryPresent(long index){
98         if(index == context.getReplicatedLog().getSnapshotIndex()){
99             return true;
100         }
101
102         ReplicatedLogEntry previousEntry = context.getReplicatedLog()
103                 .get(index);
104
105         return previousEntry != null;
106
107     }
108
109     private long getLogEntryTerm(long index){
110         if(index == context.getReplicatedLog().getSnapshotIndex()){
111             return context.getReplicatedLog().getSnapshotTerm();
112         }
113
114         ReplicatedLogEntry previousEntry = context.getReplicatedLog()
115                 .get(index);
116
117         if(previousEntry != null){
118             return previousEntry.getTerm();
119         }
120
121         return -1;
122     }
123
124     private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){
125         initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex());
126     }
127
128     @Override
129     protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) {
130
131         int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
132         if(LOG.isTraceEnabled()) {
133             LOG.trace("{}: handleAppendEntries: {}", logName(), appendEntries);
134         } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
135             LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
136         }
137
138         // TODO : Refactor this method into a bunch of smaller methods
139         // to make it easier to read. Before refactoring ensure tests
140         // cover the code properly
141
142         if (snapshotTracker != null || context.getSnapshotManager().isApplying()) {
143             // if snapshot install is in progress, follower should just acknowledge append entries with a reply.
144             AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
145                     lastIndex(), lastTerm(), context.getPayloadVersion());
146
147             if(LOG.isDebugEnabled()) {
148                 LOG.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply);
149             }
150             sender.tell(reply, actor());
151
152             return this;
153         }
154
155         // If we got here then we do appear to be talking to the leader
156         leaderId = appendEntries.getLeaderId();
157         leaderPayloadVersion = appendEntries.getPayloadVersion();
158
159         updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
160         // First check if the logs are in sync or not
161         long lastIndex = lastIndex();
162
163         if (isOutOfSync(appendEntries)) {
164             // We found that the log was out of sync so just send a negative
165             // reply and return
166
167             LOG.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}",
168                         logName(), lastIndex, lastTerm());
169
170             sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
171                     lastTerm(), context.getPayloadVersion()), actor());
172             return this;
173         }
174
175         if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
176
177             LOG.debug("{}: Number of entries to be appended = {}", logName(),
178                         appendEntries.getEntries().size());
179
180             // 3. If an existing entry conflicts with a new one (same index
181             // but different terms), delete the existing entry and all that
182             // follow it (ยง5.3)
183             int addEntriesFrom = 0;
184             if (context.getReplicatedLog().size() > 0) {
185
186                 // Find the entry up until the one that is not in the follower's log
187                 for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) {
188                     ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
189                     ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex());
190
191                     if (newEntry == null) {
192                         //newEntry not found in the log
193                         break;
194                     }
195
196                     if (newEntry.getTerm() == matchEntry.getTerm()) {
197                         continue;
198                     }
199
200                     if(!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) {
201
202                         LOG.debug("{}: Removing entries from log starting at {}", logName(),
203                                 matchEntry.getIndex());
204
205                         // Entries do not match so remove all subsequent entries
206                         context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex());
207                         break;
208                     } else {
209                         sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
210                                 lastTerm(), context.getPayloadVersion(), true), actor());
211                         return this;
212                     }
213                 }
214             }
215
216             lastIndex = lastIndex();
217             LOG.debug("{}: After cleanup entries to be added from = {}", logName(),
218                         (addEntriesFrom + lastIndex));
219
220             // 4. Append any new entries not already in the log
221             for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) {
222                 ReplicatedLogEntry entry = appendEntries.getEntries().get(i);
223
224                 LOG.debug("{}: Append entry to log {}", logName(), entry.getData());
225
226                 context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback);
227
228                 if(entry.getData() instanceof ServerConfigurationPayload) {
229                     context.updatePeerIds((ServerConfigurationPayload)entry.getData());
230                 }
231             }
232
233             LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
234         }
235
236         // 5. If leaderCommit > commitIndex, set commitIndex =
237         // min(leaderCommit, index of last new entry)
238
239         lastIndex = lastIndex();
240         long prevCommitIndex = context.getCommitIndex();
241
242         context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex));
243
244         if (prevCommitIndex != context.getCommitIndex()) {
245             LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex());
246         }
247
248         // If commitIndex > lastApplied: increment lastApplied, apply
249         // log[lastApplied] to state machine (ยง5.3)
250         // check if there are any entries to be applied. last-applied can be equal to last-index
251         if (appendEntries.getLeaderCommit() > context.getLastApplied() &&
252             context.getLastApplied() < lastIndex) {
253             if(LOG.isDebugEnabled()) {
254                 LOG.debug("{}: applyLogToStateMachine, " +
255                         "appendEntries.getLeaderCommit(): {}," +
256                         "context.getLastApplied(): {}, lastIndex(): {}", logName(),
257                     appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex);
258             }
259
260             applyLogToStateMachine(appendEntries.getLeaderCommit());
261         }
262
263         AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
264             lastIndex, lastTerm(), context.getPayloadVersion());
265
266         if(LOG.isTraceEnabled()) {
267             LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply);
268         } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
269             LOG.debug("{}: handleAppendEntries returning : {}", logName(), reply);
270         }
271
272         sender.tell(reply, actor());
273
274         if (!context.getSnapshotManager().isCapturing()) {
275             super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex());
276         }
277
278         return this;
279     }
280
281     private boolean isOutOfSync(AppendEntries appendEntries) {
282
283         long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
284         boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex());
285         long lastIndex = lastIndex();
286         int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
287         boolean outOfSync = true;
288
289         if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) {
290
291             // The follower's log is out of sync because the leader does have
292             // an entry at prevLogIndex and this follower has no entries in
293             // it's log.
294
295             LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}",
296                         logName(), appendEntries.getPrevLogIndex());
297         } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) {
298
299             // The follower's log is out of sync because the Leader's
300             // prevLogIndex entry was not found in it's log
301
302             LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it",
303                         logName(), appendEntries.getPrevLogIndex());
304         } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) {
305
306             // The follower's log is out of sync because the Leader's
307             // prevLogIndex entry does exist in the follower's log but it has
308             // a different term in it
309
310             LOG.debug(
311                     "{}: Cannot append entries because previous entry term {}  is not equal to append entries prevLogTerm {}",
312                     logName(), prevLogTerm, appendEntries.getPrevLogTerm());
313         } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
314                 && appendEntries.getReplicatedToAllIndex() != -1
315                 && !isLogEntryPresent(appendEntries.getReplicatedToAllIndex())
316                 && !context.getReplicatedLog().isInSnapshot(appendEntries.getReplicatedToAllIndex())) {
317             // This append entry comes from a leader who has it's log aggressively trimmed and so does not have
318             // the previous entry in it's in-memory journal
319
320             LOG.debug(
321                     "{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the in-memory journal",
322                     logName(), appendEntries.getReplicatedToAllIndex());
323         } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
324                 && appendEntries.getReplicatedToAllIndex() != -1 && numLogEntries > 0
325                 && !isLogEntryPresent(appendEntries.getEntries().get(0).getIndex() - 1)
326                 && !context.getReplicatedLog().isInSnapshot(appendEntries.getEntries().get(0).getIndex() - 1)) {
327             LOG.debug(
328                     "{}: Cannot append entries because the calculated previousIndex {} was not found in the in-memory journal",
329                     logName(), appendEntries.getEntries().get(0).getIndex() - 1);
330         } else {
331             outOfSync = false;
332         }
333         return outOfSync;
334     }
335
336     @Override
337     protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
338         AppendEntriesReply appendEntriesReply) {
339         return this;
340     }
341
342     @Override
343     protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
344         RequestVoteReply requestVoteReply) {
345         return this;
346     }
347
348     @Override
349     public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
350         if (originalMessage instanceof ElectionTimeout) {
351             if (canStartElection()) {
352                 LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName());
353                 return internalSwitchBehavior(RaftState.Candidate);
354             } else {
355                 return this;
356             }
357         }
358
359         final Object message = fromSerializableMessage(originalMessage);
360         if (message instanceof RaftRPC) {
361             RaftRPC rpc = (RaftRPC) message;
362             // If RPC request or response contains term T > currentTerm:
363             // set currentTerm = T, convert to follower (ยง5.1)
364             // This applies to all RPC messages and responses
365             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
366                 LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term",
367                         logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
368
369                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
370             }
371         }
372
373         if (message instanceof InstallSnapshot) {
374             InstallSnapshot installSnapshot = (InstallSnapshot) message;
375             handleInstallSnapshot(sender, installSnapshot);
376         }
377
378         if (message instanceof RaftRPC && (!(message instanceof RequestVote) || (canGrantVote((RequestVote) message)))){
379             scheduleElection(electionDuration());
380         }
381
382         return super.handleMessage(sender, message);
383     }
384
385     private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot installSnapshot) {
386
387         LOG.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot);
388
389         leaderId = installSnapshot.getLeaderId();
390
391         if(snapshotTracker == null){
392             snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks());
393         }
394
395         updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId());
396
397         try {
398             final InstallSnapshotReply reply = new InstallSnapshotReply(
399                     currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true);
400
401             if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
402                     installSnapshot.getLastChunkHashCode())){
403                 Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(),
404                         new ArrayList<ReplicatedLogEntry>(),
405                         installSnapshot.getLastIncludedIndex(),
406                         installSnapshot.getLastIncludedTerm(),
407                         installSnapshot.getLastIncludedIndex(),
408                         installSnapshot.getLastIncludedTerm(),
409                         context.getTermInformation().getCurrentTerm(),
410                         context.getTermInformation().getVotedFor(),
411                         context.getPeerServerInfo(true));
412
413                 ApplySnapshot.Callback applySnapshotCallback = new ApplySnapshot.Callback() {
414                     @Override
415                     public void onSuccess() {
416                         LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
417
418                         sender.tell(reply, actor());
419                     }
420
421                     @Override
422                     public void onFailure() {
423                         sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), -1, false), actor());
424                     }
425                 };
426
427                 actor().tell(new ApplySnapshot(snapshot, applySnapshotCallback), actor());
428
429                 snapshotTracker = null;
430             } else {
431                 LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
432
433                 sender.tell(reply, actor());
434             }
435         } catch (SnapshotTracker.InvalidChunkException e) {
436             LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e);
437
438             sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
439                     -1, false), actor());
440             snapshotTracker = null;
441
442         } catch (Exception e){
443             LOG.error("{}: Exception in InstallSnapshot of follower", logName(), e);
444
445             //send reply with success as false. The chunk will be sent again on failure
446             sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
447                     installSnapshot.getChunkIndex(), false), actor());
448
449         }
450     }
451
452     @Override
453     public void close() {
454         stopElection();
455     }
456
457     @VisibleForTesting
458     SnapshotTracker getSnapshotTracker(){
459         return snapshotTracker;
460     }
461 }