BUG 2676 : Use notification-dispatcher for DataChangeListener actors
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / utils / ActorContext.java
index c989b275df3105480b035b7972e83c0822b7182d..26e6318f6d4d14aa5944a909696bbaa8b5f7f207 100644 (file)
@@ -8,35 +8,50 @@
 
 package org.opendaylight.controller.cluster.datastore.utils;
 
+import static akka.pattern.Patterns.ask;
 import akka.actor.ActorPath;
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
 import akka.actor.ActorSystem;
+import akka.actor.Address;
 import akka.actor.PoisonPill;
+import akka.dispatch.Mapper;
+import akka.pattern.AskTimeoutException;
 import akka.util.Timeout;
-
+import com.codahale.metrics.JmxReporter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.util.concurrent.RateLimiter;
+import java.util.concurrent.TimeUnit;
+import org.opendaylight.controller.cluster.common.actor.CommonConfig;
 import org.opendaylight.controller.cluster.datastore.ClusterWrapper;
 import org.opendaylight.controller.cluster.datastore.Configuration;
+import org.opendaylight.controller.cluster.datastore.DatastoreContext;
+import org.opendaylight.controller.cluster.datastore.exceptions.LocalShardNotFoundException;
+import org.opendaylight.controller.cluster.datastore.exceptions.NotInitializedException;
 import org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException;
 import org.opendaylight.controller.cluster.datastore.exceptions.TimeoutException;
+import org.opendaylight.controller.cluster.datastore.exceptions.UnknownMessageException;
+import org.opendaylight.controller.cluster.datastore.messages.ActorNotInitialized;
 import org.opendaylight.controller.cluster.datastore.messages.FindLocalShard;
 import org.opendaylight.controller.cluster.datastore.messages.FindPrimary;
 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.PrimaryNotFound;
 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import scala.concurrent.Await;
+import scala.concurrent.ExecutionContext;
 import scala.concurrent.Future;
 import scala.concurrent.duration.Duration;
 import scala.concurrent.duration.FiniteDuration;
 
-import java.util.concurrent.TimeUnit;
-
-import static akka.pattern.Patterns.ask;
-
 /**
  * The ActorContext class contains utility methods which could be used by
  * non-actors (like DistributedDataStore) to work with actors a little more
@@ -44,28 +59,82 @@ import static akka.pattern.Patterns.ask;
  * but should not be passed to actors especially remote actors
  */
 public class ActorContext {
-    private static final Logger
-        LOG = LoggerFactory.getLogger(ActorContext.class);
-
-    private static final FiniteDuration DEFAULT_OPER_DURATION = Duration.create(5, TimeUnit.SECONDS);
+    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
+        public Throwable apply(Throwable failure) {
+            Throwable actualFailure = failure;
+            if(failure instanceof AskTimeoutException) {
+                // A timeout exception most likely means the shard isn't initialized.
+                actualFailure = new NotInitializedException(
+                        "Timed out trying to find the primary shard. Most likely cause is the " +
+                        "shard is not initialized yet.");
+            }
 
+            return actualFailure;
+        }
+    };
     public static final String MAILBOX = "bounded-mailbox";
 
     private final ActorSystem actorSystem;
     private final ActorRef shardManager;
     private final ClusterWrapper clusterWrapper;
     private final Configuration configuration;
+    private final DatastoreContext datastoreContext;
+    private final FiniteDuration operationDuration;
+    private final Timeout operationTimeout;
+    private final String selfAddressHostPort;
+    private final RateLimiter txRateLimiter;
+    private final MetricRegistry metricRegistry = new MetricRegistry();
+    private final JmxReporter jmxReporter = JmxReporter.forRegistry(metricRegistry).inDomain(DOMAIN).build();
+    private final int transactionOutstandingOperationLimit;
+    private final Timeout transactionCommitOperationTimeout;
+    private final Dispatchers dispatchers;
+
     private volatile SchemaContext schemaContext;
-    private FiniteDuration operationDuration = DEFAULT_OPER_DURATION;
-    private Timeout operationTimeout = new Timeout(operationDuration);
 
     public ActorContext(ActorSystem actorSystem, ActorRef shardManager,
-        ClusterWrapper clusterWrapper,
-        Configuration configuration) {
+            ClusterWrapper clusterWrapper, Configuration configuration) {
+        this(actorSystem, shardManager, clusterWrapper, configuration,
+                DatastoreContext.newBuilder().build());
+    }
+
+    public ActorContext(ActorSystem actorSystem, ActorRef shardManager,
+            ClusterWrapper clusterWrapper, Configuration configuration,
+            DatastoreContext datastoreContext) {
         this.actorSystem = actorSystem;
         this.shardManager = shardManager;
         this.clusterWrapper = clusterWrapper;
         this.configuration = configuration;
+        this.datastoreContext = datastoreContext;
+        this.txRateLimiter = RateLimiter.create(datastoreContext.getTransactionCreationInitialRateLimit());
+        this.dispatchers = new Dispatchers(actorSystem.dispatchers());
+
+        operationDuration = Duration.create(datastoreContext.getOperationTimeoutInSeconds(), TimeUnit.SECONDS);
+        operationTimeout = new Timeout(operationDuration);
+        transactionCommitOperationTimeout =  new Timeout(Duration.create(getDatastoreContext().getShardTransactionCommitTimeoutInSeconds(),
+                TimeUnit.SECONDS));
+
+
+        Address selfAddress = clusterWrapper.getSelfAddress();
+        if (selfAddress != null && !selfAddress.host().isEmpty()) {
+            selfAddressHostPort = selfAddress.host().get() + ":" + selfAddress.port().get();
+        } else {
+            selfAddressHostPort = null;
+        }
+
+        transactionOutstandingOperationLimit = new CommonConfig(this.getActorSystem().settings().config()).getMailBoxCapacity();
+        jmxReporter.start();
+
+    }
+
+    public DatastoreContext getDatastoreContext() {
+        return datastoreContext;
     }
 
     public ActorSystem getActorSystem() {
@@ -92,61 +161,120 @@ public class ActorContext {
         }
     }
 
-    public void setOperationTimeout(int timeoutInSeconds) {
-        operationDuration = Duration.create(timeoutInSeconds, TimeUnit.SECONDS);
-        operationTimeout = new Timeout(operationDuration);
-    }
-
     public SchemaContext getSchemaContext() {
         return schemaContext;
     }
 
     /**
-     * Finds the primary for a given shard
+     * Finds the primary shard for the given shard name
      *
      * @param shardName
      * @return
      */
-    public ActorSelection findPrimary(String shardName) {
-        String path = findPrimaryPath(shardName);
-        return actorSystem.actorSelection(path);
+    public Optional<ActorSelection> findPrimaryShard(String shardName) {
+        String path = findPrimaryPathOrNull(shardName);
+        if (path == null){
+            return Optional.absent();
+        }
+        return Optional.of(actorSystem.actorSelection(path));
+    }
+
+    public Future<ActorSelection> findPrimaryShardAsync(final String shardName) {
+        Future<Object> future = executeOperationAsync(shardManager,
+                new FindPrimary(shardName, true).toSerializable(),
+                datastoreContext.getShardInitializationTimeout());
+
+        return future.transform(new Mapper<Object, ActorSelection>() {
+            @Override
+            public ActorSelection checkedApply(Object response) throws Exception {
+                if(response.getClass().equals(PrimaryFound.SERIALIZABLE_CLASS)) {
+                    PrimaryFound found = PrimaryFound.fromSerializable(response);
+
+                    LOG.debug("Primary found {}", found.getPrimaryPath());
+                    return actorSystem.actorSelection(found.getPrimaryPath());
+                } else if(response instanceof ActorNotInitialized) {
+                    throw new NotInitializedException(
+                            String.format("Found primary shard %s but it's not initialized yet. " +
+                                          "Please try again later", shardName));
+                } else if(response instanceof PrimaryNotFound) {
+                    throw new PrimaryNotFoundException(
+                            String.format("No primary shard found for %S.", shardName));
+                }
+
+                throw new UnknownMessageException(String.format(
+                        "FindPrimary returned unkown response: %s", response));
+            }
+        }, FIND_PRIMARY_FAILURE_TRANSFORMER, getClientDispatcher());
     }
 
     /**
-     * Finds a local shard given it's shard name and return it's ActorRef
+     * Finds a local shard given its shard name and return it's ActorRef
      *
      * @param shardName the name of the local shard that needs to be found
      * @return a reference to a local shard actor which represents the shard
      *         specified by the shardName
      */
-    public ActorRef findLocalShard(String shardName) {
-        Object result = executeLocalOperation(shardManager,
-            new FindLocalShard(shardName));
+    public Optional<ActorRef> findLocalShard(String shardName) {
+        Object result = executeOperation(shardManager, new FindLocalShard(shardName, false));
 
         if (result instanceof LocalShardFound) {
             LocalShardFound found = (LocalShardFound) result;
-
             LOG.debug("Local shard found {}", found.getPath());
-
-            return found.getPath();
+            return Optional.of(found.getPath());
         }
 
-        return null;
+        return Optional.absent();
     }
 
+    /**
+     * Finds a local shard async given its shard name and return a Future from which to obtain the
+     * ActorRef.
+     *
+     * @param shardName the name of the local shard that needs to be found
+     */
+    public Future<ActorRef> findLocalShardAsync( final String shardName) {
+        Future<Object> future = executeOperationAsync(shardManager,
+                new FindLocalShard(shardName, true), datastoreContext.getShardInitializationTimeout());
+
+        return future.map(new Mapper<Object, ActorRef>() {
+            @Override
+            public ActorRef checkedApply(Object response) throws Throwable {
+                if(response instanceof LocalShardFound) {
+                    LocalShardFound found = (LocalShardFound)response;
+                    LOG.debug("Local shard found {}", found.getPath());
+                    return found.getPath();
+                } else if(response instanceof ActorNotInitialized) {
+                    throw new NotInitializedException(
+                            String.format("Found local shard for %s but it's not initialized yet.",
+                                    shardName));
+                } else if(response instanceof LocalShardNotFound) {
+                    throw new LocalShardNotFoundException(
+                            String.format("Local shard for %s does not exist.", shardName));
+                }
+
+                throw new UnknownMessageException(String.format(
+                        "FindLocalShard returned unkown response: %s", response));
+            }
+        }, getClientDispatcher());
+    }
 
-    public String findPrimaryPath(String shardName) {
-        Object result = executeLocalOperation(shardManager,
-            new FindPrimary(shardName).toSerializable());
+    private String findPrimaryPathOrNull(String shardName) {
+        Object result = executeOperation(shardManager, new FindPrimary(shardName, false).toSerializable());
 
         if (result.getClass().equals(PrimaryFound.SERIALIZABLE_CLASS)) {
             PrimaryFound found = PrimaryFound.fromSerializable(result);
 
             LOG.debug("Primary found {}", found.getPrimaryPath());
-
             return found.getPrimaryPath();
+
+        } else if (result.getClass().equals(ActorNotInitialized.class)){
+            throw new NotInitializedException(
+                String.format("Found primary shard[%s] but its not initialized yet. Please try again later", shardName)
+            );
+
+        } else {
+            return null;
         }
-        throw new PrimaryNotFoundException("Could not find primary for shardName " + shardName);
     }
 
 
@@ -157,16 +285,25 @@ public class ActorContext {
      * @param message
      * @return The response of the operation
      */
-    public Object executeLocalOperation(ActorRef actor, Object message) {
-        Future<Object> future = ask(actor, message, operationTimeout);
+    public Object executeOperation(ActorRef actor, Object message) {
+        Future<Object> future = executeOperationAsync(actor, message, operationTimeout);
 
         try {
             return Await.result(future, operationDuration);
         } catch (Exception e) {
-            throw new TimeoutException("Sending message " + message.getClass().toString() + " to actor " + actor.toString() + " failed" , e);
+            throw new TimeoutException("Sending message " + message.getClass().toString() +
+                    " to actor " + actor.toString() + " failed. Try again later.", e);
         }
     }
 
+    public Future<Object> executeOperationAsync(ActorRef actor, Object message, Timeout timeout) {
+        Preconditions.checkArgument(actor != null, "actor must not be null");
+        Preconditions.checkArgument(message != null, "message must not be null");
+
+        LOG.debug("Sending message {} to {}", message.getClass(), actor);
+        return ask(actor, message, timeout);
+    }
+
     /**
      * Execute an operation on a remote actor and wait for it's response
      *
@@ -174,18 +311,14 @@ public class ActorContext {
      * @param message
      * @return
      */
-    public Object executeRemoteOperation(ActorSelection actor, Object message) {
-
-        LOG.debug("Sending remote message {} to {}", message.getClass().toString(),
-            actor.toString());
-
-        Future<Object> future = ask(actor, message, operationTimeout);
+    public Object executeOperation(ActorSelection actor, Object message) {
+        Future<Object> future = executeOperationAsync(actor, message);
 
         try {
             return Await.result(future, operationDuration);
         } catch (Exception e) {
             throw new TimeoutException("Sending message " + message.getClass().toString() +
-                    " to actor " + actor.toString() + " failed, e);
+                    " to actor " + actor.toString() + " failed. Try again later.", e);
         }
     }
 
@@ -194,13 +327,28 @@ public class ActorContext {
      *
      * @param actor the ActorSelection
      * @param message the message to send
+     * @param timeout the operation timeout
      * @return a Future containing the eventual result
      */
-    public Future<Object> executeRemoteOperationAsync(ActorSelection actor, Object message) {
+    public Future<Object> executeOperationAsync(ActorSelection actor, Object message,
+            Timeout timeout) {
+        Preconditions.checkArgument(actor != null, "actor must not be null");
+        Preconditions.checkArgument(message != null, "message must not be null");
+
+        LOG.debug("Sending message {} to {}", message.getClass(), actor);
 
-        LOG.debug("Sending remote message {} to {}", message.getClass().toString(), actor.toString());
+        return ask(actor, message, timeout);
+    }
 
-        return ask(actor, message, operationTimeout);
+    /**
+     * Execute an operation on a remote actor asynchronously.
+     *
+     * @param actor the ActorSelection
+     * @param message the message to send
+     * @return a Future containing the eventual result
+     */
+    public Future<Object> executeOperationAsync(ActorSelection actor, Object message) {
+        return executeOperationAsync(actor, message, operationTimeout);
     }
 
     /**
@@ -210,115 +358,177 @@ public class ActorContext {
      * @param actor the ActorSelection
      * @param message the message to send
      */
-    public void sendRemoteOperationAsync(ActorSelection actor, Object message) {
+    public void sendOperationAsync(ActorSelection actor, Object message) {
+        Preconditions.checkArgument(actor != null, "actor must not be null");
+        Preconditions.checkArgument(message != null, "message must not be null");
+
+        LOG.debug("Sending message {} to {}", message.getClass(), actor);
+
         actor.tell(message, ActorRef.noSender());
     }
 
-    public void sendShardOperationAsync(String shardName, Object message) {
-        ActorSelection primary = findPrimary(shardName);
+    public void shutdown() {
+        shardManager.tell(PoisonPill.getInstance(), null);
+        actorSystem.shutdown();
+    }
 
-        primary.tell(message, ActorRef.noSender());
+    public ClusterWrapper getClusterWrapper() {
+        return clusterWrapper;
     }
 
+    public String getCurrentMemberName(){
+        return clusterWrapper.getCurrentMemberName();
+    }
 
     /**
-     * Execute an operation on the primary for a given shard
-     * <p>
-     * This method first finds the primary for a given shard ,then sends
-     * the message to the remote shard and waits for a response
-     * </p>
+     * Send the message to each and every shard
      *
-     * @param shardName
      * @param message
-     * @return
-     * @throws org.opendaylight.controller.cluster.datastore.exceptions.TimeoutException         if the message to the remote shard times out
-     * @throws org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException if the primary shard is not found
      */
-    public Object executeShardOperation(String shardName, Object message) {
-        ActorSelection primary = findPrimary(shardName);
+    public void broadcast(Object message){
+        for(String shardName : configuration.getAllShardNames()){
 
-        return executeRemoteOperation(primary, message);
+            Optional<ActorSelection> primary = findPrimaryShard(shardName);
+            if (primary.isPresent()) {
+                primary.get().tell(message, ActorRef.noSender());
+            } else {
+                LOG.warn("broadcast failed to send message {} to shard {}. Primary not found",
+                        message.getClass().getSimpleName(), shardName);
+            }
+        }
     }
 
-    /**
-     * Execute an operation on the the local shard only
-     * <p>
-     *     This method first finds the address of the local shard if any. It then
-     *     executes the operation on it.
-     * </p>
-     *
-     * @param shardName the name of the shard on which the operation needs to be executed
-     * @param message the message that needs to be sent to the shard
-     * @return the message that was returned by the local actor on which the
-     *         the operation was executed. If a local shard was not found then
-     *         null is returned
-     * @throws org.opendaylight.controller.cluster.datastore.exceptions.TimeoutException
-     *         if the operation does not complete in a specified time duration
-     */
-    public Object executeLocalShardOperation(String shardName, Object message) {
-        ActorRef local = findLocalShard(shardName);
+    public FiniteDuration getOperationDuration() {
+        return operationDuration;
+    }
 
-        if(local != null) {
-            return executeLocalOperation(local, message);
+    public boolean isPathLocal(String path) {
+        if (Strings.isNullOrEmpty(path)) {
+            return false;
         }
 
-        return null;
-    }
+        int pathAtIndex = path.indexOf('@');
+        if (pathAtIndex == -1) {
+            //if the path is of local format, then its local and is co-located
+            return true;
 
+        } else if (selfAddressHostPort != null) {
+            // self-address and tx actor path, both are of remote path format
+            int slashIndex = path.indexOf('/', pathAtIndex);
 
-    public void shutdown() {
-        shardManager.tell(PoisonPill.getInstance(), null);
-        actorSystem.shutdown();
+            if (slashIndex == -1) {
+                return false;
+            }
+
+            String hostPort = path.substring(pathAtIndex + 1, slashIndex);
+            return hostPort.equals(selfAddressHostPort);
+
+        } else {
+            // self address is local format and tx actor path is remote format
+            return false;
+        }
     }
 
     /**
-     * @deprecated Need to stop using this method. There are ways to send a
-     * remote ActorRef as a string which should be used instead of this hack
+     * @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) {
+                                            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]);
-        }
+                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;
     }
 
-    public ActorPath actorFor(String path){
-        return actorSystem.actorFor(path).path();
+    /**
+     * 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.
+     *
+     * @param operationName
+     * @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);
     }
 
-    public String getCurrentMemberName(){
-        return clusterWrapper.getCurrentMemberName();
+    /**
+     * Get the type of the data store to which this ActorContext belongs
+     *
+     * @return
+     */
+    public String getDataStoreType() {
+        return datastoreContext.getDataStoreType();
     }
 
     /**
-     * Send the message to each and every shard
+     * Set the number of transaction creation permits that are to be allowed
      *
-     * @param message
+     * @param permitsPerSecond
      */
-    public void broadcast(Object message){
-        for(String shardName : configuration.getAllShardNames()){
-            try {
-                sendShardOperationAsync(shardName, message);
-            } catch(Exception e){
-                LOG.warn("broadcast failed to send message " +  message.getClass().getSimpleName() + " to shard " + shardName, e);
-            }
-        }
+    public void setTxCreationLimit(double permitsPerSecond){
+        txRateLimiter.setRate(permitsPerSecond);
     }
 
-    public FiniteDuration getOperationDuration() {
-        return operationDuration;
+    /**
+     * Get the current transaction creation rate limit
+     * @return
+     */
+    public double getTxCreationLimit(){
+        return txRateLimiter.getRate();
     }
+
+    /**
+     * Try to acquire a transaction creation permit. Will block if no permits are available.
+     */
+    public void acquireTxCreationPermit(){
+        txRateLimiter.acquire();
+    }
+
+    /**
+     * Return the operation timeout to be used when committing transactions
+     * @return
+     */
+    public Timeout getTransactionCommitOperationTimeout(){
+        return transactionCommitOperationTimeout;
+    }
+
+    /**
+     * An akka dispatcher that is meant to be used when processing ask Futures which were triggered by client
+     * code on the datastore
+     * @return
+     */
+    public ExecutionContext getClientDispatcher() {
+        return this.dispatchers.getDispatcher(Dispatchers.DispatcherType.Client);
+    }
+
+    public String getNotificationDispatcherPath(){
+        return this.dispatchers.getDispatcherPath(Dispatchers.DispatcherType.Notification);
+    }
+
 }