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 akka.japi.Procedure;
13 import com.google.common.annotations.VisibleForTesting;
14 import com.google.common.base.Preconditions;
15 import java.util.ArrayList;
16 import org.opendaylight.controller.cluster.raft.RaftActorContext;
17 import org.opendaylight.controller.cluster.raft.RaftState;
18 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
19 import org.opendaylight.controller.cluster.raft.ServerConfigurationPayload;
20 import org.opendaylight.controller.cluster.raft.Snapshot;
21 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
22 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
23 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
24 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
25 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
26 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
27 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
28 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
29 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
32 * The behavior of a RaftActor in the Follower state
35 * <li> Respond to RPCs from candidates and leaders
36 * <li> If election timeout elapses without receiving AppendEntries
37 * RPC from current leader or granting vote to candidate:
38 * convert to candidate
41 public class Follower extends AbstractRaftActorBehavior {
42 private static final int SYNC_THRESHOLD = 10;
44 private final SyncStatusTracker initialSyncStatusTracker;
46 private final Procedure<ReplicatedLogEntry> appendAndPersistCallback = new Procedure<ReplicatedLogEntry>() {
48 public void apply(ReplicatedLogEntry logEntry) {
49 context.getReplicatedLog().captureSnapshotIfReady(logEntry);
53 private SnapshotTracker snapshotTracker = null;
54 private String leaderId;
55 private short leaderPayloadVersion;
57 public Follower(RaftActorContext context) {
58 this(context, null, (short)-1);
61 public Follower(RaftActorContext context, String initialLeaderId, short initialLeaderPayloadVersion) {
62 super(context, RaftState.Follower);
63 this.leaderId = initialLeaderId;
64 this.leaderPayloadVersion = initialLeaderPayloadVersion;
66 initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD);
68 if(canStartElection()) {
69 if (context.getPeerIds().isEmpty() && getLeaderId() == null) {
70 actor().tell(ElectionTimeout.INSTANCE, actor());
72 scheduleElection(electionDuration());
78 public final String getLeaderId() {
83 protected final void setLeaderId(final String leaderId) {
84 this.leaderId = Preconditions.checkNotNull(leaderId);
88 public short getLeaderPayloadVersion() {
89 return leaderPayloadVersion;
93 protected final void setLeaderPayloadVersion(short leaderPayloadVersion) {
94 this.leaderPayloadVersion = leaderPayloadVersion;
97 private boolean isLogEntryPresent(long index){
98 if(context.getReplicatedLog().isInSnapshot(index)) {
102 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
105 return previousEntry != null;
109 private long getLogEntryTerm(long index){
110 if(index == context.getReplicatedLog().getSnapshotIndex()){
111 return context.getReplicatedLog().getSnapshotTerm();
114 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
117 if(previousEntry != null){
118 return previousEntry.getTerm();
124 private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){
125 initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex());
129 protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) {
131 int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
132 if(LOG.isTraceEnabled()) {
133 LOG.trace("{}: handleAppendEntries: {}", logName(), appendEntries);
134 } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
135 LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
138 // TODO : Refactor this method into a bunch of smaller methods
139 // to make it easier to read. Before refactoring ensure tests
140 // cover the code properly
142 if (snapshotTracker != null || context.getSnapshotManager().isApplying()) {
143 // if snapshot install is in progress, follower should just acknowledge append entries with a reply.
144 AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
145 lastIndex(), lastTerm(), context.getPayloadVersion());
147 if(LOG.isDebugEnabled()) {
148 LOG.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply);
150 sender.tell(reply, actor());
155 // If we got here then we do appear to be talking to the leader
156 leaderId = appendEntries.getLeaderId();
157 leaderPayloadVersion = appendEntries.getPayloadVersion();
159 updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
160 // First check if the logs are in sync or not
161 long lastIndex = lastIndex();
163 if (isOutOfSync(appendEntries)) {
164 // We found that the log was out of sync so just send a negative
167 LOG.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}",
168 logName(), lastIndex, lastTerm());
170 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
171 lastTerm(), context.getPayloadVersion()), actor());
175 if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
177 LOG.debug("{}: Number of entries to be appended = {}", logName(),
178 appendEntries.getEntries().size());
180 // 3. If an existing entry conflicts with a new one (same index
181 // but different terms), delete the existing entry and all that
183 int addEntriesFrom = 0;
184 if (context.getReplicatedLog().size() > 0) {
186 // Find the entry up until the one that is not in the follower's log
187 for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) {
188 ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
189 ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex());
191 if (newEntry == null) {
192 //newEntry not found in the log
196 if (newEntry.getTerm() == matchEntry.getTerm()) {
200 if(!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) {
202 LOG.debug("{}: Removing entries from log starting at {}", logName(),
203 matchEntry.getIndex());
205 // Entries do not match so remove all subsequent entries
206 context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex());
209 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
210 lastTerm(), context.getPayloadVersion(), true), actor());
216 lastIndex = lastIndex();
217 LOG.debug("{}: After cleanup entries to be added from = {}", logName(),
218 (addEntriesFrom + lastIndex));
220 // 4. Append any new entries not already in the log
221 for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) {
222 ReplicatedLogEntry entry = appendEntries.getEntries().get(i);
224 LOG.debug("{}: Append entry to log {}", logName(), entry.getData());
226 context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback);
228 if(entry.getData() instanceof ServerConfigurationPayload) {
229 context.updatePeerIds((ServerConfigurationPayload)entry.getData());
233 LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
236 // 5. If leaderCommit > commitIndex, set commitIndex =
237 // min(leaderCommit, index of last new entry)
239 lastIndex = lastIndex();
240 long prevCommitIndex = context.getCommitIndex();
242 context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex));
244 if (prevCommitIndex != context.getCommitIndex()) {
245 LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex());
248 // If commitIndex > lastApplied: increment lastApplied, apply
249 // log[lastApplied] to state machine (ยง5.3)
250 // check if there are any entries to be applied. last-applied can be equal to last-index
251 if (appendEntries.getLeaderCommit() > context.getLastApplied() &&
252 context.getLastApplied() < lastIndex) {
253 if(LOG.isDebugEnabled()) {
254 LOG.debug("{}: applyLogToStateMachine, " +
255 "appendEntries.getLeaderCommit(): {}," +
256 "context.getLastApplied(): {}, lastIndex(): {}", logName(),
257 appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex);
260 applyLogToStateMachine(appendEntries.getLeaderCommit());
263 AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
264 lastIndex, lastTerm(), context.getPayloadVersion());
266 if(LOG.isTraceEnabled()) {
267 LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply);
268 } else if(LOG.isDebugEnabled() && numLogEntries > 0) {
269 LOG.debug("{}: handleAppendEntries returning : {}", logName(), reply);
272 sender.tell(reply, actor());
274 if (!context.getSnapshotManager().isCapturing()) {
275 super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex());
281 private boolean isOutOfSync(AppendEntries appendEntries) {
283 long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
284 boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex());
285 long lastIndex = lastIndex();
286 int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
287 boolean outOfSync = true;
289 if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) {
291 // The follower's log is out of sync because the leader does have
292 // an entry at prevLogIndex and this follower has no entries in
295 LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}",
296 logName(), appendEntries.getPrevLogIndex());
297 } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) {
299 // The follower's log is out of sync because the Leader's
300 // prevLogIndex entry was not found in it's log
302 LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it - lastIndex: {}, snapshotIndex: {}",
303 logName(), appendEntries.getPrevLogIndex(), lastIndex, context.getReplicatedLog().getSnapshotIndex());
304 } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) {
306 // The follower's log is out of sync because the Leader's
307 // prevLogIndex entry does exist in the follower's log but it has
308 // a different term in it
311 "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}",
312 logName(), prevLogTerm, appendEntries.getPrevLogTerm());
313 } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
314 && appendEntries.getReplicatedToAllIndex() != -1
315 && !isLogEntryPresent(appendEntries.getReplicatedToAllIndex())) {
316 // This append entry comes from a leader who has it's log aggressively trimmed and so does not have
317 // the previous entry in it's in-memory journal
320 "{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the in-memory journal",
321 logName(), appendEntries.getReplicatedToAllIndex());
322 } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
323 && appendEntries.getReplicatedToAllIndex() != -1 && numLogEntries > 0
324 && !isLogEntryPresent(appendEntries.getEntries().get(0).getIndex() - 1)) {
326 "{}: Cannot append entries because the calculated previousIndex {} was not found in the in-memory journal",
327 logName(), appendEntries.getEntries().get(0).getIndex() - 1);
335 protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
336 AppendEntriesReply appendEntriesReply) {
341 protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
342 RequestVoteReply requestVoteReply) {
347 public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
348 if (originalMessage instanceof ElectionTimeout) {
349 if (canStartElection()) {
350 LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName());
351 return internalSwitchBehavior(RaftState.Candidate);
357 final Object message = fromSerializableMessage(originalMessage);
358 if (!(message instanceof RaftRPC)) {
359 // The rest of the processing requires the message to be a RaftRPC
363 final RaftRPC rpc = (RaftRPC) message;
364 // If RPC request or response contains term T > currentTerm:
365 // set currentTerm = T, convert to follower (ยง5.1)
366 // This applies to all RPC messages and responses
367 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
368 LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term",
369 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
371 context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
374 if (rpc instanceof InstallSnapshot) {
375 InstallSnapshot installSnapshot = (InstallSnapshot) rpc;
376 handleInstallSnapshot(sender, installSnapshot);
377 scheduleElection(electionDuration());
381 if (!(rpc instanceof RequestVote) || canGrantVote((RequestVote) rpc)) {
382 scheduleElection(electionDuration());
385 return super.handleMessage(sender, rpc);
388 private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot installSnapshot) {
390 LOG.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot);
392 leaderId = installSnapshot.getLeaderId();
394 if(snapshotTracker == null){
395 snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks());
398 updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId());
401 final InstallSnapshotReply reply = new InstallSnapshotReply(
402 currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true);
404 if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
405 installSnapshot.getLastChunkHashCode())){
406 Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(),
407 new ArrayList<ReplicatedLogEntry>(),
408 installSnapshot.getLastIncludedIndex(),
409 installSnapshot.getLastIncludedTerm(),
410 installSnapshot.getLastIncludedIndex(),
411 installSnapshot.getLastIncludedTerm(),
412 context.getTermInformation().getCurrentTerm(),
413 context.getTermInformation().getVotedFor(),
414 context.getPeerServerInfo(true));
416 ApplySnapshot.Callback applySnapshotCallback = new ApplySnapshot.Callback() {
418 public void onSuccess() {
419 LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
421 sender.tell(reply, actor());
425 public void onFailure() {
426 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), -1, false), actor());
430 actor().tell(new ApplySnapshot(snapshot, applySnapshotCallback), actor());
432 snapshotTracker = null;
434 LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
436 sender.tell(reply, actor());
438 } catch (SnapshotTracker.InvalidChunkException e) {
439 LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e);
441 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
442 -1, false), actor());
443 snapshotTracker = null;
445 } catch (Exception e){
446 LOG.error("{}: Exception in InstallSnapshot of follower", logName(), e);
448 //send reply with success as false. The chunk will be sent again on failure
449 sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
450 installSnapshot.getChunkIndex(), false), actor());
456 public void close() {
461 SnapshotTracker getSnapshotTracker(){
462 return snapshotTracker;