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