Fix AppendEntry logic when prevLogIndex and prevLogTerm is -1
[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.base.messages.FollowerInitialSyncUpStatus;
21 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
22 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
23 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
24 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
25 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
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
41
42     private SnapshotTracker snapshotTracker = null;
43
44     private final InitialSyncStatusTracker initialSyncStatusTracker;
45
46     public Follower(RaftActorContext context) {
47         super(context, RaftState.Follower);
48
49         initialSyncStatusTracker = new InitialSyncStatusTracker(context.getActor());
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 which 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                     LOG.debug("{}: Removing entries from log starting at {}", logName(),
166                                 matchEntry.getIndex());
167
168                     // Entries do not match so remove all subsequent entries
169                     context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex());
170                     break;
171                 }
172             }
173
174             lastIndex = lastIndex();
175             LOG.debug("{}: After cleanup entries to be added from = {}", logName(),
176                         (addEntriesFrom + lastIndex));
177
178             // 4. Append any new entries not already in the log
179             for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) {
180                 ReplicatedLogEntry entry = appendEntries.getEntries().get(i);
181
182                 LOG.debug("{}: Append entry to log {}", logName(), entry.getData());
183
184                 context.getReplicatedLog().appendAndPersist(entry);
185             }
186
187             LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
188         }
189
190         // 5. If leaderCommit > commitIndex, set commitIndex =
191         // min(leaderCommit, index of last new entry)
192
193         lastIndex = lastIndex();
194         long prevCommitIndex = context.getCommitIndex();
195
196         context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex));
197
198         if (prevCommitIndex != context.getCommitIndex()) {
199             LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex());
200         }
201
202         // If commitIndex > lastApplied: increment lastApplied, apply
203         // log[lastApplied] to state machine (ยง5.3)
204         // check if there are any entries to be applied. last-applied can be equal to last-index
205         if (appendEntries.getLeaderCommit() > context.getLastApplied() &&
206             context.getLastApplied() < lastIndex) {
207             if(LOG.isDebugEnabled()) {
208                 LOG.debug("{}: applyLogToStateMachine, " +
209                         "appendEntries.getLeaderCommit(): {}," +
210                         "context.getLastApplied(): {}, lastIndex(): {}", logName(),
211                     appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex);
212             }
213
214             applyLogToStateMachine(appendEntries.getLeaderCommit());
215         }
216
217         AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
218             lastIndex, lastTerm(), context.getPayloadVersion());
219
220         if(LOG.isTraceEnabled()) {
221             LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply);
222         } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
223             LOG.debug("{}: handleAppendEntries returning : {}", logName(), reply);
224         }
225
226         sender.tell(reply, actor());
227
228         if (!context.getSnapshotManager().isCapturing()) {
229             super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex());
230         }
231
232         return this;
233     }
234
235     private boolean isOutOfSync(AppendEntries appendEntries) {
236
237         long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
238         boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex());
239         long lastIndex = lastIndex();
240         int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
241         boolean outOfSync = true;
242
243         if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) {
244
245             // The follower's log is out of sync because the leader does have
246             // an entry at prevLogIndex and this follower has no entries in
247             // it's log.
248
249             LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}",
250                         logName(), appendEntries.getPrevLogIndex());
251         } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) {
252
253             // The follower's log is out of sync because the Leader's
254             // prevLogIndex entry was not found in it's log
255
256             LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it",
257                         logName(), appendEntries.getPrevLogIndex());
258         } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) {
259
260             // The follower's log is out of sync because the Leader's
261             // prevLogIndex entry does exist in the follower's log but it has
262             // a different term in it
263
264             LOG.debug(
265                     "{}: Cannot append entries because previous entry term {}  is not equal to append entries prevLogTerm {}",
266                     logName(), prevLogTerm, appendEntries.getPrevLogTerm());
267         } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
268                 && appendEntries.getReplicatedToAllIndex() != -1
269                 && !isLogEntryPresent(appendEntries.getReplicatedToAllIndex())) {
270             // This append entry comes from a leader who has it's log aggressively trimmed and so does not have
271             // the previous entry in it's in-memory journal
272
273             LOG.debug(
274                     "{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the in-memory journal",
275                     logName(), appendEntries.getReplicatedToAllIndex());
276         } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
277                 && appendEntries.getReplicatedToAllIndex() != -1 && numLogEntries > 0 &&
278                 !isLogEntryPresent(appendEntries.getEntries().get(0).getIndex() - 1)){
279             LOG.debug(
280                     "{}: Cannot append entries because the calculated previousIndex {} was not found in the in-memory journal",
281                     logName(), appendEntries.getEntries().get(0).getIndex() - 1);
282         } else {
283             outOfSync = false;
284         }
285         return outOfSync;
286     }
287
288     @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
289         AppendEntriesReply appendEntriesReply) {
290         return this;
291     }
292
293     @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
294         RequestVoteReply requestVoteReply) {
295         return this;
296     }
297
298     @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
299
300         Object message = fromSerializableMessage(originalMessage);
301
302         if (message instanceof RaftRPC) {
303             RaftRPC rpc = (RaftRPC) message;
304             // If RPC request or response contains term T > currentTerm:
305             // set currentTerm = T, convert to follower (ยง5.1)
306             // This applies to all RPC messages and responses
307             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
308                 LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term",
309                         logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
310
311                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
312             }
313         }
314
315         if (message instanceof ElectionTimeout) {
316             LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName());
317             return switchBehavior(new Candidate(context));
318
319         } else if (message instanceof InstallSnapshot) {
320             InstallSnapshot installSnapshot = (InstallSnapshot) message;
321             handleInstallSnapshot(sender, installSnapshot);
322         }
323
324         scheduleElection(electionDuration());
325
326         return super.handleMessage(sender, message);
327     }
328
329     private void handleInstallSnapshot(ActorRef sender, InstallSnapshot installSnapshot) {
330
331         LOG.debug("{}: InstallSnapshot received from leader {}, datasize: {} , Chunk: {}/{}",
332                     logName(), installSnapshot.getLeaderId(), installSnapshot.getData().size(),
333                     installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks());
334
335         if(snapshotTracker == null){
336             snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks());
337         }
338
339         updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId());
340
341         try {
342             if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
343                     installSnapshot.getLastChunkHashCode())){
344                 Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(),
345                         new ArrayList<ReplicatedLogEntry>(),
346                         installSnapshot.getLastIncludedIndex(),
347                         installSnapshot.getLastIncludedTerm(),
348                         installSnapshot.getLastIncludedIndex(),
349                         installSnapshot.getLastIncludedTerm());
350
351                 actor().tell(new ApplySnapshot(snapshot), actor());
352
353                 snapshotTracker = null;
354
355             }
356
357             InstallSnapshotReply reply = new InstallSnapshotReply(
358                     currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true);
359
360             LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
361
362             sender.tell(reply, actor());
363
364         } catch (SnapshotTracker.InvalidChunkException e) {
365             LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e);
366
367             sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
368                     -1, false), actor());
369             snapshotTracker = null;
370
371         } catch (Exception e){
372             LOG.error("{}: Exception in InstallSnapshot of follower", logName(), e);
373
374             //send reply with success as false. The chunk will be sent again on failure
375             sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
376                     installSnapshot.getChunkIndex(), false), actor());
377
378         }
379     }
380
381     @Override
382     public void close() throws Exception {
383         stopElection();
384     }
385
386     @VisibleForTesting
387     SnapshotTracker getSnapshotTracker(){
388         return snapshotTracker;
389     }
390
391     private class InitialSyncStatusTracker {
392
393         private static final long INVALID_LOG_INDEX = -2L;
394         private long initialLeaderCommit = INVALID_LOG_INDEX;
395         private boolean initialSyncUpDone = false;
396         private String syncedLeaderId = null;
397         private final ActorRef actor;
398
399         public InitialSyncStatusTracker(ActorRef actor) {
400             this.actor = actor;
401         }
402
403         public void update(String leaderId, long leaderCommit, long commitIndex){
404
405             if(!leaderId.equals(syncedLeaderId)){
406                 initialSyncUpDone = false;
407                 initialLeaderCommit = INVALID_LOG_INDEX;
408                 syncedLeaderId = leaderId;
409             }
410
411             if(!initialSyncUpDone){
412                 if(initialLeaderCommit == INVALID_LOG_INDEX){
413                     actor.tell(new FollowerInitialSyncUpStatus(false, getId()), ActorRef.noSender());
414                     initialLeaderCommit = leaderCommit;
415                 } else if(commitIndex >= initialLeaderCommit){
416                     actor.tell(new FollowerInitialSyncUpStatus(true, getId()), ActorRef.noSender());
417                     initialSyncUpDone = true;
418                 }
419             }
420         }
421     }
422 }