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