Improve segmented journal actor metrics
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / AbstractDataStore.java
index 385297809236e3e5c2392c205d1ebd8e7b9b4d34..05edad2cf1f72867110f5a3798a6570365408682 100644 (file)
@@ -7,7 +7,6 @@
  */
 package org.opendaylight.controller.cluster.datastore;
 
-import static com.google.common.base.Preconditions.checkArgument;
 import static java.util.Objects.requireNonNull;
 
 import akka.actor.ActorRef;
@@ -20,10 +19,12 @@ import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
 import com.google.common.util.concurrent.Uninterruptibles;
-import java.util.Set;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.util.List;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import org.eclipse.jdt.annotation.NonNull;
 import org.opendaylight.controller.cluster.access.concepts.ClientIdentifier;
 import org.opendaylight.controller.cluster.common.actor.Dispatchers;
 import org.opendaylight.controller.cluster.databroker.actors.dds.DataStoreClient;
@@ -35,17 +36,16 @@ import org.opendaylight.controller.cluster.datastore.shardmanager.AbstractShardM
 import org.opendaylight.controller.cluster.datastore.shardmanager.ShardManagerCreator;
 import org.opendaylight.controller.cluster.datastore.utils.ActorUtils;
 import org.opendaylight.controller.cluster.datastore.utils.PrimaryShardInfoFutureCache;
-import org.opendaylight.mdsal.dom.api.ClusteredDOMDataTreeChangeListener;
+import org.opendaylight.mdsal.dom.api.DOMDataBroker.CommitCohortExtension;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeChangeListener;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeCommitCohort;
-import org.opendaylight.mdsal.dom.api.DOMDataTreeCommitCohortRegistration;
-import org.opendaylight.mdsal.dom.api.DOMDataTreeCommitCohortRegistry;
 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.concepts.Registration;
+import org.opendaylight.yangtools.yang.common.Empty;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.tree.api.DataTreeCandidate;
 import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext;
-import org.opendaylight.yangtools.yang.model.api.EffectiveModelContextListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.duration.Duration;
@@ -53,13 +53,12 @@ import scala.concurrent.duration.Duration;
 /**
  * Base implementation of a distributed DOMStore.
  */
-public abstract class AbstractDataStore implements DistributedDataStoreInterface, EffectiveModelContextListener,
-        DatastoreContextPropertiesUpdater.Listener, DOMStoreTreeChangePublisher,
-        DOMDataTreeCommitCohortRegistry, AutoCloseable {
-
+public abstract class AbstractDataStore implements DistributedDataStoreInterface,
+        DatastoreContextPropertiesUpdater.Listener, DOMStoreTreeChangePublisher, CommitCohortExtension,
+        AutoCloseable {
     private static final Logger LOG = LoggerFactory.getLogger(AbstractDataStore.class);
 
-    private final SettableFuture<Void> readinessFuture = SettableFuture.create();
+    private final SettableFuture<Empty> readinessFuture = SettableFuture.create();
     private final ClientIdentifier identifier;
     private final DataStoreClient client;
     private final ActorUtils actorUtils;
@@ -69,6 +68,7 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
     private DatastoreInfoMXBeanImpl datastoreInfoMXBean;
 
     @SuppressWarnings("checkstyle:IllegalCatch")
+    @SuppressFBWarnings(value = "MC_OVERRIDABLE_METHOD_CALL_IN_CONSTRUCTOR", justification = "Testing overrides")
     protected AbstractDataStore(final ActorSystem actorSystem, final ClusterWrapper cluster,
             final Configuration configuration, final DatastoreContextFactory datastoreContextFactory,
             final DatastoreSnapshot restoreFromSnapshot) {
@@ -107,7 +107,7 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
             LOG.error("Failed to get actor for {}", clientProps, e);
             clientActor.tell(PoisonPill.getInstance(), ActorRef.noSender());
             Throwables.throwIfUnchecked(e);
-            throw new RuntimeException(e);
+            throw new IllegalStateException(e);
         }
 
         identifier = client.getIdentifier();
@@ -123,21 +123,15 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
         datastoreInfoMXBean.registerMBean();
     }
 
-    @VisibleForTesting
-    protected AbstractDataStore(final ActorUtils actorUtils, final ClientIdentifier identifier) {
-        this.actorUtils = requireNonNull(actorUtils, "actorContext should not be null");
-        this.client = null;
-        this.identifier = requireNonNull(identifier);
-    }
-
     @VisibleForTesting
     protected AbstractDataStore(final ActorUtils actorUtils, final ClientIdentifier identifier,
                                 final DataStoreClient clientActor) {
         this.actorUtils = requireNonNull(actorUtils, "actorContext should not be null");
-        this.client = clientActor;
+        client = clientActor;
         this.identifier = requireNonNull(identifier);
     }
 
+    @VisibleForTesting
     protected AbstractShardManagerCreator<?> getShardManagerCreator() {
         return new ShardManagerCreator();
     }
@@ -146,17 +140,18 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
         return client;
     }
 
-    final ClientIdentifier getIdentifier() {
-        return identifier;
-    }
-
     public void setCloseable(final AutoCloseable closeable) {
         this.closeable = closeable;
     }
 
     @Override
-    public <L extends DOMDataTreeChangeListener> ListenerRegistration<L> registerTreeChangeListener(
-            final YangInstanceIdentifier treeId, final L listener) {
+    public final Registration registerTreeChangeListener(final YangInstanceIdentifier treeId,
+            final DOMDataTreeChangeListener listener) {
+        return registerTreeChangeListener(treeId, listener, true);
+    }
+
+    private @NonNull Registration registerTreeChangeListener(final YangInstanceIdentifier treeId,
+            final DOMDataTreeChangeListener listener, final boolean clustered) {
         requireNonNull(treeId, "treeId should not be null");
         requireNonNull(listener, "listener should not be null");
 
@@ -167,47 +162,51 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
         if (treeId.isEmpty()) {
             // User is targeting root of the datastore. If there is more than one shard, we have to register with them
             // all and perform data composition.
-            final Set<String> shardNames = actorUtils.getConfiguration().getAllShardNames();
+            final var shardNames = actorUtils.getConfiguration().getAllShardNames();
             if (shardNames.size() > 1) {
-                checkArgument(listener instanceof ClusteredDOMDataTreeChangeListener,
-                    "Cannot listen on root without non-clustered listener %s", listener);
+                if (!clustered) {
+                    throw new IllegalArgumentException(
+                        "Cannot listen on root without non-clustered listener " + listener);
+                }
                 return new RootDataTreeChangeListenerProxy<>(actorUtils, listener, shardNames);
             }
         }
 
-        final String shardName = actorUtils.getShardStrategyFactory().getStrategy(treeId).findShard(treeId);
+        final var shardName = actorUtils.getShardStrategyFactory().getStrategy(treeId).findShard(treeId);
         LOG.debug("Registering tree listener: {} for tree: {} shard: {}", listener, treeId, shardName);
 
-        final DataTreeChangeListenerProxy<L> listenerRegistrationProxy =
-                new DataTreeChangeListenerProxy<>(actorUtils, listener, treeId);
-        listenerRegistrationProxy.init(shardName);
+        return DataTreeChangeListenerProxy.of(actorUtils, listener, treeId, clustered, shardName);
+    }
 
-        return listenerRegistrationProxy;
+    @Override
+    @Deprecated(since = "9.0.0", forRemoval = true)
+    public final Registration registerLegacyTreeChangeListener(final YangInstanceIdentifier treeId,
+            final DOMDataTreeChangeListener listener) {
+        return registerTreeChangeListener(treeId, listener, false);
     }
 
     @Override
-    public <C extends DOMDataTreeCommitCohort> DOMDataTreeCommitCohortRegistration<C> registerCommitCohort(
-            final DOMDataTreeIdentifier subtree, final C cohort) {
-        YangInstanceIdentifier treeId = requireNonNull(subtree, "subtree should not be null").getRootIdentifier();
+    // Non-final for testing
+    public Registration registerCommitCohort(final DOMDataTreeIdentifier subtree,
+            final DOMDataTreeCommitCohort cohort) {
+        YangInstanceIdentifier treeId = requireNonNull(subtree, "subtree should not be null").path();
         requireNonNull(cohort, "listener should not be null");
 
 
         final String shardName = actorUtils.getShardStrategyFactory().getStrategy(treeId).findShard(treeId);
         LOG.debug("Registering cohort: {} for tree: {} shard: {}", cohort, treeId, shardName);
 
-        DataTreeCohortRegistrationProxy<C> cohortProxy =
-                new DataTreeCohortRegistrationProxy<>(actorUtils, subtree, cohort);
+        final var cohortProxy = new DataTreeCohortRegistrationProxy<>(actorUtils, subtree, cohort);
         cohortProxy.init(shardName);
         return cohortProxy;
     }
 
-    @Override
     public void onModelContextUpdated(final EffectiveModelContext newModelContext) {
         actorUtils.setSchemaContext(newModelContext);
     }
 
     @Override
-    public void onDatastoreContextUpdated(final DatastoreContextFactory contextFactory) {
+    public final void onDatastoreContextUpdated(final DatastoreContextFactory contextFactory) {
         LOG.info("DatastoreContext updated for data store {}", actorUtils.getDataStoreName());
 
         actorUtils.setDatastoreContext(contextFactory);
@@ -216,7 +215,7 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
 
     @Override
     @SuppressWarnings("checkstyle:IllegalCatch")
-    public void close() {
+    public final void close() {
         LOG.info("Closing data store {}", identifier);
 
         if (datastoreConfigMXBean != null) {
@@ -242,13 +241,13 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
     }
 
     @Override
-    public ActorUtils getActorUtils() {
+    public final ActorUtils getActorUtils() {
         return actorUtils;
     }
 
     // TODO: consider removing this in favor of awaitReadiness()
     @Deprecated
-    public void waitTillReady() {
+    public final void waitTillReady() {
         LOG.info("Beginning to wait for data store to become ready : {}", identifier);
 
         final Duration toWait = initialSettleTime();
@@ -267,13 +266,13 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
 
     @Beta
     @Deprecated
-    public boolean awaitReadiness() throws InterruptedException {
+    public final boolean awaitReadiness() throws InterruptedException {
         return awaitReadiness(initialSettleTime());
     }
 
     @Beta
     @Deprecated
-    public boolean awaitReadiness(final Duration toWait) throws InterruptedException {
+    public final boolean awaitReadiness(final Duration toWait) throws InterruptedException {
         try {
             if (toWait.isFinite()) {
                 try {
@@ -294,7 +293,8 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
 
     @Beta
     @Deprecated
-    public void awaitReadiness(final long timeout, final TimeUnit unit) throws InterruptedException, TimeoutException {
+    public final void awaitReadiness(final long timeout, final TimeUnit unit)
+            throws InterruptedException, TimeoutException {
         if (!awaitReadiness(Duration.create(timeout, unit))) {
             throw new TimeoutException("Shard leaders failed to settle");
         }
@@ -330,32 +330,32 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
     }
 
     @VisibleForTesting
-    SettableFuture<Void> readinessFuture() {
+    public final SettableFuture<Empty> readinessFuture() {
         return readinessFuture;
     }
 
     @Override
-    @SuppressWarnings("unchecked")
-    public <L extends DOMDataTreeChangeListener> ListenerRegistration<L> registerProxyListener(
-            final YangInstanceIdentifier shardLookup, final YangInstanceIdentifier insideShard,
-            final DOMDataTreeChangeListener delegate) {
-
+    public final Registration registerProxyListener(final YangInstanceIdentifier shardLookup,
+            final YangInstanceIdentifier insideShard, final DOMDataTreeChangeListener delegate) {
         requireNonNull(shardLookup, "shardLookup should not be null");
         requireNonNull(insideShard, "insideShard should not be null");
         requireNonNull(delegate, "delegate should not be null");
 
-        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 var 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<>(actorUtils,
-                        // wrap this in the ClusteredDOMDataTreeChangeLister interface
-                        // since we always want clustered registration
-                        (ClusteredDOMDataTreeChangeListener) delegate::onDataTreeChanged, insideShard);
-        listenerRegistrationProxy.init(shardName);
+        return DataTreeChangeListenerProxy.of(actorUtils, new DOMDataTreeChangeListener() {
+            @Override
+            public void onDataTreeChanged(final List<DataTreeCandidate> changes) {
+                delegate.onDataTreeChanged(changes);
+            }
 
-        return (ListenerRegistration<L>) listenerRegistrationProxy;
+            @Override
+            public void onInitialData() {
+                delegate.onInitialData();
+            }
+        }, insideShard, true, shardName);
     }
 
     private Duration initialSettleTime() {