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.base.messages.FollowerInitialSyncUpStatus;
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.RequestVote;
27 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
30 * The behavior of a RaftActor in the Follower state
33 * <li> Respond to RPCs from candidates and leaders
34 * <li> If election timeout elapses without receiving AppendEntries
35 * RPC from current leader or granting vote to candidate:
36 * convert to candidate
39 public class Follower extends AbstractRaftActorBehavior {
41 private SnapshotTracker snapshotTracker = null;
43 private final InitialSyncStatusTracker initialSyncStatusTracker;
45 public Follower(RaftActorContext context) {
46 super(context, RaftState.Follower);
48 initialSyncStatusTracker = new InitialSyncStatusTracker(context.getActor());
50 if(context.getRaftPolicy().automaticElectionsEnabled()) {
51 if (context.getPeerAddresses().isEmpty()) {
52 actor().tell(ELECTION_TIMEOUT, actor());
54 scheduleElection(electionDuration());
60 private boolean isLogEntryPresent(long index){
61 if(index == context.getReplicatedLog().getSnapshotIndex()){
65 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
68 return previousEntry != null;
72 private long getLogEntryTerm(long index){
73 if(index == context.getReplicatedLog().getSnapshotIndex()){
74 return context.getReplicatedLog().getSnapshotTerm();
77 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
80 if(previousEntry != null){
81 return previousEntry.getTerm();
87 private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){
88 initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex());
91 @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender,
92 AppendEntries appendEntries) {
94 int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
95 if(LOG.isTraceEnabled()) {
96 LOG.trace("{}: handleAppendEntries: {}", logName(), appendEntries);
97 } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
98 LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
101 // TODO : Refactor this method into a bunch of smaller methods
102 // to make it easier to read. Before refactoring ensure tests
103 // cover the code properly
105 if (snapshotTracker != null) {
106 // if snapshot install is in progress, follower should just acknowledge append entries with a reply.
107 AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
108 lastIndex(), lastTerm(), context.getPayloadVersion());
110 if(LOG.isDebugEnabled()) {
111 LOG.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply);
113 sender.tell(reply, actor());
118 // If we got here then we do appear to be talking to the leader
119 leaderId = appendEntries.getLeaderId();
121 setLeaderPayloadVersion(appendEntries.getPayloadVersion());
123 updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
124 // First check if the logs are in sync or not
125 long lastIndex = lastIndex();
127 if (isOutOfSync(appendEntries)) {
128 // We found that the log was out of sync so just send a negative
131 LOG.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}",
132 logName(), lastIndex, lastTerm());
134 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
135 lastTerm(), context.getPayloadVersion()), actor());
139 if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
141 LOG.debug("{}: Number of entries to be appended = {}", logName(),
142 appendEntries.getEntries().size());
144 // 3. If an existing entry conflicts with a new one (same index
145 // but different terms), delete the existing entry and all that
147 int addEntriesFrom = 0;
148 if (context.getReplicatedLog().size() > 0) {
150 // Find the entry up until which the one that is not in the follower's log
151 for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) {
152 ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
153 ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex());
155 if (newEntry == null) {
156 //newEntry not found in the log
160 if (newEntry.getTerm() == matchEntry.getTerm()) {
164 LOG.debug("{}: Removing entries from log starting at {}", logName(),
165 matchEntry.getIndex());
167 // Entries do not match so remove all subsequent entries
168 context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex());
173 lastIndex = lastIndex();
174 LOG.debug("{}: After cleanup entries to be added from = {}", logName(),
175 (addEntriesFrom + lastIndex));
177 // 4. Append any new entries not already in the log
178 for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) {
179 ReplicatedLogEntry entry = appendEntries.getEntries().get(i);
181 LOG.debug("{}: Append entry to log {}", logName(), entry.getData());
183 context.getReplicatedLog().appendAndPersist(entry);
186 LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
189 // 5. If leaderCommit > commitIndex, set commitIndex =
190 // min(leaderCommit, index of last new entry)
192 lastIndex = lastIndex();
193 long prevCommitIndex = context.getCommitIndex();
195 context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex));
197 if (prevCommitIndex != context.getCommitIndex()) {
198 LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex());
201 // If commitIndex > lastApplied: increment lastApplied, apply
202 // log[lastApplied] to state machine (ยง5.3)
203 // check if there are any entries to be applied. last-applied can be equal to last-index
204 if (appendEntries.getLeaderCommit() > context.getLastApplied() &&
205 context.getLastApplied() < lastIndex) {
206 if(LOG.isDebugEnabled()) {
207 LOG.debug("{}: applyLogToStateMachine, " +
208 "appendEntries.getLeaderCommit(): {}," +
209 "context.getLastApplied(): {}, lastIndex(): {}", logName(),
210 appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex);
213 applyLogToStateMachine(appendEntries.getLeaderCommit());
216 AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
217 lastIndex, lastTerm(), context.getPayloadVersion());
219 if(LOG.isTraceEnabled()) {
220 LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply);
221 } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
222 LOG.debug("{}: handleAppendEntries returning : {}", logName(), reply);
225 sender.tell(reply, actor());
227 if (!context.getSnapshotManager().isCapturing()) {
228 super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex());
234 private boolean isOutOfSync(AppendEntries appendEntries) {
236 long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
237 boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex());
238 long lastIndex = lastIndex();
239 int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
240 boolean outOfSync = true;
242 if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) {
244 // The follower's log is out of sync because the leader does have
245 // an entry at prevLogIndex and this follower has no entries in
248 LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}",
249 logName(), appendEntries.getPrevLogIndex());
250 } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) {
252 // The follower's log is out of sync because the Leader's
253 // prevLogIndex entry was not found in it's log
255 LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it",
256 logName(), appendEntries.getPrevLogIndex());
257 } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) {
259 // The follower's log is out of sync because the Leader's
260 // prevLogIndex entry does exist in the follower's log but it has
261 // a different term in it
264 "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}",
265 logName(), prevLogTerm, appendEntries.getPrevLogTerm());
266 } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
267 && appendEntries.getReplicatedToAllIndex() != -1
268 && !isLogEntryPresent(appendEntries.getReplicatedToAllIndex())) {
269 // This append entry comes from a leader who has it's log aggressively trimmed and so does not have
270 // the previous entry in it's in-memory journal
273 "{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the in-memory journal",
274 logName(), appendEntries.getReplicatedToAllIndex());
275 } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
276 && appendEntries.getReplicatedToAllIndex() != -1 && numLogEntries > 0 &&
277 !isLogEntryPresent(appendEntries.getEntries().get(0).getIndex() - 1)){
279 "{}: Cannot append entries because the calculated previousIndex {} was not found in the in-memory journal",
280 logName(), appendEntries.getEntries().get(0).getIndex() - 1);
287 @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
288 AppendEntriesReply appendEntriesReply) {
292 @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
293 RequestVoteReply requestVoteReply) {
297 @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
299 Object message = fromSerializableMessage(originalMessage);
301 if (message instanceof RaftRPC) {
302 RaftRPC rpc = (RaftRPC) message;
303 // If RPC request or response contains term T > currentTerm:
304 // set currentTerm = T, convert to follower (ยง5.1)
305 // This applies to all RPC messages and responses
306 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
307 LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term",
308 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
310 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
314 if (message instanceof ElectionTimeout) {
315 LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName());
316 return switchBehavior(new Candidate(context));
318 } else if (message instanceof InstallSnapshot) {
319 InstallSnapshot installSnapshot = (InstallSnapshot) message;
320 handleInstallSnapshot(sender, installSnapshot);
323 if(message instanceof RaftRPC && (!(message instanceof RequestVote) || (canGrantVote((RequestVote) message)))){
324 scheduleElection(electionDuration());
327 return super.handleMessage(sender, message);
330 private void handleInstallSnapshot(ActorRef sender, InstallSnapshot installSnapshot) {
332 LOG.debug("{}: InstallSnapshot received from leader {}, datasize: {} , Chunk: {}/{}",
333 logName(), installSnapshot.getLeaderId(), installSnapshot.getData().size(),
334 installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks());
336 if(snapshotTracker == null){
337 snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks());
340 updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId());
343 if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
344 installSnapshot.getLastChunkHashCode())){
345 Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(),
346 new ArrayList<ReplicatedLogEntry>(),
347 installSnapshot.getLastIncludedIndex(),
348 installSnapshot.getLastIncludedTerm(),
349 installSnapshot.getLastIncludedIndex(),
350 installSnapshot.getLastIncludedTerm());
352 actor().tell(new ApplySnapshot(snapshot), actor());
354 snapshotTracker = null;
358 InstallSnapshotReply reply = new InstallSnapshotReply(
359 currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true);
361 LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
363 sender.tell(reply, actor());
365 } catch (SnapshotTracker.InvalidChunkException e) {
366 LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e);
368 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
369 -1, false), actor());
370 snapshotTracker = null;
372 } catch (Exception e){
373 LOG.error("{}: Exception in InstallSnapshot of follower", logName(), e);
375 //send reply with success as false. The chunk will be sent again on failure
376 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
377 installSnapshot.getChunkIndex(), false), actor());
383 public void close() throws Exception {
388 SnapshotTracker getSnapshotTracker(){
389 return snapshotTracker;
392 private class InitialSyncStatusTracker {
394 private static final long INVALID_LOG_INDEX = -2L;
395 private long initialLeaderCommit = INVALID_LOG_INDEX;
396 private boolean initialSyncUpDone = false;
397 private String syncedLeaderId = null;
398 private final ActorRef actor;
400 public InitialSyncStatusTracker(ActorRef actor) {
404 public void update(String leaderId, long leaderCommit, long commitIndex){
406 if(!leaderId.equals(syncedLeaderId)){
407 initialSyncUpDone = false;
408 initialLeaderCommit = INVALID_LOG_INDEX;
409 syncedLeaderId = leaderId;
412 if(!initialSyncUpDone){
413 if(initialLeaderCommit == INVALID_LOG_INDEX){
414 actor.tell(new FollowerInitialSyncUpStatus(false, getId()), ActorRef.noSender());
415 initialLeaderCommit = leaderCommit;
416 } else if(commitIndex >= initialLeaderCommit){
417 actor.tell(new FollowerInitialSyncUpStatus(true, getId()), ActorRef.noSender());
418 initialSyncUpDone = true;