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) {
43 super(context, RaftState.Follower);
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 int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
79 if(LOG.isTraceEnabled()) {
80 LOG.trace("{}: handleAppendEntries: {}", logName(), appendEntries);
81 } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
82 LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
85 // TODO : Refactor this method into a bunch of smaller methods
86 // to make it easier to read. Before refactoring ensure tests
87 // cover the code properly
89 // 1. Reply false if term < currentTerm (§5.1)
90 // This is handled in the appendEntries method of the base class
92 // If we got here then we do appear to be talking to the leader
93 leaderId = appendEntries.getLeaderId();
95 // 2. Reply false if log doesn’t contain an entry at prevLogIndex
96 // whose term matches prevLogTerm (§5.3)
98 long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
99 boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex());
102 boolean outOfSync = true;
104 // First check if the logs are in sync or not
105 long lastIndex = lastIndex();
106 if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) {
108 // The follower's log is out of sync because the leader does have
109 // an entry at prevLogIndex and this follower has no entries in
112 LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}",
113 logName(), appendEntries.getPrevLogIndex());
114 } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) {
116 // The follower's log is out of sync because the Leader's
117 // prevLogIndex entry was not found in it's log
119 LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it",
120 logName(), appendEntries.getPrevLogIndex());
121 } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) {
123 // The follower's log is out of sync because the Leader's
124 // prevLogIndex entry does exist in the follower's log but it has
125 // a different term in it
128 "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}",
129 logName(), prevLogTerm, appendEntries.getPrevLogTerm());
135 // We found that the log was out of sync so just send a negative
138 LOG.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}",
139 logName(), lastIndex, lastTerm());
141 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
142 lastTerm()), actor());
146 if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
148 LOG.debug("{}: Number of entries to be appended = {}", logName(),
149 appendEntries.getEntries().size());
151 // 3. If an existing entry conflicts with a new one (same index
152 // but different terms), delete the existing entry and all that
154 int addEntriesFrom = 0;
155 if (context.getReplicatedLog().size() > 0) {
157 // Find the entry up until which the one that is not in the follower's log
158 for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) {
159 ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
160 ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex());
162 if (newEntry == null) {
163 //newEntry not found in the log
167 if (newEntry.getTerm() == matchEntry.getTerm()) {
171 LOG.debug("{}: Removing entries from log starting at {}", logName(),
172 matchEntry.getIndex());
174 // Entries do not match so remove all subsequent entries
175 context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex());
180 lastIndex = lastIndex();
181 LOG.debug("{}: After cleanup entries to be added from = {}", logName(),
182 (addEntriesFrom + lastIndex));
184 // 4. Append any new entries not already in the log
185 for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) {
186 ReplicatedLogEntry entry = appendEntries.getEntries().get(i);
188 LOG.debug("{}: Append entry to log {}", logName(), entry.getData());
190 context.getReplicatedLog().appendAndPersist(entry);
193 LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
196 // 5. If leaderCommit > commitIndex, set commitIndex =
197 // min(leaderCommit, index of last new entry)
199 lastIndex = lastIndex();
200 long prevCommitIndex = context.getCommitIndex();
202 context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex));
204 if (prevCommitIndex != context.getCommitIndex()) {
205 LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex());
208 // If commitIndex > lastApplied: increment lastApplied, apply
209 // log[lastApplied] to state machine (§5.3)
210 // check if there are any entries to be applied. last-applied can be equal to last-index
211 if (appendEntries.getLeaderCommit() > context.getLastApplied() &&
212 context.getLastApplied() < lastIndex) {
213 if(LOG.isDebugEnabled()) {
214 LOG.debug("{}: applyLogToStateMachine, " +
215 "appendEntries.getLeaderCommit(): {}," +
216 "context.getLastApplied(): {}, lastIndex(): {}", logName(),
217 appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex);
220 applyLogToStateMachine(appendEntries.getLeaderCommit());
223 AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
224 lastIndex, lastTerm());
226 if(LOG.isTraceEnabled()) {
227 LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply);
228 } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
229 LOG.debug("{}: handleAppendEntries returning : {}", logName(), reply);
232 sender.tell(reply, actor());
234 if (!context.isSnapshotCaptureInitiated()) {
235 super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex());
241 @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
242 AppendEntriesReply appendEntriesReply) {
246 @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
247 RequestVoteReply requestVoteReply) {
251 @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
253 Object message = fromSerializableMessage(originalMessage);
255 if (message instanceof RaftRPC) {
256 RaftRPC rpc = (RaftRPC) message;
257 // If RPC request or response contains term T > currentTerm:
258 // set currentTerm = T, convert to follower (§5.1)
259 // This applies to all RPC messages and responses
260 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
261 LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term",
262 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
264 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
268 if (message instanceof ElectionTimeout) {
269 LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName());
270 return switchBehavior(new Candidate(context));
272 } else if (message instanceof InstallSnapshot) {
273 InstallSnapshot installSnapshot = (InstallSnapshot) message;
274 handleInstallSnapshot(sender, installSnapshot);
277 scheduleElection(electionDuration());
279 return super.handleMessage(sender, message);
282 private void handleInstallSnapshot(ActorRef sender, InstallSnapshot installSnapshot) {
285 LOG.debug("{}: InstallSnapshot received from leader {}, datasize: {} , Chunk: {}/{}",
286 logName(), installSnapshot.getLeaderId(), installSnapshot.getData().size(),
287 installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks());
289 if(snapshotTracker == null){
290 snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks());
294 if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
295 installSnapshot.getLastChunkHashCode())){
296 Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(),
297 new ArrayList<ReplicatedLogEntry>(),
298 installSnapshot.getLastIncludedIndex(),
299 installSnapshot.getLastIncludedTerm(),
300 installSnapshot.getLastIncludedIndex(),
301 installSnapshot.getLastIncludedTerm());
303 actor().tell(new ApplySnapshot(snapshot), actor());
305 snapshotTracker = null;
309 InstallSnapshotReply reply = new InstallSnapshotReply(
310 currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true);
312 LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
314 sender.tell(reply, actor());
316 } catch (SnapshotTracker.InvalidChunkException e) {
317 LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e);
319 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
320 -1, false), actor());
321 snapshotTracker = null;
323 } catch (Exception e){
324 LOG.error("{}: Exception in InstallSnapshot of follower", logName(), e);
326 //send reply with success as false. The chunk will be sent again on failure
327 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
328 installSnapshot.getChunkIndex(), false), actor());
333 @Override public void close() throws Exception {
338 ByteString getSnapshotChunksCollected(){
339 return snapshotTracker != null ? snapshotTracker.getCollectedChunks() : ByteString.EMPTY;