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.actor.ActorSelection;
13 import akka.actor.Cancellable;
14 import com.google.common.base.Preconditions;
15 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
16 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
17 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
18 import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
19 import org.opendaylight.controller.cluster.raft.RaftActorContext;
20 import org.opendaylight.controller.cluster.raft.RaftState;
21 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
22 import org.opendaylight.controller.cluster.raft.internal.messages.ApplyState;
23 import org.opendaylight.controller.cluster.raft.internal.messages.Replicate;
24 import org.opendaylight.controller.cluster.raft.internal.messages.SendHeartBeat;
25 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
26 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
27 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
28 import scala.concurrent.duration.FiniteDuration;
30 import java.util.ArrayList;
31 import java.util.HashMap;
32 import java.util.List;
34 import java.util.concurrent.TimeUnit;
35 import java.util.concurrent.atomic.AtomicLong;
38 * The behavior of a RaftActor when it is in the Leader state
42 * <li> Upon election: send initial empty AppendEntries RPCs
43 * (heartbeat) to each server; repeat during idle periods to
44 * prevent election timeouts (§5.2)
45 * <li> If command received from client: append entry to local log,
46 * respond after entry applied to state machine (§5.3)
47 * <li> If last log index ≥ nextIndex for a follower: send
48 * AppendEntries RPC with log entries starting at nextIndex
50 * <li> If successful: update nextIndex and matchIndex for
52 * <li> If AppendEntries fails because of log inconsistency:
53 * decrement nextIndex and retry (§5.3)
55 * <li> If there exists an N such that N > commitIndex, a majority
56 * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
57 * set commitIndex = N (§5.3, §5.4).
59 public class Leader extends AbstractRaftActorBehavior {
62 private final Map<String, FollowerLogInformation> followerToLog =
65 private final Map<String, ActorSelection> followerToActor = new HashMap<>();
67 private Cancellable heartbeatCancel = null;
69 private List<ClientRequestTracker> trackerList = new ArrayList<>();
71 private final int minReplicationCount;
73 public Leader(RaftActorContext context) {
76 if(lastIndex() >= 0) {
77 context.setCommitIndex(lastIndex());
80 for (String followerId : context.getPeerAddresses().keySet()) {
81 FollowerLogInformation followerLogInformation =
82 new FollowerLogInformationImpl(followerId,
83 new AtomicLong(lastIndex()),
86 followerToActor.put(followerId,
87 context.actorSelection(context.getPeerAddress(followerId)));
89 followerToLog.put(followerId, followerLogInformation);
93 if (followerToActor.size() > 0) {
94 minReplicationCount = (followerToActor.size() + 1) / 2 + 1;
96 minReplicationCount = 0;
100 // Immediately schedule a heartbeat
101 // Upon election: send initial empty AppendEntries RPCs
102 // (heartbeat) to each server; repeat during idle periods to
103 // prevent election timeouts (§5.2)
104 scheduleHeartBeat(new FiniteDuration(0, TimeUnit.SECONDS));
109 @Override protected RaftState handleAppendEntries(ActorRef sender,
110 AppendEntries appendEntries, RaftState suggestedState) {
113 .error("An unexpected AppendEntries received in state " + state());
115 return suggestedState;
118 @Override protected RaftState handleAppendEntriesReply(ActorRef sender,
119 AppendEntriesReply appendEntriesReply, RaftState suggestedState) {
121 // Do not take any other action since a behavior change is coming
122 if (suggestedState != state())
123 return suggestedState;
125 // Update the FollowerLogInformation
126 String followerId = appendEntriesReply.getFollowerId();
127 FollowerLogInformation followerLogInformation =
128 followerToLog.get(followerId);
129 if (appendEntriesReply.isSuccess()) {
130 followerLogInformation
131 .setMatchIndex(appendEntriesReply.getLogLastIndex());
132 followerLogInformation
133 .setNextIndex(appendEntriesReply.getLogLastIndex() + 1);
135 followerLogInformation.decrNextIndex();
138 // Now figure out if this reply warrants a change in the commitIndex
139 // If there exists an N such that N > commitIndex, a majority
140 // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
141 // set commitIndex = N (§5.3, §5.4).
142 for (long N = context.getCommitIndex() + 1; ; N++) {
143 int replicatedCount = 1;
145 for (FollowerLogInformation info : followerToLog.values()) {
146 if (info.getMatchIndex().get() >= N) {
151 if (replicatedCount >= minReplicationCount){
152 ReplicatedLogEntry replicatedLogEntry =
153 context.getReplicatedLog().get(N);
154 if (replicatedLogEntry != null
155 && replicatedLogEntry.getTerm()
157 context.setCommitIndex(N);
164 if(context.getCommitIndex() > context.getLastApplied()){
165 applyLogToStateMachine(context.getCommitIndex());
168 return suggestedState;
171 protected ClientRequestTracker findClientRequestTracker(long logIndex) {
172 for (ClientRequestTracker tracker : trackerList) {
173 if (tracker.getIndex() == logIndex) {
181 @Override protected RaftState handleRequestVoteReply(ActorRef sender,
182 RequestVoteReply requestVoteReply, RaftState suggestedState) {
183 return suggestedState;
186 @Override public RaftState state() {
187 return RaftState.Leader;
190 @Override public RaftState handleMessage(ActorRef sender, Object message) {
191 Preconditions.checkNotNull(sender, "sender should not be null");
194 if (message instanceof SendHeartBeat) {
195 return sendHeartBeat();
196 } else if (message instanceof Replicate) {
198 Replicate replicate = (Replicate) message;
199 long logIndex = replicate.getReplicatedLogEntry().getIndex();
201 context.getLogger().debug("Replicate message " + logIndex);
203 if (followerToActor.size() == 0) {
204 context.setCommitIndex(
205 replicate.getReplicatedLogEntry().getIndex());
208 .tell(new ApplyState(replicate.getClientActor(),
209 replicate.getIdentifier(),
210 replicate.getReplicatedLogEntry()),
216 new ClientRequestTrackerImpl(replicate.getClientActor(),
217 replicate.getIdentifier(),
221 ReplicatedLogEntry prevEntry =
222 context.getReplicatedLog().get(lastIndex() - 1);
223 long prevLogIndex = -1;
224 long prevLogTerm = -1;
225 if (prevEntry != null) {
226 prevLogIndex = prevEntry.getIndex();
227 prevLogTerm = prevEntry.getTerm();
229 // Send an AppendEntries to all followers
230 for (String followerId : followerToActor.keySet()) {
231 ActorSelection followerActor =
232 followerToActor.get(followerId);
233 FollowerLogInformation followerLogInformation =
234 followerToLog.get(followerId);
236 new AppendEntries(currentTerm(), context.getId(),
237 prevLogIndex, prevLogTerm,
238 context.getReplicatedLog().getFrom(
239 followerLogInformation.getNextIndex()
241 ), context.getCommitIndex()
249 scheduleHeartBeat(HEART_BEAT_INTERVAL);
252 return super.handleMessage(sender, message);
255 private RaftState sendHeartBeat() {
256 if (followerToActor.size() > 0) {
257 for (String follower : followerToActor.keySet()) {
259 FollowerLogInformation followerLogInformation =
260 followerToLog.get(follower);
262 AtomicLong nextIndex =
263 followerLogInformation.getNextIndex();
265 List<ReplicatedLogEntry> entries =
266 context.getReplicatedLog().getFrom(nextIndex.get());
268 followerToActor.get(follower).tell(new AppendEntries(
269 context.getTermInformation().getCurrentTerm(),
271 context.getReplicatedLog().lastIndex(),
272 context.getReplicatedLog().lastTerm(),
273 entries, context.getCommitIndex()),
281 private void stopHeartBeat() {
282 if (heartbeatCancel != null && !heartbeatCancel.isCancelled()) {
283 heartbeatCancel.cancel();
287 private void scheduleHeartBeat(FiniteDuration interval) {
290 // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
291 // message is sent to itself.
292 // Scheduling the heartbeat only once here because heartbeats do not
293 // need to be sent if there are other messages being sent to the remote
296 context.getActorSystem().scheduler().scheduleOnce(
298 context.getActor(), new SendHeartBeat(),
299 context.getActorSystem().dispatcher(), context.getActor());
302 @Override public void close() throws Exception {
306 @Override public String getLeaderId() {
307 return context.getId();