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