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 this(context, null, (short)-1);
51 public Follower(RaftActorContext context, String initialLeaderId, short initialLeaderPayloadVersion) {
52 super(context, RaftState.Follower);
53 leaderId = initialLeaderId;
54 setLeaderPayloadVersion(initialLeaderPayloadVersion);
56 initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD);
58 if(canStartElection()) {
59 if (context.getPeerIds().isEmpty() && getLeaderId() == null) {
60 actor().tell(ElectionTimeout.INSTANCE, actor());
62 scheduleElection(electionDuration());
68 private boolean isLogEntryPresent(long index){
69 if(index == context.getReplicatedLog().getSnapshotIndex()){
73 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
76 return previousEntry != null;
80 private long getLogEntryTerm(long index){
81 if(index == context.getReplicatedLog().getSnapshotIndex()){
82 return context.getReplicatedLog().getSnapshotTerm();
85 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
88 if(previousEntry != null){
89 return previousEntry.getTerm();
95 private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){
96 initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex());
99 @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender,
100 AppendEntries appendEntries) {
102 int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
103 if(LOG.isTraceEnabled()) {
104 LOG.trace("{}: handleAppendEntries: {}", logName(), appendEntries);
105 } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
106 LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
109 // TODO : Refactor this method into a bunch of smaller methods
110 // to make it easier to read. Before refactoring ensure tests
111 // cover the code properly
113 if (snapshotTracker != null || context.getSnapshotManager().isApplying()) {
114 // if snapshot install is in progress, follower should just acknowledge append entries with a reply.
115 AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
116 lastIndex(), lastTerm(), context.getPayloadVersion());
118 if(LOG.isDebugEnabled()) {
119 LOG.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply);
121 sender.tell(reply, actor());
126 // If we got here then we do appear to be talking to the leader
127 leaderId = appendEntries.getLeaderId();
129 setLeaderPayloadVersion(appendEntries.getPayloadVersion());
131 updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
132 // First check if the logs are in sync or not
133 long lastIndex = lastIndex();
135 if (isOutOfSync(appendEntries)) {
136 // We found that the log was out of sync so just send a negative
139 LOG.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}",
140 logName(), lastIndex, lastTerm());
142 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
143 lastTerm(), context.getPayloadVersion()), actor());
147 if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
149 LOG.debug("{}: Number of entries to be appended = {}", logName(),
150 appendEntries.getEntries().size());
152 // 3. If an existing entry conflicts with a new one (same index
153 // but different terms), delete the existing entry and all that
155 int addEntriesFrom = 0;
156 if (context.getReplicatedLog().size() > 0) {
158 // Find the entry up until the one that is not in the follower's log
159 for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) {
160 ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
161 ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex());
163 if (newEntry == null) {
164 //newEntry not found in the log
168 if (newEntry.getTerm() == matchEntry.getTerm()) {
172 if(!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) {
174 LOG.debug("{}: Removing entries from log starting at {}", logName(),
175 matchEntry.getIndex());
177 // Entries do not match so remove all subsequent entries
178 context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex());
181 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
182 lastTerm(), context.getPayloadVersion(), true), actor());
188 lastIndex = lastIndex();
189 LOG.debug("{}: After cleanup entries to be added from = {}", logName(),
190 (addEntriesFrom + lastIndex));
192 // 4. Append any new entries not already in the log
193 for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) {
194 ReplicatedLogEntry entry = appendEntries.getEntries().get(i);
196 LOG.debug("{}: Append entry to log {}", logName(), entry.getData());
198 context.getReplicatedLog().appendAndPersist(entry);
200 if(entry.getData() instanceof ServerConfigurationPayload) {
201 context.updatePeerIds((ServerConfigurationPayload)entry.getData());
205 LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
208 // 5. If leaderCommit > commitIndex, set commitIndex =
209 // min(leaderCommit, index of last new entry)
211 lastIndex = lastIndex();
212 long prevCommitIndex = context.getCommitIndex();
214 context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex));
216 if (prevCommitIndex != context.getCommitIndex()) {
217 LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex());
220 // If commitIndex > lastApplied: increment lastApplied, apply
221 // log[lastApplied] to state machine (§5.3)
222 // check if there are any entries to be applied. last-applied can be equal to last-index
223 if (appendEntries.getLeaderCommit() > context.getLastApplied() &&
224 context.getLastApplied() < lastIndex) {
225 if(LOG.isDebugEnabled()) {
226 LOG.debug("{}: applyLogToStateMachine, " +
227 "appendEntries.getLeaderCommit(): {}," +
228 "context.getLastApplied(): {}, lastIndex(): {}", logName(),
229 appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex);
232 applyLogToStateMachine(appendEntries.getLeaderCommit());
235 AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
236 lastIndex, lastTerm(), context.getPayloadVersion());
238 if(LOG.isTraceEnabled()) {
239 LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply);
240 } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
241 LOG.debug("{}: handleAppendEntries returning : {}", logName(), reply);
244 sender.tell(reply, actor());
246 if (!context.getSnapshotManager().isCapturing()) {
247 super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex());
253 private boolean isOutOfSync(AppendEntries appendEntries) {
255 long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
256 boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex());
257 long lastIndex = lastIndex();
258 int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
259 boolean outOfSync = true;
261 if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) {
263 // The follower's log is out of sync because the leader does have
264 // an entry at prevLogIndex and this follower has no entries in
267 LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}",
268 logName(), appendEntries.getPrevLogIndex());
269 } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) {
271 // The follower's log is out of sync because the Leader's
272 // prevLogIndex entry was not found in it's log
274 LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it",
275 logName(), appendEntries.getPrevLogIndex());
276 } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) {
278 // The follower's log is out of sync because the Leader's
279 // prevLogIndex entry does exist in the follower's log but it has
280 // a different term in it
283 "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}",
284 logName(), prevLogTerm, appendEntries.getPrevLogTerm());
285 } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
286 && appendEntries.getReplicatedToAllIndex() != -1
287 && !isLogEntryPresent(appendEntries.getReplicatedToAllIndex())
288 && !context.getReplicatedLog().isInSnapshot(appendEntries.getReplicatedToAllIndex())) {
289 // This append entry comes from a leader who has it's log aggressively trimmed and so does not have
290 // the previous entry in it's in-memory journal
293 "{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the in-memory journal",
294 logName(), appendEntries.getReplicatedToAllIndex());
295 } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
296 && appendEntries.getReplicatedToAllIndex() != -1 && numLogEntries > 0
297 && !isLogEntryPresent(appendEntries.getEntries().get(0).getIndex() - 1)
298 && !context.getReplicatedLog().isInSnapshot(appendEntries.getEntries().get(0).getIndex() - 1)) {
300 "{}: Cannot append entries because the calculated previousIndex {} was not found in the in-memory journal",
301 logName(), appendEntries.getEntries().get(0).getIndex() - 1);
308 @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
309 AppendEntriesReply appendEntriesReply) {
313 @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
314 RequestVoteReply requestVoteReply) {
318 @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
320 Object message = fromSerializableMessage(originalMessage);
322 if (message instanceof RaftRPC) {
323 RaftRPC rpc = (RaftRPC) message;
324 // If RPC request or response contains term T > currentTerm:
325 // set currentTerm = T, convert to follower (§5.1)
326 // This applies to all RPC messages and responses
327 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
328 LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term",
329 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
331 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
335 if (message instanceof ElectionTimeout) {
336 if(canStartElection()) {
337 LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName());
338 return internalSwitchBehavior(RaftState.Candidate);
343 } else if (message instanceof InstallSnapshot) {
344 InstallSnapshot installSnapshot = (InstallSnapshot) message;
345 handleInstallSnapshot(sender, installSnapshot);
348 if(message instanceof RaftRPC && (!(message instanceof RequestVote) || (canGrantVote((RequestVote) message)))){
349 scheduleElection(electionDuration());
352 return super.handleMessage(sender, message);
355 private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot installSnapshot) {
357 LOG.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot);
359 leaderId = installSnapshot.getLeaderId();
361 if(snapshotTracker == null){
362 snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks());
365 updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId());
368 final InstallSnapshotReply reply = new InstallSnapshotReply(
369 currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true);
371 if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
372 installSnapshot.getLastChunkHashCode())){
373 Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(),
374 new ArrayList<ReplicatedLogEntry>(),
375 installSnapshot.getLastIncludedIndex(),
376 installSnapshot.getLastIncludedTerm(),
377 installSnapshot.getLastIncludedIndex(),
378 installSnapshot.getLastIncludedTerm(),
379 context.getTermInformation().getCurrentTerm(),
380 context.getTermInformation().getVotedFor(),
381 context.getPeerServerInfo(true));
383 ApplySnapshot.Callback applySnapshotCallback = new ApplySnapshot.Callback() {
385 public void onSuccess() {
386 LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
388 sender.tell(reply, actor());
392 public void onFailure() {
393 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), -1, false), actor());
397 actor().tell(new ApplySnapshot(snapshot, applySnapshotCallback), actor());
399 snapshotTracker = null;
401 LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
403 sender.tell(reply, actor());
405 } catch (SnapshotTracker.InvalidChunkException e) {
406 LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e);
408 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
409 -1, false), actor());
410 snapshotTracker = null;
412 } catch (Exception e){
413 LOG.error("{}: Exception in InstallSnapshot of follower", logName(), e);
415 //send reply with success as false. The chunk will be sent again on failure
416 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
417 installSnapshot.getChunkIndex(), false), actor());
423 public void close() throws Exception {
428 SnapshotTracker getSnapshotTracker(){
429 return snapshotTracker;