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 org.opendaylight.controller.cluster.raft.RaftActorContext;
13 import org.opendaylight.controller.cluster.raft.RaftState;
14 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
15 import org.opendaylight.controller.cluster.raft.internal.messages.ApplySnapshot;
16 import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout;
17 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
18 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
19 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
20 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
21 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
24 * The behavior of a RaftActor in the Follower state
27 * <li> Respond to RPCs from candidates and leaders
28 * <li> If election timeout elapses without receiving AppendEntries
29 * RPC from current leader or granting vote to candidate:
30 * convert to candidate
33 public class Follower extends AbstractRaftActorBehavior {
34 public Follower(RaftActorContext context) {
37 scheduleElection(electionDuration());
40 @Override protected RaftState handleAppendEntries(ActorRef sender,
41 AppendEntries appendEntries) {
43 // If we got here then we do appear to be talking to the leader
44 leaderId = appendEntries.getLeaderId();
46 // 2. Reply false if log doesn’t contain an entry at prevLogIndex
47 // whose term matches prevLogTerm (§5.3)
48 ReplicatedLogEntry previousEntry = context.getReplicatedLog()
49 .get(appendEntries.getPrevLogIndex());
52 boolean noMatchingTerms = true;
55 && appendEntries.getPrevLogIndex() != -1) {
57 context.getLogger().debug(
58 "The followers log is empty and the senders prevLogIndex is {}",
59 appendEntries.getPrevLogIndex());
61 } else if (lastIndex() > -1
62 && appendEntries.getPrevLogIndex() != -1
63 && previousEntry == null) {
65 context.getLogger().debug(
66 "The log is not empty but the prevLogIndex {} was not found in it",
67 appendEntries.getPrevLogIndex());
69 } else if (lastIndex() > -1
70 && previousEntry != null
71 && previousEntry.getTerm()!= appendEntries.getPrevLogTerm()) {
73 context.getLogger().debug(
74 "Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}"
75 , previousEntry.getTerm()
76 , appendEntries.getPrevLogTerm());
78 noMatchingTerms = false;
81 if (noMatchingTerms) {
83 new AppendEntriesReply(context.getId(), currentTerm(), false,
84 lastIndex(), lastTerm()), actor()
89 if (appendEntries.getEntries() != null
90 && appendEntries.getEntries().size() > 0) {
91 context.getLogger().debug(
92 "Number of entries to be appended = " + appendEntries
96 // 3. If an existing entry conflicts with a new one (same index
97 // but different terms), delete the existing entry and all that
99 int addEntriesFrom = 0;
100 if (context.getReplicatedLog().size() > 0) {
102 i < appendEntries.getEntries()
103 .size(); i++, addEntriesFrom++) {
104 ReplicatedLogEntry matchEntry =
105 appendEntries.getEntries().get(i);
106 ReplicatedLogEntry newEntry = context.getReplicatedLog()
107 .get(matchEntry.getIndex());
109 if (newEntry == null) {
110 //newEntry not found in the log
114 if (newEntry != null && newEntry.getTerm() == matchEntry
118 if (newEntry != null && newEntry.getTerm() != matchEntry
120 context.getLogger().debug(
121 "Removing entries from log starting at "
122 + matchEntry.getIndex()
124 context.getReplicatedLog()
125 .removeFrom(matchEntry.getIndex());
131 context.getLogger().debug(
132 "After cleanup entries to be added from = " + (addEntriesFrom
136 // 4. Append any new entries not already in the log
137 for (int i = addEntriesFrom;
138 i < appendEntries.getEntries().size(); i++) {
139 context.getLogger().debug(
140 "Append entry to log " + appendEntries.getEntries().get(i).getData()
143 context.getReplicatedLog()
144 .appendAndPersist(appendEntries.getEntries().get(i));
147 context.getLogger().debug(
148 "Log size is now " + context.getReplicatedLog().size());
152 // 5. If leaderCommit > commitIndex, set commitIndex =
153 // min(leaderCommit, index of last new entry)
155 long prevCommitIndex = context.getCommitIndex();
157 context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(),
158 context.getReplicatedLog().lastIndex()));
160 if (prevCommitIndex != context.getCommitIndex()) {
162 .debug("Commit index set to " + context.getCommitIndex());
165 // If commitIndex > lastApplied: increment lastApplied, apply
166 // log[lastApplied] to state machine (§5.3)
167 if (appendEntries.getLeaderCommit() > context.getLastApplied()) {
168 applyLogToStateMachine(appendEntries.getLeaderCommit());
171 sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), true,
172 lastIndex(), lastTerm()), actor());
177 @Override protected RaftState handleAppendEntriesReply(ActorRef sender,
178 AppendEntriesReply appendEntriesReply) {
182 @Override protected RaftState handleRequestVoteReply(ActorRef sender,
183 RequestVoteReply requestVoteReply) {
187 @Override public RaftState state() {
188 return RaftState.Follower;
191 @Override public RaftState handleMessage(ActorRef sender, Object message) {
192 if (message instanceof RaftRPC) {
193 RaftRPC rpc = (RaftRPC) message;
194 // If RPC request or response contains term T > currentTerm:
195 // set currentTerm = T, convert to follower (§5.1)
196 // This applies to all RPC messages and responses
197 if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
198 context.getTermInformation().update(rpc.getTerm(), null);
202 if (message instanceof ElectionTimeout) {
203 return RaftState.Candidate;
204 } else if (message instanceof InstallSnapshot) {
205 InstallSnapshot snapshot = (InstallSnapshot) message;
206 actor().tell(new ApplySnapshot(snapshot), actor());
209 scheduleElection(electionDuration());
211 return super.handleMessage(sender, message);
214 @Override public void close() throws Exception {