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