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.ServerConfigurationPayload;
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.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 SyncStatusTracker initialSyncStatusTracker;
45 private static final int SYNC_THRESHOLD = 10;
47 public Follower(RaftActorContext context) {
48 super(context, RaftState.Follower);
50 initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD);
52 if(context.getRaftPolicy().automaticElectionsEnabled()) {
53 if (context.getPeerIds().isEmpty()) {
54 actor().tell(ELECTION_TIMEOUT, actor());
56 scheduleElection(electionDuration());
62 private boolean isLogEntryPresent(long index){
63 if(index == context.getReplicatedLog().getSnapshotIndex()){
67 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
70 return previousEntry != null;
74 private long getLogEntryTerm(long index){
75 if(index == context.getReplicatedLog().getSnapshotIndex()){
76 return context.getReplicatedLog().getSnapshotTerm();
79 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
82 if(previousEntry != null){
83 return previousEntry.getTerm();
89 private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){
90 initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex());
93 @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender,
94 AppendEntries appendEntries) {
96 int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
97 if(LOG.isTraceEnabled()) {
98 LOG.trace("{}: handleAppendEntries: {}", logName(), appendEntries);
99 } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
100 LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
103 // TODO : Refactor this method into a bunch of smaller methods
104 // to make it easier to read. Before refactoring ensure tests
105 // cover the code properly
107 if (snapshotTracker != null || context.getSnapshotManager().isApplying()) {
108 // if snapshot install is in progress, follower should just acknowledge append entries with a reply.
109 AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
110 lastIndex(), lastTerm(), context.getPayloadVersion());
112 if(LOG.isDebugEnabled()) {
113 LOG.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply);
115 sender.tell(reply, actor());
120 // If we got here then we do appear to be talking to the leader
121 leaderId = appendEntries.getLeaderId();
123 setLeaderPayloadVersion(appendEntries.getPayloadVersion());
125 updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
126 // First check if the logs are in sync or not
127 long lastIndex = lastIndex();
129 if (isOutOfSync(appendEntries)) {
130 // We found that the log was out of sync so just send a negative
133 LOG.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}",
134 logName(), lastIndex, lastTerm());
136 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
137 lastTerm(), context.getPayloadVersion()), actor());
141 if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
143 LOG.debug("{}: Number of entries to be appended = {}", logName(),
144 appendEntries.getEntries().size());
146 // 3. If an existing entry conflicts with a new one (same index
147 // but different terms), delete the existing entry and all that
149 int addEntriesFrom = 0;
150 if (context.getReplicatedLog().size() > 0) {
152 // Find the entry up until the one that is not in the follower's log
153 for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) {
154 ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
155 ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex());
157 if (newEntry == null) {
158 //newEntry not found in the log
162 if (newEntry.getTerm() == matchEntry.getTerm()) {
166 if(!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) {
168 LOG.debug("{}: Removing entries from log starting at {}", logName(),
169 matchEntry.getIndex());
171 // Entries do not match so remove all subsequent entries
172 context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex());
175 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
176 lastTerm(), context.getPayloadVersion(), true), actor());
182 lastIndex = lastIndex();
183 LOG.debug("{}: After cleanup entries to be added from = {}", logName(),
184 (addEntriesFrom + lastIndex));
186 // 4. Append any new entries not already in the log
187 for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) {
188 ReplicatedLogEntry entry = appendEntries.getEntries().get(i);
190 LOG.debug("{}: Append entry to log {}", logName(), entry.getData());
192 context.getReplicatedLog().appendAndPersist(entry);
194 if(entry.getData() instanceof ServerConfigurationPayload) {
195 context.updatePeerIds((ServerConfigurationPayload)entry.getData());
199 LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
202 // 5. If leaderCommit > commitIndex, set commitIndex =
203 // min(leaderCommit, index of last new entry)
205 lastIndex = lastIndex();
206 long prevCommitIndex = context.getCommitIndex();
208 context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex));
210 if (prevCommitIndex != context.getCommitIndex()) {
211 LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex());
214 // If commitIndex > lastApplied: increment lastApplied, apply
215 // log[lastApplied] to state machine (ยง5.3)
216 // check if there are any entries to be applied. last-applied can be equal to last-index
217 if (appendEntries.getLeaderCommit() > context.getLastApplied() &&
218 context.getLastApplied() < lastIndex) {
219 if(LOG.isDebugEnabled()) {
220 LOG.debug("{}: applyLogToStateMachine, " +
221 "appendEntries.getLeaderCommit(): {}," +
222 "context.getLastApplied(): {}, lastIndex(): {}", logName(),
223 appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex);
226 applyLogToStateMachine(appendEntries.getLeaderCommit());
229 AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
230 lastIndex, lastTerm(), context.getPayloadVersion());
232 if(LOG.isTraceEnabled()) {
233 LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply);
234 } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
235 LOG.debug("{}: handleAppendEntries returning : {}", logName(), reply);
238 sender.tell(reply, actor());
240 if (!context.getSnapshotManager().isCapturing()) {
241 super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex());
247 private boolean isOutOfSync(AppendEntries appendEntries) {
249 long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
250 boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex());
251 long lastIndex = lastIndex();
252 int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
253 boolean outOfSync = true;
255 if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) {
257 // The follower's log is out of sync because the leader does have
258 // an entry at prevLogIndex and this follower has no entries in
261 LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}",
262 logName(), appendEntries.getPrevLogIndex());
263 } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) {
265 // The follower's log is out of sync because the Leader's
266 // prevLogIndex entry was not found in it's log
268 LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it",
269 logName(), appendEntries.getPrevLogIndex());
270 } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) {
272 // The follower's log is out of sync because the Leader's
273 // prevLogIndex entry does exist in the follower's log but it has
274 // a different term in it
277 "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}",
278 logName(), prevLogTerm, appendEntries.getPrevLogTerm());
279 } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
280 && appendEntries.getReplicatedToAllIndex() != -1
281 && !isLogEntryPresent(appendEntries.getReplicatedToAllIndex())) {
282 // This append entry comes from a leader who has it's log aggressively trimmed and so does not have
283 // the previous entry in it's in-memory journal
286 "{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the in-memory journal",
287 logName(), appendEntries.getReplicatedToAllIndex());
288 } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
289 && appendEntries.getReplicatedToAllIndex() != -1 && numLogEntries > 0 &&
290 !isLogEntryPresent(appendEntries.getEntries().get(0).getIndex() - 1)){
292 "{}: Cannot append entries because the calculated previousIndex {} was not found in the in-memory journal",
293 logName(), appendEntries.getEntries().get(0).getIndex() - 1);
300 @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
301 AppendEntriesReply appendEntriesReply) {
305 @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
306 RequestVoteReply requestVoteReply) {
310 @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
312 Object message = fromSerializableMessage(originalMessage);
314 if (message instanceof RaftRPC) {
315 RaftRPC rpc = (RaftRPC) message;
316 // If RPC request or response contains term T > currentTerm:
317 // set currentTerm = T, convert to follower (ยง5.1)
318 // This applies to all RPC messages and responses
319 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
320 LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term",
321 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
323 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
327 if (message instanceof ElectionTimeout) {
328 LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName());
329 return internalSwitchBehavior(RaftState.Candidate);
331 } else if (message instanceof InstallSnapshot) {
332 InstallSnapshot installSnapshot = (InstallSnapshot) message;
333 handleInstallSnapshot(sender, installSnapshot);
336 if(message instanceof RaftRPC && (!(message instanceof RequestVote) || (canGrantVote((RequestVote) message)))){
337 scheduleElection(electionDuration());
340 return super.handleMessage(sender, message);
343 private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot installSnapshot) {
345 LOG.debug("{}: InstallSnapshot received from leader {}, datasize: {} , Chunk: {}/{}",
346 logName(), installSnapshot.getLeaderId(), installSnapshot.getData().size(),
347 installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks());
349 if(snapshotTracker == null){
350 snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks());
353 updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId());
356 final InstallSnapshotReply reply = new InstallSnapshotReply(
357 currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true);
359 if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
360 installSnapshot.getLastChunkHashCode())){
361 Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(),
362 new ArrayList<ReplicatedLogEntry>(),
363 installSnapshot.getLastIncludedIndex(),
364 installSnapshot.getLastIncludedTerm(),
365 installSnapshot.getLastIncludedIndex(),
366 installSnapshot.getLastIncludedTerm(),
367 context.getTermInformation().getCurrentTerm(),
368 context.getTermInformation().getVotedFor());
370 ApplySnapshot.Callback applySnapshotCallback = new ApplySnapshot.Callback() {
372 public void onSuccess() {
373 LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
375 sender.tell(reply, actor());
379 public void onFailure() {
380 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), -1, false), actor());
384 actor().tell(new ApplySnapshot(snapshot, applySnapshotCallback), actor());
386 snapshotTracker = null;
388 LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
390 sender.tell(reply, actor());
392 } catch (SnapshotTracker.InvalidChunkException e) {
393 LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e);
395 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
396 -1, false), actor());
397 snapshotTracker = null;
399 } catch (Exception e){
400 LOG.error("{}: Exception in InstallSnapshot of follower", logName(), e);
402 //send reply with success as false. The chunk will be sent again on failure
403 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
404 installSnapshot.getChunkIndex(), false), actor());
410 public void close() throws Exception {
415 SnapshotTracker getSnapshotTracker(){
416 return snapshotTracker;