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