d52a15483c6161cef8bbfb38ed7eb73fc1185f4c
[controller.git] / opendaylight / md-sal / sal-cluster-admin / src / main / java / org / opendaylight / controller / cluster / datastore / admin / ClusterAdminRpcService.java
1 /*
2  * Copyright (c) 2015 Brocade Communications Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.datastore.admin;
9
10 import akka.actor.ActorRef;
11 import akka.actor.Status.Success;
12 import akka.dispatch.OnComplete;
13 import akka.pattern.Patterns;
14 import akka.util.Timeout;
15 import com.google.common.base.Function;
16 import com.google.common.base.Strings;
17 import com.google.common.base.Throwables;
18 import com.google.common.util.concurrent.FutureCallback;
19 import com.google.common.util.concurrent.Futures;
20 import com.google.common.util.concurrent.ListenableFuture;
21 import com.google.common.util.concurrent.SettableFuture;
22 import java.io.FileOutputStream;
23 import java.util.AbstractMap.SimpleEntry;
24 import java.util.ArrayList;
25 import java.util.List;
26 import java.util.Map.Entry;
27 import java.util.Set;
28 import java.util.concurrent.Future;
29 import java.util.concurrent.TimeUnit;
30 import org.apache.commons.lang3.SerializationUtils;
31 import org.opendaylight.controller.cluster.access.concepts.MemberName;
32 import org.opendaylight.controller.cluster.datastore.DistributedDataStoreInterface;
33 import org.opendaylight.controller.cluster.datastore.messages.AddShardReplica;
34 import org.opendaylight.controller.cluster.datastore.messages.DatastoreSnapshot;
35 import org.opendaylight.controller.cluster.datastore.messages.DatastoreSnapshotList;
36 import org.opendaylight.controller.cluster.datastore.messages.RemoveShardReplica;
37 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
38 import org.opendaylight.controller.cluster.raft.client.messages.GetSnapshot;
39 import org.opendaylight.controller.sal.binding.api.BindingAwareBroker.RpcRegistration;
40 import org.opendaylight.controller.sal.binding.api.RpcProviderRegistry;
41 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.AddReplicasForAllShardsOutput;
42 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.AddReplicasForAllShardsOutputBuilder;
43 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.AddShardReplicaInput;
44 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.BackupDatastoreInput;
45 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.ClusterAdminService;
46 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.ConvertMembersToNonvotingForAllShardsInput;
47 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.ConvertMembersToVotingForAllShardsInput;
48 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.DataStoreType;
49 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.RemoveAllShardReplicasInput;
50 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.RemoveAllShardReplicasOutput;
51 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.RemoveAllShardReplicasOutputBuilder;
52 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.RemoveShardReplicaInput;
53 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.shard.result.output.ShardResult;
54 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.cluster.admin.rev151013.shard.result.output.ShardResultBuilder;
55 import org.opendaylight.yangtools.yang.common.RpcError.ErrorType;
56 import org.opendaylight.yangtools.yang.common.RpcResult;
57 import org.opendaylight.yangtools.yang.common.RpcResultBuilder;
58 import org.slf4j.Logger;
59 import org.slf4j.LoggerFactory;
60
61 /**
62  * Implements the yang RPCs defined in the generated ClusterAdminService interface.
63  *
64  * @author Thomas Pantelis
65  */
66 public class ClusterAdminRpcService implements ClusterAdminService, AutoCloseable {
67     private static final Timeout SHARD_MGR_TIMEOUT = new Timeout(1, TimeUnit.MINUTES);
68
69     private static final Logger LOG = LoggerFactory.getLogger(ClusterAdminRpcService.class);
70
71     private final DistributedDataStoreInterface configDataStore;
72     private final DistributedDataStoreInterface operDataStore;
73     private RpcRegistration<ClusterAdminService> rpcRegistration;
74
75     public ClusterAdminRpcService(DistributedDataStoreInterface configDataStore,
76             DistributedDataStoreInterface operDataStore) {
77         this.configDataStore = configDataStore;
78         this.operDataStore = operDataStore;
79     }
80
81     public void start(RpcProviderRegistry rpcProviderRegistry) {
82         LOG.debug("ClusterAdminRpcService starting");
83
84         rpcRegistration = rpcProviderRegistry.addRpcImplementation(ClusterAdminService.class, this);
85     }
86
87     @Override
88     public void close() {
89         if(rpcRegistration != null) {
90             rpcRegistration.close();
91         }
92     }
93
94     @Override
95     public Future<RpcResult<Void>> addShardReplica(final AddShardReplicaInput input) {
96         final String shardName = input.getShardName();
97         if(Strings.isNullOrEmpty(shardName)) {
98             return newFailedRpcResultFuture("A valid shard name must be specified");
99         }
100
101         DataStoreType dataStoreType = input.getDataStoreType();
102         if(dataStoreType == null) {
103             return newFailedRpcResultFuture("A valid DataStoreType must be specified");
104         }
105
106         LOG.info("Adding replica for shard {}", shardName);
107
108         final SettableFuture<RpcResult<Void>> returnFuture = SettableFuture.create();
109         ListenableFuture<Success> future = sendMessageToShardManager(dataStoreType, new AddShardReplica(shardName));
110         Futures.addCallback(future, new FutureCallback<Success>() {
111             @Override
112             public void onSuccess(Success success) {
113                 LOG.info("Successfully added replica for shard {}", shardName);
114                 returnFuture.set(newSuccessfulResult());
115             }
116
117             @Override
118             public void onFailure(Throwable failure) {
119                 onMessageFailure(String.format("Failed to add replica for shard %s", shardName),
120                         returnFuture, failure);
121             }
122         });
123
124         return returnFuture;
125     }
126
127     @Override
128     public Future<RpcResult<Void>> removeShardReplica(RemoveShardReplicaInput input) {
129         final String shardName = input.getShardName();
130         if(Strings.isNullOrEmpty(shardName)) {
131             return newFailedRpcResultFuture("A valid shard name must be specified");
132         }
133
134         DataStoreType dataStoreType = input.getDataStoreType();
135         if(dataStoreType == null) {
136             return newFailedRpcResultFuture("A valid DataStoreType must be specified");
137         }
138
139         final String memberName = input.getMemberName();
140         if(Strings.isNullOrEmpty(memberName)) {
141             return newFailedRpcResultFuture("A valid member name must be specified");
142         }
143
144         LOG.info("Removing replica for shard {} memberName {}, datastoreType {}", shardName, memberName, dataStoreType);
145
146         final SettableFuture<RpcResult<Void>> returnFuture = SettableFuture.create();
147         ListenableFuture<Success> future = sendMessageToShardManager(dataStoreType,
148                 new RemoveShardReplica(shardName, MemberName.forName(memberName)));
149         Futures.addCallback(future, new FutureCallback<Success>() {
150             @Override
151             public void onSuccess(Success success) {
152                 LOG.info("Successfully removed replica for shard {}", shardName);
153                 returnFuture.set(newSuccessfulResult());
154             }
155
156             @Override
157             public void onFailure(Throwable failure) {
158                 onMessageFailure(String.format("Failed to remove replica for shard %s", shardName),
159                         returnFuture, failure);
160             }
161         });
162
163         return returnFuture;
164     }
165
166     @Override
167     public Future<RpcResult<AddReplicasForAllShardsOutput>> addReplicasForAllShards() {
168         LOG.info("Adding replicas for all shards");
169
170         final List<Entry<ListenableFuture<Success>, ShardResultBuilder>> shardResultData = new ArrayList<>();
171         Function<String, Object> messageSupplier = new Function<String, Object>() {
172             @Override
173             public Object apply(String shardName) {
174                 return new AddShardReplica(shardName);
175             }
176         };
177
178         sendMessageToManagerForConfiguredShards(DataStoreType.Config, shardResultData, messageSupplier);
179         sendMessageToManagerForConfiguredShards(DataStoreType.Operational, shardResultData, messageSupplier);
180
181         return waitForShardResults(shardResultData, new Function<List<ShardResult>, AddReplicasForAllShardsOutput>() {
182             @Override
183             public AddReplicasForAllShardsOutput apply(List<ShardResult> shardResults) {
184                 return new AddReplicasForAllShardsOutputBuilder().setShardResult(shardResults).build();
185             }
186         }, "Failed to add replica");
187     }
188
189
190     @Override
191     public Future<RpcResult<RemoveAllShardReplicasOutput>> removeAllShardReplicas(RemoveAllShardReplicasInput input) {
192         LOG.info("Removing replicas for all shards");
193
194         final String memberName = input.getMemberName();
195         if(Strings.isNullOrEmpty(memberName)) {
196             return newFailedRpcResultFuture("A valid member name must be specified");
197         }
198
199         final List<Entry<ListenableFuture<Success>, ShardResultBuilder>> shardResultData = new ArrayList<>();
200         Function<String, Object> messageSupplier = new Function<String, Object>() {
201             @Override
202             public Object apply(String shardName) {
203                 return new RemoveShardReplica(shardName, MemberName.forName(memberName));
204             }
205         };
206
207         sendMessageToManagerForConfiguredShards(DataStoreType.Config, shardResultData, messageSupplier);
208         sendMessageToManagerForConfiguredShards(DataStoreType.Operational, shardResultData, messageSupplier);
209
210         return waitForShardResults(shardResultData, new Function<List<ShardResult>, RemoveAllShardReplicasOutput>() {
211             @Override
212             public RemoveAllShardReplicasOutput apply(List<ShardResult> shardResults) {
213                 return new RemoveAllShardReplicasOutputBuilder().setShardResult(shardResults).build();
214             }
215         }, "Failed to add replica");
216     }
217
218     @Override
219     public Future<RpcResult<Void>> convertMembersToVotingForAllShards(ConvertMembersToVotingForAllShardsInput input) {
220         // TODO implement
221         return RpcResultBuilder.<Void>failed().withError(ErrorType.APPLICATION, "operation-not-supported",
222                 "Not implemented yet").buildFuture();
223     }
224
225     @Override
226     public Future<RpcResult<Void>> convertMembersToNonvotingForAllShards(
227             ConvertMembersToNonvotingForAllShardsInput input) {
228         // TODO implement
229         return RpcResultBuilder.<Void>failed().withError(ErrorType.APPLICATION, "operation-not-supported",
230                 "Not implemented yet").buildFuture();
231     }
232
233     @Override
234     public Future<RpcResult<Void>> backupDatastore(final BackupDatastoreInput input) {
235         LOG.debug("backupDatastore: {}", input);
236
237         if(Strings.isNullOrEmpty(input.getFilePath())) {
238             return newFailedRpcResultFuture("A valid file path must be specified");
239         }
240
241         final SettableFuture<RpcResult<Void>> returnFuture = SettableFuture.create();
242         ListenableFuture<List<DatastoreSnapshot>> future = sendMessageToShardManagers(GetSnapshot.INSTANCE);
243         Futures.addCallback(future, new FutureCallback<List<DatastoreSnapshot>>() {
244             @Override
245             public void onSuccess(List<DatastoreSnapshot> snapshots) {
246                 saveSnapshotsToFile(new DatastoreSnapshotList(snapshots), input.getFilePath(), returnFuture);
247             }
248
249             @Override
250             public void onFailure(Throwable failure) {
251                 onDatastoreBackupFailure(input.getFilePath(), returnFuture, failure);
252             }
253         });
254
255         return returnFuture;
256     }
257
258     private static <T> SettableFuture<RpcResult<T>> waitForShardResults(
259             final List<Entry<ListenableFuture<Success>, ShardResultBuilder>> shardResultData,
260             final Function<List<ShardResult>, T> resultDataSupplier,
261             final String failureLogMsgPrefix) {
262         final SettableFuture<RpcResult<T>> returnFuture = SettableFuture.create();
263         final List<ShardResult> shardResults = new ArrayList<>();
264         for(final Entry<ListenableFuture<Success>, ShardResultBuilder> entry: shardResultData) {
265             Futures.addCallback(entry.getKey(), new FutureCallback<Success>() {
266                 @Override
267                 public void onSuccess(Success result) {
268                     synchronized(shardResults) {
269                         ShardResultBuilder shardResult = entry.getValue();
270                         LOG.debug("onSuccess for shard {}, type {}", shardResult.getShardName(),
271                                 shardResult.getDataStoreType());
272                         shardResults.add(shardResult.setSucceeded(true).build());
273                         checkIfComplete();
274                     }
275                 }
276
277                 @Override
278                 public void onFailure(Throwable t) {
279                     synchronized(shardResults) {
280                         ShardResultBuilder shardResult = entry.getValue();
281                         LOG.warn("{} for shard {}, type {}", failureLogMsgPrefix, shardResult.getShardName(),
282                                 shardResult.getDataStoreType(), t);
283                         shardResults.add(shardResult.setSucceeded(false).setErrorMessage(
284                                 Throwables.getRootCause(t).getMessage()).build());
285                         checkIfComplete();
286                     }
287                 }
288
289                 void checkIfComplete() {
290                     LOG.debug("checkIfComplete: expected {}, actual {}", shardResultData.size(), shardResults.size());
291                     if(shardResults.size() == shardResultData.size()) {
292                         returnFuture.set(newSuccessfulResult(resultDataSupplier.apply(shardResults)));
293                     }
294                 }
295             });
296         }
297         return returnFuture;
298     }
299
300     private <T> void sendMessageToManagerForConfiguredShards(DataStoreType dataStoreType,
301             List<Entry<ListenableFuture<T>, ShardResultBuilder>> shardResultData,
302             Function<String, Object> messageSupplier) {
303         ActorContext actorContext = dataStoreType == DataStoreType.Config ?
304                 configDataStore.getActorContext() : operDataStore.getActorContext();
305         Set<String> allShardNames = actorContext.getConfiguration().getAllShardNames();
306
307         LOG.debug("Sending message to all shards {} for data store {}", allShardNames, actorContext.getDataStoreName());
308
309         for(String shardName: allShardNames) {
310             ListenableFuture<T> future = this.<T>ask(actorContext.getShardManager(), messageSupplier.apply(shardName),
311                     SHARD_MGR_TIMEOUT);
312             shardResultData.add(new SimpleEntry<ListenableFuture<T>, ShardResultBuilder>(future,
313                     new ShardResultBuilder().setShardName(shardName).setDataStoreType(dataStoreType)));
314         }
315     }
316
317     @SuppressWarnings("unchecked")
318     private <T> ListenableFuture<List<T>> sendMessageToShardManagers(Object message) {
319         Timeout timeout = SHARD_MGR_TIMEOUT;
320         ListenableFuture<T> configFuture = ask(configDataStore.getActorContext().getShardManager(), message, timeout);
321         ListenableFuture<T> operFuture = ask(operDataStore.getActorContext().getShardManager(), message, timeout);
322
323         return Futures.allAsList(configFuture, operFuture);
324     }
325
326     private <T> ListenableFuture<T> sendMessageToShardManager(DataStoreType dataStoreType, Object message) {
327         ActorRef shardManager = dataStoreType == DataStoreType.Config ?
328                 configDataStore.getActorContext().getShardManager() : operDataStore.getActorContext().getShardManager();
329         return ask(shardManager, message, SHARD_MGR_TIMEOUT);
330     }
331
332     private static void saveSnapshotsToFile(DatastoreSnapshotList snapshots, String fileName,
333             SettableFuture<RpcResult<Void>> returnFuture) {
334         try(FileOutputStream fos = new FileOutputStream(fileName)) {
335             SerializationUtils.serialize(snapshots, fos);
336
337             returnFuture.set(newSuccessfulResult());
338             LOG.info("Successfully backed up datastore to file {}", fileName);
339         } catch(Exception e) {
340             onDatastoreBackupFailure(fileName, returnFuture, e);
341         }
342     }
343
344     private static void onDatastoreBackupFailure(String fileName, SettableFuture<RpcResult<Void>> returnFuture,
345             Throwable failure) {
346         onMessageFailure(String.format("Failed to back up datastore to file %s", fileName), returnFuture, failure);
347     }
348
349     private static void onMessageFailure(String msg, final SettableFuture<RpcResult<Void>> returnFuture,
350             Throwable failure) {
351         LOG.error(msg, failure);
352         returnFuture.set(ClusterAdminRpcService.<Void>newFailedRpcResultBuilder(String.format("%s: %s", msg,
353                 failure.getMessage())).build());
354     }
355
356     private <T> ListenableFuture<T> ask(ActorRef actor, Object message, Timeout timeout) {
357         final SettableFuture<T> returnFuture = SettableFuture.create();
358
359         @SuppressWarnings("unchecked")
360         scala.concurrent.Future<T> askFuture = (scala.concurrent.Future<T>) Patterns.ask(actor, message, timeout);
361         askFuture.onComplete(new OnComplete<T>() {
362             @Override
363             public void onComplete(Throwable failure, T resp) {
364                 if(failure != null) {
365                     returnFuture.setException(failure);
366                 } else {
367                     returnFuture.set(resp);
368                 }
369             }
370         }, configDataStore.getActorContext().getClientDispatcher());
371
372         return returnFuture;
373     }
374
375     private static <T> ListenableFuture<RpcResult<T>> newFailedRpcResultFuture(String message) {
376         return ClusterAdminRpcService.<T>newFailedRpcResultBuilder(message).buildFuture();
377     }
378
379     private static <T> RpcResultBuilder<T> newFailedRpcResultBuilder(String message) {
380         return newFailedRpcResultBuilder(message, null);
381     }
382
383     private static <T> RpcResultBuilder<T> newFailedRpcResultBuilder(String message, Throwable cause) {
384         return RpcResultBuilder.<T>failed().withError(ErrorType.RPC, message, cause);
385     }
386
387     private static RpcResult<Void> newSuccessfulResult() {
388         return newSuccessfulResult((Void)null);
389     }
390
391     private static <T> RpcResult<T> newSuccessfulResult(T data) {
392         return RpcResultBuilder.<T>success(data).build();
393     }
394 }