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