2 * Copyright (c) 2014 Cisco Systems, Inc. and others. All rights reserved.
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
9 package org.opendaylight.controller.cluster.raft.behaviors;
11 import akka.actor.ActorRef;
12 import com.google.common.annotations.VisibleForTesting;
13 import com.google.protobuf.ByteString;
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.Snapshot;
19 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
20 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
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;
29 * The behavior of a RaftActor in the Follower state
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
38 public class Follower extends AbstractRaftActorBehavior {
40 private SnapshotTracker snapshotTracker = null;
42 public Follower(RaftActorContext context) {
45 scheduleElection(electionDuration());
48 @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender,
49 AppendEntries appendEntries) {
51 if(appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
52 if(LOG.isDebugEnabled()) {
53 LOG.debug(appendEntries.toString());
57 // TODO : Refactor this method into a bunch of smaller methods
58 // to make it easier to read. Before refactoring ensure tests
59 // cover the code properly
61 // 1. Reply false if term < currentTerm (§5.1)
62 // This is handled in the appendEntries method of the base class
64 // If we got here then we do appear to be talking to the leader
65 leaderId = appendEntries.getLeaderId();
67 // 2. Reply false if log doesn’t contain an entry at prevLogIndex
68 // whose term matches prevLogTerm (§5.3)
70 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
71 .get(appendEntries.getPrevLogIndex());
74 boolean outOfSync = true;
76 // First check if the logs are in sync or not
78 && appendEntries.getPrevLogIndex() != -1) {
80 // The follower's log is out of sync because the leader does have
81 // an entry at prevLogIndex and this follower has no entries in
84 if(LOG.isDebugEnabled()) {
85 LOG.debug("The followers log is empty and the senders prevLogIndex is {}",
86 appendEntries.getPrevLogIndex());
89 } else if (lastIndex() > -1
90 && appendEntries.getPrevLogIndex() != -1
91 && previousEntry == null) {
93 // The follower's log is out of sync because the Leader's
94 // prevLogIndex entry was not found in it's log
96 if(LOG.isDebugEnabled()) {
97 LOG.debug("The log is not empty but the prevLogIndex {} was not found in it",
98 appendEntries.getPrevLogIndex());
101 } else if (lastIndex() > -1
102 && previousEntry != null
103 && previousEntry.getTerm()!= appendEntries.getPrevLogTerm()) {
105 // The follower's log is out of sync because the Leader's
106 // prevLogIndex entry does exist in the follower's log but it has
107 // a different term in it
109 if(LOG.isDebugEnabled()) {
111 "Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}"
112 , previousEntry.getTerm()
113 , appendEntries.getPrevLogTerm());
120 // We found that the log was out of sync so just send a negative
122 if(LOG.isDebugEnabled()) {
123 LOG.debug("Follower is out-of-sync, " +
124 "so sending negative reply, lastIndex():{}, lastTerm():{}",
125 lastIndex(), lastTerm()
129 new AppendEntriesReply(context.getId(), currentTerm(), false,
130 lastIndex(), lastTerm()), actor()
135 if (appendEntries.getEntries() != null
136 && appendEntries.getEntries().size() > 0) {
137 if(LOG.isDebugEnabled()) {
139 "Number of entries to be appended = {}", appendEntries.getEntries().size()
143 // 3. If an existing entry conflicts with a new one (same index
144 // but different terms), delete the existing entry and all that
146 int addEntriesFrom = 0;
147 if (context.getReplicatedLog().size() > 0) {
149 // Find the entry up until which the one that is not in the follower's log
150 for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) {
151 ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
152 ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex());
154 if (newEntry == null) {
155 //newEntry not found in the log
159 if (newEntry.getTerm() == matchEntry
164 if(LOG.isDebugEnabled()) {
166 "Removing entries from log starting at {}", matchEntry.getIndex()
170 // Entries do not match so remove all subsequent entries
171 context.getReplicatedLog()
172 .removeFromAndPersist(matchEntry.getIndex());
177 if(LOG.isDebugEnabled()) {
178 LOG.debug("After cleanup entries to be added from = {}", (addEntriesFrom + lastIndex())
182 // 4. Append any new entries not already in the log
183 for (int i = addEntriesFrom;
184 i < appendEntries.getEntries().size(); i++) {
186 if(LOG.isDebugEnabled()) {
187 LOG.debug("Append entry to log {}", appendEntries.getEntries().get(i).getData());
189 context.getReplicatedLog().appendAndPersist(appendEntries.getEntries().get(i));
192 if(LOG.isDebugEnabled()) {
193 LOG.debug("Log size is now {}", context.getReplicatedLog().size());
198 // 5. If leaderCommit > commitIndex, set commitIndex =
199 // min(leaderCommit, index of last new entry)
201 long prevCommitIndex = context.getCommitIndex();
203 context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(),
204 context.getReplicatedLog().lastIndex()));
206 if (prevCommitIndex != context.getCommitIndex()) {
207 if(LOG.isDebugEnabled()) {
208 LOG.debug("Commit index set to {}", context.getCommitIndex());
212 // If commitIndex > lastApplied: increment lastApplied, apply
213 // log[lastApplied] to state machine (§5.3)
214 // check if there are any entries to be applied. last-applied can be equal to last-index
215 if (appendEntries.getLeaderCommit() > context.getLastApplied() &&
216 context.getLastApplied() < lastIndex()) {
217 if(LOG.isDebugEnabled()) {
218 LOG.debug("applyLogToStateMachine, " +
219 "appendEntries.getLeaderCommit():{}," +
220 "context.getLastApplied():{}, lastIndex():{}",
221 appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex()
225 applyLogToStateMachine(appendEntries.getLeaderCommit());
228 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), true,
229 lastIndex(), lastTerm()), actor());
234 @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
235 AppendEntriesReply appendEntriesReply) {
239 @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
240 RequestVoteReply requestVoteReply) {
244 @Override public RaftState state() {
245 return RaftState.Follower;
248 @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
250 Object message = fromSerializableMessage(originalMessage);
252 if (message instanceof RaftRPC) {
253 RaftRPC rpc = (RaftRPC) message;
254 // If RPC request or response contains term T > currentTerm:
255 // set currentTerm = T, convert to follower (§5.1)
256 // This applies to all RPC messages and responses
257 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
258 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
262 if (message instanceof ElectionTimeout) {
263 return switchBehavior(new Candidate(context));
265 } else if (message instanceof InstallSnapshot) {
266 InstallSnapshot installSnapshot = (InstallSnapshot) message;
267 handleInstallSnapshot(sender, installSnapshot);
270 scheduleElection(electionDuration());
272 return super.handleMessage(sender, message);
275 private void handleInstallSnapshot(ActorRef sender, InstallSnapshot installSnapshot) {
277 if(LOG.isDebugEnabled()) {
278 LOG.debug("InstallSnapshot received by follower " +
279 "datasize:{} , Chunk:{}/{}", installSnapshot.getData().size(),
280 installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks()
284 if(snapshotTracker == null){
285 snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks());
289 if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
290 installSnapshot.getLastChunkHashCode())){
291 Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(),
292 new ArrayList<ReplicatedLogEntry>(),
293 installSnapshot.getLastIncludedIndex(),
294 installSnapshot.getLastIncludedTerm(),
295 installSnapshot.getLastIncludedIndex(),
296 installSnapshot.getLastIncludedTerm());
298 actor().tell(new ApplySnapshot(snapshot), actor());
300 snapshotTracker = null;
304 sender.tell(new InstallSnapshotReply(
305 currentTerm(), context.getId(), installSnapshot.getChunkIndex(),
308 } catch (SnapshotTracker.InvalidChunkException e) {
310 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
311 -1, false), actor());
312 snapshotTracker = null;
314 } catch (Exception e){
316 LOG.error(e, "Exception in InstallSnapshot of follower:");
317 //send reply with success as false. The chunk will be sent again on failure
318 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
319 installSnapshot.getChunkIndex(), false), actor());
324 @Override public void close() throws Exception {
329 ByteString getSnapshotChunksCollected(){
330 return snapshotTracker != null ? snapshotTracker.getCollectedChunks() : ByteString.EMPTY;