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 java.util.ArrayList;
14 import org.opendaylight.controller.cluster.raft.RaftActorContext;
15 import org.opendaylight.controller.cluster.raft.RaftState;
16 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
17 import org.opendaylight.controller.cluster.raft.Snapshot;
18 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
19 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
20 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
21 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
22 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
23 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
24 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
25 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
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 private final SyncStatusTracker initialSyncStatusTracker;
44 private static final int SYNC_THRESHOLD = 10;
46 public Follower(RaftActorContext context) {
47 super(context, RaftState.Follower);
49 initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD);
51 if(context.getRaftPolicy().automaticElectionsEnabled()) {
52 if (context.getPeerAddresses().isEmpty()) {
53 actor().tell(ELECTION_TIMEOUT, actor());
55 scheduleElection(electionDuration());
61 private boolean isLogEntryPresent(long index){
62 if(index == context.getReplicatedLog().getSnapshotIndex()){
66 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
69 return previousEntry != null;
73 private long getLogEntryTerm(long index){
74 if(index == context.getReplicatedLog().getSnapshotIndex()){
75 return context.getReplicatedLog().getSnapshotTerm();
78 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
81 if(previousEntry != null){
82 return previousEntry.getTerm();
88 private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){
89 initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex());
92 @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender,
93 AppendEntries appendEntries) {
95 int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
96 if(LOG.isTraceEnabled()) {
97 LOG.trace("{}: handleAppendEntries: {}", logName(), appendEntries);
98 } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
99 LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
102 // TODO : Refactor this method into a bunch of smaller methods
103 // to make it easier to read. Before refactoring ensure tests
104 // cover the code properly
106 if (snapshotTracker != null) {
107 // if snapshot install is in progress, follower should just acknowledge append entries with a reply.
108 AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
109 lastIndex(), lastTerm(), context.getPayloadVersion());
111 if(LOG.isDebugEnabled()) {
112 LOG.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply);
114 sender.tell(reply, actor());
119 // If we got here then we do appear to be talking to the leader
120 leaderId = appendEntries.getLeaderId();
122 setLeaderPayloadVersion(appendEntries.getPayloadVersion());
124 updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
125 // First check if the logs are in sync or not
126 long lastIndex = lastIndex();
128 if (isOutOfSync(appendEntries)) {
129 // We found that the log was out of sync so just send a negative
132 LOG.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}",
133 logName(), lastIndex, lastTerm());
135 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
136 lastTerm(), context.getPayloadVersion()), actor());
140 if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
142 LOG.debug("{}: Number of entries to be appended = {}", logName(),
143 appendEntries.getEntries().size());
145 // 3. If an existing entry conflicts with a new one (same index
146 // but different terms), delete the existing entry and all that
148 int addEntriesFrom = 0;
149 if (context.getReplicatedLog().size() > 0) {
151 // Find the entry up until the one that is not in the follower's log
152 for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) {
153 ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
154 ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex());
156 if (newEntry == null) {
157 //newEntry not found in the log
161 if (newEntry.getTerm() == matchEntry.getTerm()) {
165 if(!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) {
167 LOG.debug("{}: Removing entries from log starting at {}", logName(),
168 matchEntry.getIndex());
170 // Entries do not match so remove all subsequent entries
171 context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex());
174 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
175 lastTerm(), context.getPayloadVersion(), true), actor());
181 lastIndex = lastIndex();
182 LOG.debug("{}: After cleanup entries to be added from = {}", logName(),
183 (addEntriesFrom + lastIndex));
185 // 4. Append any new entries not already in the log
186 for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) {
187 ReplicatedLogEntry entry = appendEntries.getEntries().get(i);
189 LOG.debug("{}: Append entry to log {}", logName(), entry.getData());
191 context.getReplicatedLog().appendAndPersist(entry);
194 LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
197 // 5. If leaderCommit > commitIndex, set commitIndex =
198 // min(leaderCommit, index of last new entry)
200 lastIndex = lastIndex();
201 long prevCommitIndex = context.getCommitIndex();
203 context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex));
205 if (prevCommitIndex != context.getCommitIndex()) {
206 LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex());
209 // If commitIndex > lastApplied: increment lastApplied, apply
210 // log[lastApplied] to state machine (ยง5.3)
211 // check if there are any entries to be applied. last-applied can be equal to last-index
212 if (appendEntries.getLeaderCommit() > context.getLastApplied() &&
213 context.getLastApplied() < lastIndex) {
214 if(LOG.isDebugEnabled()) {
215 LOG.debug("{}: applyLogToStateMachine, " +
216 "appendEntries.getLeaderCommit(): {}," +
217 "context.getLastApplied(): {}, lastIndex(): {}", logName(),
218 appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex);
221 applyLogToStateMachine(appendEntries.getLeaderCommit());
224 AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
225 lastIndex, lastTerm(), context.getPayloadVersion());
227 if(LOG.isTraceEnabled()) {
228 LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply);
229 } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
230 LOG.debug("{}: handleAppendEntries returning : {}", logName(), reply);
233 sender.tell(reply, actor());
235 if (!context.getSnapshotManager().isCapturing()) {
236 super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex());
242 private boolean isOutOfSync(AppendEntries appendEntries) {
244 long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
245 boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex());
246 long lastIndex = lastIndex();
247 int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
248 boolean outOfSync = true;
250 if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) {
252 // The follower's log is out of sync because the leader does have
253 // an entry at prevLogIndex and this follower has no entries in
256 LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}",
257 logName(), appendEntries.getPrevLogIndex());
258 } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) {
260 // The follower's log is out of sync because the Leader's
261 // prevLogIndex entry was not found in it's log
263 LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it",
264 logName(), appendEntries.getPrevLogIndex());
265 } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) {
267 // The follower's log is out of sync because the Leader's
268 // prevLogIndex entry does exist in the follower's log but it has
269 // a different term in it
272 "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}",
273 logName(), prevLogTerm, appendEntries.getPrevLogTerm());
274 } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
275 && appendEntries.getReplicatedToAllIndex() != -1
276 && !isLogEntryPresent(appendEntries.getReplicatedToAllIndex())) {
277 // This append entry comes from a leader who has it's log aggressively trimmed and so does not have
278 // the previous entry in it's in-memory journal
281 "{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the in-memory journal",
282 logName(), appendEntries.getReplicatedToAllIndex());
283 } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
284 && appendEntries.getReplicatedToAllIndex() != -1 && numLogEntries > 0 &&
285 !isLogEntryPresent(appendEntries.getEntries().get(0).getIndex() - 1)){
287 "{}: Cannot append entries because the calculated previousIndex {} was not found in the in-memory journal",
288 logName(), appendEntries.getEntries().get(0).getIndex() - 1);
295 @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
296 AppendEntriesReply appendEntriesReply) {
300 @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
301 RequestVoteReply requestVoteReply) {
305 @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
307 Object message = fromSerializableMessage(originalMessage);
309 if (message instanceof RaftRPC) {
310 RaftRPC rpc = (RaftRPC) message;
311 // If RPC request or response contains term T > currentTerm:
312 // set currentTerm = T, convert to follower (ยง5.1)
313 // This applies to all RPC messages and responses
314 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
315 LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term",
316 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
318 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
322 if (message instanceof ElectionTimeout) {
323 LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName());
324 return internalSwitchBehavior(RaftState.Candidate);
326 } else if (message instanceof InstallSnapshot) {
327 InstallSnapshot installSnapshot = (InstallSnapshot) message;
328 handleInstallSnapshot(sender, installSnapshot);
331 if(message instanceof RaftRPC && (!(message instanceof RequestVote) || (canGrantVote((RequestVote) message)))){
332 scheduleElection(electionDuration());
335 return super.handleMessage(sender, message);
338 private void handleInstallSnapshot(ActorRef sender, InstallSnapshot installSnapshot) {
340 LOG.debug("{}: InstallSnapshot received from leader {}, datasize: {} , Chunk: {}/{}",
341 logName(), installSnapshot.getLeaderId(), installSnapshot.getData().size(),
342 installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks());
344 if(snapshotTracker == null){
345 snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks());
348 updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId());
351 if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
352 installSnapshot.getLastChunkHashCode())){
353 Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(),
354 new ArrayList<ReplicatedLogEntry>(),
355 installSnapshot.getLastIncludedIndex(),
356 installSnapshot.getLastIncludedTerm(),
357 installSnapshot.getLastIncludedIndex(),
358 installSnapshot.getLastIncludedTerm());
360 actor().tell(new ApplySnapshot(snapshot), actor());
362 snapshotTracker = null;
366 InstallSnapshotReply reply = new InstallSnapshotReply(
367 currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true);
369 LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
371 sender.tell(reply, actor());
373 } catch (SnapshotTracker.InvalidChunkException e) {
374 LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e);
376 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
377 -1, false), actor());
378 snapshotTracker = null;
380 } catch (Exception e){
381 LOG.error("{}: Exception in InstallSnapshot of follower", logName(), e);
383 //send reply with success as false. The chunk will be sent again on failure
384 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
385 installSnapshot.getChunkIndex(), false), actor());
391 public void close() throws Exception {
396 SnapshotTracker getSnapshotTracker(){
397 return snapshotTracker;