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