Migrate to use openCursor()
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / databroker / actors / dds / AbstractShardBackendResolver.java
index 0ce4df0f134f0a95191cd582e94f102397d8c238..eddbba6acdd92992bf4e145c8fb3f20596fb3000 100644 (file)
@@ -7,15 +7,20 @@
  */
 package org.opendaylight.controller.cluster.databroker.actors.dds;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
+
 import akka.actor.ActorRef;
 import akka.util.Timeout;
-import com.google.common.base.Preconditions;
 import com.google.common.primitives.UnsignedLong;
+import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 import javax.annotation.concurrent.GuardedBy;
@@ -32,7 +37,8 @@ import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderExc
 import org.opendaylight.controller.cluster.datastore.exceptions.NotInitializedException;
 import org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException;
 import org.opendaylight.controller.cluster.datastore.messages.PrimaryShardInfo;
-import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
+import org.opendaylight.controller.cluster.datastore.utils.ActorUtils;
+import org.opendaylight.yangtools.concepts.Registration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.Function1;
@@ -41,7 +47,7 @@ import scala.compat.java8.FutureConverters;
 /**
  * {@link BackendInfoResolver} implementation for static shard configuration based on ShardManager. Each string-named
  * shard is assigned a single cookie and this mapping is stored in a bidirectional map. Information about corresponding
- * shard leader is resolved via {@link ActorContext}. The product of resolution is {@link ShardBackendInfo}.
+ * shard leader is resolved via {@link ActorUtils}. The product of resolution is {@link ShardBackendInfo}.
  *
  * @author Robert Varga
  */
@@ -53,7 +59,7 @@ abstract class AbstractShardBackendResolver extends BackendInfoResolver<ShardBac
         private ShardBackendInfo result;
 
         ShardState(final CompletionStage<ShardBackendInfo> stage) {
-            this.stage = Preconditions.checkNotNull(stage);
+            this.stage = requireNonNull(stage);
             stage.whenComplete(this::onStageResolved);
         }
 
@@ -65,9 +71,9 @@ abstract class AbstractShardBackendResolver extends BackendInfoResolver<ShardBac
             return result;
         }
 
-        private synchronized void onStageResolved(final ShardBackendInfo result, final Throwable failure) {
+        private synchronized void onStageResolved(final ShardBackendInfo info, final Throwable failure) {
             if (failure == null) {
-                this.result = Preconditions.checkNotNull(result);
+                this.result = requireNonNull(info);
             } else {
                 LOG.warn("Failed to resolve shard", failure);
             }
@@ -84,24 +90,39 @@ abstract class AbstractShardBackendResolver extends BackendInfoResolver<ShardBac
 
     private final AtomicLong nextSessionId = new AtomicLong();
     private final Function1<ActorRef, ?> connectFunction;
-    private final ActorContext actorContext;
+    private final ActorUtils actorUtils;
+    private final Set<Consumer<Long>> staleBackendInfoCallbacks = ConcurrentHashMap.newKeySet();
 
     // FIXME: we really need just ActorContext.findPrimaryShardAsync()
-    AbstractShardBackendResolver(final ClientIdentifier clientId, final ActorContext actorContext) {
-        this.actorContext = Preconditions.checkNotNull(actorContext);
+    AbstractShardBackendResolver(final ClientIdentifier clientId, final ActorUtils actorUtils) {
+        this.actorUtils = requireNonNull(actorUtils);
         this.connectFunction = ExplicitAsk.toScala(t -> new ConnectClientRequest(clientId, t, ABIVersion.BORON,
             ABIVersion.current()));
     }
 
+    @Override
+    public Registration notifyWhenBackendInfoIsStale(final Consumer<Long> callback) {
+        staleBackendInfoCallbacks.add(callback);
+        return () -> staleBackendInfoCallbacks.remove(callback);
+    }
+
+    protected void notifyStaleBackendInfoCallbacks(Long cookie) {
+        staleBackendInfoCallbacks.forEach(callback -> callback.accept(cookie));
+    }
+
+    protected ActorUtils actorUtils() {
+        return actorUtils;
+    }
+
     protected final void flushCache(final String shardName) {
-        actorContext.getPrimaryShardInfoCache().remove(shardName);
+        actorUtils.getPrimaryShardInfoCache().remove(shardName);
     }
 
     protected final ShardState resolveBackendInfo(final String shardName, final long cookie) {
         LOG.debug("Resolving cookie {} to shard {}", cookie, shardName);
 
         final CompletableFuture<ShardBackendInfo> future = new CompletableFuture<>();
-        FutureConverters.toJava(actorContext.findPrimaryShardAsync(shardName)).whenComplete((info, failure) -> {
+        FutureConverters.toJava(actorUtils.findPrimaryShardAsync(shardName)).whenComplete((info, failure) -> {
             if (failure == null) {
                 connectShard(shardName, cookie, info, future);
                 return;
@@ -127,7 +148,7 @@ abstract class AbstractShardBackendResolver extends BackendInfoResolver<ShardBac
 
     private static TimeoutException wrap(final String message, final Throwable cause) {
         final TimeoutException ret = new TimeoutException(message);
-        ret.initCause(Preconditions.checkNotNull(cause));
+        ret.initCause(requireNonNull(cause));
         return ret;
     }
 
@@ -136,9 +157,7 @@ abstract class AbstractShardBackendResolver extends BackendInfoResolver<ShardBac
         LOG.debug("Shard {} resolved to {}, attempting to connect", shardName, info);
 
         FutureConverters.toJava(ExplicitAsk.ask(info.getPrimaryShardActor(), connectFunction, CONNECT_TIMEOUT))
-            .whenComplete((response, failure) -> {
-                onConnectResponse(shardName, cookie, future, response, failure);
-            });
+            .whenComplete((response, failure) -> onConnectResponse(shardName, cookie, future, response, failure));
     }
 
     private void onConnectResponse(final String shardName, final long cookie,
@@ -158,8 +177,7 @@ abstract class AbstractShardBackendResolver extends BackendInfoResolver<ShardBac
         }
 
         LOG.debug("Resolved backend information to {}", response);
-        Preconditions.checkArgument(response instanceof ConnectClientSuccess, "Unhandled response %s",
-            response);
+        checkArgument(response instanceof ConnectClientSuccess, "Unhandled response %s", response);
         final ConnectClientSuccess success = (ConnectClientSuccess) response;
         future.complete(new ShardBackendInfo(success.getBackend(), nextSessionId.getAndIncrement(),
             success.getVersion(), shardName, UnsignedLong.fromLongBits(cookie), success.getDataTree(),