X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2Futils%2FActorContext.java;h=26e6318f6d4d14aa5944a909696bbaa8b5f7f207;hb=0f3e97c63d11379ef36505733aacf3c926d4a085;hp=d8af09c86b6ec01479f038104fdad350170f4991;hpb=b3c034675957f963c5878ce1e5e183ec2de8b5e2;p=controller.git 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 d8af09c86b..68ac6412e8 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,36 +8,57 @@ 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.PoisonPill; +import akka.actor.Address; +import akka.dispatch.Mapper; +import akka.dispatch.OnComplete; +import akka.pattern.AskTimeoutException; +import akka.pattern.Patterns; import akka.util.Timeout; +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 java.util.concurrent.TimeUnit; +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; +import org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException; import org.opendaylight.controller.cluster.datastore.exceptions.TimeoutException; -import org.opendaylight.controller.cluster.datastore.messages.ActorNotInitialized; +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.PrimaryFound; +import org.opendaylight.controller.cluster.datastore.messages.LocalShardNotFound; +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; 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 @@ -45,28 +66,90 @@ 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 DISTRIBUTED_DATA_STORE_METRIC_REGISTRY = "distributed-data-store"; + private static final String METRIC_RATE = "rate"; + private static final Mapper FIND_PRIMARY_FAILURE_TRANSFORMER = + new Mapper() { + @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."); + } - public static final String MAILBOX = "bounded-mailbox"; + return actualFailure; + } + }; + public static final String BOUNDED_MAILBOX = "bounded-mailbox"; + public static final String COMMIT = "commit"; private final ActorSystem actorSystem; private final ActorRef shardManager; private final ClusterWrapper clusterWrapper; private final Configuration configuration; + private DatastoreContext datastoreContext; + private FiniteDuration operationDuration; + private Timeout operationTimeout; + private final String selfAddressHostPort; + private TransactionRateLimiter txRateLimiter; + private Timeout transactionCommitOperationTimeout; + private Timeout shardInitializationTimeout; + private final Dispatchers dispatchers; + private volatile SchemaContext schemaContext; - private FiniteDuration operationDuration = DEFAULT_OPER_DURATION; - private Timeout operationTimeout = new Timeout(operationDuration); + 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(), new PrimaryShardInfoFutureCache()); + } public ActorContext(ActorSystem actorSystem, ActorRef shardManager, - ClusterWrapper clusterWrapper, - Configuration configuration) { + ClusterWrapper clusterWrapper, Configuration configuration, + 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(); + + Address selfAddress = clusterWrapper.getSelfAddress(); + if (selfAddress != null && !selfAddress.host().isEmpty()) { + selfAddressHostPort = selfAddress.host().get() + ":" + selfAddress.port().get(); + } else { + selfAddressHostPort = null; + } + + } + + private void setCachedProperties() { + txRateLimiter = new TransactionRateLimiter(this); + + 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)); + } + + public DatastoreContext getDatastoreContext() { + return datastoreContext; } public ActorSystem getActorSystem() { @@ -89,42 +172,84 @@ public class ActorContext { this.schemaContext = schemaContext; if(shardManager != null) { - shardManager.tell(new UpdateSchemaContext(schemaContext), null); + shardManager.tell(new UpdateSchemaContext(schemaContext), ActorRef.noSender()); } } - public void setOperationTimeout(int timeoutInSeconds) { - operationDuration = Duration.create(timeoutInSeconds, TimeUnit.SECONDS); - operationTimeout = new Timeout(operationDuration); + 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 + // 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(contextFactory, ActorRef.noSender()); + } } public SchemaContext getSchemaContext() { return schemaContext; } - /** - * 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(); + public Future findPrimaryShardAsync(final String shardName) { + Future ret = primaryShardInfoCache.getIfPresent(shardName); + if(ret != null){ + return ret; } - return Optional.of(actorSystem.actorSelection(path)); + Future future = executeOperationAsync(shardManager, + new FindPrimary(shardName, true), shardInitializationTimeout); + + return future.transform(new Mapper() { + @Override + 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) { + 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, 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 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 Optional findLocalShard(String shardName) { - Object result = executeOperation(shardManager, new FindLocalShard(shardName)); + Object result = executeOperation(shardManager, new FindLocalShard(shardName, false)); if (result instanceof LocalShardFound) { LocalShardFound found = (LocalShardFound) result; @@ -135,27 +260,36 @@ public class ActorContext { return Optional.absent(); } - - private String findPrimaryPathOrNull(String shardName) { - Object result = executeOperation(shardManager, new FindPrimary(shardName).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; - } + /** + * 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 findLocalShardAsync( final String shardName) { + Future future = executeOperationAsync(shardManager, + new FindLocalShard(shardName, true), shardInitializationTimeout); + + return future.map(new Mapper() { + @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 NotInitializedException) { + throw (NotInitializedException)response; + } 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()); } - /** * Executes an operation on a local actor and wait for it's response * @@ -178,8 +312,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); } /** @@ -205,15 +339,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 executeOperationAsync(ActorSelection actor, Object message) { + public Future 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().toString(), actor.toString()); + LOG.debug("Sending message {} to {}", message.getClass(), actor); + + return doAsk(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 executeOperationAsync(ActorSelection actor, Object message) { + return executeOperationAsync(actor, message, operationTimeout); } /** @@ -227,17 +374,25 @@ 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()); } 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(); } @@ -246,20 +401,139 @@ 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 Function messageSupplier){ + for(final String shardName : configuration.getAllShardNames()){ + + Future primaryFuture = findPrimaryShardAsync(shardName); + primaryFuture.onComplete(new OnComplete() { + @Override + 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 { + primaryShardInfo.getPrimaryShardActor().tell(message, ActorRef.noSender()); + } + } + }, getClientDispatcher()); } } public FiniteDuration getOperationDuration() { return operationDuration; } + + public Timeout getOperationTimeout() { + return operationTimeout; + } + + public boolean isPathLocal(String path) { + if (Strings.isNullOrEmpty(path)) { + return false; + } + + 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); + + 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; + } + } + + /** + * 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){ + return getOperationTimer(datastoreContext.getDataStoreName(), operationName); + } + + 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); + } + + /** + * Get the name of the data store to which this ActorContext belongs + * + * @return + */ + public String getDataStoreName() { + return datastoreContext.getDataStoreName(); + } + + /** + * Get the current transaction creation rate limit + * @return + */ + public double getTxCreationLimit(){ + return txRateLimiter.getTxCreationLimit(); + } + + /** + * 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); + } + + public Configuration getConfiguration() { + return configuration; + } + + public ShardStrategyFactory getShardStrategyFactory() { + return shardStrategyFactory; + } + + 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); + } + + public PrimaryShardInfoFutureCache getPrimaryShardInfoCache() { + return primaryShardInfoCache; + } }