Merge "BUG 2221 : Add metering to ShardTransaction actor"
[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.protobuf.ByteString;
13 import org.opendaylight.controller.cluster.raft.RaftActorContext;
14 import org.opendaylight.controller.cluster.raft.RaftState;
15 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
16 import org.opendaylight.controller.cluster.raft.Snapshot;
17 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
18 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
19 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
20 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
21 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
22 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
23 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
24 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
25
26 import java.util.ArrayList;
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     private ByteString snapshotChunksCollected = ByteString.EMPTY;
40
41     public Follower(RaftActorContext context) {
42         super(context);
43
44         scheduleElection(electionDuration());
45     }
46
47     @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender,
48         AppendEntries appendEntries) {
49
50         if(appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
51             if(LOG.isDebugEnabled()) {
52                 LOG.debug(appendEntries.toString());
53             }
54         }
55
56         // TODO : Refactor this method into a bunch of smaller methods
57         // to make it easier to read. Before refactoring ensure tests
58         // cover the code properly
59
60         // 1. Reply false if term < currentTerm (§5.1)
61         // This is handled in the appendEntries method of the base class
62
63         // If we got here then we do appear to be talking to the leader
64         leaderId = appendEntries.getLeaderId();
65
66         // 2. Reply false if log doesn’t contain an entry at prevLogIndex
67         // whose term matches prevLogTerm (§5.3)
68
69         ReplicatedLogEntry previousEntry = context.getReplicatedLog()
70             .get(appendEntries.getPrevLogIndex());
71
72
73         boolean outOfSync = true;
74
75         // First check if the logs are in sync or not
76         if (lastIndex() == -1
77             && appendEntries.getPrevLogIndex() != -1) {
78
79             // The follower's log is out of sync because the leader does have
80             // an entry at prevLogIndex and this follower has no entries in
81             // it's log.
82
83             if(LOG.isDebugEnabled()) {
84                 LOG.debug("The followers log is empty and the senders prevLogIndex is {}",
85                     appendEntries.getPrevLogIndex());
86             }
87
88         } else if (lastIndex() > -1
89             && appendEntries.getPrevLogIndex() != -1
90             && previousEntry == null) {
91
92             // The follower's log is out of sync because the Leader's
93             // prevLogIndex entry was not found in it's log
94
95             if(LOG.isDebugEnabled()) {
96                 LOG.debug("The log is not empty but the prevLogIndex {} was not found in it",
97                     appendEntries.getPrevLogIndex());
98             }
99
100         } else if (lastIndex() > -1
101             && previousEntry != null
102             && previousEntry.getTerm()!= appendEntries.getPrevLogTerm()) {
103
104             // The follower's log is out of sync because the Leader's
105             // prevLogIndex entry does exist in the follower's log but it has
106             // a different term in it
107
108             if(LOG.isDebugEnabled()) {
109                 LOG.debug(
110                     "Cannot append entries because previous entry term {}  is not equal to append entries prevLogTerm {}"
111                     , previousEntry.getTerm()
112                     , appendEntries.getPrevLogTerm());
113             }
114         } else {
115             outOfSync = false;
116         }
117
118         if (outOfSync) {
119             // We found that the log was out of sync so just send a negative
120             // reply and return
121             if(LOG.isDebugEnabled()) {
122                 LOG.debug("Follower is out-of-sync, " +
123                         "so sending negative reply, lastIndex():{}, lastTerm():{}",
124                     lastIndex(), lastTerm()
125                 );
126             }
127             sender.tell(
128                 new AppendEntriesReply(context.getId(), currentTerm(), false,
129                     lastIndex(), lastTerm()), actor()
130             );
131             return this;
132         }
133
134         if (appendEntries.getEntries() != null
135             && appendEntries.getEntries().size() > 0) {
136             if(LOG.isDebugEnabled()) {
137                 LOG.debug(
138                     "Number of entries to be appended = {}", appendEntries.getEntries().size()
139                 );
140             }
141
142             // 3. If an existing entry conflicts with a new one (same index
143             // but different terms), delete the existing entry and all that
144             // follow it (§5.3)
145             int addEntriesFrom = 0;
146             if (context.getReplicatedLog().size() > 0) {
147
148                 // Find the entry up until which the one that is not in the follower's log
149                 for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) {
150                     ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
151                     ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex());
152
153                     if (newEntry == null) {
154                         //newEntry not found in the log
155                         break;
156                     }
157
158                     if (newEntry.getTerm() == matchEntry
159                         .getTerm()) {
160                         continue;
161                     }
162
163                     if(LOG.isDebugEnabled()) {
164                         LOG.debug(
165                             "Removing entries from log starting at {}", matchEntry.getIndex()
166                         );
167                     }
168
169                     // Entries do not match so remove all subsequent entries
170                     context.getReplicatedLog()
171                         .removeFromAndPersist(matchEntry.getIndex());
172                     break;
173                 }
174             }
175
176             if(LOG.isDebugEnabled()) {
177                 LOG.debug("After cleanup entries to be added from = {}", (addEntriesFrom + lastIndex())
178                 );
179             }
180
181             // 4. Append any new entries not already in the log
182             for (int i = addEntriesFrom;
183                  i < appendEntries.getEntries().size(); i++) {
184
185                 if(LOG.isDebugEnabled()) {
186                     LOG.debug("Append entry to log {}", appendEntries.getEntries().get(i).getData());
187                 }
188                 context.getReplicatedLog().appendAndPersist(appendEntries.getEntries().get(i));
189             }
190
191             if(LOG.isDebugEnabled()) {
192                 LOG.debug("Log size is now {}", context.getReplicatedLog().size());
193             }
194         }
195
196
197         // 5. If leaderCommit > commitIndex, set commitIndex =
198         // min(leaderCommit, index of last new entry)
199
200         long prevCommitIndex = context.getCommitIndex();
201
202         context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(),
203             context.getReplicatedLog().lastIndex()));
204
205         if (prevCommitIndex != context.getCommitIndex()) {
206             if(LOG.isDebugEnabled()) {
207                 LOG.debug("Commit index set to {}", context.getCommitIndex());
208             }
209         }
210
211         // If commitIndex > lastApplied: increment lastApplied, apply
212         // log[lastApplied] to state machine (§5.3)
213         // check if there are any entries to be applied. last-applied can be equal to last-index
214         if (appendEntries.getLeaderCommit() > context.getLastApplied() &&
215             context.getLastApplied() < lastIndex()) {
216             if(LOG.isDebugEnabled()) {
217                 LOG.debug("applyLogToStateMachine, " +
218                         "appendEntries.getLeaderCommit():{}," +
219                         "context.getLastApplied():{}, lastIndex():{}",
220                     appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex()
221                 );
222             }
223
224             applyLogToStateMachine(appendEntries.getLeaderCommit());
225         }
226
227         sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), true,
228             lastIndex(), lastTerm()), actor());
229
230         return this;
231     }
232
233     @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
234         AppendEntriesReply appendEntriesReply) {
235         return this;
236     }
237
238     @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
239         RequestVoteReply requestVoteReply) {
240         return this;
241     }
242
243     @Override public RaftState state() {
244         return RaftState.Follower;
245     }
246
247     @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
248
249         Object message = fromSerializableMessage(originalMessage);
250
251         if (message instanceof RaftRPC) {
252             RaftRPC rpc = (RaftRPC) message;
253             // If RPC request or response contains term T > currentTerm:
254             // set currentTerm = T, convert to follower (§5.1)
255             // This applies to all RPC messages and responses
256             if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
257                 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
258             }
259         }
260
261         if (message instanceof ElectionTimeout) {
262             return switchBehavior(new Candidate(context));
263
264         } else if (message instanceof InstallSnapshot) {
265             InstallSnapshot installSnapshot = (InstallSnapshot) message;
266             handleInstallSnapshot(sender, installSnapshot);
267         }
268
269         scheduleElection(electionDuration());
270
271         return super.handleMessage(sender, message);
272     }
273
274     private void handleInstallSnapshot(ActorRef sender, InstallSnapshot installSnapshot) {
275
276         if(LOG.isDebugEnabled()) {
277             LOG.debug("InstallSnapshot received by follower " +
278                     "datasize:{} , Chunk:{}/{}", installSnapshot.getData().size(),
279                 installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks()
280             );
281         }
282
283         try {
284             if (installSnapshot.getChunkIndex() == installSnapshot.getTotalChunks()) {
285                 // this is the last chunk, create a snapshot object and apply
286
287                 snapshotChunksCollected = snapshotChunksCollected.concat(installSnapshot.getData());
288                 if(LOG.isDebugEnabled()) {
289                     LOG.debug("Last chunk received: snapshotChunksCollected.size:{}",
290                             snapshotChunksCollected.size());
291                 }
292
293                 Snapshot snapshot = Snapshot.create(snapshotChunksCollected.toByteArray(),
294                     new ArrayList<ReplicatedLogEntry>(),
295                     installSnapshot.getLastIncludedIndex(),
296                     installSnapshot.getLastIncludedTerm(),
297                     installSnapshot.getLastIncludedIndex(),
298                     installSnapshot.getLastIncludedTerm());
299
300                 actor().tell(new ApplySnapshot(snapshot), actor());
301
302             } else {
303                 // we have more to go
304                 snapshotChunksCollected = snapshotChunksCollected.concat(installSnapshot.getData());
305
306                 if(LOG.isDebugEnabled()) {
307                     LOG.debug("Chunk={},snapshotChunksCollected.size:{}",
308                         installSnapshot.getChunkIndex(), snapshotChunksCollected.size());
309                 }
310             }
311
312             sender.tell(new InstallSnapshotReply(
313                 currentTerm(), context.getId(), installSnapshot.getChunkIndex(),
314                 true), actor());
315
316         } catch (Exception e) {
317             LOG.error(e, "Exception in InstallSnapshot of follower:");
318             //send reply with success as false. The chunk will be sent again on failure
319             sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
320                 installSnapshot.getChunkIndex(), false), actor());
321         }
322     }
323
324     @Override public void close() throws Exception {
325         stopElection();
326     }
327 }