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