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