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