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