Deprecate DOMDataTreeProducer-related classes
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / sharding / DistributedShardedDOMDataTree.java
index 235cc5df306162ef9d9af9aa4b62eafff4b98ad4..bb51203e4a73a92b36d66e7e31b5813f9b01058d 100644 (file)
@@ -5,10 +5,12 @@
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
-
 package org.opendaylight.controller.cluster.sharding;
 
 import static akka.actor.ActorRef.noSender;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import static java.util.Objects.requireNonNull;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
@@ -18,8 +20,6 @@ import akka.dispatch.Mapper;
 import akka.dispatch.OnComplete;
 import akka.pattern.Patterns;
 import akka.util.Timeout;
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ClassToInstanceMap;
 import com.google.common.collect.ForwardingObject;
@@ -31,33 +31,32 @@ import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.SettableFuture;
 import com.google.common.util.concurrent.Uninterruptibles;
 import java.util.AbstractMap.SimpleEntry;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.EnumMap;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.CompletionStage;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
-import javax.annotation.concurrent.GuardedBy;
+import org.checkerframework.checker.lock.qual.GuardedBy;
 import org.opendaylight.controller.cluster.ActorSystemProvider;
 import org.opendaylight.controller.cluster.access.concepts.MemberName;
 import org.opendaylight.controller.cluster.databroker.actors.dds.DataStoreClient;
 import org.opendaylight.controller.cluster.databroker.actors.dds.SimpleDataStoreClientActor;
-import org.opendaylight.controller.cluster.datastore.AbstractDataStore;
+import org.opendaylight.controller.cluster.datastore.DistributedDataStoreInterface;
 import org.opendaylight.controller.cluster.datastore.Shard;
 import org.opendaylight.controller.cluster.datastore.config.Configuration;
 import org.opendaylight.controller.cluster.datastore.config.ModuleShardConfiguration;
 import org.opendaylight.controller.cluster.datastore.messages.CreateShard;
 import org.opendaylight.controller.cluster.datastore.shardstrategy.ModuleShardStrategy;
-import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
+import org.opendaylight.controller.cluster.datastore.utils.ActorUtils;
 import org.opendaylight.controller.cluster.datastore.utils.ClusterUtils;
 import org.opendaylight.controller.cluster.dom.api.CDSDataTreeProducer;
 import org.opendaylight.controller.cluster.dom.api.CDSShardAccess;
@@ -98,6 +97,7 @@ import scala.concurrent.duration.FiniteDuration;
  * A layer on top of DOMDataTreeService that distributes producer/shard registrations to remote nodes via
  * {@link ShardedDataTreeActor}. Also provides QoL method for addition of prefix based clustered shard into the system.
  */
+@Deprecated(forRemoval = true)
 public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDataTreeShardingService,
         DistributedShardFactory {
 
@@ -115,8 +115,8 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
 
     private final ShardedDOMDataTree shardedDOMDataTree;
     private final ActorSystem actorSystem;
-    private final AbstractDataStore distributedOperDatastore;
-    private final AbstractDataStore distributedConfigDatastore;
+    private final DistributedDataStoreInterface distributedOperDatastore;
+    private final DistributedDataStoreInterface distributedConfigDatastore;
 
     private final ActorRef shardedDataTreeActor;
     private final MemberName memberName;
@@ -134,43 +134,43 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
     private final PrefixedShardConfigUpdateHandler updateHandler;
 
     public DistributedShardedDOMDataTree(final ActorSystemProvider actorSystemProvider,
-                                         final AbstractDataStore distributedOperDatastore,
-                                         final AbstractDataStore distributedConfigDatastore) {
-        this.actorSystem = Preconditions.checkNotNull(actorSystemProvider).getActorSystem();
-        this.distributedOperDatastore = Preconditions.checkNotNull(distributedOperDatastore);
-        this.distributedConfigDatastore = Preconditions.checkNotNull(distributedConfigDatastore);
+                                         final DistributedDataStoreInterface distributedOperDatastore,
+                                         final DistributedDataStoreInterface distributedConfigDatastore) {
+        this.actorSystem = requireNonNull(actorSystemProvider).getActorSystem();
+        this.distributedOperDatastore = requireNonNull(distributedOperDatastore);
+        this.distributedConfigDatastore = requireNonNull(distributedConfigDatastore);
         shardedDOMDataTree = new ShardedDOMDataTree();
 
         shardedDataTreeActor = createShardedDataTreeActor(actorSystem,
                 new ShardedDataTreeActorCreator()
                         .setShardingService(this)
                         .setActorSystem(actorSystem)
-                        .setClusterWrapper(distributedConfigDatastore.getActorContext().getClusterWrapper())
+                        .setClusterWrapper(distributedConfigDatastore.getActorUtils().getClusterWrapper())
                         .setDistributedConfigDatastore(distributedConfigDatastore)
                         .setDistributedOperDatastore(distributedOperDatastore)
                         .setLookupTaskMaxRetries(LOOKUP_TASK_MAX_RETRIES),
                 ACTOR_ID);
 
-        this.memberName = distributedConfigDatastore.getActorContext().getCurrentMemberName();
+        this.memberName = distributedConfigDatastore.getActorUtils().getCurrentMemberName();
 
         updateHandler = new PrefixedShardConfigUpdateHandler(shardedDataTreeActor,
-                distributedConfigDatastore.getActorContext().getCurrentMemberName());
+                distributedConfigDatastore.getActorUtils().getCurrentMemberName());
 
         LOG.debug("{} - Starting prefix configuration shards", memberName);
         createPrefixConfigShard(distributedConfigDatastore);
         createPrefixConfigShard(distributedOperDatastore);
     }
 
-    private static void createPrefixConfigShard(final AbstractDataStore dataStore) {
-        Configuration configuration = dataStore.getActorContext().getConfiguration();
+    private static void createPrefixConfigShard(final DistributedDataStoreInterface dataStore) {
+        Configuration configuration = dataStore.getActorUtils().getConfiguration();
         Collection<MemberName> memberNames = configuration.getUniqueMemberNamesForAllShards();
         CreateShard createShardMessage =
                 new CreateShard(new ModuleShardConfiguration(PrefixShards.QNAME.getNamespace(),
                         "prefix-shard-configuration", ClusterUtils.PREFIX_CONFIG_SHARD_ID, ModuleShardStrategy.NAME,
                         memberNames),
-                        Shard.builder(), dataStore.getActorContext().getDatastoreContext());
+                        Shard.builder(), dataStore.getActorUtils().getDatastoreContext());
 
-        dataStore.getActorContext().getShardManager().tell(createShardMessage, noSender());
+        dataStore.getActorUtils().getShardManager().tell(createShardMessage, noSender());
     }
 
     /**
@@ -208,7 +208,7 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
             LOG.debug("{}: Prefix configuration shards ready - creating clients", memberName);
             configurationShardMap.put(LogicalDatastoreType.CONFIGURATION,
                     createDatastoreClient(ClusterUtils.PREFIX_CONFIG_SHARD_ID,
-                            distributedConfigDatastore.getActorContext()));
+                            distributedConfigDatastore.getActorUtils()));
         } catch (final DOMDataTreeShardCreationFailedException e) {
             throw new IllegalStateException(
                     "Unable to create datastoreClient for config DS prefix configuration shard.", e);
@@ -217,7 +217,7 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
         try {
             configurationShardMap.put(LogicalDatastoreType.OPERATIONAL,
                     createDatastoreClient(ClusterUtils.PREFIX_CONFIG_SHARD_ID,
-                            distributedOperDatastore.getActorContext()));
+                            distributedOperDatastore.getActorUtils()));
 
         } catch (final DOMDataTreeShardCreationFailedException e) {
             throw new IllegalStateException(
@@ -233,22 +233,13 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
         updateHandler.initListener(distributedConfigDatastore, LogicalDatastoreType.CONFIGURATION);
         updateHandler.initListener(distributedOperDatastore, LogicalDatastoreType.OPERATIONAL);
 
-        distributedConfigDatastore.getActorContext().getShardManager().tell(InitConfigListener.INSTANCE, noSender());
-        distributedOperDatastore.getActorContext().getShardManager().tell(InitConfigListener.INSTANCE, noSender());
+        distributedConfigDatastore.getActorUtils().getShardManager().tell(InitConfigListener.INSTANCE, noSender());
+        distributedOperDatastore.getActorUtils().getShardManager().tell(InitConfigListener.INSTANCE, noSender());
 
 
         //create shard registration for DEFAULT_SHARD
-        try {
-            initDefaultShard(LogicalDatastoreType.CONFIGURATION);
-        } catch (final InterruptedException | ExecutionException e) {
-            throw new IllegalStateException("Unable to create default shard frontend for config shard", e);
-        }
-
-        try {
-            initDefaultShard(LogicalDatastoreType.OPERATIONAL);
-        } catch (final InterruptedException | ExecutionException e) {
-            throw new IllegalStateException("Unable to create default shard frontend for operational shard", e);
-        }
+        initDefaultShard(LogicalDatastoreType.CONFIGURATION);
+        initDefaultShard(LogicalDatastoreType.OPERATIONAL);
     }
 
     private ListenableFuture<List<Void>> handleConfigShardLookup() {
@@ -265,9 +256,9 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
         final Future<Object> ask =
                 Patterns.ask(shardedDataTreeActor, new StartConfigShardLookup(type), SHARD_FUTURE_TIMEOUT);
 
-        ask.onComplete(new OnComplete<Object>() {
+        ask.onComplete(new OnComplete<>() {
             @Override
-            public void onComplete(final Throwable throwable, final Object result) throws Throwable {
+            public void onComplete(final Throwable throwable, final Object result) {
                 if (throwable != null) {
                     future.setException(throwable);
                 } else {
@@ -279,7 +270,6 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
         return future;
     }
 
-    @Nonnull
     @Override
     public <T extends DOMDataTreeListener> ListenerRegistration<T> registerListener(
             final T listener, final Collection<DOMDataTreeIdentifier> subtrees,
@@ -293,18 +283,17 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
         return ImmutableClassToInstanceMap.of();
     }
 
-    @Nonnull
     @Override
-    public DOMDataTreeProducer createProducer(@Nonnull final Collection<DOMDataTreeIdentifier> subtrees) {
+    public DOMDataTreeProducer createProducer(final Collection<DOMDataTreeIdentifier> subtrees) {
         LOG.debug("{} - Creating producer for {}", memberName, subtrees);
         final DOMDataTreeProducer producer = shardedDOMDataTree.createProducer(subtrees);
 
-        final Object response = distributedConfigDatastore.getActorContext()
+        final Object response = distributedConfigDatastore.getActorUtils()
                 .executeOperation(shardedDataTreeActor, new ProducerCreated(subtrees));
         if (response == null) {
             LOG.debug("{} - Received success from remote nodes, creating producer:{}", memberName, subtrees);
             return new ProxyProducer(producer, subtrees, shardedDataTreeActor,
-                    distributedConfigDatastore.getActorContext(), shards);
+                    distributedConfigDatastore.getActorUtils(), shards);
         }
 
         closeProducer(producer);
@@ -338,7 +327,7 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
         final Promise<DistributedShardRegistration> shardRegistrationPromise = akka.dispatch.Futures.promise();
         Futures.addCallback(writeFuture, new FutureCallback<Void>() {
             @Override
-            public void onSuccess(@Nullable final Void result) {
+            public void onSuccess(final Void result) {
 
                 final Future<Object> ask =
                         Patterns.ask(shardedDataTreeActor, new LookupPrefixShard(prefix), SHARD_FUTURE_TIMEOUT);
@@ -372,19 +361,11 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
 
     void resolveShardAdditions(final Set<DOMDataTreeIdentifier> additions) {
         LOG.debug("{}: Resolving additions : {}", memberName, additions);
-        final ArrayList<DOMDataTreeIdentifier> list = new ArrayList<>(additions);
         // we need to register the shards from top to bottom, so we need to atleast make sure the ordering reflects that
-        Collections.sort(list, (o1, o2) -> {
-            if (o1.getRootIdentifier().getPathArguments().size() < o2.getRootIdentifier().getPathArguments().size()) {
-                return -1;
-            } else if (o1.getRootIdentifier().getPathArguments().size()
-                    == o2.getRootIdentifier().getPathArguments().size()) {
-                return 0;
-            } else {
-                return 1;
-            }
-        });
-        list.forEach(this::createShardFrontend);
+        additions
+            .stream()
+            .sorted(Comparator.comparingInt(o -> o.getRootIdentifier().getPathArguments().size()))
+            .forEachOrdered(this::createShardFrontend);
     }
 
     void resolveShardRemovals(final Set<DOMDataTreeIdentifier> removals) {
@@ -397,13 +378,13 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
     private void createShardFrontend(final DOMDataTreeIdentifier prefix) {
         LOG.debug("{}: Creating CDS shard for prefix: {}", memberName, prefix);
         final String shardName = ClusterUtils.getCleanShardName(prefix.getRootIdentifier());
-        final AbstractDataStore distributedDataStore =
-                prefix.getDatastoreType().equals(org.opendaylight.mdsal.common.api.LogicalDatastoreType.CONFIGURATION)
+        final DistributedDataStoreInterface distributedDataStore =
+                prefix.getDatastoreType().equals(LogicalDatastoreType.CONFIGURATION)
                         ? distributedConfigDatastore : distributedOperDatastore;
 
         try (DOMDataTreeProducer producer = localCreateProducer(Collections.singletonList(prefix))) {
             final Entry<DataStoreClient, ActorRef> entry =
-                    createDatastoreClient(shardName, distributedDataStore.getActorContext());
+                    createDatastoreClient(shardName, distributedDataStore.getActorUtils());
 
             final DistributedShardFrontend shard =
                     new DistributedShardFrontend(distributedDataStore, entry.getKey(), prefix);
@@ -417,7 +398,7 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
 
         } catch (final DOMDataTreeShardingConflictException e) {
             LOG.error("{}: Prefix {} is already occupied by another shard",
-                    distributedConfigDatastore.getActorContext().getClusterWrapper().getCurrentMemberName(), prefix, e);
+                    distributedConfigDatastore.getActorUtils().getClusterWrapper().getCurrentMemberName(), prefix, e);
         } catch (DOMDataTreeProducerException e) {
             LOG.error("Unable to close producer", e);
         } catch (DOMDataTreeShardCreationFailedException e) {
@@ -449,7 +430,7 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
 
         Futures.addCallback(future, new FutureCallback<Void>() {
             @Override
-            public void onSuccess(@Nullable final Void result) {
+            public void onSuccess(final Void result) {
                 LOG.debug("{} - Succesfuly removed shard for {}", memberName, prefix);
             }
 
@@ -471,13 +452,10 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
         return shardedDOMDataTree.createProducer(prefix);
     }
 
-    @Nonnull
     @Override
     public <T extends DOMDataTreeShard> ListenerRegistration<T> registerDataTreeShard(
-            @Nonnull final DOMDataTreeIdentifier prefix,
-            @Nonnull final T shard,
-            @Nonnull final DOMDataTreeProducer producer)
-            throws DOMDataTreeShardingConflictException {
+            final DOMDataTreeIdentifier prefix, final T shard, final DOMDataTreeProducer producer)
+                    throws DOMDataTreeShardingConflictException {
 
         LOG.debug("Registering shard[{}] at prefix: {}", shard, prefix);
 
@@ -489,13 +467,12 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
     }
 
     @SuppressWarnings("checkstyle:IllegalCatch")
-    private Entry<DataStoreClient, ActorRef> createDatastoreClient(
-            final String shardName, final ActorContext actorContext)
+    private Entry<DataStoreClient, ActorRef> createDatastoreClient(final String shardName, final ActorUtils actorUtils)
             throws DOMDataTreeShardCreationFailedException {
 
         LOG.debug("{}: Creating distributed datastore client for shard {}", memberName, shardName);
         final Props distributedDataStoreClientProps =
-                SimpleDataStoreClientActor.props(memberName, "Shard-" + shardName, actorContext, shardName);
+                SimpleDataStoreClientActor.props(memberName, "Shard-" + shardName, actorUtils, shardName);
 
         final ActorRef clientActor = actorSystem.actorOf(distributedDataStoreClientProps);
         try {
@@ -510,8 +487,7 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
     }
 
     @SuppressWarnings("checkstyle:IllegalCatch")
-    private void initDefaultShard(final LogicalDatastoreType logicalDatastoreType)
-            throws ExecutionException, InterruptedException {
+    private void initDefaultShard(final LogicalDatastoreType logicalDatastoreType) {
 
         final PrefixedShardConfigWriter writer = writerMap.get(logicalDatastoreType);
 
@@ -525,16 +501,17 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
 
                 // TODO we don't have to do it for config and operational default shard separately. Just one of them
                 // should be enough
-                final ActorContext actorContext = logicalDatastoreType == LogicalDatastoreType.CONFIGURATION
-                        ? distributedConfigDatastore.getActorContext() : distributedOperDatastore.getActorContext();
+                final ActorUtils actorUtils = logicalDatastoreType == LogicalDatastoreType.CONFIGURATION
+                        ? distributedConfigDatastore.getActorUtils() : distributedOperDatastore.getActorUtils();
 
                 final Optional<ActorRef> defaultLocalShardOptional =
-                        actorContext.findLocalShard(ClusterUtils.getCleanShardName(YangInstanceIdentifier.EMPTY));
+                        actorUtils.findLocalShard(ClusterUtils.getCleanShardName(YangInstanceIdentifier.empty()));
 
                 if (defaultLocalShardOptional.isPresent()) {
                     LOG.debug("{}: Default shard for {} is already started, creating just frontend", memberName,
                             logicalDatastoreType);
-                    createShardFrontend(new DOMDataTreeIdentifier(logicalDatastoreType, YangInstanceIdentifier.EMPTY));
+                    createShardFrontend(new DOMDataTreeIdentifier(logicalDatastoreType,
+                                YangInstanceIdentifier.empty()));
                 }
 
                 // The local shard isn't present - we assume that means the local member isn't in the replica list
@@ -543,10 +520,10 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
                 // the default shard as a prefix shard is problematic in this scenario so it is commented out. Since
                 // the default shard is a module-based shard by default, it makes sense to always treat it as such,
                 // ie bootstrap it in the same manner as the special prefix-configuration and EOS shards.
-//                final Collection<MemberName> names = distributedConfigDatastore.getActorContext().getConfiguration()
+//                final Collection<MemberName> names = distributedConfigDatastore.getActorUtils().getConfiguration()
 //                        .getUniqueMemberNamesForAllShards();
 //                Await.result(FutureConverters.toScala(createDistributedShard(
-//                        new DOMDataTreeIdentifier(logicalDatastoreType, YangInstanceIdentifier.EMPTY), names)),
+//                        new DOMDataTreeIdentifier(logicalDatastoreType, YangInstanceIdentifier.empty()), names)),
 //                        SHARD_FUTURE_TIMEOUT_DURATION);
 //            } catch (DOMDataTreeShardingConflictException e) {
 //                LOG.debug("{}: Default shard for {} already registered, possibly due to other node doing it faster",
@@ -634,7 +611,7 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
         private final DOMDataTreeProducer delegate;
         private final Collection<DOMDataTreeIdentifier> subtrees;
         private final ActorRef shardDataTreeActor;
-        private final ActorContext actorContext;
+        private final ActorUtils actorUtils;
         @GuardedBy("shardAccessMap")
         private final Map<DOMDataTreeIdentifier, CDSShardAccessImpl> shardAccessMap = new HashMap<>();
 
@@ -646,25 +623,23 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
         ProxyProducer(final DOMDataTreeProducer delegate,
                       final Collection<DOMDataTreeIdentifier> subtrees,
                       final ActorRef shardDataTreeActor,
-                      final ActorContext actorContext,
+                      final ActorUtils actorUtils,
                       final DOMDataTreePrefixTable<DOMDataTreeShardRegistration<DOMDataTreeShard>> shardLayout) {
-            this.delegate = Preconditions.checkNotNull(delegate);
-            this.subtrees = Preconditions.checkNotNull(subtrees);
-            this.shardDataTreeActor = Preconditions.checkNotNull(shardDataTreeActor);
-            this.actorContext = Preconditions.checkNotNull(actorContext);
-            this.shardTable = Preconditions.checkNotNull(shardLayout);
+            this.delegate = requireNonNull(delegate);
+            this.subtrees = requireNonNull(subtrees);
+            this.shardDataTreeActor = requireNonNull(shardDataTreeActor);
+            this.actorUtils = requireNonNull(actorUtils);
+            this.shardTable = requireNonNull(shardLayout);
         }
 
-        @Nonnull
         @Override
         public DOMDataTreeCursorAwareTransaction createTransaction(final boolean isolated) {
             return delegate.createTransaction(isolated);
         }
 
-        @Nonnull
         @Override
         @SuppressWarnings("checkstyle:hiddenField")
-        public DOMDataTreeProducer createProducer(@Nonnull final Collection<DOMDataTreeIdentifier> subtrees) {
+        public DOMDataTreeProducer createProducer(final Collection<DOMDataTreeIdentifier> subtrees) {
             // TODO we probably don't need to distribute this on the remote nodes since once we have this producer
             // open we surely have the rights to all the subtrees.
             return delegate.createProducer(subtrees);
@@ -679,7 +654,7 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
                 shardAccessMap.values().forEach(CDSShardAccessImpl::close);
             }
 
-            final Object o = actorContext.executeOperation(shardDataTreeActor, new ProducerRemoved(subtrees));
+            final Object o = actorUtils.executeOperation(shardDataTreeActor, new ProducerRemoved(subtrees));
             if (o instanceof DOMDataTreeProducerException) {
                 throw (DOMDataTreeProducerException) o;
             } else if (o instanceof Throwable) {
@@ -692,16 +667,14 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
             return delegate;
         }
 
-        @Nonnull
         @Override
-        public CDSShardAccess getShardAccess(@Nonnull final DOMDataTreeIdentifier subtree) {
-            Preconditions.checkArgument(
-                    subtrees.stream().anyMatch(dataTreeIdentifier -> dataTreeIdentifier.contains(subtree)),
-                    "Subtree %s is not controlled by this producer %s", subtree, this);
+        public CDSShardAccess getShardAccess(final DOMDataTreeIdentifier subtree) {
+            checkArgument(subtrees.stream().anyMatch(dataTreeIdentifier -> dataTreeIdentifier.contains(subtree)),
+                "Subtree %s is not controlled by this producer %s", subtree, this);
 
             final DOMDataTreePrefixTableEntry<DOMDataTreeShardRegistration<DOMDataTreeShard>> lookup =
                     shardTable.lookup(subtree);
-            Preconditions.checkState(lookup != null, "Subtree %s is not contained in any registered shard.", subtree);
+            checkState(lookup != null, "Subtree %s is not contained in any registered shard.", subtree);
 
             final DOMDataTreeIdentifier lookupId = lookup.getValue().getPrefix();
 
@@ -714,7 +687,7 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
                 // for same subtrees. But maybe it is not needed since there can be only one
                 // producer attached to some subtree at a time. And also how we can close ShardAccess
                 // then
-                final CDSShardAccessImpl shardAccess = new CDSShardAccessImpl(lookupId, actorContext);
+                final CDSShardAccessImpl shardAccess = new CDSShardAccessImpl(lookupId, actorUtils);
                 shardAccessMap.put(lookupId, shardAccess);
                 return shardAccess;
             }