X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2Futils%2FActorContext.java;h=b6250fc1cccbfbde152a8af7e1e2f4b9b3c6f7cd;hp=f81c2a87cd8694dab4c5c194a3a3be05af6bc282;hb=c9d0ae2a8ce525d150f360cf49f21b7c0187cfbf;hpb=d802cf2cdf94d60ae7a4336a7d629ffa42356d8a diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/utils/ActorContext.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/utils/ActorContext.java index f81c2a87cd..b6250fc1cc 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/utils/ActorContext.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/utils/ActorContext.java @@ -8,45 +8,54 @@ 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.Futures; import akka.dispatch.Mapper; +import akka.dispatch.OnComplete; 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.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 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.NoShardLeaderException; 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 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 @@ -54,11 +63,11 @@ 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); - - public static final String MAILBOX = "bounded-mailbox"; - + 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 FIND_PRIMARY_FAILURE_TRANSFORMER = new Mapper() { @Override @@ -74,16 +83,27 @@ public class ActorContext { 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 volatile SchemaContext schemaContext; - private final FiniteDuration operationDuration; - private final Timeout operationTimeout; + private DatastoreContext datastoreContext; + 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 Timeout transactionCommitOperationTimeout; + private Timeout shardInitializationTimeout; + private final Dispatchers dispatchers; + private Cache> primaryShardActorSelectionCache; + + private volatile SchemaContext schemaContext; + private volatile boolean updated; public ActorContext(ActorSystem actorSystem, ActorRef shardManager, ClusterWrapper clusterWrapper, Configuration configuration) { @@ -99,10 +119,9 @@ public class ActorContext { this.clusterWrapper = clusterWrapper; this.configuration = configuration; this.datastoreContext = datastoreContext; + this.dispatchers = new Dispatchers(actorSystem.dispatchers()); - operationDuration = Duration.create(datastoreContext.getOperationTimeoutInSeconds(), - TimeUnit.SECONDS); - operationTimeout = new Timeout(operationDuration); + setCachedProperties(); Address selfAddress = clusterWrapper.getSelfAddress(); if (selfAddress != null && !selfAddress.host().isEmpty()) { @@ -110,6 +129,26 @@ public class ActorContext { } else { selfAddressHostPort = null; } + + transactionOutstandingOperationLimit = new CommonConfig(this.getActorSystem().settings().config()).getMailBoxCapacity(); + jmxReporter.start(); + + } + + private void setCachedProperties() { + txRateLimiter = RateLimiter.create(datastoreContext.getTransactionCreationInitialRateLimit()); + + operationDuration = Duration.create(datastoreContext.getOperationTimeoutInSeconds(), TimeUnit.SECONDS); + 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() { @@ -136,54 +175,62 @@ public class ActorContext { this.schemaContext = schemaContext; if(shardManager != null) { - shardManager.tell(new UpdateSchemaContext(schemaContext), null); + shardManager.tell(new UpdateSchemaContext(schemaContext), ActorRef.noSender()); } } - public SchemaContext getSchemaContext() { - return schemaContext; - } + public void setDatastoreContext(DatastoreContext context) { + this.datastoreContext = context; + setCachedProperties(); - /** - * Finds the primary shard for the given shard name - * - * @param shardName - * @return - */ - public Optional findPrimaryShard(String shardName) { - String path = findPrimaryPathOrNull(shardName); - if (path == null){ - return Optional.absent(); + // We write the 'updated' volatile to trigger a write memory barrier so that the writes above + // will be published immediately even though they may not be immediately visible to other + // threads due to unsynchronized reads. That's OK though - we're going for eventual + // consistency here as immediately visible updates to these members aren't critical. These + // members could've been made volatile but wanted to avoid volatile reads as these are + // accessed often and updates will be infrequent. + + updated = true; + + if(shardManager != null) { + shardManager.tell(context, ActorRef.noSender()); } - return Optional.of(actorSystem.actorSelection(path)); + } + + public SchemaContext getSchemaContext() { + return schemaContext; } public Future findPrimaryShardAsync(final String shardName) { + Future ret = primaryShardActorSelectionCache.getIfPresent(shardName); + if(ret != null){ + return ret; + } Future future = executeOperationAsync(shardManager, - new FindPrimary(shardName, true).toSerializable(), - datastoreContext.getShardInitializationTimeout()); + new FindPrimary(shardName, true), shardInitializationTimeout); return future.transform(new Mapper() { @Override public ActorSelection checkedApply(Object response) throws Exception { - if(response.getClass().equals(PrimaryFound.SERIALIZABLE_CLASS)) { - PrimaryFound found = PrimaryFound.fromSerializable(response); + if(response instanceof PrimaryFound) { + PrimaryFound found = (PrimaryFound)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)); + ActorSelection actorSelection = actorSystem.actorSelection(found.getPrimaryPath()); + primaryShardActorSelectionCache.put(shardName, Futures.successful(actorSelection)); + return actorSelection; + } else if(response instanceof NotInitializedException) { + throw (NotInitializedException)response; + } else if(response instanceof PrimaryNotFoundException) { + throw (PrimaryNotFoundException)response; + } else if(response instanceof NoShardLeaderException) { + throw (NoShardLeaderException)response; } throw new UnknownMessageException(String.format( "FindPrimary returned unkown response: %s", response)); } - }, FIND_PRIMARY_FAILURE_TRANSFORMER, getActorSystem().dispatcher()); + }, FIND_PRIMARY_FAILURE_TRANSFORMER, getClientDispatcher()); } /** @@ -213,7 +260,7 @@ public class ActorContext { */ public Future findLocalShardAsync( final String shardName) { Future future = executeOperationAsync(shardManager, - new FindLocalShard(shardName, true), datastoreContext.getShardInitializationTimeout()); + new FindLocalShard(shardName, true), shardInitializationTimeout); return future.map(new Mapper() { @Override @@ -222,10 +269,8 @@ public class ActorContext { 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 NotInitializedException) { + throw (NotInitializedException)response; } else if(response instanceof LocalShardNotFound) { throw new LocalShardNotFoundException( String.format("Local shard for %s does not exist.", shardName)); @@ -234,29 +279,9 @@ public class ActorContext { throw new UnknownMessageException(String.format( "FindLocalShard returned unkown response: %s", response)); } - }, getActorSystem().dispatcher()); - } - - 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; - } + }, getClientDispatcher()); } - /** * Executes an operation on a local actor and wait for it's response * @@ -279,8 +304,8 @@ public class ActorContext { 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().toString(), actor.toString()); - return ask(actor, message, timeout); + LOG.debug("Sending message {} to {}", message.getClass(), actor); + return doAsk(actor, message, timeout); } /** @@ -314,9 +339,9 @@ public class ActorContext { 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().toString(), actor.toString()); + LOG.debug("Sending message {} to {}", message.getClass(), actor); - return ask(actor, message, timeout); + return doAsk(actor, message, timeout); } /** @@ -341,7 +366,7 @@ public class ActorContext { 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().toString(), actor.toString()); + LOG.debug("Sending message {} to {}", message.getClass(), actor); actor.tell(message, ActorRef.noSender()); } @@ -364,16 +389,21 @@ public class ActorContext { * * @param message */ - public void broadcast(Object message){ - for(String shardName : configuration.getAllShardNames()){ - - Optional 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); - } + public void broadcast(final Object message){ + for(final String shardName : configuration.getAllShardNames()){ + + Future primaryFuture = findPrimaryShardAsync(shardName); + primaryFuture.onComplete(new OnComplete() { + @Override + public void onComplete(Throwable failure, ActorSelection primaryShard) { + if(failure != null) { + LOG.warn("broadcast failed to send message {} to shard {}: {}", + message.getClass().getSimpleName(), shardName, failure); + } else { + primaryShard.tell(message, ActorRef.noSender()); + } + } + }, getClientDispatcher()); } } @@ -386,14 +416,14 @@ public class ActorContext { return false; } - int pathAtIndex = path.indexOf("@"); + 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); + int slashIndex = path.indexOf('/', pathAtIndex); if (slashIndex == -1) { return false; @@ -431,4 +461,95 @@ public class ActorContext { 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. + * + * @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); + } + + /** + * Get the type of the data store to which this ActorContext belongs + * + * @return + */ + public String getDataStoreType() { + return datastoreContext.getDataStoreType(); + } + + /** + * Set the number of transaction creation permits that are to be allowed + * + * @param permitsPerSecond + */ + public void setTxCreationLimit(double permitsPerSecond){ + txRateLimiter.setRate(permitsPerSecond); + } + + /** + * 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); + } + + protected Future doAsk(ActorRef actorRef, Object message, Timeout timeout){ + return ask(actorRef, message, timeout); + } + + protected Future doAsk(ActorSelection actorRef, Object message, Timeout timeout){ + return ask(actorRef, message, timeout); + } + + @VisibleForTesting + Cache> getPrimaryShardActorSelectionCache() { + return primaryShardActorSelectionCache; + } }