Add unit test for FrontedMetadata memory leaks
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / AbstractDataStore.java
index 5c0daca19920768adbf23feaecb05ff59bc72e5b..43314cd3ba0c7ab84987ea2a695533f852bf850a 100644 (file)
@@ -5,15 +5,15 @@
  * 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.datastore;
 
+import static java.util.Objects.requireNonNull;
+
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
 import akka.actor.PoisonPill;
 import akka.actor.Props;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.Uninterruptibles;
 import java.util.concurrent.CountDownLatch;
@@ -27,12 +27,11 @@ import org.opendaylight.controller.cluster.datastore.identifiers.ShardManagerIde
 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.DatastoreConfigurationMXBeanImpl;
 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.DatastoreInfoMXBeanImpl;
 import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot;
+import org.opendaylight.controller.cluster.datastore.shardmanager.AbstractShardManagerCreator;
 import org.opendaylight.controller.cluster.datastore.shardmanager.ShardManagerCreator;
-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.datastore.utils.PrimaryShardInfoFutureCache;
-import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker;
-import org.opendaylight.controller.md.sal.common.api.data.AsyncDataChangeListener;
 import org.opendaylight.mdsal.dom.api.ClusteredDOMDataTreeChangeListener;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeChangeListener;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeCommitCohort;
@@ -42,7 +41,6 @@ import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
 import org.opendaylight.mdsal.dom.spi.store.DOMStoreTreeChangePublisher;
 import org.opendaylight.yangtools.concepts.ListenerRegistration;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
-import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.opendaylight.yangtools.yang.model.api.SchemaContextListener;
 import org.slf4j.Logger;
@@ -59,7 +57,7 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
 
     private static final long READY_WAIT_FACTOR = 3;
 
-    private final ActorContext actorContext;
+    private final ActorUtils actorUtils;
     private final long waitTillReadyTimeInMillis;
 
     private AutoCloseable closeable;
@@ -77,10 +75,10 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
     protected AbstractDataStore(final ActorSystem actorSystem, final ClusterWrapper cluster,
             final Configuration configuration, final DatastoreContextFactory datastoreContextFactory,
             final DatastoreSnapshot restoreFromSnapshot) {
-        Preconditions.checkNotNull(actorSystem, "actorSystem should not be null");
-        Preconditions.checkNotNull(cluster, "cluster should not be null");
-        Preconditions.checkNotNull(configuration, "configuration should not be null");
-        Preconditions.checkNotNull(datastoreContextFactory, "datastoreContextFactory should not be null");
+        requireNonNull(actorSystem, "actorSystem should not be null");
+        requireNonNull(cluster, "cluster should not be null");
+        requireNonNull(configuration, "configuration should not be null");
+        requireNonNull(datastoreContextFactory, "datastoreContextFactory should not be null");
 
         String shardManagerId = ShardManagerIdentifier.builder()
                 .type(datastoreContextFactory.getBaseDatastoreContext().getDataStoreName()).build().toString();
@@ -92,19 +90,19 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
 
         PrimaryShardInfoFutureCache primaryShardInfoCache = new PrimaryShardInfoFutureCache();
 
-        ShardManagerCreator creator = new ShardManagerCreator().cluster(cluster).configuration(configuration)
+        AbstractShardManagerCreator<?> creator = getShardManagerCreator().cluster(cluster).configuration(configuration)
                 .datastoreContextFactory(datastoreContextFactory)
                 .waitTillReadyCountDownLatch(waitTillReadyCountDownLatch)
                 .primaryShardInfoCache(primaryShardInfoCache)
                 .restoreFromSnapshot(restoreFromSnapshot)
                 .distributedDataStore(this);
 
-        actorContext = new ActorContext(actorSystem, createShardManager(actorSystem, creator, shardDispatcher,
+        actorUtils = new ActorUtils(actorSystem, createShardManager(actorSystem, creator, shardDispatcher,
                 shardManagerId), cluster, configuration, datastoreContextFactory.getBaseDatastoreContext(),
                 primaryShardInfoCache);
 
         final Props clientProps = DistributedDataStoreClientActor.props(cluster.getCurrentMemberName(),
-            datastoreContextFactory.getBaseDatastoreContext().getDataStoreName(), actorContext);
+            datastoreContextFactory.getBaseDatastoreContext().getDataStoreName(), actorUtils);
         final ActorRef clientActor = actorSystem.actorOf(clientProps);
         try {
             client = DistributedDataStoreClientActor.getDistributedDataStoreClient(clientActor, 30, TimeUnit.SECONDS);
@@ -118,7 +116,7 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
         identifier = client.getIdentifier();
         LOG.debug("Distributed data store client {} started", identifier);
 
-        this.waitTillReadyTimeInMillis = actorContext.getDatastoreContext().getShardLeaderElectionTimeout()
+        this.waitTillReadyTimeInMillis = actorUtils.getDatastoreContext().getShardLeaderElectionTimeout()
                 .duration().toMillis() * READY_WAIT_FACTOR;
 
         datastoreConfigMXBean = new DatastoreConfigurationMXBeanImpl(
@@ -127,29 +125,33 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
         datastoreConfigMXBean.registerMBean();
 
         datastoreInfoMXBean = new DatastoreInfoMXBeanImpl(datastoreContextFactory.getBaseDatastoreContext()
-                .getDataStoreMXBeanType(), actorContext);
+                .getDataStoreMXBeanType(), actorUtils);
         datastoreInfoMXBean.registerMBean();
     }
 
     @VisibleForTesting
-    protected AbstractDataStore(final ActorContext actorContext, final ClientIdentifier identifier) {
-        this.actorContext = Preconditions.checkNotNull(actorContext, "actorContext should not be null");
+    protected AbstractDataStore(final ActorUtils actorUtils, final ClientIdentifier identifier) {
+        this.actorUtils = requireNonNull(actorUtils, "actorContext should not be null");
         this.client = null;
-        this.identifier = Preconditions.checkNotNull(identifier);
-        this.waitTillReadyTimeInMillis = actorContext.getDatastoreContext().getShardLeaderElectionTimeout()
+        this.identifier = requireNonNull(identifier);
+        this.waitTillReadyTimeInMillis = actorUtils.getDatastoreContext().getShardLeaderElectionTimeout()
                 .duration().toMillis() * READY_WAIT_FACTOR;
     }
 
     @VisibleForTesting
-    protected AbstractDataStore(final ActorContext actorContext, final ClientIdentifier identifier,
+    protected AbstractDataStore(final ActorUtils actorUtils, final ClientIdentifier identifier,
                                 final DataStoreClient clientActor) {
-        this.actorContext = Preconditions.checkNotNull(actorContext, "actorContext should not be null");
+        this.actorUtils = requireNonNull(actorUtils, "actorContext should not be null");
         this.client = clientActor;
-        this.identifier = Preconditions.checkNotNull(identifier);
-        this.waitTillReadyTimeInMillis = actorContext.getDatastoreContext().getShardLeaderElectionTimeout()
+        this.identifier = requireNonNull(identifier);
+        this.waitTillReadyTimeInMillis = actorUtils.getDatastoreContext().getShardLeaderElectionTimeout()
                 .duration().toMillis() * READY_WAIT_FACTOR;
     }
 
+    protected AbstractShardManagerCreator<?> getShardManagerCreator() {
+        return new ShardManagerCreator();
+    }
+
     protected final DataStoreClient getClient() {
         return client;
     }
@@ -162,39 +164,17 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
         this.closeable = closeable;
     }
 
-    @SuppressWarnings("unchecked")
-    @Override
-    @Deprecated
-    public <L extends AsyncDataChangeListener<YangInstanceIdentifier, NormalizedNode<?, ?>>>
-                                              ListenerRegistration<L> registerChangeListener(
-        final YangInstanceIdentifier path, final L listener,
-        final AsyncDataBroker.DataChangeScope scope) {
-
-        Preconditions.checkNotNull(path, "path should not be null");
-        Preconditions.checkNotNull(listener, "listener should not be null");
-
-        LOG.debug("Registering listener: {} for path: {} scope: {}", listener, path, scope);
-
-        String shardName = actorContext.getShardStrategyFactory().getStrategy(path).findShard(path);
-
-        final DataChangeListenerRegistrationProxy listenerRegistrationProxy =
-                new DataChangeListenerRegistrationProxy(shardName, actorContext, listener);
-        listenerRegistrationProxy.init(path, scope);
-
-        return listenerRegistrationProxy;
-    }
-
     @Override
     public <L extends DOMDataTreeChangeListener> ListenerRegistration<L> registerTreeChangeListener(
             final YangInstanceIdentifier treeId, final L listener) {
-        Preconditions.checkNotNull(treeId, "treeId should not be null");
-        Preconditions.checkNotNull(listener, "listener should not be null");
+        requireNonNull(treeId, "treeId should not be null");
+        requireNonNull(listener, "listener should not be null");
 
-        final String shardName = actorContext.getShardStrategyFactory().getStrategy(treeId).findShard(treeId);
+        final String shardName = actorUtils.getShardStrategyFactory().getStrategy(treeId).findShard(treeId);
         LOG.debug("Registering tree listener: {} for tree: {} shard: {}", listener, treeId, shardName);
 
         final DataTreeChangeListenerProxy<L> listenerRegistrationProxy =
-                new DataTreeChangeListenerProxy<>(actorContext, listener, treeId);
+                new DataTreeChangeListenerProxy<>(actorUtils, listener, treeId);
         listenerRegistrationProxy.init(shardName);
 
         return listenerRegistrationProxy;
@@ -204,30 +184,29 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
     @Override
     public <C extends DOMDataTreeCommitCohort> DOMDataTreeCommitCohortRegistration<C> registerCommitCohort(
             final DOMDataTreeIdentifier subtree, final C cohort) {
-        YangInstanceIdentifier treeId =
-                Preconditions.checkNotNull(subtree, "subtree should not be null").getRootIdentifier();
-        Preconditions.checkNotNull(cohort, "listener should not be null");
+        YangInstanceIdentifier treeId = requireNonNull(subtree, "subtree should not be null").getRootIdentifier();
+        requireNonNull(cohort, "listener should not be null");
 
 
-        final String shardName = actorContext.getShardStrategyFactory().getStrategy(treeId).findShard(treeId);
+        final String shardName = actorUtils.getShardStrategyFactory().getStrategy(treeId).findShard(treeId);
         LOG.debug("Registering cohort: {} for tree: {} shard: {}", cohort, treeId, shardName);
 
         DataTreeCohortRegistrationProxy<C> cohortProxy =
-                new DataTreeCohortRegistrationProxy<>(actorContext, subtree, cohort);
+                new DataTreeCohortRegistrationProxy<>(actorUtils, subtree, cohort);
         cohortProxy.init(shardName);
         return cohortProxy;
     }
 
     @Override
     public void onGlobalContextUpdated(final SchemaContext schemaContext) {
-        actorContext.setSchemaContext(schemaContext);
+        actorUtils.setSchemaContext(schemaContext);
     }
 
     @Override
     public void onDatastoreContextUpdated(final DatastoreContextFactory contextFactory) {
-        LOG.info("DatastoreContext updated for data store {}", actorContext.getDataStoreName());
+        LOG.info("DatastoreContext updated for data store {}", actorUtils.getDataStoreName());
 
-        actorContext.setDatastoreContext(contextFactory);
+        actorUtils.setDatastoreContext(contextFactory);
         datastoreConfigMXBean.setContext(contextFactory.getBaseDatastoreContext());
     }
 
@@ -251,7 +230,7 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
             }
         }
 
-        actorContext.shutdown();
+        actorUtils.shutdown();
 
         if (client != null) {
             client.close();
@@ -259,8 +238,8 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
     }
 
     @Override
-    public ActorContext getActorContext() {
-        return actorContext;
+    public ActorUtils getActorUtils() {
+        return actorUtils;
     }
 
     public void waitTillReady() {
@@ -279,8 +258,9 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
     }
 
     @SuppressWarnings("checkstyle:IllegalCatch")
-    private static ActorRef createShardManager(final ActorSystem actorSystem, final ShardManagerCreator creator,
-            final String shardDispatcher, final String shardManagerId) {
+    private static ActorRef createShardManager(final ActorSystem actorSystem,
+            final AbstractShardManagerCreator<?> creator, final String shardDispatcher,
+            final String shardManagerId) {
         Exception lastException = null;
 
         for (int i = 0; i < 100; i++) {
@@ -308,16 +288,16 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
             final YangInstanceIdentifier insideShard,
             final org.opendaylight.mdsal.dom.api.DOMDataTreeChangeListener delegate) {
 
-        Preconditions.checkNotNull(shardLookup, "shardLookup should not be null");
-        Preconditions.checkNotNull(insideShard, "insideShard should not be null");
-        Preconditions.checkNotNull(delegate, "delegate should not be null");
+        requireNonNull(shardLookup, "shardLookup should not be null");
+        requireNonNull(insideShard, "insideShard should not be null");
+        requireNonNull(delegate, "delegate should not be null");
 
-        final String shardName = actorContext.getShardStrategyFactory().getStrategy(shardLookup).findShard(shardLookup);
+        final String shardName = actorUtils.getShardStrategyFactory().getStrategy(shardLookup).findShard(shardLookup);
         LOG.debug("Registering tree listener: {} for tree: {} shard: {}, path inside shard: {}",
                 delegate,shardLookup, shardName, insideShard);
 
         final DataTreeChangeListenerProxy<DOMDataTreeChangeListener> listenerRegistrationProxy =
-                new DataTreeChangeListenerProxy<>(actorContext,
+                new DataTreeChangeListenerProxy<>(actorUtils,
                         // wrap this in the ClusteredDOMDataTreeChangeLister interface
                         // since we always want clustered registration
                         (ClusteredDOMDataTreeChangeListener) delegate::onDataTreeChanged, insideShard);
@@ -330,12 +310,12 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
     public <L extends DOMDataTreeChangeListener> ListenerRegistration<L> registerShardConfigListener(
             final YangInstanceIdentifier internalPath,
             final DOMDataTreeChangeListener delegate) {
-        Preconditions.checkNotNull(delegate, "delegate should not be null");
+        requireNonNull(delegate, "delegate should not be null");
 
         LOG.debug("Registering a listener for the configuration shard: {}", internalPath);
 
         final DataTreeChangeListenerProxy<DOMDataTreeChangeListener> proxy =
-                new DataTreeChangeListenerProxy<>(actorContext, delegate, internalPath);
+                new DataTreeChangeListenerProxy<>(actorUtils, delegate, internalPath);
         proxy.init(ClusterUtils.PREFIX_CONFIG_SHARD_ID);
 
         return (ListenerRegistration<L>) proxy;