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