Bug 7407 - Add request leadership functionality to shards
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActorLeadershipTransferCohort.java
1 /*
2  * Copyright (c) 2015 Brocade Communications 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;
9
10 import akka.actor.ActorRef;
11 import akka.actor.ActorSelection;
12 import akka.actor.Cancellable;
13 import com.google.common.annotations.VisibleForTesting;
14 import com.google.common.base.Optional;
15 import com.google.common.base.Stopwatch;
16 import java.util.ArrayList;
17 import java.util.List;
18 import java.util.concurrent.TimeUnit;
19 import javax.annotation.Nullable;
20 import org.opendaylight.controller.cluster.raft.base.messages.LeaderTransitioning;
21 import org.opendaylight.controller.cluster.raft.behaviors.Leader;
22 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
23 import org.slf4j.Logger;
24 import org.slf4j.LoggerFactory;
25 import scala.concurrent.duration.FiniteDuration;
26
27 /**
28  * A raft actor support class that participates in leadership transfer. An instance is created upon
29  * initialization of leadership transfer.
30  *
31  * <p>
32  * The transfer process is as follows:
33  * <ol>
34  * <li>Send a LeaderStateChanged message with a null leader Id to the local RoleChangeNotifier to notify
35  *     clients that we no longer have a working leader.</li>
36  * <li>Send a LeaderTransitioning message to each follower so each can send LeaderStateChanged messages to
37  *     their local RoleChangeNotifiers.</li>
38  * <li>Call {@link RaftActor#pauseLeader} passing this RaftActorLeadershipTransferCohort
39  *     instance. This allows derived classes to perform work prior to transferring leadership.</li>
40  * <li>When the pause is complete, the run method is called which in turn calls
41  *     {@link Leader#transferLeadership(RaftActorLeadershipTransferCohort)}.</li>
42  * <li>The Leader calls {@link #transferComplete} on successful completion.</li>
43  * <li>Wait a short period of time for the new leader to be elected to give the derived class a chance to
44  *     possibly complete work that was suspended while we were transferring.</li>
45  * <li>On notification of the new leader from the RaftActor or on time out, notify {@link OnComplete} callbacks.</li>
46  * </ol>
47  *
48  * <p>
49  * NOTE: All methods on this class must be called on the actor's thread dispatcher as they may access/modify
50  * internal state.
51  *
52  * @author Thomas Pantelis
53  */
54 public class RaftActorLeadershipTransferCohort {
55     private static final Logger LOG = LoggerFactory.getLogger(RaftActorLeadershipTransferCohort.class);
56
57     private final List<OnComplete> onCompleteCallbacks = new ArrayList<>();
58     private final Stopwatch transferTimer = Stopwatch.createUnstarted();
59     private final RaftActor raftActor;
60     private final String requestedFollowerId;
61
62     private long newLeaderTimeoutInMillis = 2000;
63     private Cancellable newLeaderTimer;
64     private boolean isTransferring;
65
66     RaftActorLeadershipTransferCohort(final RaftActor raftActor) {
67         this(raftActor, null);
68     }
69
70     RaftActorLeadershipTransferCohort(final RaftActor raftActor, @Nullable final String requestedFollowerId) {
71         this.raftActor = raftActor;
72         this.requestedFollowerId = requestedFollowerId;
73     }
74
75     void init() {
76         RaftActorContext context = raftActor.getRaftActorContext();
77         RaftActorBehavior currentBehavior = raftActor.getCurrentBehavior();
78
79         transferTimer.start();
80
81         Optional<ActorRef> roleChangeNotifier = raftActor.getRoleChangeNotifier();
82         if (roleChangeNotifier.isPresent()) {
83             roleChangeNotifier.get().tell(raftActor.newLeaderStateChanged(context.getId(), null,
84                     currentBehavior.getLeaderPayloadVersion()), raftActor.self());
85         }
86
87         for (String peerId: context.getPeerIds()) {
88             ActorSelection followerActor = context.getPeerActorSelection(peerId);
89             if (followerActor != null) {
90                 followerActor.tell(new LeaderTransitioning(context.getId()), context.getActor());
91             }
92         }
93
94         raftActor.pauseLeader(new TimedRunnable(context.getConfigParams().getElectionTimeOutInterval(), raftActor) {
95             @Override
96             protected void doRun() {
97                 doTransfer();
98             }
99
100             @Override
101             protected void doCancel() {
102                 LOG.debug("{}: pauseLeader timed out - aborting transfer", raftActor.persistenceId());
103                 abortTransfer();
104             }
105         });
106     }
107
108     /**
109      * This method is invoked to perform the leadership transfer.
110      */
111     @VisibleForTesting
112     void doTransfer() {
113         RaftActorBehavior behavior = raftActor.getCurrentBehavior();
114         // Sanity check...
115         if (behavior instanceof Leader) {
116             isTransferring = true;
117             ((Leader)behavior).transferLeadership(this);
118         } else {
119             LOG.debug("{}: No longer the leader - skipping transfer", raftActor.persistenceId());
120             finish(true);
121         }
122     }
123
124     /**
125      * This method is invoked to abort leadership transfer on failure.
126      */
127     public void abortTransfer() {
128         LOG.debug("{}: leader transfer aborted", raftActor.persistenceId());
129         finish(false);
130     }
131
132     /**
133      * This method is invoked when leadership transfer was carried out and complete.
134      */
135     public void transferComplete() {
136         LOG.debug("{}: leader transfer complete - waiting for new leader", raftActor.persistenceId());
137
138         // We'll give it a little time for the new leader to be elected to give the derived class a
139         // chance to possibly complete work that was suspended while we were transferring. The
140         // RequestVote message from the new leader candidate should cause us to step down as leader
141         // and convert to follower due to higher term. We should then get an AppendEntries heart
142         // beat with the new leader id.
143
144         // Add a timer in case we don't get a leader change - 2 sec should be plenty of time if a new
145         // leader is elected. Note: the Runnable is sent as a message to the raftActor which executes it
146         // safely run on the actor's thread dispatcher.
147         FiniteDuration timeout = FiniteDuration.create(newLeaderTimeoutInMillis, TimeUnit.MILLISECONDS);
148         newLeaderTimer = raftActor.getContext().system().scheduler().scheduleOnce(timeout, raftActor.self(),
149             (Runnable) () -> {
150                 LOG.debug("{}: leader not elected in time", raftActor.persistenceId());
151                 finish(true);
152             }, raftActor.getContext().system().dispatcher(), raftActor.self());
153     }
154
155     void onNewLeader(final String newLeader) {
156         if (newLeader != null && newLeaderTimer != null) {
157             LOG.debug("{}: leader changed to {}", raftActor.persistenceId(), newLeader);
158             newLeaderTimer.cancel();
159             finish(true);
160         }
161     }
162
163     private void finish(final boolean success) {
164         isTransferring = false;
165         if (transferTimer.isRunning()) {
166             transferTimer.stop();
167             if (success) {
168                 LOG.info("{}: Successfully transferred leadership to {} in {}", raftActor.persistenceId(),
169                         raftActor.getLeaderId(), transferTimer);
170             } else {
171                 LOG.warn("{}: Failed to transfer leadership in {}", raftActor.persistenceId(), transferTimer);
172             }
173         }
174
175         for (OnComplete onComplete: onCompleteCallbacks) {
176             if (success) {
177                 onComplete.onSuccess(raftActor.self());
178             } else {
179                 onComplete.onFailure(raftActor.self());
180             }
181         }
182     }
183
184     void addOnComplete(final OnComplete onComplete) {
185         onCompleteCallbacks.add(onComplete);
186     }
187
188     boolean isTransferring() {
189         return isTransferring;
190     }
191
192     @VisibleForTesting
193     void setNewLeaderTimeoutInMillis(final long newLeaderTimeoutInMillis) {
194         this.newLeaderTimeoutInMillis = newLeaderTimeoutInMillis;
195     }
196
197     public Optional<String> getRequestedFollowerId() {
198         return Optional.fromNullable(requestedFollowerId);
199     }
200
201     interface OnComplete {
202         void onSuccess(ActorRef raftActorRef);
203
204         void onFailure(ActorRef raftActorRef);
205     }
206 }