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