e58d4f57c6cc426b4ca9808883d7aff1129f3b15
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / Leader.java
1 /*
2  * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
3  *
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
7  */
8 package org.opendaylight.controller.cluster.raft.behaviors;
9
10 import akka.actor.ActorRef;
11 import akka.actor.ActorSelection;
12 import com.google.common.annotations.VisibleForTesting;
13 import com.google.common.base.Preconditions;
14 import com.google.common.base.Stopwatch;
15 import java.util.concurrent.TimeUnit;
16 import javax.annotation.Nonnull;
17 import javax.annotation.Nullable;
18 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
19 import org.opendaylight.controller.cluster.raft.RaftActorContext;
20 import org.opendaylight.controller.cluster.raft.RaftActorLeadershipTransferCohort;
21 import org.opendaylight.controller.cluster.raft.RaftState;
22 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
23 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
24
25 /**
26  * The behavior of a RaftActor when it is in the Leader state
27  * <p/>
28  * Leaders:
29  * <ul>
30  * <li> Upon election: send initial empty AppendEntries RPCs
31  * (heartbeat) to each server; repeat during idle periods to
32  * prevent election timeouts (§5.2)
33  * <li> If command received from client: append entry to local log,
34  * respond after entry applied to state machine (§5.3)
35  * <li> If last log index ≥ nextIndex for a follower: send
36  * AppendEntries RPC with log entries starting at nextIndex
37  * <ul>
38  * <li> If successful: update nextIndex and matchIndex for
39  * follower (§5.3)
40  * <li> If AppendEntries fails because of log inconsistency:
41  * decrement nextIndex and retry (§5.3)
42  * </ul>
43  * <li> If there exists an N such that N > commitIndex, a majority
44  * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
45  * set commitIndex = N (§5.3, §5.4).
46  */
47 public class Leader extends AbstractLeader {
48     /**
49      * Internal message sent to periodically check if this leader has become isolated and should transition
50      * to {@link IsolatedLeader}.
51      */
52     @VisibleForTesting
53     static final Object ISOLATED_LEADER_CHECK = new Object();
54
55     private final Stopwatch isolatedLeaderCheck;
56     private @Nullable LeadershipTransferContext leadershipTransferContext;
57
58     public Leader(RaftActorContext context) {
59         super(context, RaftState.Leader);
60         isolatedLeaderCheck = Stopwatch.createStarted();
61     }
62
63     @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
64         Preconditions.checkNotNull(sender, "sender should not be null");
65
66         if (ISOLATED_LEADER_CHECK.equals(originalMessage)) {
67             if (isLeaderIsolated()) {
68                 LOG.warn("{}: At least {} followers need to be active, Switching {} from Leader to IsolatedLeader",
69                         context.getId(), getMinIsolatedLeaderPeerCount(), leaderId);
70
71                 return internalSwitchBehavior(RaftState.IsolatedLeader);
72             }
73         }
74
75         return super.handleMessage(sender, originalMessage);
76     }
77
78     @Override
79     protected void beforeSendHeartbeat(){
80         if(isolatedLeaderCheck.elapsed(TimeUnit.MILLISECONDS) > context.getConfigParams().getIsolatedCheckIntervalInMillis()){
81             context.getActor().tell(ISOLATED_LEADER_CHECK, context.getActor());
82             isolatedLeaderCheck.reset().start();
83         }
84
85         if(leadershipTransferContext != null && leadershipTransferContext.isExpired(
86                 context.getConfigParams().getElectionTimeOutInterval().toMillis())) {
87             LOG.debug("{}: Leadership transfer expired", logName());
88             leadershipTransferContext = null;
89         }
90     }
91
92     @Override
93     protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) {
94         RaftActorBehavior returnBehavior = super.handleAppendEntriesReply(sender, appendEntriesReply);
95         tryToCompleteLeadershipTransfer(appendEntriesReply.getFollowerId());
96         return returnBehavior;
97     }
98
99     /**
100      * Attempts to transfer leadership to a follower as per the raft paper (§3.10) as follows:
101      * <ul>
102      * <li>Start a timer (Stopwatch).</li>
103      * <li>Send an initial AppendEntries heartbeat to all followers.</li>
104      * <li>On AppendEntriesReply, check if the follower's new match Index matches the leader's last index</li>
105      * <li>If it matches, </li>
106      *   <ul>
107      *   <li>Send an additional AppendEntries to ensure the follower has applied all its log entries to its state.</li>
108      *   <li>Send an ElectionTimeout to the follower to immediately start an election.</li>
109      *   <li>Notify {@link RaftActorLeadershipTransferCohort#transferComplete}.</li>
110      *   </ul>
111      * <li>Otherwise if the election time out period elapses, notify
112      *     {@link RaftActorLeadershipTransferCohort#abortTtransfer}.</li>
113      * </ul>
114      *
115      * @param leadershipTransferCohort
116      */
117     public void transferLeadership(@Nonnull RaftActorLeadershipTransferCohort leadershipTransferCohort) {
118         LOG.debug("{}: Attempting to transfer leadership", logName());
119
120         leadershipTransferContext = new LeadershipTransferContext(leadershipTransferCohort);
121
122         // Send an immediate heart beat to the followers.
123         sendAppendEntries(0, false);
124     }
125
126     private void tryToCompleteLeadershipTransfer(String followerId) {
127         if(leadershipTransferContext == null) {
128             return;
129         }
130
131         FollowerLogInformation followerInfo = getFollower(followerId);
132         if(followerInfo == null) {
133             return;
134         }
135
136         long lastIndex = context.getReplicatedLog().lastIndex();
137         boolean isVoting = context.getPeerInfo(followerId).isVoting();
138
139         LOG.debug("{}: tryToCompleteLeadershipTransfer: followerId: {}, matchIndex: {}, lastIndex: {}, isVoting: {}",
140                 logName(), followerId, followerInfo.getMatchIndex(), lastIndex, isVoting);
141
142         if(isVoting && followerInfo.getMatchIndex() == lastIndex) {
143             LOG.debug("{}: Follower's log matches - sending ElectionTimeout", logName());
144
145             // We can't be sure if the follower has applied all its log entries to its state so send an
146             // additional AppendEntries with the latest commit index.
147             sendAppendEntries(0, false);
148
149             // Now send an ElectionTimeout to the matching follower to immediately start an election.
150             ActorSelection followerActor = context.getPeerActorSelection(followerId);
151             followerActor.tell(ElectionTimeout.INSTANCE, context.getActor());
152
153             LOG.debug("{}: Leader transfer complete", logName());
154
155             leadershipTransferContext.transferCohort.transferComplete();
156             leadershipTransferContext = null;
157         }
158     }
159
160     @Override
161     public void close() {
162         if(leadershipTransferContext != null) {
163             leadershipTransferContext.transferCohort.abortTransfer();
164         }
165
166         super.close();
167     }
168
169     @VisibleForTesting
170     void markFollowerActive(String followerId) {
171         getFollower(followerId).markFollowerActive();
172     }
173
174     @VisibleForTesting
175     void markFollowerInActive(String followerId) {
176         getFollower(followerId).markFollowerInActive();
177     }
178
179     private static class LeadershipTransferContext {
180         RaftActorLeadershipTransferCohort transferCohort;
181         Stopwatch timer = Stopwatch.createStarted();
182
183         LeadershipTransferContext(RaftActorLeadershipTransferCohort transferCohort) {
184             this.transferCohort = transferCohort;
185         }
186
187         boolean isExpired(long timeout) {
188             if(timer.elapsed(TimeUnit.MILLISECONDS) >= timeout) {
189                 transferCohort.abortTransfer();
190                 return true;
191             }
192
193             return false;
194         }
195     }
196 }