Rename ActorContext to ActorUtils
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / sharding / DistributedShardedDOMDataTree.java
index 64c3f14dfb5dde59806c31dd3f92e92af745ebc7..df21b901da6676e29e2bf7352e5a734228b9d965 100644 (file)
@@ -9,6 +9,9 @@
 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;
@@ -57,7 +60,7 @@ 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;
@@ -145,16 +148,16 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
                 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);
@@ -162,15 +165,15 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
     }
 
     private static void createPrefixConfigShard(final AbstractDataStore dataStore) {
-        Configuration configuration = dataStore.getActorContext().getConfiguration();
+        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 +211,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 +220,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,8 +236,8 @@ 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
@@ -290,12 +293,12 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
         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);
@@ -386,7 +389,7 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
 
         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);
@@ -400,7 +403,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) {
@@ -472,13 +475,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 {
@@ -507,11 +509,11 @@ 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,
@@ -525,7 +527,7 @@ 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)),
@@ -616,7 +618,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<>();
 
@@ -628,13 +630,13 @@ 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
@@ -661,7 +663,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) {
@@ -677,13 +679,12 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
         @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);
+            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();
 
@@ -696,7 +697,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;
             }