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.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;
26 import java.util.ArrayList;
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 {
39 private ByteString snapshotChunksCollected = ByteString.EMPTY;
41 public Follower(RaftActorContext context) {
44 scheduleElection(electionDuration());
47 @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender,
48 AppendEntries appendEntries) {
50 if(appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
51 if(LOG.isDebugEnabled()) {
52 LOG.debug(appendEntries.toString());
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
60 // 1. Reply false if term < currentTerm (§5.1)
61 // This is handled in the appendEntries method of the base class
63 // If we got here then we do appear to be talking to the leader
64 leaderId = appendEntries.getLeaderId();
66 // 2. Reply false if log doesn’t contain an entry at prevLogIndex
67 // whose term matches prevLogTerm (§5.3)
69 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
70 .get(appendEntries.getPrevLogIndex());
73 boolean outOfSync = true;
75 // First check if the logs are in sync or not
77 && appendEntries.getPrevLogIndex() != -1) {
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
83 if(LOG.isDebugEnabled()) {
84 LOG.debug("The followers log is empty and the senders prevLogIndex is {}",
85 appendEntries.getPrevLogIndex());
88 } else if (lastIndex() > -1
89 && appendEntries.getPrevLogIndex() != -1
90 && previousEntry == null) {
92 // The follower's log is out of sync because the Leader's
93 // prevLogIndex entry was not found in it's log
95 if(LOG.isDebugEnabled()) {
96 LOG.debug("The log is not empty but the prevLogIndex {} was not found in it",
97 appendEntries.getPrevLogIndex());
100 } else if (lastIndex() > -1
101 && previousEntry != null
102 && previousEntry.getTerm()!= appendEntries.getPrevLogTerm()) {
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
108 if(LOG.isDebugEnabled()) {
110 "Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}"
111 , previousEntry.getTerm()
112 , appendEntries.getPrevLogTerm());
119 // We found that the log was out of sync so just send a negative
121 if(LOG.isDebugEnabled()) {
122 LOG.debug("Follower is out-of-sync, " +
123 "so sending negative reply, lastIndex():{}, lastTerm():{}",
124 lastIndex(), lastTerm()
128 new AppendEntriesReply(context.getId(), currentTerm(), false,
129 lastIndex(), lastTerm()), actor()
134 if (appendEntries.getEntries() != null
135 && appendEntries.getEntries().size() > 0) {
136 if(LOG.isDebugEnabled()) {
138 "Number of entries to be appended = {}", appendEntries.getEntries().size()
142 // 3. If an existing entry conflicts with a new one (same index
143 // but different terms), delete the existing entry and all that
145 int addEntriesFrom = 0;
146 if (context.getReplicatedLog().size() > 0) {
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());
153 if (newEntry == null) {
154 //newEntry not found in the log
158 if (newEntry.getTerm() == matchEntry
163 if(LOG.isDebugEnabled()) {
165 "Removing entries from log starting at {}", matchEntry.getIndex()
169 // Entries do not match so remove all subsequent entries
170 context.getReplicatedLog()
171 .removeFromAndPersist(matchEntry.getIndex());
176 if(LOG.isDebugEnabled()) {
177 LOG.debug("After cleanup entries to be added from = {}", (addEntriesFrom + lastIndex())
181 // 4. Append any new entries not already in the log
182 for (int i = addEntriesFrom;
183 i < appendEntries.getEntries().size(); i++) {
185 if(LOG.isDebugEnabled()) {
186 LOG.debug("Append entry to log {}", appendEntries.getEntries().get(i).getData());
188 context.getReplicatedLog().appendAndPersist(appendEntries.getEntries().get(i));
191 if(LOG.isDebugEnabled()) {
192 LOG.debug("Log size is now {}", context.getReplicatedLog().size());
197 // 5. If leaderCommit > commitIndex, set commitIndex =
198 // min(leaderCommit, index of last new entry)
200 long prevCommitIndex = context.getCommitIndex();
202 context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(),
203 context.getReplicatedLog().lastIndex()));
205 if (prevCommitIndex != context.getCommitIndex()) {
206 if(LOG.isDebugEnabled()) {
207 LOG.debug("Commit index set to {}", context.getCommitIndex());
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()
224 applyLogToStateMachine(appendEntries.getLeaderCommit());
227 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), true,
228 lastIndex(), lastTerm()), actor());
233 @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
234 AppendEntriesReply appendEntriesReply) {
238 @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
239 RequestVoteReply requestVoteReply) {
243 @Override public RaftState state() {
244 return RaftState.Follower;
247 @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
249 Object message = fromSerializableMessage(originalMessage);
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);
261 if (message instanceof ElectionTimeout) {
262 return switchBehavior(new Candidate(context));
264 } else if (message instanceof InstallSnapshot) {
265 InstallSnapshot installSnapshot = (InstallSnapshot) message;
266 handleInstallSnapshot(sender, installSnapshot);
269 scheduleElection(electionDuration());
271 return super.handleMessage(sender, message);
274 private void handleInstallSnapshot(ActorRef sender, InstallSnapshot installSnapshot) {
276 if(LOG.isDebugEnabled()) {
277 LOG.debug("InstallSnapshot received by follower " +
278 "datasize:{} , Chunk:{}/{}", installSnapshot.getData().size(),
279 installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks()
284 if (installSnapshot.getChunkIndex() == installSnapshot.getTotalChunks()) {
285 // this is the last chunk, create a snapshot object and apply
287 snapshotChunksCollected = snapshotChunksCollected.concat(installSnapshot.getData());
288 if(LOG.isDebugEnabled()) {
289 LOG.debug("Last chunk received: snapshotChunksCollected.size:{}",
290 snapshotChunksCollected.size());
293 Snapshot snapshot = Snapshot.create(snapshotChunksCollected.toByteArray(),
294 new ArrayList<ReplicatedLogEntry>(),
295 installSnapshot.getLastIncludedIndex(),
296 installSnapshot.getLastIncludedTerm(),
297 installSnapshot.getLastIncludedIndex(),
298 installSnapshot.getLastIncludedTerm());
300 actor().tell(new ApplySnapshot(snapshot), actor());
303 // we have more to go
304 snapshotChunksCollected = snapshotChunksCollected.concat(installSnapshot.getData());
306 if(LOG.isDebugEnabled()) {
307 LOG.debug("Chunk={},snapshotChunksCollected.size:{}",
308 installSnapshot.getChunkIndex(), snapshotChunksCollected.size());
312 sender.tell(new InstallSnapshotReply(
313 currentTerm(), context.getId(), installSnapshot.getChunkIndex(),
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());
324 @Override public void close() throws Exception {