Change declared exception
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / utils / ActorContext.java
index b6250fc1cccbfbde152a8af7e1e2f4b9b3c6f7cd..68ac6412e8ad1346f46686e7d5103be5d0824b76 100644 (file)
@@ -14,27 +14,24 @@ import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
 import akka.actor.ActorSystem;
 import akka.actor.Address;
-import akka.actor.PoisonPill;
-import akka.dispatch.Futures;
 import akka.dispatch.Mapper;
 import akka.dispatch.OnComplete;
 import akka.pattern.AskTimeoutException;
+import akka.pattern.Patterns;
 import akka.util.Timeout;
-import com.codahale.metrics.JmxReporter;
 import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.util.concurrent.RateLimiter;
 import java.util.concurrent.TimeUnit;
-import org.opendaylight.controller.cluster.common.actor.CommonConfig;
+import java.util.function.Function;
+import org.opendaylight.controller.cluster.access.concepts.MemberName;
 import org.opendaylight.controller.cluster.datastore.ClusterWrapper;
-import org.opendaylight.controller.cluster.datastore.Configuration;
+import org.opendaylight.controller.cluster.datastore.DataStoreVersions;
 import org.opendaylight.controller.cluster.datastore.DatastoreContext;
+import org.opendaylight.controller.cluster.datastore.DatastoreContextFactory;
+import org.opendaylight.controller.cluster.datastore.config.Configuration;
 import org.opendaylight.controller.cluster.datastore.exceptions.LocalShardNotFoundException;
 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
 import org.opendaylight.controller.cluster.datastore.exceptions.NotInitializedException;
@@ -43,10 +40,16 @@ import org.opendaylight.controller.cluster.datastore.exceptions.TimeoutException
 import org.opendaylight.controller.cluster.datastore.exceptions.UnknownMessageException;
 import org.opendaylight.controller.cluster.datastore.messages.FindLocalShard;
 import org.opendaylight.controller.cluster.datastore.messages.FindPrimary;
+import org.opendaylight.controller.cluster.datastore.messages.LocalPrimaryShardFound;
 import org.opendaylight.controller.cluster.datastore.messages.LocalShardFound;
 import org.opendaylight.controller.cluster.datastore.messages.LocalShardNotFound;
-import org.opendaylight.controller.cluster.datastore.messages.PrimaryFound;
+import org.opendaylight.controller.cluster.datastore.messages.PrimaryShardInfo;
+import org.opendaylight.controller.cluster.datastore.messages.RemotePrimaryShardFound;
 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
+import org.opendaylight.controller.cluster.datastore.shardstrategy.ShardStrategyFactory;
+import org.opendaylight.controller.cluster.raft.client.messages.Shutdown;
+import org.opendaylight.controller.cluster.reporting.MetricsReporter;
+import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -64,10 +67,8 @@ import scala.concurrent.duration.FiniteDuration;
  */
 public class ActorContext {
     private static final Logger LOG = LoggerFactory.getLogger(ActorContext.class);
-    private static final String UNKNOWN_DATA_STORE_TYPE = "unknown";
     private static final String DISTRIBUTED_DATA_STORE_METRIC_REGISTRY = "distributed-data-store";
     private static final String METRIC_RATE = "rate";
-    private static final String DOMAIN = "org.opendaylight.controller.cluster.datastore";
     private static final Mapper<Throwable, Throwable> FIND_PRIMARY_FAILURE_TRANSFORMER =
                                                               new Mapper<Throwable, Throwable>() {
         @Override
@@ -83,7 +84,8 @@ public class ActorContext {
             return actualFailure;
         }
     };
-    public static final String MAILBOX = "bounded-mailbox";
+    public static final String BOUNDED_MAILBOX = "bounded-mailbox";
+    public static final String COMMIT = "commit";
 
     private final ActorSystem actorSystem;
     private final ActorRef shardManager;
@@ -93,33 +95,35 @@ public class ActorContext {
     private FiniteDuration operationDuration;
     private Timeout operationTimeout;
     private final String selfAddressHostPort;
-    private RateLimiter txRateLimiter;
-    private final MetricRegistry metricRegistry = new MetricRegistry();
-    private final JmxReporter jmxReporter = JmxReporter.forRegistry(metricRegistry).inDomain(DOMAIN).build();
-    private final int transactionOutstandingOperationLimit;
+    private TransactionRateLimiter txRateLimiter;
     private Timeout transactionCommitOperationTimeout;
     private Timeout shardInitializationTimeout;
     private final Dispatchers dispatchers;
-    private Cache<String, Future<ActorSelection>> primaryShardActorSelectionCache;
 
     private volatile SchemaContext schemaContext;
     private volatile boolean updated;
+    private final MetricRegistry metricRegistry = MetricsReporter.getInstance(DatastoreContext.METRICS_DOMAIN).getMetricsRegistry();
+
+    private final PrimaryShardInfoFutureCache primaryShardInfoCache;
+    private final ShardStrategyFactory shardStrategyFactory;
 
     public ActorContext(ActorSystem actorSystem, ActorRef shardManager,
             ClusterWrapper clusterWrapper, Configuration configuration) {
         this(actorSystem, shardManager, clusterWrapper, configuration,
-                DatastoreContext.newBuilder().build());
+                DatastoreContext.newBuilder().build(), new PrimaryShardInfoFutureCache());
     }
 
     public ActorContext(ActorSystem actorSystem, ActorRef shardManager,
             ClusterWrapper clusterWrapper, Configuration configuration,
-            DatastoreContext datastoreContext) {
+            DatastoreContext datastoreContext, PrimaryShardInfoFutureCache primaryShardInfoCache) {
         this.actorSystem = actorSystem;
         this.shardManager = shardManager;
         this.clusterWrapper = clusterWrapper;
         this.configuration = configuration;
         this.datastoreContext = datastoreContext;
         this.dispatchers = new Dispatchers(actorSystem.dispatchers());
+        this.primaryShardInfoCache = primaryShardInfoCache;
+        this.shardStrategyFactory = new ShardStrategyFactory(configuration);
 
         setCachedProperties();
 
@@ -130,25 +134,18 @@ public class ActorContext {
             selfAddressHostPort = null;
         }
 
-        transactionOutstandingOperationLimit = new CommonConfig(this.getActorSystem().settings().config()).getMailBoxCapacity();
-        jmxReporter.start();
-
     }
 
     private void setCachedProperties() {
-        txRateLimiter = RateLimiter.create(datastoreContext.getTransactionCreationInitialRateLimit());
+        txRateLimiter = new TransactionRateLimiter(this);
 
-        operationDuration = Duration.create(datastoreContext.getOperationTimeoutInSeconds(), TimeUnit.SECONDS);
+        operationDuration = Duration.create(datastoreContext.getOperationTimeoutInMillis(), TimeUnit.MILLISECONDS);
         operationTimeout = new Timeout(operationDuration);
 
         transactionCommitOperationTimeout =  new Timeout(Duration.create(
                 datastoreContext.getShardTransactionCommitTimeoutInSeconds(), TimeUnit.SECONDS));
 
         shardInitializationTimeout = new Timeout(datastoreContext.getShardInitializationTimeout().duration().$times(2));
-
-        primaryShardActorSelectionCache = CacheBuilder.newBuilder()
-                .expireAfterWrite(datastoreContext.getShardLeaderElectionTimeout().duration().toMillis(), TimeUnit.MILLISECONDS)
-                .build();
     }
 
     public DatastoreContext getDatastoreContext() {
@@ -179,8 +176,8 @@ public class ActorContext {
         }
     }
 
-    public void setDatastoreContext(DatastoreContext context) {
-        this.datastoreContext = context;
+    public void setDatastoreContext(DatastoreContextFactory contextFactory) {
+        this.datastoreContext = contextFactory.getBaseDatastoreContext();
         setCachedProperties();
 
         // We write the 'updated' volatile to trigger a write memory barrier so that the writes above
@@ -193,7 +190,7 @@ public class ActorContext {
         updated = true;
 
         if(shardManager != null) {
-            shardManager.tell(context, ActorRef.noSender());
+            shardManager.tell(contextFactory, ActorRef.noSender());
         }
     }
 
@@ -201,24 +198,26 @@ public class ActorContext {
         return schemaContext;
     }
 
-    public Future<ActorSelection> findPrimaryShardAsync(final String shardName) {
-        Future<ActorSelection> ret = primaryShardActorSelectionCache.getIfPresent(shardName);
+    public Future<PrimaryShardInfo> findPrimaryShardAsync(final String shardName) {
+        Future<PrimaryShardInfo> ret = primaryShardInfoCache.getIfPresent(shardName);
         if(ret != null){
             return ret;
         }
         Future<Object> future = executeOperationAsync(shardManager,
                 new FindPrimary(shardName, true), shardInitializationTimeout);
 
-        return future.transform(new Mapper<Object, ActorSelection>() {
+        return future.transform(new Mapper<Object, PrimaryShardInfo>() {
             @Override
-            public ActorSelection checkedApply(Object response) throws Exception {
-                if(response instanceof PrimaryFound) {
-                    PrimaryFound found = (PrimaryFound)response;
-
-                    LOG.debug("Primary found {}", found.getPrimaryPath());
-                    ActorSelection actorSelection = actorSystem.actorSelection(found.getPrimaryPath());
-                    primaryShardActorSelectionCache.put(shardName, Futures.successful(actorSelection));
-                    return actorSelection;
+            public PrimaryShardInfo checkedApply(Object response) throws UnknownMessageException {
+                if(response instanceof RemotePrimaryShardFound) {
+                    LOG.debug("findPrimaryShardAsync received: {}", response);
+                    RemotePrimaryShardFound found = (RemotePrimaryShardFound)response;
+                    return onPrimaryShardFound(shardName, found.getPrimaryPath(), found.getPrimaryVersion(), null);
+                } else if(response instanceof LocalPrimaryShardFound) {
+                    LOG.debug("findPrimaryShardAsync received: {}", response);
+                    LocalPrimaryShardFound found = (LocalPrimaryShardFound)response;
+                    return onPrimaryShardFound(shardName, found.getPrimaryPath(), DataStoreVersions.CURRENT_VERSION,
+                            found.getLocalShardDataTree());
                 } else if(response instanceof NotInitializedException) {
                     throw (NotInitializedException)response;
                 } else if(response instanceof PrimaryNotFoundException) {
@@ -233,6 +232,15 @@ public class ActorContext {
         }, FIND_PRIMARY_FAILURE_TRANSFORMER, getClientDispatcher());
     }
 
+    private PrimaryShardInfo onPrimaryShardFound(String shardName, String primaryActorPath,
+            short primaryVersion, DataTree localShardDataTree) {
+        ActorSelection actorSelection = actorSystem.actorSelection(primaryActorPath);
+        PrimaryShardInfo info = localShardDataTree == null ? new PrimaryShardInfo(actorSelection, primaryVersion) :
+            new PrimaryShardInfo(actorSelection, primaryVersion, localShardDataTree);
+        primaryShardInfoCache.putSuccessful(shardName, info);
+        return info;
+    }
+
     /**
      * Finds a local shard given its shard name and return it's ActorRef
      *
@@ -372,15 +380,19 @@ public class ActorContext {
     }
 
     public void shutdown() {
-        shardManager.tell(PoisonPill.getInstance(), null);
-        actorSystem.shutdown();
+        FiniteDuration duration = datastoreContext.getShardRaftConfig().getElectionTimeOutInterval().$times(3);
+        try {
+            Await.ready(Patterns.gracefulStop(shardManager, duration, Shutdown.INSTANCE), duration);
+        } catch(Exception e) {
+            LOG.warn("ShardManager for {} data store did not shutdown gracefully", getDataStoreName(), e);
+        }
     }
 
     public ClusterWrapper getClusterWrapper() {
         return clusterWrapper;
     }
 
-    public String getCurrentMemberName(){
+    public MemberName getCurrentMemberName(){
         return clusterWrapper.getCurrentMemberName();
     }
 
@@ -389,18 +401,19 @@ public class ActorContext {
      *
      * @param message
      */
-    public void broadcast(final Object message){
+    public void broadcast(final Function<Short, Object> messageSupplier){
         for(final String shardName : configuration.getAllShardNames()){
 
-            Future<ActorSelection> primaryFuture = findPrimaryShardAsync(shardName);
-            primaryFuture.onComplete(new OnComplete<ActorSelection>() {
+            Future<PrimaryShardInfo> primaryFuture = findPrimaryShardAsync(shardName);
+            primaryFuture.onComplete(new OnComplete<PrimaryShardInfo>() {
                 @Override
-                public void onComplete(Throwable failure, ActorSelection primaryShard) {
+                public void onComplete(Throwable failure, PrimaryShardInfo primaryShardInfo) {
+                    Object message = messageSupplier.apply(primaryShardInfo.getPrimaryShardVersion());
                     if(failure != null) {
                         LOG.warn("broadcast failed to send message {} to shard {}:  {}",
                                 message.getClass().getSimpleName(), shardName, failure);
                     } else {
-                        primaryShard.tell(message, ActorRef.noSender());
+                        primaryShardInfo.getPrimaryShardActor().tell(message, ActorRef.noSender());
                     }
                 }
             }, getClientDispatcher());
@@ -411,6 +424,10 @@ public class ActorContext {
         return operationDuration;
     }
 
+    public Timeout getOperationTimeout() {
+        return operationTimeout;
+    }
+
     public boolean isPathLocal(String path) {
         if (Strings.isNullOrEmpty(path)) {
             return false;
@@ -438,42 +455,6 @@ public class ActorContext {
         }
     }
 
-    /**
-     * @deprecated This method is present only to support backward compatibility with Helium and should not be
-     * used any further
-     *
-     *
-     * @param primaryPath
-     * @param localPathOfRemoteActor
-     * @return
-    */
-    @Deprecated
-    public String resolvePath(final String primaryPath,
-                                            final String localPathOfRemoteActor) {
-        StringBuilder builder = new StringBuilder();
-        String[] primaryPathElements = primaryPath.split("/");
-        builder.append(primaryPathElements[0]).append("//")
-            .append(primaryPathElements[1]).append(primaryPathElements[2]);
-        String[] remotePathElements = localPathOfRemoteActor.split("/");
-        for (int i = 3; i < remotePathElements.length; i++) {
-                builder.append("/").append(remotePathElements[i]);
-            }
-
-        return builder.toString();
-    }
-
-    /**
-     * Get the maximum number of operations that are to be permitted within a transaction before the transaction
-     * should begin throttling the operations
-     *
-     * Parking reading this configuration here because we need to get to the actor system settings
-     *
-     * @return
-     */
-    public int getTransactionOutstandingOperationLimit(){
-        return transactionOutstandingOperationLimit;
-    }
-
     /**
      * This is a utility method that lets us get a Timer object for any operation. This is a little open-ended to allow
      * us to create a timer for pretty much anything.
@@ -482,26 +463,22 @@ public class ActorContext {
      * @return
      */
     public Timer getOperationTimer(String operationName){
-        final String rate = MetricRegistry.name(DISTRIBUTED_DATA_STORE_METRIC_REGISTRY, datastoreContext.getDataStoreType(), operationName, METRIC_RATE);
-        return metricRegistry.timer(rate);
+        return getOperationTimer(datastoreContext.getDataStoreName(), operationName);
     }
 
-    /**
-     * Get the type of the data store to which this ActorContext belongs
-     *
-     * @return
-     */
-    public String getDataStoreType() {
-        return datastoreContext.getDataStoreType();
+    public Timer getOperationTimer(String dataStoreType, String operationName){
+        final String rate = MetricRegistry.name(DISTRIBUTED_DATA_STORE_METRIC_REGISTRY, dataStoreType,
+                operationName, METRIC_RATE);
+        return metricRegistry.timer(rate);
     }
 
     /**
-     * Set the number of transaction creation permits that are to be allowed
+     * Get the name of the data store to which this ActorContext belongs
      *
-     * @param permitsPerSecond
+     * @return
      */
-    public void setTxCreationLimit(double permitsPerSecond){
-        txRateLimiter.setRate(permitsPerSecond);
+    public String getDataStoreName() {
+        return datastoreContext.getDataStoreName();
     }
 
     /**
@@ -509,7 +486,7 @@ public class ActorContext {
      * @return
      */
     public double getTxCreationLimit(){
-        return txRateLimiter.getRate();
+        return txRateLimiter.getTxCreationLimit();
     }
 
     /**
@@ -540,6 +517,14 @@ public class ActorContext {
         return this.dispatchers.getDispatcherPath(Dispatchers.DispatcherType.Notification);
     }
 
+    public Configuration getConfiguration() {
+        return configuration;
+    }
+
+    public ShardStrategyFactory getShardStrategyFactory() {
+        return shardStrategyFactory;
+    }
+
     protected Future<Object> doAsk(ActorRef actorRef, Object message, Timeout timeout){
         return ask(actorRef, message, timeout);
     }
@@ -548,8 +533,7 @@ public class ActorContext {
         return ask(actorRef, message, timeout);
     }
 
-    @VisibleForTesting
-    Cache<String, Future<ActorSelection>> getPrimaryShardActorSelectionCache() {
-        return primaryShardActorSelectionCache;
+    public PrimaryShardInfoFutureCache getPrimaryShardInfoCache() {
+        return primaryShardInfoCache;
     }
 }