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