BUG-2138: Create blueprint wiring for cds shard manager.
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / sharding / DistributedShardedDOMDataTree.java
1 /*
2  * Copyright (c) 2016 Cisco 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
9 package org.opendaylight.controller.cluster.sharding;
10
11 import static akka.actor.ActorRef.noSender;
12
13 import akka.actor.ActorRef;
14 import akka.actor.ActorSystem;
15 import akka.actor.PoisonPill;
16 import akka.actor.Props;
17 import akka.dispatch.Mapper;
18 import akka.dispatch.OnComplete;
19 import akka.pattern.Patterns;
20 import akka.util.Timeout;
21 import com.google.common.base.Optional;
22 import com.google.common.base.Preconditions;
23 import com.google.common.base.Throwables;
24 import com.google.common.collect.ForwardingObject;
25 import com.google.common.util.concurrent.FutureCallback;
26 import com.google.common.util.concurrent.Futures;
27 import com.google.common.util.concurrent.ListenableFuture;
28 import com.google.common.util.concurrent.SettableFuture;
29 import com.google.common.util.concurrent.Uninterruptibles;
30 import java.util.AbstractMap.SimpleEntry;
31 import java.util.ArrayList;
32 import java.util.Collection;
33 import java.util.Collections;
34 import java.util.EnumMap;
35 import java.util.List;
36 import java.util.Map.Entry;
37 import java.util.Set;
38 import java.util.concurrent.CompletionStage;
39 import java.util.concurrent.ExecutionException;
40 import java.util.concurrent.TimeUnit;
41 import java.util.concurrent.TimeoutException;
42 import javax.annotation.Nonnull;
43 import javax.annotation.Nullable;
44 import javax.annotation.concurrent.GuardedBy;
45 import org.opendaylight.controller.cluster.ActorSystemProvider;
46 import org.opendaylight.controller.cluster.access.concepts.MemberName;
47 import org.opendaylight.controller.cluster.databroker.actors.dds.DataStoreClient;
48 import org.opendaylight.controller.cluster.databroker.actors.dds.SimpleDataStoreClientActor;
49 import org.opendaylight.controller.cluster.datastore.DistributedDataStore;
50 import org.opendaylight.controller.cluster.datastore.Shard;
51 import org.opendaylight.controller.cluster.datastore.config.Configuration;
52 import org.opendaylight.controller.cluster.datastore.config.ModuleShardConfiguration;
53 import org.opendaylight.controller.cluster.datastore.messages.CreateShard;
54 import org.opendaylight.controller.cluster.datastore.shardstrategy.ModuleShardStrategy;
55 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
56 import org.opendaylight.controller.cluster.datastore.utils.ClusterUtils;
57 import org.opendaylight.controller.cluster.sharding.ShardedDataTreeActor.ShardedDataTreeActorCreator;
58 import org.opendaylight.controller.cluster.sharding.messages.InitConfigListener;
59 import org.opendaylight.controller.cluster.sharding.messages.LookupPrefixShard;
60 import org.opendaylight.controller.cluster.sharding.messages.PrefixShardRemovalLookup;
61 import org.opendaylight.controller.cluster.sharding.messages.ProducerCreated;
62 import org.opendaylight.controller.cluster.sharding.messages.ProducerRemoved;
63 import org.opendaylight.controller.cluster.sharding.messages.StartConfigShardLookup;
64 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
65 import org.opendaylight.mdsal.dom.api.DOMDataTreeCursorAwareTransaction;
66 import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
67 import org.opendaylight.mdsal.dom.api.DOMDataTreeListener;
68 import org.opendaylight.mdsal.dom.api.DOMDataTreeLoopException;
69 import org.opendaylight.mdsal.dom.api.DOMDataTreeProducer;
70 import org.opendaylight.mdsal.dom.api.DOMDataTreeProducerException;
71 import org.opendaylight.mdsal.dom.api.DOMDataTreeService;
72 import org.opendaylight.mdsal.dom.api.DOMDataTreeShard;
73 import org.opendaylight.mdsal.dom.api.DOMDataTreeShardingConflictException;
74 import org.opendaylight.mdsal.dom.api.DOMDataTreeShardingService;
75 import org.opendaylight.mdsal.dom.broker.DOMDataTreeShardRegistration;
76 import org.opendaylight.mdsal.dom.broker.ShardedDOMDataTree;
77 import org.opendaylight.mdsal.dom.spi.DOMDataTreePrefixTable;
78 import org.opendaylight.mdsal.dom.spi.DOMDataTreePrefixTableEntry;
79 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.controller.md.sal.clustering.prefix.shard.configuration.rev170110.PrefixShards;
80 import org.opendaylight.yangtools.concepts.ListenerRegistration;
81 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
82 import org.slf4j.Logger;
83 import org.slf4j.LoggerFactory;
84 import scala.compat.java8.FutureConverters;
85 import scala.concurrent.Await;
86 import scala.concurrent.Future;
87 import scala.concurrent.Promise;
88 import scala.concurrent.duration.FiniteDuration;
89
90 /**
91  * A layer on top of DOMDataTreeService that distributes producer/shard registrations to remote nodes via
92  * {@link ShardedDataTreeActor}. Also provides QoL method for addition of prefix based clustered shard into the system.
93  */
94 public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDataTreeShardingService,
95         DistributedShardFactory {
96
97     private static final Logger LOG = LoggerFactory.getLogger(DistributedShardedDOMDataTree.class);
98
99     private static final int MAX_ACTOR_CREATION_RETRIES = 100;
100     private static final int ACTOR_RETRY_DELAY = 100;
101     private static final TimeUnit ACTOR_RETRY_TIME_UNIT = TimeUnit.MILLISECONDS;
102     private static final int LOOKUP_TASK_MAX_RETRIES = 100;
103     static final FiniteDuration SHARD_FUTURE_TIMEOUT_DURATION =
104             new FiniteDuration(LOOKUP_TASK_MAX_RETRIES * LOOKUP_TASK_MAX_RETRIES * 3, TimeUnit.SECONDS);
105     static final Timeout SHARD_FUTURE_TIMEOUT = new Timeout(SHARD_FUTURE_TIMEOUT_DURATION);
106
107     static final String ACTOR_ID = "ShardedDOMDataTreeFrontend";
108
109     private final ShardedDOMDataTree shardedDOMDataTree;
110     private final ActorSystem actorSystem;
111     private final DistributedDataStore distributedOperDatastore;
112     private final DistributedDataStore distributedConfigDatastore;
113
114     private final ActorRef shardedDataTreeActor;
115     private final MemberName memberName;
116
117     @GuardedBy("shards")
118     private final DOMDataTreePrefixTable<DOMDataTreeShardRegistration<DOMDataTreeShard>> shards =
119             DOMDataTreePrefixTable.create();
120
121     private final EnumMap<LogicalDatastoreType, DistributedShardRegistration> defaultShardRegistrations =
122             new EnumMap<>(LogicalDatastoreType.class);
123
124     private final EnumMap<LogicalDatastoreType, Entry<DataStoreClient, ActorRef>> configurationShardMap =
125             new EnumMap<>(LogicalDatastoreType.class);
126
127     private final EnumMap<LogicalDatastoreType, PrefixedShardConfigWriter> writerMap =
128             new EnumMap<>(LogicalDatastoreType.class);
129
130     private final PrefixedShardConfigUpdateHandler updateHandler;
131
132     public DistributedShardedDOMDataTree(final ActorSystemProvider actorSystemProvider,
133                                          final DistributedDataStore distributedOperDatastore,
134                                          final DistributedDataStore distributedConfigDatastore) {
135         this.actorSystem = Preconditions.checkNotNull(actorSystemProvider).getActorSystem();
136         this.distributedOperDatastore = Preconditions.checkNotNull(distributedOperDatastore);
137         this.distributedConfigDatastore = Preconditions.checkNotNull(distributedConfigDatastore);
138         shardedDOMDataTree = new ShardedDOMDataTree();
139
140         shardedDataTreeActor = createShardedDataTreeActor(actorSystem,
141                 new ShardedDataTreeActorCreator()
142                         .setShardingService(this)
143                         .setActorSystem(actorSystem)
144                         .setClusterWrapper(distributedConfigDatastore.getActorContext().getClusterWrapper())
145                         .setDistributedConfigDatastore(distributedConfigDatastore)
146                         .setDistributedOperDatastore(distributedOperDatastore)
147                         .setLookupTaskMaxRetries(LOOKUP_TASK_MAX_RETRIES),
148                 ACTOR_ID);
149
150         this.memberName = distributedConfigDatastore.getActorContext().getCurrentMemberName();
151
152         updateHandler = new PrefixedShardConfigUpdateHandler(shardedDataTreeActor,
153                 distributedConfigDatastore.getActorContext().getCurrentMemberName());
154
155         LOG.debug("{} - Starting prefix configuration shards", memberName);
156         createPrefixConfigShard(distributedConfigDatastore);
157         createPrefixConfigShard(distributedOperDatastore);
158     }
159
160     private void createPrefixConfigShard(final DistributedDataStore dataStore) {
161         Configuration configuration = dataStore.getActorContext().getConfiguration();
162         Collection<MemberName> memberNames = configuration.getUniqueMemberNamesForAllShards();
163         CreateShard createShardMessage =
164                 new CreateShard(new ModuleShardConfiguration(PrefixShards.QNAME.getNamespace(),
165                         "prefix-shard-configuration", ClusterUtils.PREFIX_CONFIG_SHARD_ID, ModuleShardStrategy.NAME,
166                         memberNames),
167                         Shard.builder(), dataStore.getActorContext().getDatastoreContext());
168
169         dataStore.getActorContext().getShardManager().tell(createShardMessage, noSender());
170     }
171
172     /**
173      * This will try to initialize prefix configuration shards upon their
174      * successful start. We need to create writers to these shards, so we can
175      * satisfy future {@link #createDistributedShard} and
176      * {@link #resolveShardAdditions} requests and update prefix configuration
177      * shards accordingly.
178      *
179      * <p>
180      * We also need to initialize listeners on these shards, so we can react
181      * on changes made on them by other cluster members or even by ourselves.
182      *
183      * <p>
184      * Finally, we need to be sure that default shards for both operational and
185      * configuration data stores are up and running and we have distributed
186      * shards frontend created for them.
187      *
188      * <p>
189      * This is intended to be invoked by blueprint as initialization method.
190      */
191     public void init() {
192         // create our writers to the configuration
193         try {
194             LOG.debug("{} - starting config shard lookup.",
195                     distributedConfigDatastore.getActorContext().getCurrentMemberName());
196
197             // We have to wait for prefix config shards to be up and running
198             // so we can create datastore clients for them
199             handleConfigShardLookup().get(SHARD_FUTURE_TIMEOUT_DURATION.length(), SHARD_FUTURE_TIMEOUT_DURATION.unit());
200
201             LOG.debug("Prefix configuration shards ready - creating clients");
202
203         } catch (InterruptedException | ExecutionException | TimeoutException e) {
204             throw new IllegalStateException("Prefix config shards not found", e);
205         }
206
207         try {
208             LOG.debug("Prefix configuration shards ready - creating clients");
209             configurationShardMap.put(LogicalDatastoreType.CONFIGURATION,
210                     createDatastoreClient(ClusterUtils.PREFIX_CONFIG_SHARD_ID,
211                             distributedConfigDatastore.getActorContext()));
212         } catch (final DOMDataTreeShardCreationFailedException e) {
213             throw new IllegalStateException(
214                     "Unable to create datastoreClient for config DS prefix configuration shard.", e);
215         }
216
217         try {
218             configurationShardMap.put(LogicalDatastoreType.OPERATIONAL,
219                     createDatastoreClient(ClusterUtils.PREFIX_CONFIG_SHARD_ID,
220                             distributedOperDatastore.getActorContext()));
221
222         } catch (final DOMDataTreeShardCreationFailedException e) {
223             throw new IllegalStateException(
224                         "Unable to create datastoreClient for oper DS prefix configuration shard.", e);
225         }
226
227         writerMap.put(LogicalDatastoreType.CONFIGURATION, new PrefixedShardConfigWriter(
228                 configurationShardMap.get(LogicalDatastoreType.CONFIGURATION).getKey()));
229
230         writerMap.put(LogicalDatastoreType.OPERATIONAL, new PrefixedShardConfigWriter(
231                 configurationShardMap.get(LogicalDatastoreType.OPERATIONAL).getKey()));
232
233         updateHandler.initListener(distributedConfigDatastore, LogicalDatastoreType.CONFIGURATION);
234         updateHandler.initListener(distributedOperDatastore, LogicalDatastoreType.OPERATIONAL);
235
236         distributedConfigDatastore.getActorContext().getShardManager().tell(InitConfigListener.INSTANCE, noSender());
237         distributedOperDatastore.getActorContext().getShardManager().tell(InitConfigListener.INSTANCE, noSender());
238
239
240         //create shard registration for DEFAULT_SHARD
241         try {
242             defaultShardRegistrations.put(LogicalDatastoreType.CONFIGURATION,
243                     initDefaultShard(LogicalDatastoreType.CONFIGURATION));
244         } catch (final InterruptedException | ExecutionException e) {
245             throw new IllegalStateException("Unable to create default shard frontend for config shard", e);
246         }
247
248         try {
249             defaultShardRegistrations.put(LogicalDatastoreType.OPERATIONAL,
250                     initDefaultShard(LogicalDatastoreType.OPERATIONAL));
251         } catch (final InterruptedException | ExecutionException e) {
252             throw new IllegalStateException("Unable to create default shard frontend for operational shard", e);
253         }
254     }
255
256     private ListenableFuture<List<Void>> handleConfigShardLookup() {
257
258         final ListenableFuture<Void> configFuture = lookupConfigShard(LogicalDatastoreType.CONFIGURATION);
259         final ListenableFuture<Void> operFuture = lookupConfigShard(LogicalDatastoreType.OPERATIONAL);
260
261         return Futures.allAsList(configFuture, operFuture);
262     }
263
264     private ListenableFuture<Void> lookupConfigShard(final LogicalDatastoreType type) {
265         final SettableFuture<Void> future = SettableFuture.create();
266
267         final Future<Object> ask =
268                 Patterns.ask(shardedDataTreeActor, new StartConfigShardLookup(type), SHARD_FUTURE_TIMEOUT);
269
270         ask.onComplete(new OnComplete<Object>() {
271             @Override
272             public void onComplete(final Throwable throwable, final Object result) throws Throwable {
273                 if (throwable != null) {
274                     future.setException(throwable);
275                 } else {
276                     future.set(null);
277                 }
278             }
279         }, actorSystem.dispatcher());
280
281         return future;
282     }
283
284     @Nonnull
285     @Override
286     public <T extends DOMDataTreeListener> ListenerRegistration<T> registerListener(
287             final T listener, final Collection<DOMDataTreeIdentifier> subtrees,
288             final boolean allowRxMerges, final Collection<DOMDataTreeProducer> producers)
289             throws DOMDataTreeLoopException {
290         return shardedDOMDataTree.registerListener(listener, subtrees, allowRxMerges, producers);
291     }
292
293     @Nonnull
294     @Override
295     public DOMDataTreeProducer createProducer(@Nonnull final Collection<DOMDataTreeIdentifier> subtrees) {
296         LOG.debug("{} - Creating producer for {}",
297                 distributedConfigDatastore.getActorContext().getClusterWrapper().getCurrentMemberName(), subtrees);
298         final DOMDataTreeProducer producer = shardedDOMDataTree.createProducer(subtrees);
299
300         final Object response = distributedConfigDatastore.getActorContext()
301                 .executeOperation(shardedDataTreeActor, new ProducerCreated(subtrees));
302         if (response == null) {
303             LOG.debug("{} - Received success from remote nodes, creating producer:{}",
304                     distributedConfigDatastore.getActorContext().getClusterWrapper().getCurrentMemberName(), subtrees);
305             return new ProxyProducer(producer, subtrees, shardedDataTreeActor,
306                     distributedConfigDatastore.getActorContext());
307         } else if (response instanceof Exception) {
308             closeProducer(producer);
309             throw Throwables.propagate((Exception) response);
310         } else {
311             closeProducer(producer);
312             throw new RuntimeException("Unexpected response to create producer received." + response);
313         }
314     }
315
316     @Override
317     public CompletionStage<DistributedShardRegistration> createDistributedShard(
318             final DOMDataTreeIdentifier prefix, final Collection<MemberName> replicaMembers)
319             throws DOMDataTreeShardingConflictException {
320
321         synchronized (shards) {
322             final DOMDataTreePrefixTableEntry<DOMDataTreeShardRegistration<DOMDataTreeShard>> lookup =
323                     shards.lookup(prefix);
324             if (lookup != null && lookup.getValue().getPrefix().equals(prefix)) {
325                 throw new DOMDataTreeShardingConflictException(
326                         "Prefix " + prefix + " is already occupied by another shard.");
327             }
328         }
329
330         final PrefixedShardConfigWriter writer = writerMap.get(prefix.getDatastoreType());
331
332         final ListenableFuture<Void> writeFuture =
333                 writer.writeConfig(prefix.getRootIdentifier(), replicaMembers);
334
335         final Promise<DistributedShardRegistration> shardRegistrationPromise = akka.dispatch.Futures.promise();
336         Futures.addCallback(writeFuture, new FutureCallback<Void>() {
337             @Override
338             public void onSuccess(@Nullable final Void result) {
339
340                 final Future<Object> ask =
341                         Patterns.ask(shardedDataTreeActor, new LookupPrefixShard(prefix), SHARD_FUTURE_TIMEOUT);
342
343                 shardRegistrationPromise.completeWith(ask.transform(
344                         new Mapper<Object, DistributedShardRegistration>() {
345                             @Override
346                             public DistributedShardRegistration apply(final Object parameter) {
347                                 return new DistributedShardRegistrationImpl(
348                                         prefix, shardedDataTreeActor, DistributedShardedDOMDataTree.this);
349                             }
350                         },
351                         new Mapper<Throwable, Throwable>() {
352                             @Override
353                             public Throwable apply(final Throwable throwable) {
354                                 return new DOMDataTreeShardCreationFailedException(
355                                         "Unable to create a cds shard.", throwable);
356                             }
357                         }, actorSystem.dispatcher()));
358             }
359
360             @Override
361             public void onFailure(final Throwable throwable) {
362                 shardRegistrationPromise.failure(
363                         new DOMDataTreeShardCreationFailedException("Unable to create a cds shard.", throwable));
364             }
365         });
366
367         return FutureConverters.toJava(shardRegistrationPromise.future());
368     }
369
370     void resolveShardAdditions(final Set<DOMDataTreeIdentifier> additions) {
371         LOG.debug("Member {}: Resolving additions : {}", memberName, additions);
372         final ArrayList<DOMDataTreeIdentifier> list = new ArrayList<>(additions);
373         // we need to register the shards from top to bottom, so we need to atleast make sure the ordering reflects that
374         Collections.sort(list, (o1, o2) -> {
375             if (o1.getRootIdentifier().getPathArguments().size() < o2.getRootIdentifier().getPathArguments().size()) {
376                 return -1;
377             } else if (o1.getRootIdentifier().getPathArguments().size()
378                     == o2.getRootIdentifier().getPathArguments().size()) {
379                 return 0;
380             } else {
381                 return 1;
382             }
383         });
384         list.forEach(this::createShardFrontend);
385     }
386
387     void resolveShardRemovals(final Set<DOMDataTreeIdentifier> removals) {
388         LOG.debug("Member {}: Resolving removals : {}", memberName, removals);
389
390         // do we need to go from bottom to top?
391         removals.forEach(this::despawnShardFrontend);
392     }
393
394     private void createShardFrontend(final DOMDataTreeIdentifier prefix) {
395         LOG.debug("Member {}: Creating CDS shard for prefix: {}", memberName, prefix);
396         final String shardName = ClusterUtils.getCleanShardName(prefix.getRootIdentifier());
397         final DistributedDataStore distributedDataStore =
398                 prefix.getDatastoreType().equals(org.opendaylight.mdsal.common.api.LogicalDatastoreType.CONFIGURATION)
399                         ? distributedConfigDatastore : distributedOperDatastore;
400
401         try (final DOMDataTreeProducer producer = localCreateProducer(Collections.singletonList(prefix))) {
402             final Entry<DataStoreClient, ActorRef> entry =
403                     createDatastoreClient(shardName, distributedDataStore.getActorContext());
404
405             final DistributedShardFrontend shard =
406                     new DistributedShardFrontend(distributedDataStore, entry.getKey(), prefix);
407
408             @SuppressWarnings("unchecked")
409             final DOMDataTreeShardRegistration<DOMDataTreeShard> reg =
410                     (DOMDataTreeShardRegistration) shardedDOMDataTree.registerDataTreeShard(prefix, shard, producer);
411
412             synchronized (shards) {
413                 shards.store(prefix, reg);
414             }
415
416         } catch (final DOMDataTreeShardingConflictException e) {
417             LOG.error("{}: Prefix {} is already occupied by another shard",
418                     distributedConfigDatastore.getActorContext().getClusterWrapper().getCurrentMemberName(), prefix, e);
419         } catch (DOMDataTreeProducerException e) {
420             LOG.error("Unable to close producer", e);
421         } catch (DOMDataTreeShardCreationFailedException e) {
422             LOG.error("Unable to create datastore client for shard {}", prefix, e);
423         }
424     }
425
426     private void despawnShardFrontend(final DOMDataTreeIdentifier prefix) {
427         LOG.debug("Member {}: Removing CDS shard for prefix: {}", memberName, prefix);
428         final DOMDataTreePrefixTableEntry<DOMDataTreeShardRegistration<DOMDataTreeShard>> lookup;
429         synchronized (shards) {
430             lookup = shards.lookup(prefix);
431         }
432
433         if (lookup == null || !lookup.getValue().getPrefix().equals(prefix)) {
434             LOG.debug("Member {}: Received despawn for non-existing CDS shard frontend, prefix: {}, ignoring..",
435                     memberName, prefix);
436             return;
437         }
438
439         lookup.getValue().close();
440         // need to remove from our local table thats used for tracking
441         synchronized (shards) {
442             shards.remove(prefix);
443         }
444
445         final PrefixedShardConfigWriter writer = writerMap.get(prefix.getDatastoreType());
446         final ListenableFuture<Void> future = writer.removeConfig(prefix.getRootIdentifier());
447
448         Futures.addCallback(future, new FutureCallback<Void>() {
449             @Override
450             public void onSuccess(@Nullable Void result) {
451                 LOG.debug("{} - Succesfuly removed shard for {}", memberName, prefix);
452             }
453
454             @Override
455             public void onFailure(Throwable throwable) {
456                 LOG.error("Removal of shard {} from configuration failed.", prefix, throwable);
457             }
458         });
459     }
460
461     DOMDataTreePrefixTableEntry<DOMDataTreeShardRegistration<DOMDataTreeShard>> lookupShardFrontend(
462             final DOMDataTreeIdentifier prefix) {
463         return shards.lookup(prefix);
464
465     }
466
467     DOMDataTreeProducer localCreateProducer(final Collection<DOMDataTreeIdentifier> prefix) {
468         return shardedDOMDataTree.createProducer(prefix);
469     }
470
471     @Nonnull
472     @Override
473     public <T extends DOMDataTreeShard> ListenerRegistration<T> registerDataTreeShard(
474             @Nonnull final DOMDataTreeIdentifier prefix,
475             @Nonnull final T shard,
476             @Nonnull final DOMDataTreeProducer producer)
477             throws DOMDataTreeShardingConflictException {
478
479         LOG.debug("Registering shard[{}] at prefix: {}", shard, prefix);
480
481         if (producer instanceof ProxyProducer) {
482             return shardedDOMDataTree.registerDataTreeShard(prefix, shard, ((ProxyProducer) producer).delegate());
483         }
484
485         return shardedDOMDataTree.registerDataTreeShard(prefix, shard, producer);
486     }
487
488     @SuppressWarnings("checkstyle:IllegalCatch")
489     private Entry<DataStoreClient, ActorRef> createDatastoreClient(
490             final String shardName, final ActorContext actorContext)
491             throws DOMDataTreeShardCreationFailedException {
492
493         LOG.debug("Creating distributed datastore client for shard {}", shardName);
494         final Props distributedDataStoreClientProps =
495                 SimpleDataStoreClientActor.props(memberName, "Shard-" + shardName, actorContext, shardName);
496
497         final ActorRef clientActor = actorSystem.actorOf(distributedDataStoreClientProps);
498         try {
499             return new SimpleEntry<>(SimpleDataStoreClientActor
500                     .getDistributedDataStoreClient(clientActor, 30, TimeUnit.SECONDS), clientActor);
501         } catch (final Exception e) {
502             LOG.error("Failed to get actor for {}", distributedDataStoreClientProps, e);
503             clientActor.tell(PoisonPill.getInstance(), noSender());
504             throw new DOMDataTreeShardCreationFailedException(
505                     "Unable to create datastore client for shard{" + shardName + "}", e);
506         }
507     }
508
509     @SuppressWarnings("checkstyle:IllegalCatch")
510     private DistributedShardRegistration initDefaultShard(final LogicalDatastoreType logicalDatastoreType)
511             throws ExecutionException, InterruptedException {
512         final Collection<MemberName> names =
513                 distributedConfigDatastore.getActorContext().getConfiguration().getUniqueMemberNamesForAllShards();
514
515         final PrefixedShardConfigWriter writer = writerMap.get(logicalDatastoreType);
516
517         if (writer.checkDefaultIsPresent()) {
518             LOG.debug("Default shard for {} is already present in the config. Possibly saved in snapshot.",
519                     logicalDatastoreType);
520             return new DistributedShardRegistrationImpl(
521                     new DOMDataTreeIdentifier(logicalDatastoreType, YangInstanceIdentifier.EMPTY),
522                     shardedDataTreeActor, this);
523         } else {
524             try {
525                 // There can be situation when there is already started default shard
526                 // because it is present in modules.conf. In that case we have to create
527                 // just frontend for default shard, but not shard itself
528                 // TODO we don't have to do it for config and operational default shard
529                 // separately. Just one of them should be enough
530                 final ActorContext actorContext = logicalDatastoreType == LogicalDatastoreType.CONFIGURATION
531                         ? distributedConfigDatastore.getActorContext() : distributedOperDatastore.getActorContext();
532
533                 final Optional<ActorRef> defaultLocalShardOptional =
534                         actorContext.findLocalShard(ClusterUtils.getCleanShardName(YangInstanceIdentifier.EMPTY));
535
536                 if (defaultLocalShardOptional.isPresent()) {
537                     LOG.debug("{} Default shard is already started, creating just frontend", logicalDatastoreType);
538                     createShardFrontend(new DOMDataTreeIdentifier(logicalDatastoreType, YangInstanceIdentifier.EMPTY));
539                     return new DistributedShardRegistrationImpl(
540                             new DOMDataTreeIdentifier(logicalDatastoreType, YangInstanceIdentifier.EMPTY),
541                             shardedDataTreeActor, this);
542                 }
543
544                 // we should probably only have one node create the default shards
545                 return Await.result(FutureConverters.toScala(createDistributedShard(
546                         new DOMDataTreeIdentifier(logicalDatastoreType, YangInstanceIdentifier.EMPTY), names)),
547                         SHARD_FUTURE_TIMEOUT_DURATION);
548             } catch (DOMDataTreeShardingConflictException e) {
549                 LOG.debug("Default shard already registered, possibly due to other node doing it faster");
550                 return new DistributedShardRegistrationImpl(
551                         new DOMDataTreeIdentifier(logicalDatastoreType, YangInstanceIdentifier.EMPTY),
552                         shardedDataTreeActor, this);
553             } catch (Exception e) {
554                 LOG.error("{} default shard initialization failed", logicalDatastoreType, e);
555                 throw new RuntimeException(e);
556             }
557         }
558     }
559
560     private static void closeProducer(final DOMDataTreeProducer producer) {
561         try {
562             producer.close();
563         } catch (final DOMDataTreeProducerException e) {
564             LOG.error("Unable to close producer", e);
565         }
566     }
567
568     @SuppressWarnings("checkstyle:IllegalCatch")
569     private static ActorRef createShardedDataTreeActor(final ActorSystem actorSystem,
570                                                        final ShardedDataTreeActorCreator creator,
571                                                        final String shardDataTreeActorId) {
572         Exception lastException = null;
573
574         for (int i = 0; i < MAX_ACTOR_CREATION_RETRIES; i++) {
575             try {
576                 return actorSystem.actorOf(creator.props(), shardDataTreeActorId);
577             } catch (final Exception e) {
578                 lastException = e;
579                 Uninterruptibles.sleepUninterruptibly(ACTOR_RETRY_DELAY, ACTOR_RETRY_TIME_UNIT);
580                 LOG.debug("Could not create actor {} because of {} -"
581                                 + " waiting for sometime before retrying (retry count = {})",
582                         shardDataTreeActorId, e.getMessage(), i);
583             }
584         }
585
586         throw new IllegalStateException("Failed to create actor for ShardedDOMDataTree", lastException);
587     }
588
589     private class DistributedShardRegistrationImpl implements DistributedShardRegistration {
590
591         private final DOMDataTreeIdentifier prefix;
592         private final ActorRef shardedDataTreeActor;
593         private final DistributedShardedDOMDataTree distributedShardedDOMDataTree;
594
595         DistributedShardRegistrationImpl(final DOMDataTreeIdentifier prefix,
596                                          final ActorRef shardedDataTreeActor,
597                                          final DistributedShardedDOMDataTree distributedShardedDOMDataTree) {
598             this.prefix = prefix;
599             this.shardedDataTreeActor = shardedDataTreeActor;
600             this.distributedShardedDOMDataTree = distributedShardedDOMDataTree;
601         }
602
603         @Override
604         public CompletionStage<Void> close() {
605             // first despawn on the local node
606             distributedShardedDOMDataTree.despawnShardFrontend(prefix);
607             // update the config so the remote nodes are updated
608             final Future<Object> ask =
609                     Patterns.ask(shardedDataTreeActor, new PrefixShardRemovalLookup(prefix), SHARD_FUTURE_TIMEOUT);
610
611             final Future<Void> closeFuture = ask.transform(
612                     new Mapper<Object, Void>() {
613                         @Override
614                         public Void apply(Object parameter) {
615                             return null;
616                         }
617                     },
618                     new Mapper<Throwable, Throwable>() {
619                         @Override
620                         public Throwable apply(Throwable throwable) {
621                             return throwable;
622                         }
623                     }, actorSystem.dispatcher());
624
625             return FutureConverters.toJava(closeFuture);
626         }
627     }
628
629     private static final class ProxyProducer extends ForwardingObject implements DOMDataTreeProducer {
630
631         private final DOMDataTreeProducer delegate;
632         private final Collection<DOMDataTreeIdentifier> subtrees;
633         private final ActorRef shardDataTreeActor;
634         private final ActorContext actorContext;
635
636         ProxyProducer(final DOMDataTreeProducer delegate,
637                       final Collection<DOMDataTreeIdentifier> subtrees,
638                       final ActorRef shardDataTreeActor,
639                       final ActorContext actorContext) {
640             this.delegate = Preconditions.checkNotNull(delegate);
641             this.subtrees = Preconditions.checkNotNull(subtrees);
642             this.shardDataTreeActor = Preconditions.checkNotNull(shardDataTreeActor);
643             this.actorContext = Preconditions.checkNotNull(actorContext);
644         }
645
646         @Nonnull
647         @Override
648         public DOMDataTreeCursorAwareTransaction createTransaction(final boolean isolated) {
649             return delegate.createTransaction(isolated);
650         }
651
652         @Nonnull
653         @Override
654         public DOMDataTreeProducer createProducer(@Nonnull final Collection<DOMDataTreeIdentifier> subtrees) {
655             // TODO we probably don't need to distribute this on the remote nodes since once we have this producer
656             // open we surely have the rights to all the subtrees.
657             return delegate.createProducer(subtrees);
658         }
659
660         @Override
661         public void close() throws DOMDataTreeProducerException {
662             delegate.close();
663
664             final Object o = actorContext.executeOperation(shardDataTreeActor, new ProducerRemoved(subtrees));
665             if (o instanceof DOMDataTreeProducerException) {
666                 throw ((DOMDataTreeProducerException) o);
667             } else if (o instanceof Throwable) {
668                 throw new DOMDataTreeProducerException("Unable to close producer", (Throwable) o);
669             }
670         }
671
672         @Override
673         protected DOMDataTreeProducer delegate() {
674             return delegate;
675         }
676     }
677 }