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