f76c7d70d528a3357afb1d90869b1245d0701449
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActorServerConfigurationSupport.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.base.Preconditions;
14 import java.util.ArrayDeque;
15 import java.util.ArrayList;
16 import java.util.Collection;
17 import java.util.HashSet;
18 import java.util.List;
19 import java.util.Map;
20 import java.util.Queue;
21 import java.util.UUID;
22 import javax.annotation.Nullable;
23 import org.opendaylight.controller.cluster.raft.ServerConfigurationPayload.ServerInfo;
24 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
25 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
26 import org.opendaylight.controller.cluster.raft.base.messages.SnapshotComplete;
27 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
28 import org.opendaylight.controller.cluster.raft.messages.AddServer;
29 import org.opendaylight.controller.cluster.raft.messages.AddServerReply;
30 import org.opendaylight.controller.cluster.raft.messages.ChangeServersVotingStatus;
31 import org.opendaylight.controller.cluster.raft.messages.RemoveServer;
32 import org.opendaylight.controller.cluster.raft.messages.RemoveServerReply;
33 import org.opendaylight.controller.cluster.raft.messages.ServerChangeReply;
34 import org.opendaylight.controller.cluster.raft.messages.ServerChangeStatus;
35 import org.opendaylight.controller.cluster.raft.messages.ServerRemoved;
36 import org.opendaylight.controller.cluster.raft.messages.UnInitializedFollowerSnapshotReply;
37 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
38 import org.opendaylight.yangtools.concepts.Identifier;
39 import org.opendaylight.yangtools.util.UUIDIdentifier;
40 import org.slf4j.Logger;
41 import org.slf4j.LoggerFactory;
42 import scala.concurrent.duration.FiniteDuration;
43
44 /**
45  * Handles server configuration related messages for a RaftActor.
46  *
47  * @author Thomas Pantelis
48  */
49 class RaftActorServerConfigurationSupport {
50     private static final Logger LOG = LoggerFactory.getLogger(RaftActorServerConfigurationSupport.class);
51
52     private final OperationState IDLE = new Idle();
53
54     private final RaftActor raftActor;
55
56     private final RaftActorContext raftContext;
57
58     private final Queue<ServerOperationContext<?>> pendingOperationsQueue = new ArrayDeque<>();
59
60     private OperationState currentOperationState = IDLE;
61
62     RaftActorServerConfigurationSupport(RaftActor raftActor) {
63         this.raftActor = raftActor;
64         this.raftContext = raftActor.getRaftActorContext();
65     }
66
67     boolean handleMessage(Object message, ActorRef sender) {
68         if(message instanceof AddServer) {
69             onAddServer((AddServer) message, sender);
70             return true;
71         } else if(message instanceof RemoveServer) {
72             onRemoveServer((RemoveServer) message, sender);
73             return true;
74         } else if(message instanceof ChangeServersVotingStatus) {
75             onChangeServersVotingStatus((ChangeServersVotingStatus) message, sender);
76             return true;
77         } else if (message instanceof ServerOperationTimeout) {
78             currentOperationState.onServerOperationTimeout((ServerOperationTimeout) message);
79             return true;
80         } else if (message instanceof UnInitializedFollowerSnapshotReply) {
81             currentOperationState.onUnInitializedFollowerSnapshotReply((UnInitializedFollowerSnapshotReply) message);
82             return true;
83         } else if(message instanceof ApplyState) {
84             return onApplyState((ApplyState) message);
85         } else if(message instanceof SnapshotComplete) {
86             currentOperationState.onSnapshotComplete();
87             return false;
88         } else {
89             return false;
90         }
91     }
92
93     void onNewLeader(String leaderId) {
94         currentOperationState.onNewLeader(leaderId);
95     }
96
97     private void onChangeServersVotingStatus(ChangeServersVotingStatus message, ActorRef sender) {
98         LOG.debug("{}: onChangeServersVotingStatus: {}, state: {}", raftContext.getId(), message,
99                 currentOperationState);
100
101         // The following check is a special case. Normally we fail an operation if there's no leader.
102         // Consider a scenario where one has 2 geographically-separated 3-node clusters, one a primary and
103         // the other a backup such that if the primary cluster is lost, the backup can take over. In this
104         // scenario, we have a logical 6-node cluster where the primary sub-cluster is configured as voting
105         // and the backup sub-cluster as non-voting such that the primary cluster can make progress without
106         // consensus from the backup cluster while still replicating to the backup. On fail-over to the backup,
107         // a request would be sent to a member of the backup cluster to flip the voting states, ie make the
108         // backup sub-cluster voting and the lost primary non-voting. However since the primary majority
109         // cluster is lost, there would be no leader to apply, persist and replicate the server config change.
110         // Therefore, if the local server is currently non-voting and is to be changed to voting and there is
111         // no current leader, we will try to elect a leader using the new server config in order to replicate
112         // the change and progress.
113         boolean localServerChangingToVoting = Boolean.TRUE.equals(message.
114                 getServerVotingStatusMap().get(raftActor.getRaftActorContext().getId()));
115         boolean hasNoLeader = raftActor.getLeaderId() == null;
116         if(localServerChangingToVoting && !raftContext.isVotingMember() && hasNoLeader) {
117             currentOperationState.onNewOperation(new ChangeServersVotingStatusContext(message, sender, true));
118         } else {
119             onNewOperation(new ChangeServersVotingStatusContext(message, sender, false));
120         }
121     }
122
123     private void onRemoveServer(RemoveServer removeServer, ActorRef sender) {
124         LOG.debug("{}: onRemoveServer: {}, state: {}", raftContext.getId(), removeServer, currentOperationState);
125         boolean isSelf = removeServer.getServerId().equals(raftContext.getId());
126         if(isSelf && !raftContext.hasFollowers()) {
127             sender.tell(new RemoveServerReply(ServerChangeStatus.NOT_SUPPORTED, raftActor.getLeaderId()),
128                     raftActor.getSelf());
129         } else if(!isSelf && !raftContext.getPeerIds().contains(removeServer.getServerId())) {
130             sender.tell(new RemoveServerReply(ServerChangeStatus.DOES_NOT_EXIST, raftActor.getLeaderId()),
131                     raftActor.getSelf());
132         } else {
133             String serverAddress = isSelf ? raftActor.self().path().toString() :
134                 raftContext.getPeerAddress(removeServer.getServerId());
135             onNewOperation(new RemoveServerContext(removeServer, serverAddress, sender));
136         }
137     }
138
139     private boolean onApplyState(ApplyState applyState) {
140         Payload data = applyState.getReplicatedLogEntry().getData();
141         if(data instanceof ServerConfigurationPayload) {
142             currentOperationState.onApplyState(applyState);
143             return true;
144         }
145
146         return false;
147     }
148
149     /**
150      * The algorithm for AddServer is as follows:
151      * <ul>
152      * <li>Add the new server as a peer.</li>
153      * <li>Add the new follower to the leader.</li>
154      * <li>If new server should be voting member</li>
155      * <ul>
156      *     <li>Initialize FollowerState to VOTING_NOT_INITIALIZED.</li>
157      *     <li>Initiate install snapshot to the new follower.</li>
158      *     <li>When install snapshot complete, mark the follower as VOTING and re-calculate majority vote count.</li>
159      * </ul>
160      * <li>Persist and replicate ServerConfigurationPayload with the new server list.</li>
161      * <li>On replication consensus, respond to caller with OK.</li>
162      * </ul>
163      * If the install snapshot times out after a period of 2 * election time out
164      * <ul>
165      *     <li>Remove the new server as a peer.</li>
166      *     <li>Remove the new follower from the leader.</li>
167      *     <li>Respond to caller with TIMEOUT.</li>
168      * </ul>
169      */
170     private void onAddServer(AddServer addServer, ActorRef sender) {
171         LOG.debug("{}: onAddServer: {}, state: {}", raftContext.getId(), addServer, currentOperationState);
172
173         onNewOperation(new AddServerContext(addServer, sender));
174     }
175
176     private void onNewOperation(ServerOperationContext<?> operationContext) {
177         if (raftActor.isLeader()) {
178             currentOperationState.onNewOperation(operationContext);
179         } else {
180             ActorSelection leader = raftActor.getLeader();
181             if (leader != null) {
182                 LOG.debug("{}: Not leader - forwarding to leader {}", raftContext.getId(), leader);
183                 leader.tell(operationContext.getOperation(), operationContext.getClientRequestor());
184             } else {
185                 LOG.debug("{}: No leader - returning NO_LEADER reply", raftContext.getId());
186                 operationContext.getClientRequestor().tell(operationContext.newReply(
187                         ServerChangeStatus.NO_LEADER, null), raftActor.self());
188             }
189         }
190     }
191
192     /**
193      * Interface for the initial state for a server operation.
194      */
195     private interface InitialOperationState {
196         void initiate();
197     }
198
199     /**
200      * Abstract base class for a server operation FSM state. Handles common behavior for all states.
201      */
202     private abstract class OperationState {
203         void onNewOperation(ServerOperationContext<?> operationContext) {
204             // We're currently processing another operation so queue it to be processed later.
205
206             LOG.debug("{}: Server operation already in progress - queueing {}", raftContext.getId(),
207                     operationContext.getOperation());
208
209             pendingOperationsQueue.add(operationContext);
210         }
211
212         void onServerOperationTimeout(ServerOperationTimeout timeout) {
213             LOG.debug("onServerOperationTimeout should not be called in state {}", this);
214         }
215
216         void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) {
217             LOG.debug("onUnInitializedFollowerSnapshotReply was called in state {}", this);
218         }
219
220         void onApplyState(ApplyState applyState) {
221             LOG.debug("onApplyState was called in state {}", this);
222         }
223
224         void onSnapshotComplete() {
225
226         }
227
228         void onNewLeader(String newLeader) {
229         }
230
231         protected void persistNewServerConfiguration(ServerOperationContext<?> operationContext){
232             raftContext.setDynamicServerConfigurationInUse();
233
234             ServerConfigurationPayload payload = raftContext.getPeerServerInfo(
235                     operationContext.includeSelfInNewConfiguration(raftActor));
236             LOG.debug("{}: New server configuration : {}", raftContext.getId(), payload.getServerConfig());
237
238             raftActor.persistData(operationContext.getClientRequestor(), operationContext.getContextId(), payload);
239
240             currentOperationState = new Persisting(operationContext, newTimer(new ServerOperationTimeout(
241                     operationContext.getLoggingContext())));
242
243             sendReply(operationContext, ServerChangeStatus.OK);
244         }
245
246         protected void operationComplete(ServerOperationContext<?> operationContext, @Nullable ServerChangeStatus replyStatus) {
247             if(replyStatus != null) {
248                 sendReply(operationContext, replyStatus);
249             }
250
251             operationContext.operationComplete(raftActor, replyStatus == null || replyStatus == ServerChangeStatus.OK);
252
253             changeToIdleState();
254         }
255
256         protected void changeToIdleState() {
257             currentOperationState = IDLE;
258
259             ServerOperationContext<?> nextOperation = pendingOperationsQueue.poll();
260             if(nextOperation != null) {
261                 RaftActorServerConfigurationSupport.this.onNewOperation(nextOperation);
262             }
263         }
264
265         protected void sendReply(ServerOperationContext<?> operationContext, ServerChangeStatus status) {
266             LOG.debug("{}: Returning {} for operation {}", raftContext.getId(), status, operationContext.getOperation());
267
268             operationContext.getClientRequestor().tell(operationContext.newReply(status, raftActor.getLeaderId()),
269                     raftActor.self());
270         }
271
272         Cancellable newTimer(Object message) {
273             return newTimer(raftContext.getConfigParams().getElectionTimeOutInterval().$times(2), message);
274         }
275
276         Cancellable newTimer(FiniteDuration timeout, Object message) {
277             return raftContext.getActorSystem().scheduler().scheduleOnce(
278                     timeout, raftContext.getActor(), message,
279                             raftContext.getActorSystem().dispatcher(), raftContext.getActor());
280         }
281
282         @Override
283         public String toString() {
284             return getClass().getSimpleName();
285         }
286     }
287
288     /**
289      * The state when no server operation is in progress. It immediately initiates new server operations.
290      */
291     private final class Idle extends OperationState {
292         @Override
293         public void onNewOperation(ServerOperationContext<?> operationContext) {
294             operationContext.newInitialOperationState(RaftActorServerConfigurationSupport.this).initiate();
295         }
296
297         @Override
298         public void onApplyState(ApplyState applyState) {
299             // Noop - we override b/c ApplyState is called normally for followers in the idle state.
300         }
301     }
302
303     /**
304      * The state when a new server configuration is being persisted and replicated.
305      */
306     private final class Persisting extends OperationState {
307         private final ServerOperationContext<?> operationContext;
308         private final Cancellable timer;
309         private boolean timedOut = false;
310
311         Persisting(ServerOperationContext<?> operationContext, Cancellable timer) {
312             this.operationContext = operationContext;
313             this.timer = timer;
314         }
315
316         @Override
317         public void onApplyState(ApplyState applyState) {
318             // Sanity check - we could get an ApplyState from a previous operation that timed out so make
319             // sure it's meant for us.
320             if(operationContext.getContextId().equals(applyState.getIdentifier())) {
321                 LOG.info("{}: {} has been successfully replicated to a majority of followers", raftContext.getId(),
322                         applyState.getReplicatedLogEntry().getData());
323
324                 timer.cancel();
325                 operationComplete(operationContext, null);
326             }
327         }
328
329         @Override
330         public void onServerOperationTimeout(ServerOperationTimeout timeout) {
331             LOG.warn("{}: Timeout occured while replicating the new server configuration for {}", raftContext.getId(),
332                     timeout.getLoggingContext());
333
334             timedOut = true;
335
336             // Fail any pending operations
337             ServerOperationContext<?> nextOperation = pendingOperationsQueue.poll();
338             while(nextOperation != null) {
339                 sendReply(nextOperation, ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT);
340                 nextOperation = pendingOperationsQueue.poll();
341             }
342         }
343
344         @Override
345         public void onNewOperation(ServerOperationContext<?> operationContext) {
346             if(timedOut) {
347                 sendReply(operationContext, ServerChangeStatus.PRIOR_REQUEST_CONSENSUS_TIMEOUT);
348             } else {
349                 super.onNewOperation(operationContext);
350             }
351         }
352     }
353
354     /**
355      * Abstract base class for an AddServer operation state.
356      */
357     private abstract class AddServerState extends OperationState {
358         private final AddServerContext addServerContext;
359
360         AddServerState(AddServerContext addServerContext) {
361             this.addServerContext = addServerContext;
362         }
363
364         AddServerContext getAddServerContext() {
365             return addServerContext;
366         }
367
368         Cancellable newInstallSnapshotTimer() {
369             return newTimer(new ServerOperationTimeout(addServerContext.getOperation().getNewServerId()));
370         }
371
372         void handleInstallSnapshotTimeout(ServerOperationTimeout timeout) {
373             String serverId = timeout.getLoggingContext();
374
375             LOG.debug("{}: handleInstallSnapshotTimeout for new server {}", raftContext.getId(), serverId);
376
377             // cleanup
378             raftContext.removePeer(serverId);
379
380             boolean isLeader = raftActor.isLeader();
381             if(isLeader) {
382                 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
383                 leader.removeFollower(serverId);
384             }
385
386             operationComplete(getAddServerContext(), isLeader ? ServerChangeStatus.TIMEOUT : ServerChangeStatus.NO_LEADER);
387         }
388
389     }
390
391     /**
392      * The initial state for the AddServer operation. It adds the new follower as a peer and initiates
393      * snapshot capture, if necessary.
394      */
395     private final class InitialAddServerState extends AddServerState implements InitialOperationState {
396         InitialAddServerState(AddServerContext addServerContext) {
397             super(addServerContext);
398         }
399
400         @Override
401         public void initiate() {
402             AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
403             AddServer addServer = getAddServerContext().getOperation();
404
405             LOG.debug("{}: Initiating {}", raftContext.getId(), addServer);
406
407             if(raftContext.getPeerInfo(addServer.getNewServerId()) != null) {
408                 operationComplete(getAddServerContext(), ServerChangeStatus.ALREADY_EXISTS);
409                 return;
410             }
411
412             VotingState votingState = addServer.isVotingMember() ? VotingState.VOTING_NOT_INITIALIZED :
413                     VotingState.NON_VOTING;
414             raftContext.addToPeers(addServer.getNewServerId(), addServer.getNewServerAddress(), votingState);
415
416             leader.addFollower(addServer.getNewServerId());
417
418             if(votingState == VotingState.VOTING_NOT_INITIALIZED){
419                 // schedule the install snapshot timeout timer
420                 Cancellable installSnapshotTimer = newInstallSnapshotTimer();
421                 if(leader.initiateCaptureSnapshot(addServer.getNewServerId())) {
422                     LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(),
423                             addServer.getNewServerId());
424
425                     currentOperationState = new InstallingSnapshot(getAddServerContext(), installSnapshotTimer);
426                 } else {
427                     LOG.debug("{}: Snapshot already in progress - waiting for completion", raftContext.getId());
428
429                     currentOperationState = new WaitingForPriorSnapshotComplete(getAddServerContext(),
430                             installSnapshotTimer);
431                 }
432             } else {
433                 LOG.debug("{}: New follower is non-voting - directly persisting new server configuration",
434                         raftContext.getId());
435
436                 persistNewServerConfiguration(getAddServerContext());
437             }
438         }
439     }
440
441     /**
442      * The AddServer operation state for when the catch-up snapshot is being installed. It handles successful
443      * reply or timeout.
444      */
445     private final class InstallingSnapshot extends AddServerState {
446         private final Cancellable installSnapshotTimer;
447
448         InstallingSnapshot(AddServerContext addServerContext, Cancellable installSnapshotTimer) {
449             super(addServerContext);
450             this.installSnapshotTimer = Preconditions.checkNotNull(installSnapshotTimer);
451         }
452
453         @Override
454         public void onServerOperationTimeout(ServerOperationTimeout timeout) {
455             handleInstallSnapshotTimeout(timeout);
456
457             LOG.warn("{}: Timeout occured for new server {} while installing snapshot", raftContext.getId(),
458                     timeout.getLoggingContext());
459         }
460
461         @Override
462         public void onUnInitializedFollowerSnapshotReply(UnInitializedFollowerSnapshotReply reply) {
463             LOG.debug("{}: onUnInitializedFollowerSnapshotReply: {}", raftContext.getId(), reply);
464
465             String followerId = reply.getFollowerId();
466
467             // Sanity check to guard against receiving an UnInitializedFollowerSnapshotReply from a prior
468             // add server operation that timed out.
469             if(getAddServerContext().getOperation().getNewServerId().equals(followerId) && raftActor.isLeader()) {
470                 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
471                 raftContext.getPeerInfo(followerId).setVotingState(VotingState.VOTING);
472                 leader.updateMinReplicaCount();
473
474                 persistNewServerConfiguration(getAddServerContext());
475
476                 installSnapshotTimer.cancel();
477             } else {
478                 LOG.debug("{}: Dropping UnInitializedFollowerSnapshotReply for server {}: {}",
479                         raftContext.getId(), followerId,
480                         !raftActor.isLeader() ? "not leader" : "server Id doesn't match");
481             }
482         }
483     }
484
485     /**
486      * The AddServer operation state for when there is a snapshot already in progress. When the current
487      * snapshot completes, it initiates an install snapshot.
488      */
489     private final class WaitingForPriorSnapshotComplete extends AddServerState {
490         private final Cancellable snapshotTimer;
491
492         WaitingForPriorSnapshotComplete(AddServerContext addServerContext, Cancellable snapshotTimer) {
493             super(addServerContext);
494             this.snapshotTimer = Preconditions.checkNotNull(snapshotTimer);
495         }
496
497         @Override
498         public void onSnapshotComplete() {
499             LOG.debug("{}: onSnapshotComplete", raftContext.getId());
500
501             if(!raftActor.isLeader()) {
502                 LOG.debug("{}: No longer the leader", raftContext.getId());
503                 return;
504             }
505
506             AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
507             if(leader.initiateCaptureSnapshot(getAddServerContext().getOperation().getNewServerId())) {
508                 LOG.debug("{}: Initiating capture snapshot for new server {}", raftContext.getId(),
509                         getAddServerContext().getOperation().getNewServerId());
510
511                 currentOperationState = new InstallingSnapshot(getAddServerContext(),
512                         newInstallSnapshotTimer());
513
514                 snapshotTimer.cancel();
515             }
516         }
517
518         @Override
519         public void onServerOperationTimeout(ServerOperationTimeout timeout) {
520             handleInstallSnapshotTimeout(timeout);
521
522             LOG.warn("{}: Timeout occured for new server {} while waiting for prior snapshot to complete",
523                     raftContext.getId(), timeout.getLoggingContext());
524         }
525     }
526
527     /**
528      * Stores context information for a server operation.
529      *
530      * @param <T> the operation type
531      */
532     private static abstract class ServerOperationContext<T> {
533         private final T operation;
534         private final ActorRef clientRequestor;
535         private final Identifier contextId;
536
537         ServerOperationContext(T operation, ActorRef clientRequestor){
538             this.operation = operation;
539             this.clientRequestor = clientRequestor;
540             contextId = new UUIDIdentifier(UUID.randomUUID());
541         }
542
543         Identifier getContextId() {
544             return contextId;
545         }
546
547         T getOperation() {
548             return operation;
549         }
550
551         ActorRef getClientRequestor() {
552             return clientRequestor;
553         }
554
555         void operationComplete(RaftActor raftActor, boolean succeeded) {
556         }
557
558         boolean includeSelfInNewConfiguration(RaftActor raftActor) {
559             return true;
560         }
561
562         abstract Object newReply(ServerChangeStatus status, String leaderId);
563
564         abstract InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support);
565
566         abstract String getLoggingContext();
567     }
568
569     /**
570      * Stores context information for an AddServer operation.
571      */
572     private static class AddServerContext extends ServerOperationContext<AddServer> {
573         AddServerContext(AddServer addServer, ActorRef clientRequestor) {
574             super(addServer, clientRequestor);
575         }
576
577         @Override
578         Object newReply(ServerChangeStatus status, String leaderId) {
579             return new AddServerReply(status, leaderId);
580         }
581
582         @Override
583         InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) {
584             return support.new InitialAddServerState(this);
585         }
586
587         @Override
588         String getLoggingContext() {
589             return getOperation().getNewServerId();
590         }
591     }
592
593     private abstract class RemoveServerState extends OperationState {
594         private final RemoveServerContext removeServerContext;
595
596         protected RemoveServerState(RemoveServerContext removeServerContext) {
597             this.removeServerContext = Preconditions.checkNotNull(removeServerContext);
598
599         }
600
601         public RemoveServerContext getRemoveServerContext() {
602             return removeServerContext;
603         }
604     }
605
606     private final class InitialRemoveServerState extends RemoveServerState implements InitialOperationState{
607
608         protected InitialRemoveServerState(RemoveServerContext removeServerContext) {
609             super(removeServerContext);
610         }
611
612         @Override
613         public void initiate() {
614             String serverId = getRemoveServerContext().getOperation().getServerId();
615             raftContext.removePeer(serverId);
616             ((AbstractLeader)raftActor.getCurrentBehavior()).removeFollower(serverId);
617
618             persistNewServerConfiguration(getRemoveServerContext());
619         }
620     }
621
622     private static class RemoveServerContext extends ServerOperationContext<RemoveServer> {
623         private final String peerAddress;
624
625         RemoveServerContext(RemoveServer operation, String peerAddress, ActorRef clientRequestor) {
626             super(operation, clientRequestor);
627             this.peerAddress = peerAddress;
628         }
629
630         @Override
631         Object newReply(ServerChangeStatus status, String leaderId) {
632             return new RemoveServerReply(status, leaderId);
633         }
634
635         @Override
636         InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) {
637             return support.new InitialRemoveServerState(this);
638         }
639
640         @Override
641         void operationComplete(RaftActor raftActor, boolean succeeded) {
642             if(peerAddress != null) {
643                 raftActor.context().actorSelection(peerAddress).tell(new ServerRemoved(getOperation().getServerId()), raftActor.getSelf());
644             }
645         }
646
647         @Override
648         boolean includeSelfInNewConfiguration(RaftActor raftActor) {
649             return !getOperation().getServerId().equals(raftActor.getId());
650         }
651
652         @Override
653         String getLoggingContext() {
654             return getOperation().getServerId();
655         }
656     }
657
658     private static class ChangeServersVotingStatusContext extends ServerOperationContext<ChangeServersVotingStatus> {
659         private final boolean tryToElectLeader;
660
661         ChangeServersVotingStatusContext(ChangeServersVotingStatus convertMessage, ActorRef clientRequestor,
662                 boolean tryToElectLeader) {
663             super(convertMessage, clientRequestor);
664             this.tryToElectLeader = tryToElectLeader;
665         }
666
667         @Override
668         InitialOperationState newInitialOperationState(RaftActorServerConfigurationSupport support) {
669             return support.new ChangeServersVotingStatusState(this, tryToElectLeader);
670         }
671
672         @Override
673         Object newReply(ServerChangeStatus status, String leaderId) {
674             return new ServerChangeReply(status, leaderId);
675         }
676
677         @Override
678         void operationComplete(final RaftActor raftActor, boolean succeeded) {
679             // If this leader changed to non-voting we need to step down as leader so we'll try to transfer
680             // leadership.
681             boolean localServerChangedToNonVoting = Boolean.FALSE.equals(getOperation().
682                     getServerVotingStatusMap().get(raftActor.getRaftActorContext().getId()));
683             if (succeeded && localServerChangedToNonVoting) {
684                 LOG.debug("Leader changed to non-voting - trying leadership transfer");
685                 raftActor.becomeNonVoting();
686             }
687         }
688
689         @Override
690         String getLoggingContext() {
691             return getOperation().toString();
692         }
693     }
694
695     private class ChangeServersVotingStatusState extends OperationState implements InitialOperationState {
696         private final ChangeServersVotingStatusContext changeVotingStatusContext;
697         private final boolean tryToElectLeader;
698
699         ChangeServersVotingStatusState(ChangeServersVotingStatusContext changeVotingStatusContext,
700                 boolean tryToElectLeader) {
701             this.changeVotingStatusContext = changeVotingStatusContext;
702             this.tryToElectLeader = tryToElectLeader;
703         }
704
705         @Override
706         public void initiate() {
707             LOG.debug("Initiating ChangeServersVotingStatusState");
708
709             if(tryToElectLeader) {
710                 initiateLocalLeaderElection();
711             } else if(updateLocalPeerInfo()) {
712                 persistNewServerConfiguration(changeVotingStatusContext);
713             }
714         }
715
716         private void initiateLocalLeaderElection() {
717             LOG.debug("{}: Sending local ElectionTimeout to start leader election", raftContext.getId());
718
719             ServerConfigurationPayload previousServerConfig = raftContext.getPeerServerInfo(true);
720             if(!updateLocalPeerInfo()) {
721                 return;
722             }
723
724             raftContext.getActor().tell(ElectionTimeout.INSTANCE, raftContext.getActor());
725
726             currentOperationState = new WaitingForLeaderElected(changeVotingStatusContext, previousServerConfig);
727         }
728
729         private boolean updateLocalPeerInfo() {
730             List<ServerInfo> newServerInfoList = newServerInfoList();
731
732             // Check if new voting state would leave us with no voting members.
733             boolean atLeastOneVoting = false;
734             for(ServerInfo info: newServerInfoList) {
735                 if(info.isVoting()) {
736                     atLeastOneVoting = true;
737                     break;
738                 }
739             }
740
741             if(!atLeastOneVoting) {
742                 operationComplete(changeVotingStatusContext, ServerChangeStatus.INVALID_REQUEST);
743                 return false;
744             }
745
746             raftContext.updatePeerIds(new ServerConfigurationPayload(newServerInfoList));
747             if(raftActor.getCurrentBehavior() instanceof AbstractLeader) {
748                 AbstractLeader leader = (AbstractLeader) raftActor.getCurrentBehavior();
749                 leader.updateMinReplicaCount();
750             }
751
752             return true;
753         }
754
755         private List<ServerInfo> newServerInfoList() {
756             Map<String, Boolean> serverVotingStatusMap = changeVotingStatusContext.getOperation().getServerVotingStatusMap();
757             List<ServerInfo> newServerInfoList = new ArrayList<>();
758             for(String peerId: raftContext.getPeerIds()) {
759                 newServerInfoList.add(new ServerInfo(peerId, serverVotingStatusMap.containsKey(peerId) ?
760                         serverVotingStatusMap.get(peerId) : raftContext.getPeerInfo(peerId).isVoting()));
761             }
762
763             newServerInfoList.add(new ServerInfo(raftContext.getId(), serverVotingStatusMap.containsKey(
764                     raftContext.getId()) ? serverVotingStatusMap.get(raftContext.getId()) : raftContext.isVotingMember()));
765
766             return newServerInfoList;
767         }
768     }
769
770     private class WaitingForLeaderElected extends OperationState {
771         private final ServerConfigurationPayload previousServerConfig;
772         private final ChangeServersVotingStatusContext operationContext;
773         private final Cancellable timer;
774
775         WaitingForLeaderElected(ChangeServersVotingStatusContext operationContext,
776                 ServerConfigurationPayload previousServerConfig) {
777             this.operationContext = operationContext;
778             this.previousServerConfig = previousServerConfig;
779
780             timer = newTimer(raftContext.getConfigParams().getElectionTimeOutInterval(),
781                     new ServerOperationTimeout(operationContext.getLoggingContext()));
782         }
783
784         @Override
785         void onNewLeader(String newLeader) {
786             LOG.debug("{}: New leader {} elected", raftContext.getId(), newLeader);
787
788             timer.cancel();
789
790             if(raftActor.isLeader()) {
791                 persistNewServerConfiguration(operationContext);
792             } else {
793                 // Edge case - some other node became leader so forward the operation.
794                 LOG.debug("{}: Forwarding {} to new leader", raftContext.getId(), operationContext.getOperation());
795
796                 // Revert the local server config change.
797                 raftContext.updatePeerIds(previousServerConfig);
798
799                 changeToIdleState();
800                 RaftActorServerConfigurationSupport.this.onNewOperation(operationContext);
801             }
802         }
803
804         @Override
805         void onServerOperationTimeout(ServerOperationTimeout timeout) {
806             LOG.warn("{}: Leader election timed out - cannot apply operation {}",
807                     raftContext.getId(), timeout.getLoggingContext());
808
809             // Revert the local server config change.
810             raftContext.updatePeerIds(previousServerConfig);
811             raftActor.initializeBehavior();
812
813             tryToForwardOperationToAnotherServer();
814         }
815
816         private void tryToForwardOperationToAnotherServer() {
817             Collection<String> serversVisited = new HashSet<>(operationContext.getOperation().getServersVisited());
818
819             LOG.debug("{}: tryToForwardOperationToAnotherServer - servers already visited {}", raftContext.getId(),
820                     serversVisited);
821
822             serversVisited.add(raftContext.getId());
823
824             // Try to find another whose state is being changed from non-voting to voting and that we haven't
825             // tried yet.
826             Map<String, Boolean> serverVotingStatusMap = operationContext.getOperation().getServerVotingStatusMap();
827             ActorSelection forwardToPeerActor = null;
828             for(Map.Entry<String, Boolean> e: serverVotingStatusMap.entrySet()) {
829                 Boolean isVoting = e.getValue();
830                 String serverId = e.getKey();
831                 PeerInfo peerInfo = raftContext.getPeerInfo(serverId);
832                 if(isVoting && peerInfo != null && !peerInfo.isVoting() && !serversVisited.contains(serverId)) {
833                     ActorSelection actor = raftContext.getPeerActorSelection(serverId);
834                     if(actor != null) {
835                         forwardToPeerActor = actor;
836                         break;
837                     }
838                 }
839             }
840
841             if(forwardToPeerActor != null) {
842                 LOG.debug("{}: Found server {} to forward to", raftContext.getId(), forwardToPeerActor);
843
844                 forwardToPeerActor.tell(new ChangeServersVotingStatus(serverVotingStatusMap, serversVisited),
845                         operationContext.getClientRequestor());
846                 changeToIdleState();
847             } else {
848                 operationComplete(operationContext, ServerChangeStatus.NO_LEADER);
849             }
850         }
851     }
852
853     static class ServerOperationTimeout {
854         private final String loggingContext;
855
856         ServerOperationTimeout(String loggingContext){
857            this.loggingContext = Preconditions.checkNotNull(loggingContext, "loggingContext should not be null");
858         }
859
860         String getLoggingContext() {
861             return loggingContext;
862         }
863     }
864 }