NormalizedNodeAggregator should also report empty
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / AbstractDataStore.java
index d6958a70d72030ce20b01b8ba30e39149535ab06..cf60b089d832a8b0c2bf6a9f1c1d76443c119204 100644 (file)
@@ -17,9 +17,13 @@ import akka.actor.Props;
 import com.google.common.annotations.Beta;
 import com.google.common.annotations.VisibleForTesting;
 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 edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.util.List;
 import java.util.Set;
-import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import org.opendaylight.controller.cluster.access.concepts.ClientIdentifier;
@@ -28,13 +32,10 @@ import org.opendaylight.controller.cluster.databroker.actors.dds.DataStoreClient
 import org.opendaylight.controller.cluster.databroker.actors.dds.DistributedDataStoreClientActor;
 import org.opendaylight.controller.cluster.datastore.config.Configuration;
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardManagerIdentifier;
-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.ActorUtils;
-import org.opendaylight.controller.cluster.datastore.utils.ClusterUtils;
 import org.opendaylight.controller.cluster.datastore.utils.PrimaryShardInfoFutureCache;
 import org.opendaylight.mdsal.dom.api.ClusteredDOMDataTreeChangeListener;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeChangeListener;
@@ -44,7 +45,9 @@ 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.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;
@@ -60,7 +63,7 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
 
     private static final Logger LOG = LoggerFactory.getLogger(AbstractDataStore.class);
 
-    private final CountDownLatch waitTillReadyCountDownLatch = new CountDownLatch(1);
+    private final SettableFuture<Empty> readinessFuture = SettableFuture.create();
     private final ClientIdentifier identifier;
     private final DataStoreClient client;
     private final ActorUtils actorUtils;
@@ -70,6 +73,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) {
@@ -90,7 +94,7 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
 
         AbstractShardManagerCreator<?> creator = getShardManagerCreator().cluster(cluster).configuration(configuration)
                 .datastoreContextFactory(datastoreContextFactory)
-                .waitTillReadyCountDownLatch(waitTillReadyCountDownLatch)
+                .readinessFuture(readinessFuture)
                 .primaryShardInfoCache(primaryShardInfoCache)
                 .restoreFromSnapshot(restoreFromSnapshot)
                 .distributedDataStore(this);
@@ -127,7 +131,7 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
     @VisibleForTesting
     protected AbstractDataStore(final ActorUtils actorUtils, final ClientIdentifier identifier) {
         this.actorUtils = requireNonNull(actorUtils, "actorContext should not be null");
-        this.client = null;
+        client = null;
         this.identifier = requireNonNull(identifier);
     }
 
@@ -135,7 +139,7 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
     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);
     }
 
@@ -243,11 +247,12 @@ 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() {
         LOG.info("Beginning to wait for data store to become ready : {}", identifier);
 
@@ -266,23 +271,36 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
     }
 
     @Beta
+    @Deprecated
     public boolean awaitReadiness() throws InterruptedException {
         return awaitReadiness(initialSettleTime());
     }
 
     @Beta
+    @Deprecated
     public boolean awaitReadiness(final Duration toWait) throws InterruptedException {
-        if (toWait.isFinite()) {
-            return waitTillReadyCountDownLatch.await(toWait.toNanos(), TimeUnit.NANOSECONDS);
+        try {
+            if (toWait.isFinite()) {
+                try {
+                    readinessFuture.get(toWait.toNanos(), TimeUnit.NANOSECONDS);
+                } catch (TimeoutException e) {
+                    LOG.debug("Timed out waiting for shards to settle", e);
+                    return false;
+                }
+            } else {
+                readinessFuture.get();
+            }
+        } catch (ExecutionException e) {
+            LOG.warn("Unexpected readiness failure, assuming convergence", e);
         }
 
-        waitTillReadyCountDownLatch.await();
         return true;
     }
 
     @Beta
+    @Deprecated
     public void awaitReadiness(final long timeout, final TimeUnit unit) throws InterruptedException, TimeoutException {
-        if (!waitTillReadyCountDownLatch.await(timeout, unit)) {
+        if (!awaitReadiness(Duration.create(timeout, unit))) {
             throw new TimeoutException("Shard leaders failed to settle");
         }
     }
@@ -307,9 +325,18 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
         throw new IllegalStateException("Failed to create Shard Manager", lastException);
     }
 
+    /**
+     * Future which completes when all shards settle for the first time.
+     *
+     * @return A Listenable future.
+     */
+    public final ListenableFuture<?> initialSettleFuture() {
+        return readinessFuture;
+    }
+
     @VisibleForTesting
-    public CountDownLatch getWaitTillReadyCountDownLatch() {
-        return waitTillReadyCountDownLatch;
+    SettableFuture<Empty> readinessFuture() {
+        return readinessFuture;
     }
 
     @Override
@@ -326,31 +353,25 @@ public abstract class AbstractDataStore implements DistributedDataStoreInterface
         LOG.debug("Registering tree listener: {} for tree: {} shard: {}, path inside shard: {}",
                 delegate,shardLookup, shardName, insideShard);
 
+        // wrap this in the ClusteredDOMDataTreeChangeLister interface
+        // since we always want clustered registration
         final DataTreeChangeListenerProxy<DOMDataTreeChangeListener> listenerRegistrationProxy =
-                new DataTreeChangeListenerProxy<>(actorUtils,
-                        // wrap this in the ClusteredDOMDataTreeChangeLister interface
-                        // since we always want clustered registration
-                        (ClusteredDOMDataTreeChangeListener) delegate::onDataTreeChanged, insideShard);
+                new DataTreeChangeListenerProxy<>(actorUtils, new ClusteredDOMDataTreeChangeListener() {
+                    @Override
+                    public void onDataTreeChanged(final List<DataTreeCandidate> changes) {
+                        delegate.onDataTreeChanged(changes);
+                    }
+
+                    @Override
+                    public void onInitialData() {
+                        delegate.onInitialData();
+                    }
+                }, insideShard);
         listenerRegistrationProxy.init(shardName);
 
         return (ListenerRegistration<L>) listenerRegistrationProxy;
     }
 
-    @Override
-    @SuppressWarnings("unchecked")
-    public <L extends DOMDataTreeChangeListener> ListenerRegistration<L> registerShardConfigListener(
-            final YangInstanceIdentifier internalPath, final DOMDataTreeChangeListener delegate) {
-        requireNonNull(delegate, "delegate should not be null");
-
-        LOG.debug("Registering a listener for the configuration shard: {}", internalPath);
-
-        final DataTreeChangeListenerProxy<DOMDataTreeChangeListener> proxy =
-                new DataTreeChangeListenerProxy<>(actorUtils, delegate, internalPath);
-        proxy.init(ClusterUtils.PREFIX_CONFIG_SHARD_ID);
-
-        return (ListenerRegistration<L>) proxy;
-    }
-
     private Duration initialSettleTime() {
         final DatastoreContext context = actorUtils.getDatastoreContext();
         final int multiplier = context.getInitialSettleTimeoutMultiplier();