Remove prefix shard leftovers
[controller.git] / opendaylight / md-sal / sal-cluster-admin-impl / src / main / java / org / opendaylight / controller / cluster / datastore / admin / ClusterAdminRpcService.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.datastore.admin;
9
10 import akka.actor.ActorRef;
11 import akka.actor.ActorSelection;
12 import akka.actor.Status.Success;
13 import akka.dispatch.OnComplete;
14 import akka.pattern.Patterns;
15 import akka.util.Timeout;
16 import com.google.common.base.Strings;
17 import com.google.common.base.Throwables;
18 import com.google.common.collect.ImmutableMap;
19 import com.google.common.collect.Maps;
20 import com.google.common.util.concurrent.FutureCallback;
21 import com.google.common.util.concurrent.Futures;
22 import com.google.common.util.concurrent.ListenableFuture;
23 import com.google.common.util.concurrent.MoreExecutors;
24 import com.google.common.util.concurrent.SettableFuture;
25 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
26 import java.io.FileOutputStream;
27 import java.io.IOException;
28 import java.util.AbstractMap.SimpleEntry;
29 import java.util.ArrayList;
30 import java.util.HashMap;
31 import java.util.List;
32 import java.util.Map;
33 import java.util.Map.Entry;
34 import java.util.Set;
35 import java.util.concurrent.ExecutionException;
36 import java.util.concurrent.TimeUnit;
37 import java.util.function.Function;
38 import java.util.stream.Collectors;
39 import org.apache.commons.lang3.SerializationUtils;
40 import org.eclipse.jdt.annotation.NonNull;
41 import org.opendaylight.controller.cluster.access.concepts.MemberName;
42 import org.opendaylight.controller.cluster.datastore.DistributedDataStoreInterface;
43 import org.opendaylight.controller.cluster.datastore.messages.AddShardReplica;
44 import org.opendaylight.controller.cluster.datastore.messages.ChangeShardMembersVotingStatus;
45 import org.opendaylight.controller.cluster.datastore.messages.FlipShardMembersVotingStatus;
46 import org.opendaylight.controller.cluster.datastore.messages.GetKnownClients;
47 import org.opendaylight.controller.cluster.datastore.messages.GetKnownClientsReply;
48 import org.opendaylight.controller.cluster.datastore.messages.GetShardRole;
49 import org.opendaylight.controller.cluster.datastore.messages.GetShardRoleReply;
50 import org.opendaylight.controller.cluster.datastore.messages.MakeLeaderLocal;
51 import org.opendaylight.controller.cluster.datastore.messages.PrimaryShardInfo;
52 import org.opendaylight.controller.cluster.datastore.messages.RemoveShardReplica;
53 import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot;
54 import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshotList;
55 import org.opendaylight.controller.cluster.datastore.utils.ActorUtils;
56 import org.opendaylight.controller.cluster.raft.client.messages.GetSnapshot;
57 import org.opendaylight.mdsal.binding.dom.codec.api.BindingNormalizedNodeSerializer;
58 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.AddReplicasForAllShardsInput;
59 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.AddReplicasForAllShardsOutput;
60 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.AddReplicasForAllShardsOutputBuilder;
61 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.AddShardReplicaInput;
62 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.AddShardReplicaOutput;
63 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.AddShardReplicaOutputBuilder;
64 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.BackupDatastoreInput;
65 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.BackupDatastoreOutput;
66 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.BackupDatastoreOutputBuilder;
67 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.ChangeMemberVotingStatesForAllShardsInput;
68 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.ChangeMemberVotingStatesForAllShardsOutput;
69 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.ChangeMemberVotingStatesForAllShardsOutputBuilder;
70 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.ChangeMemberVotingStatesForShardInput;
71 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.ChangeMemberVotingStatesForShardOutput;
72 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.ChangeMemberVotingStatesForShardOutputBuilder;
73 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.ClusterAdminService;
74 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.DataStoreType;
75 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.FlipMemberVotingStatesForAllShardsInput;
76 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.FlipMemberVotingStatesForAllShardsOutput;
77 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.FlipMemberVotingStatesForAllShardsOutputBuilder;
78 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.GetKnownClientsForAllShardsInput;
79 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.GetKnownClientsForAllShardsOutput;
80 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.GetKnownClientsForAllShardsOutputBuilder;
81 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.GetShardRoleInput;
82 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.GetShardRoleOutput;
83 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.GetShardRoleOutputBuilder;
84 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.LocateShardInput;
85 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.LocateShardOutput;
86 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.LocateShardOutputBuilder;
87 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.MakeLeaderLocalInput;
88 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.MakeLeaderLocalOutput;
89 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.MakeLeaderLocalOutputBuilder;
90 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.RemoveAllShardReplicasInput;
91 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.RemoveAllShardReplicasOutput;
92 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.RemoveAllShardReplicasOutputBuilder;
93 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.RemoveShardReplicaInput;
94 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.RemoveShardReplicaOutput;
95 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.RemoveShardReplicaOutputBuilder;
96 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.get.known.clients._for.all.shards.output.ShardResult1Builder;
97 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.get.known.clients._for.all.shards.output.shard.result.KnownClients;
98 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.get.known.clients._for.all.shards.output.shard.result.KnownClientsBuilder;
99 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.locate.shard.output.member.node.LeaderActorRefBuilder;
100 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.locate.shard.output.member.node.LocalBuilder;
101 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.member.voting.states.input.MemberVotingState;
102 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.shard.result.output.ShardResult;
103 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.shard.result.output.ShardResultBuilder;
104 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.shard.result.output.ShardResultKey;
105 import org.opendaylight.yangtools.yang.common.Empty;
106 import org.opendaylight.yangtools.yang.common.RpcError.ErrorType;
107 import org.opendaylight.yangtools.yang.common.RpcResult;
108 import org.opendaylight.yangtools.yang.common.RpcResultBuilder;
109 import org.opendaylight.yangtools.yang.common.Uint32;
110 import org.slf4j.Logger;
111 import org.slf4j.LoggerFactory;
112 import scala.concurrent.Future;
113
114 /**
115  * Implements the yang RPCs defined in the generated ClusterAdminService interface.
116  *
117  * @author Thomas Pantelis
118  */
119 public class ClusterAdminRpcService implements ClusterAdminService {
120     private static final Timeout SHARD_MGR_TIMEOUT = new Timeout(1, TimeUnit.MINUTES);
121
122     private static final Logger LOG = LoggerFactory.getLogger(ClusterAdminRpcService.class);
123     private static final @NonNull RpcResult<LocateShardOutput> LOCAL_SHARD_RESULT =
124             RpcResultBuilder.success(new LocateShardOutputBuilder()
125                 .setMemberNode(new LocalBuilder().setLocal(Empty.getInstance()).build())
126                 .build())
127             .build();
128
129     private final DistributedDataStoreInterface configDataStore;
130     private final DistributedDataStoreInterface operDataStore;
131     private final BindingNormalizedNodeSerializer serializer;
132     private final Timeout makeLeaderLocalTimeout;
133
134     public ClusterAdminRpcService(final DistributedDataStoreInterface configDataStore,
135             final DistributedDataStoreInterface operDataStore,
136             final BindingNormalizedNodeSerializer serializer) {
137         this.configDataStore = configDataStore;
138         this.operDataStore = operDataStore;
139         this.serializer = serializer;
140
141         this.makeLeaderLocalTimeout =
142                 new Timeout(configDataStore.getActorUtils().getDatastoreContext()
143                         .getShardLeaderElectionTimeout().duration().$times(2));
144     }
145
146     @Override
147     public ListenableFuture<RpcResult<AddShardReplicaOutput>> addShardReplica(final AddShardReplicaInput input) {
148         final String shardName = input.getShardName();
149         if (Strings.isNullOrEmpty(shardName)) {
150             return newFailedRpcResultFuture("A valid shard name must be specified");
151         }
152
153         DataStoreType dataStoreType = input.getDataStoreType();
154         if (dataStoreType == null) {
155             return newFailedRpcResultFuture("A valid DataStoreType must be specified");
156         }
157
158         LOG.info("Adding replica for shard {}", shardName);
159
160         final SettableFuture<RpcResult<AddShardReplicaOutput>> returnFuture = SettableFuture.create();
161         ListenableFuture<Success> future = sendMessageToShardManager(dataStoreType, new AddShardReplica(shardName));
162         Futures.addCallback(future, new FutureCallback<Success>() {
163             @Override
164             public void onSuccess(final Success success) {
165                 LOG.info("Successfully added replica for shard {}", shardName);
166                 returnFuture.set(newSuccessfulResult(new AddShardReplicaOutputBuilder().build()));
167             }
168
169             @Override
170             public void onFailure(final Throwable failure) {
171                 onMessageFailure(String.format("Failed to add replica for shard %s", shardName),
172                         returnFuture, failure);
173             }
174         }, MoreExecutors.directExecutor());
175
176         return returnFuture;
177     }
178
179     @Override
180     public ListenableFuture<RpcResult<RemoveShardReplicaOutput>> removeShardReplica(
181             final RemoveShardReplicaInput input) {
182         final String shardName = input.getShardName();
183         if (Strings.isNullOrEmpty(shardName)) {
184             return newFailedRpcResultFuture("A valid shard name must be specified");
185         }
186
187         DataStoreType dataStoreType = input.getDataStoreType();
188         if (dataStoreType == null) {
189             return newFailedRpcResultFuture("A valid DataStoreType must be specified");
190         }
191
192         final String memberName = input.getMemberName();
193         if (Strings.isNullOrEmpty(memberName)) {
194             return newFailedRpcResultFuture("A valid member name must be specified");
195         }
196
197         LOG.info("Removing replica for shard {} memberName {}, datastoreType {}", shardName, memberName, dataStoreType);
198
199         final SettableFuture<RpcResult<RemoveShardReplicaOutput>> returnFuture = SettableFuture.create();
200         ListenableFuture<Success> future = sendMessageToShardManager(dataStoreType,
201                 new RemoveShardReplica(shardName, MemberName.forName(memberName)));
202         Futures.addCallback(future, new FutureCallback<Success>() {
203             @Override
204             public void onSuccess(final Success success) {
205                 LOG.info("Successfully removed replica for shard {}", shardName);
206                 returnFuture.set(newSuccessfulResult(new RemoveShardReplicaOutputBuilder().build()));
207             }
208
209             @Override
210             public void onFailure(final Throwable failure) {
211                 onMessageFailure(String.format("Failed to remove replica for shard %s", shardName),
212                         returnFuture, failure);
213             }
214         }, MoreExecutors.directExecutor());
215
216         return returnFuture;
217     }
218
219     @Override
220     public ListenableFuture<RpcResult<LocateShardOutput>> locateShard(final LocateShardInput input) {
221         final ActorUtils utils;
222         switch (input.getDataStoreType()) {
223             case Config:
224                 utils = configDataStore.getActorUtils();
225                 break;
226             case Operational:
227                 utils = operDataStore.getActorUtils();
228                 break;
229             default:
230                 return newFailedRpcResultFuture("Unhandled datastore in " + input);
231         }
232
233         final SettableFuture<RpcResult<LocateShardOutput>> ret = SettableFuture.create();
234         utils.findPrimaryShardAsync(input.getShardName()).onComplete(new OnComplete<PrimaryShardInfo>() {
235             @Override
236             public void onComplete(final Throwable failure, final PrimaryShardInfo success) throws Throwable {
237                 if (failure != null) {
238                     LOG.debug("Failed to find shard for {}", input, failure);
239                     ret.setException(failure);
240                     return;
241                 }
242
243                 // Data tree implies local leak
244                 if (success.getLocalShardDataTree().isPresent()) {
245                     ret.set(LOCAL_SHARD_RESULT);
246                     return;
247                 }
248
249                 final ActorSelection actorPath = success.getPrimaryShardActor();
250                 ret.set(newSuccessfulResult(new LocateShardOutputBuilder()
251                     .setMemberNode(new LeaderActorRefBuilder()
252                         .setLeaderActorRef(actorPath.toSerializationFormat())
253                         .build())
254                     .build()));
255             }
256         }, utils.getClientDispatcher());
257
258         return ret;
259     }
260
261     @Override
262     public ListenableFuture<RpcResult<MakeLeaderLocalOutput>> makeLeaderLocal(final MakeLeaderLocalInput input) {
263         final String shardName = input.getShardName();
264         if (Strings.isNullOrEmpty(shardName)) {
265             return newFailedRpcResultFuture("A valid shard name must be specified");
266         }
267
268         DataStoreType dataStoreType = input.getDataStoreType();
269         if (dataStoreType == null) {
270             return newFailedRpcResultFuture("A valid DataStoreType must be specified");
271         }
272
273         ActorUtils actorUtils = dataStoreType == DataStoreType.Config
274                 ? configDataStore.getActorUtils() : operDataStore.getActorUtils();
275
276         LOG.info("Moving leader to local node {} for shard {}, datastoreType {}",
277                 actorUtils.getCurrentMemberName().getName(), shardName, dataStoreType);
278
279         final Future<ActorRef> localShardReply = actorUtils.findLocalShardAsync(shardName);
280
281         final scala.concurrent.Promise<Object> makeLeaderLocalAsk = akka.dispatch.Futures.promise();
282         localShardReply.onComplete(new OnComplete<ActorRef>() {
283             @Override
284             public void onComplete(final Throwable failure, final ActorRef actorRef) {
285                 if (failure != null) {
286                     LOG.warn("No local shard found for {} datastoreType {} - Cannot request leadership transfer to"
287                             + " local shard.", shardName, dataStoreType, failure);
288                     makeLeaderLocalAsk.failure(failure);
289                 } else {
290                     makeLeaderLocalAsk
291                             .completeWith(actorUtils
292                                     .executeOperationAsync(actorRef, MakeLeaderLocal.INSTANCE, makeLeaderLocalTimeout));
293                 }
294             }
295         }, actorUtils.getClientDispatcher());
296
297         final SettableFuture<RpcResult<MakeLeaderLocalOutput>> future = SettableFuture.create();
298         makeLeaderLocalAsk.future().onComplete(new OnComplete<>() {
299             @Override
300             public void onComplete(final Throwable failure, final Object success) {
301                 if (failure != null) {
302                     LOG.error("Leadership transfer failed for shard {}.", shardName, failure);
303                     future.set(RpcResultBuilder.<MakeLeaderLocalOutput>failed().withError(ErrorType.APPLICATION,
304                             "leadership transfer failed", failure).build());
305                     return;
306                 }
307
308                 LOG.debug("Leadership transfer complete");
309                 future.set(RpcResultBuilder.success(new MakeLeaderLocalOutputBuilder().build()).build());
310             }
311         }, actorUtils.getClientDispatcher());
312
313         return future;
314     }
315
316     @Override
317     public ListenableFuture<RpcResult<AddReplicasForAllShardsOutput>> addReplicasForAllShards(
318             final AddReplicasForAllShardsInput input) {
319         LOG.info("Adding replicas for all shards");
320
321         final List<Entry<ListenableFuture<Success>, ShardResultBuilder>> shardResultData = new ArrayList<>();
322
323         sendMessageToManagerForConfiguredShards(DataStoreType.Config, shardResultData, AddShardReplica::new);
324         sendMessageToManagerForConfiguredShards(DataStoreType.Operational, shardResultData, AddShardReplica::new);
325
326         return waitForShardResults(shardResultData, shardResults ->
327                 new AddReplicasForAllShardsOutputBuilder().setShardResult(shardResults).build(),
328                 "Failed to add replica");
329     }
330
331
332     @Override
333     public ListenableFuture<RpcResult<RemoveAllShardReplicasOutput>> removeAllShardReplicas(
334             final RemoveAllShardReplicasInput input) {
335         LOG.info("Removing replicas for all shards");
336
337         final String memberName = input.getMemberName();
338         if (Strings.isNullOrEmpty(memberName)) {
339             return newFailedRpcResultFuture("A valid member name must be specified");
340         }
341
342         final List<Entry<ListenableFuture<Success>, ShardResultBuilder>> shardResultData = new ArrayList<>();
343         Function<String, Object> messageSupplier = shardName ->
344                 new RemoveShardReplica(shardName, MemberName.forName(memberName));
345
346         sendMessageToManagerForConfiguredShards(DataStoreType.Config, shardResultData, messageSupplier);
347         sendMessageToManagerForConfiguredShards(DataStoreType.Operational, shardResultData, messageSupplier);
348
349         return waitForShardResults(shardResultData, shardResults ->
350                 new RemoveAllShardReplicasOutputBuilder().setShardResult(shardResults).build(),
351         "       Failed to remove replica");
352     }
353
354     @Override
355     public ListenableFuture<RpcResult<ChangeMemberVotingStatesForShardOutput>> changeMemberVotingStatesForShard(
356             final ChangeMemberVotingStatesForShardInput input) {
357         final String shardName = input.getShardName();
358         if (Strings.isNullOrEmpty(shardName)) {
359             return newFailedRpcResultFuture("A valid shard name must be specified");
360         }
361
362         DataStoreType dataStoreType = input.getDataStoreType();
363         if (dataStoreType == null) {
364             return newFailedRpcResultFuture("A valid DataStoreType must be specified");
365         }
366
367         List<MemberVotingState> memberVotingStates = input.getMemberVotingState();
368         if (memberVotingStates == null || memberVotingStates.isEmpty()) {
369             return newFailedRpcResultFuture("No member voting state input was specified");
370         }
371
372         ChangeShardMembersVotingStatus changeVotingStatus = toChangeShardMembersVotingStatus(shardName,
373                 memberVotingStates);
374
375         LOG.info("Change member voting states for shard {}: {}", shardName,
376                 changeVotingStatus.getMeberVotingStatusMap());
377
378         final SettableFuture<RpcResult<ChangeMemberVotingStatesForShardOutput>> returnFuture = SettableFuture.create();
379         ListenableFuture<Success> future = sendMessageToShardManager(dataStoreType, changeVotingStatus);
380         Futures.addCallback(future, new FutureCallback<Success>() {
381             @Override
382             public void onSuccess(final Success success) {
383                 LOG.info("Successfully changed member voting states for shard {}", shardName);
384                 returnFuture.set(newSuccessfulResult(new ChangeMemberVotingStatesForShardOutputBuilder().build()));
385             }
386
387             @Override
388             public void onFailure(final Throwable failure) {
389                 onMessageFailure(String.format("Failed to change member voting states for shard %s", shardName),
390                         returnFuture, failure);
391             }
392         }, MoreExecutors.directExecutor());
393
394         return returnFuture;
395     }
396
397     @Override
398     public ListenableFuture<RpcResult<ChangeMemberVotingStatesForAllShardsOutput>> changeMemberVotingStatesForAllShards(
399             final ChangeMemberVotingStatesForAllShardsInput input) {
400         List<MemberVotingState> memberVotingStates = input.getMemberVotingState();
401         if (memberVotingStates == null || memberVotingStates.isEmpty()) {
402             return newFailedRpcResultFuture("No member voting state input was specified");
403         }
404
405         final List<Entry<ListenableFuture<Success>, ShardResultBuilder>> shardResultData = new ArrayList<>();
406         Function<String, Object> messageSupplier = shardName ->
407                 toChangeShardMembersVotingStatus(shardName, memberVotingStates);
408
409         LOG.info("Change member voting states for all shards");
410
411         sendMessageToManagerForConfiguredShards(DataStoreType.Config, shardResultData, messageSupplier);
412         sendMessageToManagerForConfiguredShards(DataStoreType.Operational, shardResultData, messageSupplier);
413
414         return waitForShardResults(shardResultData, shardResults ->
415                 new ChangeMemberVotingStatesForAllShardsOutputBuilder().setShardResult(shardResults).build(),
416                 "Failed to change member voting states");
417     }
418
419     @Override
420     public ListenableFuture<RpcResult<FlipMemberVotingStatesForAllShardsOutput>> flipMemberVotingStatesForAllShards(
421             final FlipMemberVotingStatesForAllShardsInput input) {
422         final List<Entry<ListenableFuture<Success>, ShardResultBuilder>> shardResultData = new ArrayList<>();
423         Function<String, Object> messageSupplier = FlipShardMembersVotingStatus::new;
424
425         LOG.info("Flip member voting states for all shards");
426
427         sendMessageToManagerForConfiguredShards(DataStoreType.Config, shardResultData, messageSupplier);
428         sendMessageToManagerForConfiguredShards(DataStoreType.Operational, shardResultData, messageSupplier);
429
430         return waitForShardResults(shardResultData, shardResults ->
431                 new FlipMemberVotingStatesForAllShardsOutputBuilder().setShardResult(shardResults).build(),
432                 "Failed to change member voting states");
433     }
434
435     @Override
436     public ListenableFuture<RpcResult<GetShardRoleOutput>> getShardRole(final GetShardRoleInput input) {
437         final String shardName = input.getShardName();
438         if (Strings.isNullOrEmpty(shardName)) {
439             return newFailedRpcResultFuture("A valid shard name must be specified");
440         }
441
442         DataStoreType dataStoreType = input.getDataStoreType();
443         if (dataStoreType == null) {
444             return newFailedRpcResultFuture("A valid DataStoreType must be specified");
445         }
446
447         LOG.info("Getting role for shard {}, datastore type {}", shardName, dataStoreType);
448
449         final SettableFuture<RpcResult<GetShardRoleOutput>> returnFuture = SettableFuture.create();
450         ListenableFuture<GetShardRoleReply> future = sendMessageToShardManager(dataStoreType,
451                 new GetShardRole(shardName));
452         Futures.addCallback(future, new FutureCallback<GetShardRoleReply>() {
453             @Override
454             public void onSuccess(final GetShardRoleReply reply) {
455                 if (reply == null) {
456                     returnFuture.set(ClusterAdminRpcService.<GetShardRoleOutput>newFailedRpcResultBuilder(
457                             "No Shard role present. Please retry..").build());
458                     return;
459                 }
460                 LOG.info("Successfully received role:{} for shard {}", reply.getRole(), shardName);
461                 final GetShardRoleOutputBuilder builder = new GetShardRoleOutputBuilder();
462                 if (reply.getRole() != null) {
463                     builder.setRole(reply.getRole());
464                 }
465                 returnFuture.set(newSuccessfulResult(builder.build()));
466             }
467
468             @Override
469             public void onFailure(final Throwable failure) {
470                 returnFuture.set(ClusterAdminRpcService.<GetShardRoleOutput>newFailedRpcResultBuilder(
471                         "Failed to get shard role.", failure).build());
472             }
473         }, MoreExecutors.directExecutor());
474
475         return returnFuture;
476     }
477
478     @Override
479     public ListenableFuture<RpcResult<BackupDatastoreOutput>> backupDatastore(final BackupDatastoreInput input) {
480         LOG.debug("backupDatastore: {}", input);
481
482         if (Strings.isNullOrEmpty(input.getFilePath())) {
483             return newFailedRpcResultFuture("A valid file path must be specified");
484         }
485
486         final Uint32 timeout = input.getTimeout();
487         final Timeout opTimeout = timeout != null ? Timeout.apply(timeout.longValue(), TimeUnit.SECONDS)
488                 : SHARD_MGR_TIMEOUT;
489
490         final SettableFuture<RpcResult<BackupDatastoreOutput>> returnFuture = SettableFuture.create();
491         ListenableFuture<List<DatastoreSnapshot>> future = sendMessageToShardManagers(new GetSnapshot(opTimeout));
492         Futures.addCallback(future, new FutureCallback<>() {
493             @Override
494             public void onSuccess(final List<DatastoreSnapshot> snapshots) {
495                 saveSnapshotsToFile(new DatastoreSnapshotList(snapshots), input.getFilePath(), returnFuture);
496             }
497
498             @Override
499             public void onFailure(final Throwable failure) {
500                 onDatastoreBackupFailure(input.getFilePath(), returnFuture, failure);
501             }
502         }, MoreExecutors.directExecutor());
503
504         return returnFuture;
505     }
506
507
508     @Override
509     public ListenableFuture<RpcResult<GetKnownClientsForAllShardsOutput>> getKnownClientsForAllShards(
510             final GetKnownClientsForAllShardsInput input) {
511         final ImmutableMap<ShardIdentifier, ListenableFuture<GetKnownClientsReply>> allShardReplies =
512                 getAllShardLeadersClients();
513         return Futures.whenAllComplete(allShardReplies.values()).call(() -> processReplies(allShardReplies),
514             MoreExecutors.directExecutor());
515     }
516
517     private static RpcResult<GetKnownClientsForAllShardsOutput> processReplies(
518             final ImmutableMap<ShardIdentifier, ListenableFuture<GetKnownClientsReply>> allShardReplies) {
519         final Map<ShardResultKey, ShardResult> result = Maps.newHashMapWithExpectedSize(allShardReplies.size());
520         for (Entry<ShardIdentifier, ListenableFuture<GetKnownClientsReply>> entry : allShardReplies.entrySet()) {
521             final ListenableFuture<GetKnownClientsReply> future = entry.getValue();
522             final ShardResultBuilder builder = new ShardResultBuilder()
523                     .setDataStoreType(entry.getKey().getDataStoreType())
524                     .setShardName(entry.getKey().getShardName());
525
526             final GetKnownClientsReply reply;
527             try {
528                 reply = Futures.getDone(future);
529             } catch (ExecutionException e) {
530                 LOG.debug("Shard {} failed to answer", entry.getKey(), e);
531                 final ShardResult sr = builder
532                         .setSucceeded(Boolean.FALSE)
533                         .setErrorMessage(e.getCause().getMessage())
534                         .build();
535                 result.put(sr.key(), sr);
536                 continue;
537             }
538
539             final ShardResult sr = builder
540                     .setSucceeded(Boolean.TRUE)
541                     .addAugmentation(new ShardResult1Builder()
542                         .setKnownClients(reply.getClients().stream()
543                             .map(client -> new KnownClientsBuilder()
544                                 .setMember(client.getFrontendId().getMemberName().toYang())
545                                 .setType(client.getFrontendId().getClientType().toYang())
546                                 .setGeneration(client.getYangGeneration())
547                                 .build())
548                             .collect(Collectors.toMap(KnownClients::key, Function.identity())))
549                         .build())
550                     .build();
551
552             result.put(sr.key(), sr);
553         }
554
555         return RpcResultBuilder.success(new GetKnownClientsForAllShardsOutputBuilder().setShardResult(result).build())
556                 .build();
557     }
558
559     private static ChangeShardMembersVotingStatus toChangeShardMembersVotingStatus(final String shardName,
560             final List<MemberVotingState> memberVotingStatus) {
561         Map<String, Boolean> serverVotingStatusMap = new HashMap<>();
562         for (MemberVotingState memberStatus: memberVotingStatus) {
563             serverVotingStatusMap.put(memberStatus.getMemberName(), memberStatus.getVoting());
564         }
565         return new ChangeShardMembersVotingStatus(shardName, serverVotingStatusMap);
566     }
567
568     private static <T> SettableFuture<RpcResult<T>> waitForShardResults(
569             final List<Entry<ListenableFuture<Success>, ShardResultBuilder>> shardResultData,
570             final Function<Map<ShardResultKey, ShardResult>, T> resultDataSupplier,
571             final String failureLogMsgPrefix) {
572         final SettableFuture<RpcResult<T>> returnFuture = SettableFuture.create();
573         final Map<ShardResultKey, ShardResult> shardResults = new HashMap<>();
574         for (final Entry<ListenableFuture<Success>, ShardResultBuilder> entry : shardResultData) {
575             Futures.addCallback(entry.getKey(), new FutureCallback<Success>() {
576                 @Override
577                 public void onSuccess(final Success result) {
578                     synchronized (shardResults) {
579                         final ShardResultBuilder builder = entry.getValue();
580                         LOG.debug("onSuccess for shard {}, type {}", builder.getShardName(),
581                             builder.getDataStoreType());
582                         final ShardResult sr = builder.setSucceeded(Boolean.TRUE).build();
583                         shardResults.put(sr.key(), sr);
584                         checkIfComplete();
585                     }
586                 }
587
588                 @Override
589                 public void onFailure(final Throwable failure) {
590                     synchronized (shardResults) {
591                         ShardResultBuilder builder = entry.getValue();
592                         LOG.warn("{} for shard {}, type {}", failureLogMsgPrefix, builder.getShardName(),
593                                 builder.getDataStoreType(), failure);
594                         final ShardResult sr = builder
595                                 .setSucceeded(Boolean.FALSE)
596                                 .setErrorMessage(Throwables.getRootCause(failure).getMessage())
597                                 .build();
598                         shardResults.put(sr.key(), sr);
599                         checkIfComplete();
600                     }
601                 }
602
603                 void checkIfComplete() {
604                     LOG.debug("checkIfComplete: expected {}, actual {}", shardResultData.size(), shardResults.size());
605                     if (shardResults.size() == shardResultData.size()) {
606                         returnFuture.set(newSuccessfulResult(resultDataSupplier.apply(shardResults)));
607                     }
608                 }
609             }, MoreExecutors.directExecutor());
610         }
611         return returnFuture;
612     }
613
614     private <T> void sendMessageToManagerForConfiguredShards(final DataStoreType dataStoreType,
615             final List<Entry<ListenableFuture<T>, ShardResultBuilder>> shardResultData,
616             final Function<String, Object> messageSupplier) {
617         ActorUtils actorUtils = dataStoreType == DataStoreType.Config ? configDataStore.getActorUtils()
618                 : operDataStore.getActorUtils();
619         Set<String> allShardNames = actorUtils.getConfiguration().getAllShardNames();
620
621         LOG.debug("Sending message to all shards {} for data store {}", allShardNames, actorUtils.getDataStoreName());
622
623         for (String shardName: allShardNames) {
624             ListenableFuture<T> future = this.ask(actorUtils.getShardManager(), messageSupplier.apply(shardName),
625                                                   SHARD_MGR_TIMEOUT);
626             shardResultData.add(new SimpleEntry<>(future,
627                     new ShardResultBuilder().setShardName(shardName).setDataStoreType(dataStoreType)));
628         }
629     }
630
631     private <T> ListenableFuture<List<T>> sendMessageToShardManagers(final Object message) {
632         Timeout timeout = SHARD_MGR_TIMEOUT;
633         ListenableFuture<T> configFuture = ask(configDataStore.getActorUtils().getShardManager(), message, timeout);
634         ListenableFuture<T> operFuture = ask(operDataStore.getActorUtils().getShardManager(), message, timeout);
635
636         return Futures.allAsList(configFuture, operFuture);
637     }
638
639     private <T> ListenableFuture<T> sendMessageToShardManager(final DataStoreType dataStoreType, final Object message) {
640         ActorRef shardManager = dataStoreType == DataStoreType.Config
641                 ? configDataStore.getActorUtils().getShardManager()
642                         : operDataStore.getActorUtils().getShardManager();
643         return ask(shardManager, message, SHARD_MGR_TIMEOUT);
644     }
645
646     @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
647             justification = "https://github.com/spotbugs/spotbugs/issues/811")
648     @SuppressWarnings("checkstyle:IllegalCatch")
649     private static void saveSnapshotsToFile(final DatastoreSnapshotList snapshots, final String fileName,
650             final SettableFuture<RpcResult<BackupDatastoreOutput>> returnFuture) {
651         try (FileOutputStream fos = new FileOutputStream(fileName)) {
652             SerializationUtils.serialize(snapshots, fos);
653
654             returnFuture.set(newSuccessfulResult(new BackupDatastoreOutputBuilder().build()));
655             LOG.info("Successfully backed up datastore to file {}", fileName);
656         } catch (IOException | RuntimeException e) {
657             onDatastoreBackupFailure(fileName, returnFuture, e);
658         }
659     }
660
661     private static <T> void onDatastoreBackupFailure(final String fileName,
662             final SettableFuture<RpcResult<T>> returnFuture, final Throwable failure) {
663         onMessageFailure(String.format("Failed to back up datastore to file %s", fileName), returnFuture, failure);
664     }
665
666     @SuppressFBWarnings("SLF4J_SIGN_ONLY_FORMAT")
667     private static <T> void onMessageFailure(final String msg, final SettableFuture<RpcResult<T>> returnFuture,
668             final Throwable failure) {
669         LOG.error("{}", msg, failure);
670         returnFuture.set(ClusterAdminRpcService.<T>newFailedRpcResultBuilder(String.format("%s: %s", msg,
671                 failure.getMessage())).build());
672     }
673
674     private <T> ListenableFuture<T> ask(final ActorRef actor, final Object message, final Timeout timeout) {
675         final SettableFuture<T> returnFuture = SettableFuture.create();
676
677         @SuppressWarnings("unchecked")
678         Future<T> askFuture = (Future<T>) Patterns.ask(actor, message, timeout);
679         askFuture.onComplete(new OnComplete<T>() {
680             @Override
681             public void onComplete(final Throwable failure, final T resp) {
682                 if (failure != null) {
683                     returnFuture.setException(failure);
684                 } else {
685                     returnFuture.set(resp);
686                 }
687             }
688         }, configDataStore.getActorUtils().getClientDispatcher());
689
690         return returnFuture;
691     }
692
693     private ImmutableMap<ShardIdentifier, ListenableFuture<GetKnownClientsReply>> getAllShardLeadersClients() {
694         final ImmutableMap.Builder<ShardIdentifier, ListenableFuture<GetKnownClientsReply>> builder =
695                 ImmutableMap.builder();
696
697         addAllShardsClients(builder, DataStoreType.Config, configDataStore.getActorUtils());
698         addAllShardsClients(builder, DataStoreType.Operational, operDataStore.getActorUtils());
699
700         return builder.build();
701     }
702
703     private static void addAllShardsClients(
704             final ImmutableMap.Builder<ShardIdentifier, ListenableFuture<GetKnownClientsReply>> builder,
705             final DataStoreType type, final ActorUtils utils) {
706         for (String shardName : utils.getConfiguration().getAllShardNames()) {
707             final SettableFuture<GetKnownClientsReply> future = SettableFuture.create();
708             builder.put(new ShardIdentifier(type, shardName), future);
709
710             utils.findPrimaryShardAsync(shardName).flatMap(
711                 info -> Patterns.ask(info.getPrimaryShardActor(), GetKnownClients.INSTANCE, SHARD_MGR_TIMEOUT),
712                 utils.getClientDispatcher()).onComplete(new OnComplete<>() {
713                     @Override
714                     public void onComplete(final Throwable failure, final Object success) {
715                         if (failure == null) {
716                             future.set((GetKnownClientsReply) success);
717                         } else {
718                             future.setException(failure);
719                         }
720                     }
721                 }, utils.getClientDispatcher());
722         }
723     }
724
725     private static <T> ListenableFuture<RpcResult<T>> newFailedRpcResultFuture(final String message) {
726         return ClusterAdminRpcService.<T>newFailedRpcResultBuilder(message).buildFuture();
727     }
728
729     private static <T> RpcResultBuilder<T> newFailedRpcResultBuilder(final String message) {
730         return newFailedRpcResultBuilder(message, null);
731     }
732
733     private static <T> RpcResultBuilder<T> newFailedRpcResultBuilder(final String message, final Throwable cause) {
734         return RpcResultBuilder.<T>failed().withError(ErrorType.RPC, message, cause);
735     }
736
737     private static <T> RpcResult<T> newSuccessfulResult(final T data) {
738         return RpcResultBuilder.success(data).build();
739     }
740 }