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 private boolean isLogEntryPresent(long index){
49 if(index == context.getReplicatedLog().getSnapshotIndex()){
53 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
56 return previousEntry != null;
60 private long getLogEntryTerm(long index){
61 if(index == context.getReplicatedLog().getSnapshotIndex()){
62 return context.getReplicatedLog().getSnapshotTerm();
65 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
68 if(previousEntry != null){
69 return previousEntry.getTerm();
75 @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender,
76 AppendEntries appendEntries) {
78 if(appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
79 if(LOG.isDebugEnabled()) {
80 LOG.debug("{}: handleAppendEntries: {}", context.getId(), appendEntries);
84 // TODO : Refactor this method into a bunch of smaller methods
85 // to make it easier to read. Before refactoring ensure tests
86 // cover the code properly
88 // 1. Reply false if term < currentTerm (§5.1)
89 // This is handled in the appendEntries method of the base class
91 // If we got here then we do appear to be talking to the leader
92 leaderId = appendEntries.getLeaderId();
94 // 2. Reply false if log doesn’t contain an entry at prevLogIndex
95 // whose term matches prevLogTerm (§5.3)
97 long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
98 boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex());
101 boolean outOfSync = true;
103 // First check if the logs are in sync or not
104 if (lastIndex() == -1
105 && appendEntries.getPrevLogIndex() != -1) {
107 // The follower's log is out of sync because the leader does have
108 // an entry at prevLogIndex and this follower has no entries in
111 if(LOG.isDebugEnabled()) {
112 LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}",
113 context.getId(), appendEntries.getPrevLogIndex());
116 } else if (lastIndex() > -1
117 && appendEntries.getPrevLogIndex() != -1
118 && !prevEntryPresent) {
120 // The follower's log is out of sync because the Leader's
121 // prevLogIndex entry was not found in it's log
123 if(LOG.isDebugEnabled()) {
124 LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it",
125 context.getId(), appendEntries.getPrevLogIndex());
128 } else if (lastIndex() > -1
130 && prevLogTerm != appendEntries.getPrevLogTerm()) {
132 // The follower's log is out of sync because the Leader's
133 // prevLogIndex entry does exist in the follower's log but it has
134 // a different term in it
136 if (LOG.isDebugEnabled()) {
138 "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}"
139 , context.getId(), prevLogTerm
140 , appendEntries.getPrevLogTerm());
147 // We found that the log was out of sync so just send a negative
149 if(LOG.isDebugEnabled()) {
150 LOG.debug("{}: Follower ({}) is out-of-sync, " +
151 "so sending negative reply, lastIndex():{}, lastTerm():{}",
152 context.getId(), context.getId(), lastIndex(), lastTerm()
156 new AppendEntriesReply(context.getId(), currentTerm(), false,
157 lastIndex(), lastTerm()), actor()
162 if (appendEntries.getEntries() != null
163 && appendEntries.getEntries().size() > 0) {
164 if(LOG.isDebugEnabled()) {
165 LOG.debug("{}: Number of entries to be appended = {}", context.getId(),
166 appendEntries.getEntries().size());
169 // 3. If an existing entry conflicts with a new one (same index
170 // but different terms), delete the existing entry and all that
172 int addEntriesFrom = 0;
173 if (context.getReplicatedLog().size() > 0) {
175 // Find the entry up until which the one that is not in the follower's log
176 for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) {
177 ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
178 ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex());
180 if (newEntry == null) {
181 //newEntry not found in the log
185 if (newEntry.getTerm() == matchEntry
190 if(LOG.isDebugEnabled()) {
191 LOG.debug("{}: Removing entries from log starting at {}", context.getId(),
192 matchEntry.getIndex());
195 // Entries do not match so remove all subsequent entries
196 context.getReplicatedLog()
197 .removeFromAndPersist(matchEntry.getIndex());
202 if(LOG.isDebugEnabled()) {
203 LOG.debug("{}: After cleanup entries to be added from = {}", context.getId(),
204 (addEntriesFrom + lastIndex()));
207 // 4. Append any new entries not already in the log
208 for (int i = addEntriesFrom;
209 i < appendEntries.getEntries().size(); i++) {
211 if(LOG.isDebugEnabled()) {
212 LOG.debug("{}: Append entry to log {}", context.getId(),
213 appendEntries.getEntries().get(i).getData());
215 context.getReplicatedLog().appendAndPersist(appendEntries.getEntries().get(i));
218 if(LOG.isDebugEnabled()) {
219 LOG.debug("{}: Log size is now {}", context.getId(), context.getReplicatedLog().size());
224 // 5. If leaderCommit > commitIndex, set commitIndex =
225 // min(leaderCommit, index of last new entry)
227 long prevCommitIndex = context.getCommitIndex();
229 context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(),
230 context.getReplicatedLog().lastIndex()));
232 if (prevCommitIndex != context.getCommitIndex()) {
233 if(LOG.isDebugEnabled()) {
234 LOG.debug("{}: Commit index set to {}", context.getId(), context.getCommitIndex());
238 // If commitIndex > lastApplied: increment lastApplied, apply
239 // log[lastApplied] to state machine (§5.3)
240 // check if there are any entries to be applied. last-applied can be equal to last-index
241 if (appendEntries.getLeaderCommit() > context.getLastApplied() &&
242 context.getLastApplied() < lastIndex()) {
243 if(LOG.isDebugEnabled()) {
244 LOG.debug("{}: applyLogToStateMachine, " +
245 "appendEntries.getLeaderCommit():{}," +
246 "context.getLastApplied():{}, lastIndex():{}", context.getId(),
247 appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex()
251 applyLogToStateMachine(appendEntries.getLeaderCommit());
254 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), true,
255 lastIndex(), lastTerm()), actor());
257 if (!context.isSnapshotCaptureInitiated()) {
258 fakeSnapshot(appendEntries.getReplicatedToAllIndex(), appendEntries.getReplicatedToAllIndex());
264 @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
265 AppendEntriesReply appendEntriesReply) {
269 @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
270 RequestVoteReply requestVoteReply) {
274 @Override public RaftState state() {
275 return RaftState.Follower;
278 @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
280 Object message = fromSerializableMessage(originalMessage);
282 if (message instanceof RaftRPC) {
283 RaftRPC rpc = (RaftRPC) message;
284 // If RPC request or response contains term T > currentTerm:
285 // set currentTerm = T, convert to follower (§5.1)
286 // This applies to all RPC messages and responses
287 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
288 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
292 if (message instanceof ElectionTimeout) {
293 return switchBehavior(new Candidate(context));
295 } else if (message instanceof InstallSnapshot) {
296 InstallSnapshot installSnapshot = (InstallSnapshot) message;
297 handleInstallSnapshot(sender, installSnapshot);
300 scheduleElection(electionDuration());
302 return super.handleMessage(sender, message);
305 private void handleInstallSnapshot(ActorRef sender, InstallSnapshot installSnapshot) {
307 if(LOG.isDebugEnabled()) {
308 LOG.debug("{}: InstallSnapshot received by follower " +
309 "datasize:{} , Chunk:{}/{}", context.getId(), installSnapshot.getData().size(),
310 installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks()
314 if(snapshotTracker == null){
315 snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks());
319 if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
320 installSnapshot.getLastChunkHashCode())){
321 Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(),
322 new ArrayList<ReplicatedLogEntry>(),
323 installSnapshot.getLastIncludedIndex(),
324 installSnapshot.getLastIncludedTerm(),
325 installSnapshot.getLastIncludedIndex(),
326 installSnapshot.getLastIncludedTerm());
328 actor().tell(new ApplySnapshot(snapshot), actor());
330 snapshotTracker = null;
334 sender.tell(new InstallSnapshotReply(
335 currentTerm(), context.getId(), installSnapshot.getChunkIndex(),
338 } catch (SnapshotTracker.InvalidChunkException e) {
340 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
341 -1, false), actor());
342 snapshotTracker = null;
344 } catch (Exception e){
345 LOG.error("{}: Exception in InstallSnapshot of follower", context.getId(), e);
346 //send reply with success as false. The chunk will be sent again on failure
347 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
348 installSnapshot.getChunkIndex(), false), actor());
353 @Override public void close() throws Exception {
358 ByteString getSnapshotChunksCollected(){
359 return snapshotTracker != null ? snapshotTracker.getCollectedChunks() : ByteString.EMPTY;