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%2Fshardmanager%2FShardManager.java;h=d1b9aba9d666268c6ab159f11e543ebc674fefe5;hp=7653aea3fb814b2c69e36684800902df013aab72;hb=6ef0b898f2117a4bb3a510c0df7af340f4fc8eca;hpb=925cb4a228d0fda99c7bfeb432eb25285a223887 diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/shardmanager/ShardManager.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/shardmanager/ShardManager.java index 7653aea3fb..d1b9aba9d6 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/shardmanager/ShardManager.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/shardmanager/ShardManager.java @@ -9,6 +9,7 @@ package org.opendaylight.controller.cluster.datastore.shardmanager; import static akka.pattern.Patterns.ask; +import static java.util.Objects.requireNonNull; import akka.actor.ActorRef; import akka.actor.Address; @@ -25,6 +26,8 @@ import akka.dispatch.Futures; import akka.dispatch.OnComplete; import akka.japi.Function; import akka.pattern.Patterns; +import akka.persistence.DeleteSnapshotsFailure; +import akka.persistence.DeleteSnapshotsSuccess; import akka.persistence.RecoveryCompleted; import akka.persistence.SaveSnapshotFailure; import akka.persistence.SaveSnapshotSuccess; @@ -32,9 +35,7 @@ import akka.persistence.SnapshotOffer; import akka.persistence.SnapshotSelectionCriteria; import akka.util.Timeout; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import java.io.ByteArrayInputStream; -import java.io.ObjectInputStream; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -49,39 +50,49 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.Consumer; import java.util.function.Supplier; -import org.apache.commons.lang3.SerializationUtils; import org.opendaylight.controller.cluster.access.concepts.MemberName; import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActorWithMetering; +import org.opendaylight.controller.cluster.common.actor.Dispatchers; +import org.opendaylight.controller.cluster.datastore.AbstractDataStore; import org.opendaylight.controller.cluster.datastore.ClusterWrapper; import org.opendaylight.controller.cluster.datastore.DatastoreContext; +import org.opendaylight.controller.cluster.datastore.DatastoreContext.Builder; import org.opendaylight.controller.cluster.datastore.DatastoreContextFactory; import org.opendaylight.controller.cluster.datastore.Shard; import org.opendaylight.controller.cluster.datastore.config.Configuration; import org.opendaylight.controller.cluster.datastore.config.ModuleShardConfiguration; +import org.opendaylight.controller.cluster.datastore.config.PrefixShardConfiguration; import org.opendaylight.controller.cluster.datastore.exceptions.AlreadyExistsException; 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.identifiers.ShardIdentifier; import org.opendaylight.controller.cluster.datastore.messages.ActorInitialized; +import org.opendaylight.controller.cluster.datastore.messages.AddPrefixShardReplica; import org.opendaylight.controller.cluster.datastore.messages.AddShardReplica; import org.opendaylight.controller.cluster.datastore.messages.ChangeShardMembersVotingStatus; import org.opendaylight.controller.cluster.datastore.messages.CreateShard; -import org.opendaylight.controller.cluster.datastore.messages.DatastoreSnapshot; import org.opendaylight.controller.cluster.datastore.messages.FindLocalShard; import org.opendaylight.controller.cluster.datastore.messages.FindPrimary; import org.opendaylight.controller.cluster.datastore.messages.FlipShardMembersVotingStatus; +import org.opendaylight.controller.cluster.datastore.messages.GetShardRole; +import org.opendaylight.controller.cluster.datastore.messages.GetShardRoleReply; 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.RemoteFindPrimary; import org.opendaylight.controller.cluster.datastore.messages.RemotePrimaryShardFound; +import org.opendaylight.controller.cluster.datastore.messages.RemovePrefixShardReplica; import org.opendaylight.controller.cluster.datastore.messages.RemoveShardReplica; import org.opendaylight.controller.cluster.datastore.messages.ShardLeaderStateChanged; import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext; -import org.opendaylight.controller.cluster.datastore.utils.Dispatchers; +import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot; +import org.opendaylight.controller.cluster.datastore.persisted.ShardManagerSnapshot; +import org.opendaylight.controller.cluster.datastore.utils.ClusterUtils; +import org.opendaylight.controller.cluster.datastore.utils.CompositeOnComplete; import org.opendaylight.controller.cluster.datastore.utils.PrimaryShardInfoFutureCache; import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener; +import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListenerReply; import org.opendaylight.controller.cluster.notifications.RoleChangeNotification; import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus; import org.opendaylight.controller.cluster.raft.base.messages.SwitchBehavior; @@ -98,12 +109,18 @@ import org.opendaylight.controller.cluster.raft.messages.ServerChangeReply; import org.opendaylight.controller.cluster.raft.messages.ServerChangeStatus; import org.opendaylight.controller.cluster.raft.messages.ServerRemoved; import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy; -import org.opendaylight.yangtools.yang.model.api.SchemaContext; +import org.opendaylight.controller.cluster.sharding.PrefixedShardConfigUpdateHandler; +import org.opendaylight.controller.cluster.sharding.messages.InitConfigListener; +import org.opendaylight.controller.cluster.sharding.messages.PrefixShardCreated; +import org.opendaylight.controller.cluster.sharding.messages.PrefixShardRemoved; +import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier; +import org.opendaylight.yangtools.concepts.Registration; +import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier; +import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.concurrent.ExecutionContext; import scala.concurrent.Future; -import scala.concurrent.duration.Duration; import scala.concurrent.duration.FiniteDuration; /** @@ -121,7 +138,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { // Stores a mapping between a shard name and it's corresponding information // Shard names look like inventory, topology etc and are as specified in // configuration - private final Map localShards = new HashMap<>(); + @VisibleForTesting + final Map localShards = new HashMap<>(); // The type of a ShardManager reflects the type of the datastore itself // A data store could be of type config/operational @@ -131,7 +149,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { private final Configuration configuration; - private final String shardDispatcherPath; + @VisibleForTesting + final String shardDispatcherPath; private final ShardManagerInfo shardManagerMBean; @@ -141,9 +160,10 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { private final PrimaryShardInfoFutureCache primaryShardInfoCache; - private final ShardPeerAddressResolver peerAddressResolver; + @VisibleForTesting + final ShardPeerAddressResolver peerAddressResolver; - private SchemaContext schemaContext; + private EffectiveModelContext schemaContext; private DatastoreSnapshot restoreFromSnapshot; @@ -151,16 +171,23 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { private final Set shardReplicaOperationsInProgress = new HashSet<>(); + private final Map> shardActorsStopping = new HashMap<>(); + + private final Set> shardAvailabilityCallbacks = new HashSet<>(); + private final String persistenceId; + private final AbstractDataStore dataStore; + + private PrefixedShardConfigUpdateHandler configUpdateHandler; - ShardManager(AbstractShardManagerCreator builder) { + ShardManager(final AbstractShardManagerCreator builder) { this.cluster = builder.getCluster(); this.configuration = builder.getConfiguration(); this.datastoreContextFactory = builder.getDatastoreContextFactory(); this.type = datastoreContextFactory.getBaseDatastoreContext().getDataStoreName(); this.shardDispatcherPath = new Dispatchers(context().system().dispatchers()).getDispatcherPath(Dispatchers.DispatcherType.Shard); - this.waitTillReadyCountdownLatch = builder.getWaitTillReadyCountdownLatch(); + this.waitTillReadyCountdownLatch = builder.getWaitTillReadyCountDownLatch(); this.primaryShardInfoCache = builder.getPrimaryShardInfoCache(); this.restoreFromSnapshot = builder.getRestoreFromSnapshot(); @@ -176,6 +203,13 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { "shard-manager-" + this.type, datastoreContextFactory.getBaseDatastoreContext().getDataStoreMXBeanType()); shardManagerMBean.registerMBean(); + + dataStore = builder.getDistributedDataStore(); + } + + @Override + public void preStart() { + LOG.info("Starting ShardManager {}", persistenceId); } @Override @@ -186,7 +220,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } @Override - public void handleCommand(Object message) throws Exception { + public void handleCommand(final Object message) throws Exception { if (message instanceof FindPrimary) { findPrimary((FindPrimary)message); } else if (message instanceof FindLocalShard) { @@ -220,21 +254,32 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } else if (message instanceof SwitchShardBehavior) { onSwitchShardBehavior((SwitchShardBehavior) message); } else if (message instanceof CreateShard) { - onCreateShard((CreateShard) message); + onCreateShard((CreateShard)message); } else if (message instanceof AddShardReplica) { onAddShardReplica((AddShardReplica) message); + } else if (message instanceof AddPrefixShardReplica) { + onAddPrefixShardReplica((AddPrefixShardReplica) message); + } else if (message instanceof PrefixShardCreated) { + onPrefixShardCreated((PrefixShardCreated) message); + } else if (message instanceof PrefixShardRemoved) { + onPrefixShardRemoved((PrefixShardRemoved) message); + } else if (message instanceof InitConfigListener) { + onInitConfigListener(); } else if (message instanceof ForwardedAddServerReply) { - ForwardedAddServerReply msg = (ForwardedAddServerReply) message; - onAddServerReply(msg.shardInfo, msg.addServerReply, getSender(), msg.leaderPath, msg.removeShardOnFailure); + ForwardedAddServerReply msg = (ForwardedAddServerReply)message; + onAddServerReply(msg.shardInfo, msg.addServerReply, getSender(), msg.leaderPath, + msg.removeShardOnFailure); } else if (message instanceof ForwardedAddServerFailure) { - ForwardedAddServerFailure msg = (ForwardedAddServerFailure) message; + ForwardedAddServerFailure msg = (ForwardedAddServerFailure)message; onAddServerFailure(msg.shardName, msg.failureMessage, msg.failure, getSender(), msg.removeShardOnFailure); } else if (message instanceof RemoveShardReplica) { onRemoveShardReplica((RemoveShardReplica) message); + } else if (message instanceof RemovePrefixShardReplica) { + onRemovePrefixShardReplica((RemovePrefixShardReplica) message); } else if (message instanceof WrappedShardResponse) { onWrappedShardResponse((WrappedShardResponse) message); } else if (message instanceof GetSnapshot) { - onGetSnapshot(); + onGetSnapshot((GetSnapshot) message); } else if (message instanceof ServerRemoved) { onShardReplicaRemoved((ServerRemoved) message); } else if (message instanceof ChangeShardMembersVotingStatus) { @@ -250,14 +295,69 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { onShutDown(); } else if (message instanceof GetLocalShardIds) { onGetLocalShardIds(); + } else if (message instanceof GetShardRole) { + onGetShardRole((GetShardRole) message); } else if (message instanceof RunnableMessage) { ((RunnableMessage)message).run(); + } else if (message instanceof RegisterForShardAvailabilityChanges) { + onRegisterForShardAvailabilityChanges((RegisterForShardAvailabilityChanges)message); + } else if (message instanceof DeleteSnapshotsFailure) { + LOG.warn("{}: Failed to delete prior snapshots", persistenceId(), + ((DeleteSnapshotsFailure) message).cause()); + } else if (message instanceof DeleteSnapshotsSuccess) { + LOG.debug("{}: Successfully deleted prior snapshots", persistenceId()); + } else if (message instanceof RegisterRoleChangeListenerReply) { + LOG.trace("{}: Received RegisterRoleChangeListenerReply", persistenceId()); + } else if (message instanceof ClusterEvent.MemberEvent) { + LOG.trace("{}: Received other ClusterEvent.MemberEvent: {}", persistenceId(), message); } else { unknownMessage(message); } } - private void onShutDown() { + private void onRegisterForShardAvailabilityChanges(final RegisterForShardAvailabilityChanges message) { + LOG.debug("{}: onRegisterForShardAvailabilityChanges: {}", persistenceId(), message); + + final Consumer callback = message.getCallback(); + shardAvailabilityCallbacks.add(callback); + + getSender().tell(new Status.Success((Registration) + () -> executeInSelf(() -> shardAvailabilityCallbacks.remove(callback))), self()); + } + + private void onGetShardRole(final GetShardRole message) { + LOG.debug("{}: onGetShardRole for shard: {}", persistenceId(), message.getName()); + + final String name = message.getName(); + + final ShardInformation shardInformation = localShards.get(name); + + if (shardInformation == null) { + LOG.info("{}: no shard information for {} found", persistenceId(), name); + getSender().tell(new Status.Failure( + new IllegalArgumentException("Shard with name " + name + " not present.")), ActorRef.noSender()); + return; + } + + getSender().tell(new GetShardRoleReply(shardInformation.getRole()), ActorRef.noSender()); + } + + private void onInitConfigListener() { + LOG.debug("{}: Initializing config listener on {}", persistenceId(), cluster.getCurrentMemberName()); + + final org.opendaylight.mdsal.common.api.LogicalDatastoreType datastoreType = + org.opendaylight.mdsal.common.api.LogicalDatastoreType + .valueOf(datastoreContextFactory.getBaseDatastoreContext().getLogicalStoreType().name()); + + if (configUpdateHandler != null) { + configUpdateHandler.close(); + } + + configUpdateHandler = new PrefixedShardConfigUpdateHandler(self(), cluster.getCurrentMemberName()); + configUpdateHandler.initListener(dataStore, datastoreType); + } + + void onShutDown() { List> stopFutures = new ArrayList<>(localShards.size()); for (ShardInformation info : localShards.values()) { if (info.getActor() != null) { @@ -277,7 +377,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { combinedFutures.onComplete(new OnComplete>() { @Override - public void onComplete(Throwable failure, Iterable results) { + public void onComplete(final Throwable failure, final Iterable results) { LOG.debug("{}: All shards shutdown - sending PoisonPill to self", persistenceId()); self().tell(PoisonPill.getInstance(), self()); @@ -286,7 +386,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { LOG.warn("{}: An error occurred attempting to shut down the shards", persistenceId(), failure); } else { int nfailed = 0; - for (Boolean result: results) { + for (Boolean result : results) { if (!result) { nfailed++; } @@ -300,15 +400,15 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { }, dispatcher); } - private void onWrappedShardResponse(WrappedShardResponse message) { + private void onWrappedShardResponse(final WrappedShardResponse message) { if (message.getResponse() instanceof RemoveServerReply) { onRemoveServerReply(getSender(), message.getShardId(), (RemoveServerReply) message.getResponse(), message.getLeaderPath()); } } - private void onRemoveServerReply(ActorRef originalSender, ShardIdentifier shardId, RemoveServerReply replyMsg, - String leaderPath) { + private void onRemoveServerReply(final ActorRef originalSender, final ShardIdentifier shardId, + final RemoveServerReply replyMsg, final String leaderPath) { shardReplicaOperationsInProgress.remove(shardId.getShardName()); LOG.debug("{}: Received {} for shard {}", persistenceId(), replyMsg, shardId.getShardName()); @@ -326,8 +426,10 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } - private void removeShardReplica(RemoveShardReplica contextMessage, final String shardName, final String primaryPath, - final ActorRef sender) { + @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD", + justification = "https://github.com/spotbugs/spotbugs/issues/811") + private void removePrefixShardReplica(final RemovePrefixShardReplica contextMessage, final String shardName, + final String primaryPath, final ActorRef sender) { if (isShardReplicaOperationInProgress(shardName, sender)) { return; } @@ -346,18 +448,19 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { Future futureObj = ask(getContext().actorSelection(primaryPath), new RemoveServer(shardId.toString()), removeServerTimeout); - futureObj.onComplete(new OnComplete() { + futureObj.onComplete(new OnComplete<>() { @Override - public void onComplete(Throwable failure, Object response) { + public void onComplete(final Throwable failure, final Object response) { if (failure != null) { shardReplicaOperationsInProgress.remove(shardName); - String msg = String.format("RemoveServer request to leader %s for shard %s failed", - primaryPath, shardName); - LOG.debug("{}: {}", persistenceId(), msg, failure); + LOG.debug("{}: RemoveServer request to leader {} for shard {} failed", persistenceId(), primaryPath, + shardName, failure); // FAILURE - sender.tell(new Status.Failure(new RuntimeException(msg, failure)), self()); + sender.tell(new Status.Failure(new RuntimeException( + String.format("RemoveServer request to leader %s for shard %s failed", primaryPath, shardName), + failure)), self()); } else { // SUCCESS self().tell(new WrappedShardResponse(shardId, response, primaryPath), sender); @@ -366,21 +469,101 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client)); } - private void onShardReplicaRemoved(ServerRemoved message) { - final ShardIdentifier shardId = new ShardIdentifier.Builder().fromShardIdString(message.getServerId()).build(); - final ShardInformation shardInformation = localShards.remove(shardId.getShardName()); + @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD", + justification = "https://github.com/spotbugs/spotbugs/issues/811") + private void removeShardReplica(final RemoveShardReplica contextMessage, final String shardName, + final String primaryPath, final ActorRef sender) { + if (isShardReplicaOperationInProgress(shardName, sender)) { + return; + } + + shardReplicaOperationsInProgress.add(shardName); + + final ShardIdentifier shardId = getShardIdentifier(contextMessage.getMemberName(), shardName); + + final DatastoreContext datastoreContext = newShardDatastoreContextBuilder(shardName).build(); + + //inform ShardLeader to remove this shard as a replica by sending an RemoveServer message + LOG.debug("{}: Sending RemoveServer message to peer {} for shard {}", persistenceId(), + primaryPath, shardId); + + Timeout removeServerTimeout = new Timeout(datastoreContext.getShardLeaderElectionTimeout().duration()); + Future futureObj = ask(getContext().actorSelection(primaryPath), + new RemoveServer(shardId.toString()), removeServerTimeout); + + futureObj.onComplete(new OnComplete<>() { + @Override + public void onComplete(final Throwable failure, final Object response) { + if (failure != null) { + shardReplicaOperationsInProgress.remove(shardName); + LOG.debug("{}: RemoveServer request to leader {} for shard {} failed", persistenceId(), primaryPath, + shardName, failure); + + // FAILURE + sender.tell(new Status.Failure(new RuntimeException( + String.format("RemoveServer request to leader %s for shard %s failed", primaryPath, shardName), + failure)), self()); + } else { + // SUCCESS + self().tell(new WrappedShardResponse(shardId, response, primaryPath), sender); + } + } + }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client)); + } + + private void onShardReplicaRemoved(final ServerRemoved message) { + removeShard(new ShardIdentifier.Builder().fromShardIdString(message.getServerId()).build()); + } + + @SuppressWarnings("checkstyle:IllegalCatch") + private void removeShard(final ShardIdentifier shardId) { + final String shardName = shardId.getShardName(); + final ShardInformation shardInformation = localShards.remove(shardName); if (shardInformation == null) { LOG.debug("{} : Shard replica {} is not present in list", persistenceId(), shardId.toString()); return; - } else if (shardInformation.getActor() != null) { - LOG.debug("{} : Sending Shutdown to Shard actor {}", persistenceId(), shardInformation.getActor()); - shardInformation.getActor().tell(Shutdown.INSTANCE, self()); } - LOG.debug("{} : Local Shard replica for shard {} has been removed", persistenceId(), shardId.getShardName()); + + final ActorRef shardActor = shardInformation.getActor(); + if (shardActor != null) { + long timeoutInMS = Math.max(shardInformation.getDatastoreContext().getShardRaftConfig() + .getElectionTimeOutInterval().$times(3).toMillis(), 10000); + + LOG.debug("{} : Sending Shutdown to Shard actor {} with {} ms timeout", persistenceId(), shardActor, + timeoutInMS); + + final Future stopFuture = Patterns.gracefulStop(shardActor, + FiniteDuration.apply(timeoutInMS, TimeUnit.MILLISECONDS), Shutdown.INSTANCE); + + final CompositeOnComplete onComplete = new CompositeOnComplete<>() { + @Override + public void onComplete(final Throwable failure, final Boolean result) { + if (failure == null) { + LOG.debug("{} : Successfully shut down Shard actor {}", persistenceId(), shardActor); + } else { + LOG.warn("{}: Failed to shut down Shard actor {}", persistenceId(), shardActor, failure); + } + + self().tell((RunnableMessage) () -> { + // At any rate, invalidate primaryShardInfo cache + primaryShardInfoCache.remove(shardName); + + shardActorsStopping.remove(shardName); + notifyOnCompleteTasks(failure, result); + }, ActorRef.noSender()); + } + }; + + shardActorsStopping.put(shardName, onComplete); + stopFuture.onComplete(onComplete, new Dispatchers(context().system().dispatchers()) + .getDispatcher(Dispatchers.DispatcherType.Client)); + } + + LOG.debug("{} : Local Shard replica for shard {} has been removed", persistenceId(), shardName); persistShardList(); } - private void onGetSnapshot() { + private void onGetSnapshot(final GetSnapshot getSnapshot) { LOG.debug("{}: onGetSnapshot", persistenceId()); List notInitialized = null; @@ -400,22 +583,17 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { return; } - byte[] shardManagerSnapshot = null; - if (currentSnapshot != null) { - shardManagerSnapshot = SerializationUtils.serialize(currentSnapshot); - } - ActorRef replyActor = getContext().actorOf(ShardManagerGetSnapshotReplyActor.props( - new ArrayList<>(localShards.keySet()), type, shardManagerSnapshot , getSender(), persistenceId(), + new ArrayList<>(localShards.keySet()), type, currentSnapshot , getSender(), persistenceId(), datastoreContextFactory.getBaseDatastoreContext().getShardInitializationTimeout().duration())); for (ShardInformation shardInfo: localShards.values()) { - shardInfo.getActor().tell(GetSnapshot.INSTANCE, replyActor); + shardInfo.getActor().tell(getSnapshot, replyActor); } } @SuppressWarnings("checkstyle:IllegalCatch") - private void onCreateShard(CreateShard createShard) { + private void onCreateShard(final CreateShard createShard) { LOG.debug("{}: onCreateShard: {}", persistenceId(), createShard); Object reply; @@ -438,9 +616,92 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } - private void doCreateShard(CreateShard createShard) { - ModuleShardConfiguration moduleShardConfig = createShard.getModuleShardConfig(); - String shardName = moduleShardConfig.getShardName(); + private void onPrefixShardCreated(final PrefixShardCreated message) { + LOG.debug("{}: onPrefixShardCreated: {}", persistenceId(), message); + + final PrefixShardConfiguration config = message.getConfiguration(); + final ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), + ClusterUtils.getCleanShardName(config.getPrefix().getRootIdentifier())); + final String shardName = shardId.getShardName(); + + if (isPreviousShardActorStopInProgress(shardName, message)) { + return; + } + + if (localShards.containsKey(shardName)) { + LOG.debug("{}: Received create for an already existing shard {}", persistenceId(), shardName); + final PrefixShardConfiguration existing = + configuration.getAllPrefixShardConfigurations().get(config.getPrefix()); + + if (existing != null && existing.equals(config)) { + // we don't have to do nothing here + return; + } + } + + doCreatePrefixShard(config, shardId, shardName); + } + + private boolean isPreviousShardActorStopInProgress(final String shardName, final Object messageToDefer) { + final CompositeOnComplete stopOnComplete = shardActorsStopping.get(shardName); + if (stopOnComplete == null) { + return false; + } + + LOG.debug("{} : Stop is in progress for shard {} - adding OnComplete callback to defer {}", persistenceId(), + shardName, messageToDefer); + final ActorRef sender = getSender(); + stopOnComplete.addOnComplete(new OnComplete() { + @Override + public void onComplete(final Throwable failure, final Boolean result) { + LOG.debug("{} : Stop complete for shard {} - re-queing {}", persistenceId(), shardName, messageToDefer); + self().tell(messageToDefer, sender); + } + }); + + return true; + } + + private void doCreatePrefixShard(final PrefixShardConfiguration config, final ShardIdentifier shardId, + final String shardName) { + configuration.addPrefixShardConfiguration(config); + + final Builder builder = newShardDatastoreContextBuilder(shardName); + builder.logicalStoreType(config.getPrefix().getDatastoreType()) + .storeRoot(config.getPrefix().getRootIdentifier()); + DatastoreContext shardDatastoreContext = builder.build(); + + final Map peerAddresses = getPeerAddresses(shardName); + final boolean isActiveMember = true; + + LOG.debug("{} doCreatePrefixShard: shardId: {}, memberNames: {}, peerAddresses: {}, isActiveMember: {}", + persistenceId(), shardId, config.getShardMemberNames(), peerAddresses, isActiveMember); + + final ShardInformation info = new ShardInformation(shardName, shardId, peerAddresses, + shardDatastoreContext, Shard.builder(), peerAddressResolver); + info.setActiveMember(isActiveMember); + localShards.put(info.getShardName(), info); + + if (schemaContext != null) { + info.setSchemaContext(schemaContext); + info.setActor(newShardActor(info)); + } + } + + private void onPrefixShardRemoved(final PrefixShardRemoved message) { + LOG.debug("{}: onPrefixShardRemoved : {}", persistenceId(), message); + + final DOMDataTreeIdentifier prefix = message.getPrefix(); + final ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), + ClusterUtils.getCleanShardName(prefix.getRootIdentifier())); + + configuration.removePrefixShardConfiguration(prefix); + removeShard(shardId); + } + + private void doCreateShard(final CreateShard createShard) { + final ModuleShardConfiguration moduleShardConfig = createShard.getModuleShardConfig(); + final String shardName = moduleShardConfig.getShardName(); configuration.addModuleShardConfiguration(moduleShardConfig); @@ -484,16 +745,17 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { localShards.put(info.getShardName(), info); if (schemaContext != null) { - info.setActor(newShardActor(schemaContext, info)); + info.setSchemaContext(schemaContext); + info.setActor(newShardActor(info)); } } - private DatastoreContext.Builder newShardDatastoreContextBuilder(String shardName) { + private DatastoreContext.Builder newShardDatastoreContextBuilder(final String shardName) { return DatastoreContext.newBuilderFrom(datastoreContextFactory.getShardDatastoreContext(shardName)) .shardPeerAddressResolver(peerAddressResolver); } - private DatastoreContext newShardDatastoreContext(String shardName) { + private DatastoreContext newShardDatastoreContext(final String shardName) { return newShardDatastoreContextBuilder(shardName).build(); } @@ -506,7 +768,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } - private void onLeaderStateChanged(ShardLeaderStateChanged leaderStateChanged) { + private void onLeaderStateChanged(final ShardLeaderStateChanged leaderStateChanged) { LOG.info("{}: Received LeaderStateChanged message: {}", persistenceId(), leaderStateChanged); ShardInformation shardInformation = findShardInformation(leaderStateChanged.getMemberId()); @@ -515,6 +777,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { shardInformation.setLeaderVersion(leaderStateChanged.getLeaderPayloadVersion()); if (shardInformation.setLeaderId(leaderStateChanged.getLeaderId())) { primaryShardInfoCache.remove(shardInformation.getShardName()); + + notifyShardAvailabilityCallbacks(shardInformation); } checkReady(); @@ -523,7 +787,11 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } - private void onShardNotInitializedTimeout(ShardNotInitializedTimeout message) { + private void notifyShardAvailabilityCallbacks(final ShardInformation shardInformation) { + shardAvailabilityCallbacks.forEach(callback -> callback.accept(shardInformation.getShardName())); + } + + private void onShardNotInitializedTimeout(final ShardNotInitializedTimeout message) { ShardInformation shardInfo = message.getShardInfo(); LOG.debug("{}: Received ShardNotInitializedTimeout message for shard {}", persistenceId(), @@ -536,11 +804,11 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { message.getSender().tell(createNotInitializedException(shardInfo.getShardId()), getSelf()); } else { LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(), shardInfo.getShardName()); - message.getSender().tell(createNoShardLeaderException(shardInfo.getShardId()), getSelf()); + message.getSender().tell(new NoShardLeaderException(shardInfo.getShardId()), getSelf()); } } - private void onFollowerInitialSyncStatus(FollowerInitialSyncUpStatus status) { + private void onFollowerInitialSyncStatus(final FollowerInitialSyncUpStatus status) { LOG.info("{} Received follower initial sync status for {} status sync done {}", persistenceId(), status.getName(), status.isInitialSyncDone()); @@ -554,7 +822,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } - private void onRoleChangeNotification(RoleChangeNotification roleChanged) { + private void onRoleChangeNotification(final RoleChangeNotification roleChanged) { LOG.info("{}: Received role changed for {} from {} to {}", persistenceId(), roleChanged.getMemberId(), roleChanged.getOldRole(), roleChanged.getNewRole()); @@ -567,7 +835,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } - private ShardInformation findShardInformation(String memberId) { + private ShardInformation findShardInformation(final String memberId) { for (ShardInformation info : localShards.values()) { if (info.getShardId().toString().equals(memberId)) { return info; @@ -597,11 +865,12 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { return true; } - private void onActorInitialized(Object message) { + private void onActorInitialized(final Object message) { final ActorRef sender = getSender(); if (sender == null) { - return; //why is a non-actor sending this message? Just ignore. + // why is a non-actor sending this message? Just ignore. + return; } String actorName = sender.path().name(); @@ -611,14 +880,14 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { try { shardId = ShardIdentifier.fromShardIdString(actorName); } catch (IllegalArgumentException e) { - LOG.debug("{}: ignoring actor {}", actorName, e); + LOG.debug("{}: ignoring actor {}", persistenceId, actorName, e); return; } markShardAsInitialized(shardId.getShardName()); } - private void markShardAsInitialized(String shardName) { + private void markShardAsInitialized(final String shardName) { LOG.debug("{}: Initializing shard [{}]", persistenceId(), shardName); ShardInformation shardInformation = localShards.get(shardName); @@ -630,7 +899,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } @Override - protected void handleRecover(Object message) throws Exception { + protected void handleRecover(final Object message) throws Exception { if (message instanceof RecoveryCompleted) { onRecoveryCompleted(); } else if (message instanceof SnapshotOffer) { @@ -642,29 +911,20 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { private void onRecoveryCompleted() { LOG.info("Recovery complete : {}", persistenceId()); - // We no longer persist SchemaContext modules so delete all the prior messages from the akka - // journal on upgrade from Helium. - deleteMessages(lastSequenceNr()); - if (currentSnapshot == null && restoreFromSnapshot != null && restoreFromSnapshot.getShardManagerSnapshot() != null) { - try (ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream( - restoreFromSnapshot.getShardManagerSnapshot()))) { - ShardManagerSnapshot snapshot = (ShardManagerSnapshot) ois.readObject(); + ShardManagerSnapshot snapshot = restoreFromSnapshot.getShardManagerSnapshot(); - LOG.debug("{}: Deserialized restored ShardManagerSnapshot: {}", persistenceId(), snapshot); + LOG.debug("{}: Restoring from ShardManagerSnapshot: {}", persistenceId(), snapshot); - applyShardManagerSnapshot(snapshot); - } catch (Exception e) { - LOG.error("{}: Error deserializing restored ShardManagerSnapshot", persistenceId(), e); - } + applyShardManagerSnapshot(snapshot); } createLocalShards(); } - private void sendResponse(ShardInformation shardInformation, boolean doWait, - boolean wantShardReady, final Supplier messageSupplier) { + private void sendResponse(final ShardInformation shardInformation, final boolean doWait, + final boolean wantShardReady, final Supplier messageSupplier) { if (!shardInformation.isShardInitialized() || wantShardReady && !shardInformation.isShardReadyWithLeaderId()) { if (doWait) { final ActorRef sender = getSender(); @@ -687,7 +947,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } LOG.debug("{}: Scheduling {} ms timer to wait for shard {}", persistenceId(), timeout.toMillis(), - shardInformation.getShardName()); + shardInformation); Cancellable timeoutSchedule = getContext().system().scheduler().scheduleOnce( timeout, getSelf(), @@ -703,7 +963,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } else { LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(), shardInformation.getShardName()); - getSender().tell(createNoShardLeaderException(shardInformation.getShardId()), getSelf()); + getSender().tell(new NoShardLeaderException(shardInformation.getShardId()), getSelf()); } return; @@ -712,11 +972,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { getSender().tell(messageSupplier.get(), getSelf()); } - private static NoShardLeaderException createNoShardLeaderException(ShardIdentifier shardId) { - return new NoShardLeaderException(null, shardId.toString()); - } - - private static NotInitializedException createNotInitializedException(ShardIdentifier shardId) { + private static NotInitializedException createNotInitializedException(final ShardIdentifier shardId) { return new NotInitializedException(String.format( "Found primary shard %s but it's not initialized yet. Please try again later", shardId)); } @@ -726,7 +982,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { return MemberName.forName(member.roles().iterator().next()); } - private void memberRemoved(ClusterEvent.MemberRemoved message) { + private void memberRemoved(final ClusterEvent.MemberRemoved message) { MemberName memberName = memberToName(message.member()); LOG.info("{}: Received MemberRemoved: memberName: {}, address: {}", persistenceId(), memberName, @@ -739,7 +995,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } - private void memberExited(ClusterEvent.MemberExited message) { + private void memberExited(final ClusterEvent.MemberExited message) { MemberName memberName = memberToName(message.member()); LOG.info("{}: Received MemberExited: memberName: {}, address: {}", persistenceId(), memberName, @@ -752,7 +1008,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } - private void memberUp(ClusterEvent.MemberUp message) { + private void memberUp(final ClusterEvent.MemberUp message) { MemberName memberName = memberToName(message.member()); LOG.info("{}: Received MemberUp: memberName: {}, address: {}", persistenceId(), memberName, @@ -761,12 +1017,12 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { memberUp(memberName, message.member().address()); } - private void memberUp(MemberName memberName, Address address) { + private void memberUp(final MemberName memberName, final Address address) { addPeerAddress(memberName, address); checkReady(); } - private void memberWeaklyUp(MemberWeaklyUp message) { + private void memberWeaklyUp(final MemberWeaklyUp message) { MemberName memberName = memberToName(message.member()); LOG.info("{}: Received MemberWeaklyUp: memberName: {}, address: {}", persistenceId(), memberName, @@ -775,7 +1031,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { memberUp(memberName, message.member().address()); } - private void addPeerAddress(MemberName memberName, Address address) { + private void addPeerAddress(final MemberName memberName, final Address address) { peerAddressResolver.addPeerAddress(memberName, address); for (ShardInformation info : localShards.values()) { @@ -787,7 +1043,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } - private void memberReachable(ClusterEvent.ReachableMember message) { + private void memberReachable(final ClusterEvent.ReachableMember message) { MemberName memberName = memberToName(message.member()); LOG.info("Received ReachableMember: memberName {}, address: {}", memberName, message.member().address()); @@ -796,7 +1052,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { markMemberAvailable(memberName); } - private void memberUnreachable(ClusterEvent.UnreachableMember message) { + private void memberUnreachable(final ClusterEvent.UnreachableMember message) { MemberName memberName = memberToName(message.member()); LOG.info("Received UnreachableMember: memberName {}, address: {}", memberName, message.member().address()); @@ -804,15 +1060,15 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } private void markMemberUnavailable(final MemberName memberName) { - final String memberStr = memberName.getName(); for (ShardInformation info : localShards.values()) { String leaderId = info.getLeaderId(); - // XXX: why are we using String#contains() here? - if (leaderId != null && leaderId.contains(memberStr)) { + if (leaderId != null && ShardIdentifier.fromShardIdString(leaderId).getMemberName().equals(memberName)) { LOG.debug("Marking Leader {} as unavailable.", leaderId); info.setLeaderAvailable(false); primaryShardInfoCache.remove(info.getShardName()); + + notifyShardAvailabilityCallbacks(info); } info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf()); @@ -820,11 +1076,9 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } private void markMemberAvailable(final MemberName memberName) { - final String memberStr = memberName.getName(); for (ShardInformation info : localShards.values()) { String leaderId = info.getLeaderId(); - // XXX: why are we using String#contains() here? - if (leaderId != null && leaderId.contains(memberStr)) { + if (leaderId != null && ShardIdentifier.fromShardIdString(leaderId).getMemberName().equals(memberName)) { LOG.debug("Marking Leader {} as available.", leaderId); info.setLeaderAvailable(true); } @@ -833,7 +1087,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } - private void onDatastoreContextFactory(DatastoreContextFactory factory) { + private void onDatastoreContextFactory(final DatastoreContextFactory factory) { datastoreContextFactory = factory; for (ShardInformation info : localShards.values()) { info.setDatastoreContext(newShardDatastoreContext(info.getShardName()), getSelf()); @@ -887,14 +1141,27 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { * @param message the message to send */ private void updateSchemaContext(final Object message) { - schemaContext = ((UpdateSchemaContext) message).getSchemaContext(); + schemaContext = ((UpdateSchemaContext) message).getEffectiveModelContext(); - LOG.debug("Got updated SchemaContext: # of modules {}", schemaContext.getAllModuleIdentifiers().size()); + LOG.debug("Got updated SchemaContext: # of modules {}", schemaContext.getModules().size()); for (ShardInformation info : localShards.values()) { + info.setSchemaContext(schemaContext); + if (info.getActor() == null) { LOG.debug("Creating Shard {}", info.getShardId()); - info.setActor(newShardActor(schemaContext, info)); + info.setActor(newShardActor(info)); + // Update peer address for every existing peer memeber to avoid missing sending + // PeerAddressResolved and PeerUp to this shard while UpdateSchemaContext comes after MemberUp. + String shardName = info.getShardName(); + for (MemberName memberName : peerAddressResolver.getPeerMembers()) { + String peerId = getShardIdentifier(memberName, shardName).toString() ; + String peerAddress = peerAddressResolver.getShardActorAddress(shardName, memberName); + info.updatePeerAddress(peerId, peerAddress, getSelf()); + info.peerUp(memberName, peerId, getSelf()); + LOG.debug("{}: updated peer {} on member {} with address {} on shard {} whose actor address is {}", + persistenceId(), peerId, memberName, peerAddress, info.getShardId(), info.getActor()); + } } else { info.getActor().tell(message, getSelf()); } @@ -907,12 +1174,12 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } @VisibleForTesting - protected ActorRef newShardActor(final SchemaContext schemaContext, ShardInformation info) { - return getContext().actorOf(info.newProps(schemaContext) - .withDispatcher(shardDispatcherPath), info.getShardId().toString()); + protected ActorRef newShardActor(final ShardInformation info) { + return getContext().actorOf(info.newProps().withDispatcher(shardDispatcherPath), + info.getShardId().toString()); } - private void findPrimary(FindPrimary message) { + private void findPrimary(final FindPrimary message) { LOG.debug("{}: In findPrimary: {}", persistenceId(), message); final String shardName = message.getShardName(); @@ -967,9 +1234,9 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { .getShardInitializationTimeout().duration().$times(2)); Future futureObj = ask(getSelf(), new FindPrimary(shardName, true), findPrimaryTimeout); - futureObj.onComplete(new OnComplete() { + futureObj.onComplete(new OnComplete<>() { @Override - public void onComplete(Throwable failure, Object response) { + public void onComplete(final Throwable failure, final Object response) { if (failure != null) { handler.onFailure(failure); } else { @@ -993,7 +1260,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { * @param shardName the shard name * @return a b */ - private ShardIdentifier getShardIdentifier(MemberName memberName, String shardName) { + private ShardIdentifier getShardIdentifier(final MemberName memberName, final String shardName) { return peerAddressResolver.getShardIdentifier(memberName, shardName); } @@ -1011,7 +1278,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } - restoreFromSnapshot = null; // null out to GC + // null out to GC + restoreFromSnapshot = null; for (String shardName : memberShardNames) { ShardIdentifier shardId = getShardIdentifier(memberName, shardName); @@ -1019,21 +1287,33 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { LOG.debug("{}: Creating local shard: {}", persistenceId(), shardId); Map peerAddresses = getPeerAddresses(shardName); - localShards.put(shardName, new ShardInformation(shardName, shardId, peerAddresses, - newShardDatastoreContext(shardName), Shard.builder().restoreFromSnapshot( - shardSnapshots.get(shardName)), peerAddressResolver)); + localShards.put(shardName, createShardInfoFor(shardName, shardId, peerAddresses, + newShardDatastoreContext(shardName), shardSnapshots)); } } + @VisibleForTesting + ShardInformation createShardInfoFor(final String shardName, final ShardIdentifier shardId, + final Map peerAddresses, + final DatastoreContext datastoreContext, + final Map shardSnapshots) { + return new ShardInformation(shardName, shardId, peerAddresses, + datastoreContext, Shard.builder().restoreFromSnapshot(shardSnapshots.get(shardName)), + peerAddressResolver); + } + /** * Given the name of the shard find the addresses of all it's peers. * * @param shardName the shard name */ - private Map getPeerAddresses(String shardName) { - Collection members = configuration.getMembersFromShardName(shardName); - Map peerAddresses = new HashMap<>(); + Map getPeerAddresses(final String shardName) { + final Collection members = configuration.getMembersFromShardName(shardName); + return getPeerAddresses(shardName, members); + } + private Map getPeerAddresses(final String shardName, final Collection members) { + Map peerAddresses = new HashMap<>(); MemberName currentMemberName = this.cluster.getCurrentMemberName(); for (MemberName memberName : members) { @@ -1049,7 +1329,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { @Override public SupervisorStrategy supervisorStrategy() { - return new OneForOneStrategy(10, Duration.create("1 minute"), + return new OneForOneStrategy(10, FiniteDuration.create(1, TimeUnit.MINUTES), (Function) t -> { LOG.warn("Supervisor Strategy caught unexpected exception - resuming", t); return SupervisorStrategy.resume(); @@ -1068,15 +1348,50 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { private boolean isShardReplicaOperationInProgress(final String shardName, final ActorRef sender) { if (shardReplicaOperationsInProgress.contains(shardName)) { - String msg = String.format("A shard replica operation for %s is already in progress", shardName); - LOG.debug("{}: {}", persistenceId(), msg); - sender.tell(new Status.Failure(new IllegalStateException(msg)), getSelf()); + LOG.debug("{}: A shard replica operation for {} is already in progress", persistenceId(), shardName); + sender.tell(new Status.Failure(new IllegalStateException( + String.format("A shard replica operation for %s is already in progress", shardName))), getSelf()); return true; } return false; } + private void onAddPrefixShardReplica(final AddPrefixShardReplica message) { + LOG.debug("{}: onAddPrefixShardReplica: {}", persistenceId(), message); + + final ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), + ClusterUtils.getCleanShardName(message.getShardPrefix())); + final String shardName = shardId.getShardName(); + + // Create the localShard + if (schemaContext == null) { + LOG.debug("{}: No SchemaContext is available in order to create a local shard instance for {}", + persistenceId(), shardName); + getSender().tell(new Status.Failure(new IllegalStateException( + "No SchemaContext is available in order to create a local shard instance for " + shardName)), + getSelf()); + return; + } + + findPrimary(shardName, new AutoFindPrimaryFailureResponseHandler(getSender(), shardName, persistenceId(), + getSelf()) { + @Override + public void onRemotePrimaryShardFound(final RemotePrimaryShardFound response) { + final RunnableMessage runnable = (RunnableMessage) () -> addPrefixShard(getShardName(), + message.getShardPrefix(), response, getSender()); + if (!isPreviousShardActorStopInProgress(getShardName(), runnable)) { + getSelf().tell(runnable, getTargetActor()); + } + } + + @Override + public void onLocalPrimaryFound(final LocalPrimaryShardFound message) { + sendLocalReplicaAlreadyExistsReply(getShardName(), getTargetActor()); + } + }); + } + private void onAddShardReplica(final AddShardReplica shardReplicaMsg) { final String shardName = shardReplicaMsg.getShardName(); @@ -1084,43 +1399,86 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { // verify the shard with the specified name is present in the cluster configuration if (!this.configuration.isShardConfigured(shardName)) { - String msg = String.format("No module configuration exists for shard %s", shardName); - LOG.debug("{}: {}", persistenceId(), msg); - getSender().tell(new Status.Failure(new IllegalArgumentException(msg)), getSelf()); + LOG.debug("{}: No module configuration exists for shard {}", persistenceId(), shardName); + getSender().tell(new Status.Failure(new IllegalArgumentException( + "No module configuration exists for shard " + shardName)), getSelf()); return; } // Create the localShard if (schemaContext == null) { - String msg = String.format( - "No SchemaContext is available in order to create a local shard instance for %s", shardName); - LOG.debug("{}: {}", persistenceId(), msg); - getSender().tell(new Status.Failure(new IllegalStateException(msg)), getSelf()); + LOG.debug("{}: No SchemaContext is available in order to create a local shard instance for {}", + persistenceId(), shardName); + getSender().tell(new Status.Failure(new IllegalStateException( + "No SchemaContext is available in order to create a local shard instance for " + shardName)), + getSelf()); return; } findPrimary(shardName, new AutoFindPrimaryFailureResponseHandler(getSender(), shardName, persistenceId(), getSelf()) { @Override - public void onRemotePrimaryShardFound(RemotePrimaryShardFound response) { - getSelf().tell((RunnableMessage) () -> addShard(getShardName(), response, getSender()), - getTargetActor()); + public void onRemotePrimaryShardFound(final RemotePrimaryShardFound response) { + final RunnableMessage runnable = (RunnableMessage) () -> + addShard(getShardName(), response, getSender()); + if (!isPreviousShardActorStopInProgress(getShardName(), runnable)) { + getSelf().tell(runnable, getTargetActor()); + } } @Override - public void onLocalPrimaryFound(LocalPrimaryShardFound message) { + public void onLocalPrimaryFound(final LocalPrimaryShardFound message) { sendLocalReplicaAlreadyExistsReply(getShardName(), getTargetActor()); } - }); } - private void sendLocalReplicaAlreadyExistsReply(String shardName, ActorRef sender) { - String msg = String.format("Local shard %s already exists", shardName); - LOG.debug("{}: {}", persistenceId(), msg); - sender.tell(new Status.Failure(new AlreadyExistsException(msg)), getSelf()); + @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD", + justification = "https://github.com/spotbugs/spotbugs/issues/811") + private void sendLocalReplicaAlreadyExistsReply(final String shardName, final ActorRef sender) { + LOG.debug("{}: Local shard {} already exists", persistenceId(), shardName); + sender.tell(new Status.Failure(new AlreadyExistsException( + String.format("Local shard %s already exists", shardName))), getSelf()); } + @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD", + justification = "https://github.com/spotbugs/spotbugs/issues/811") + private void addPrefixShard(final String shardName, final YangInstanceIdentifier shardPrefix, + final RemotePrimaryShardFound response, final ActorRef sender) { + if (isShardReplicaOperationInProgress(shardName, sender)) { + return; + } + + shardReplicaOperationsInProgress.add(shardName); + + final ShardInformation shardInfo; + final boolean removeShardOnFailure; + ShardInformation existingShardInfo = localShards.get(shardName); + if (existingShardInfo == null) { + removeShardOnFailure = true; + ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), shardName); + + final Builder builder = newShardDatastoreContextBuilder(shardName); + builder.storeRoot(shardPrefix).customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName()); + + DatastoreContext datastoreContext = builder.build(); + + shardInfo = new ShardInformation(shardName, shardId, getPeerAddresses(shardName), datastoreContext, + Shard.builder(), peerAddressResolver); + shardInfo.setActiveMember(false); + shardInfo.setSchemaContext(schemaContext); + localShards.put(shardName, shardInfo); + shardInfo.setActor(newShardActor(shardInfo)); + } else { + removeShardOnFailure = false; + shardInfo = existingShardInfo; + } + + execAddShard(shardName, shardInfo, response, removeShardOnFailure, sender); + } + + @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD", + justification = "https://github.com/spotbugs/spotbugs/issues/811") private void addShard(final String shardName, final RemotePrimaryShardFound response, final ActorRef sender) { if (isShardReplicaOperationInProgress(shardName, sender)) { return; @@ -1141,32 +1499,43 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { shardInfo = new ShardInformation(shardName, shardId, getPeerAddresses(shardName), datastoreContext, Shard.builder(), peerAddressResolver); shardInfo.setActiveMember(false); + shardInfo.setSchemaContext(schemaContext); localShards.put(shardName, shardInfo); - shardInfo.setActor(newShardActor(schemaContext, shardInfo)); + shardInfo.setActor(newShardActor(shardInfo)); } else { removeShardOnFailure = false; shardInfo = existingShardInfo; } - String localShardAddress = peerAddressResolver.getShardActorAddress(shardName, cluster.getCurrentMemberName()); + execAddShard(shardName, shardInfo, response, removeShardOnFailure, sender); + } + + private void execAddShard(final String shardName, + final ShardInformation shardInfo, + final RemotePrimaryShardFound response, + final boolean removeShardOnFailure, + final ActorRef sender) { + + final String localShardAddress = + peerAddressResolver.getShardActorAddress(shardName, cluster.getCurrentMemberName()); //inform ShardLeader to add this shard as a replica by sending an AddServer message LOG.debug("{}: Sending AddServer message to peer {} for shard {}", persistenceId(), response.getPrimaryPath(), shardInfo.getShardId()); - Timeout addServerTimeout = new Timeout(shardInfo.getDatastoreContext() + final Timeout addServerTimeout = new Timeout(shardInfo.getDatastoreContext() .getShardLeaderElectionTimeout().duration()); - Future futureObj = ask(getContext().actorSelection(response.getPrimaryPath()), - new AddServer(shardInfo.getShardId().toString(), localShardAddress, true), addServerTimeout); + final Future futureObj = ask(getContext().actorSelection(response.getPrimaryPath()), + new AddServer(shardInfo.getShardId().toString(), localShardAddress, true), addServerTimeout); - futureObj.onComplete(new OnComplete() { + futureObj.onComplete(new OnComplete<>() { @Override - public void onComplete(Throwable failure, Object addServerResponse) { + public void onComplete(final Throwable failure, final Object addServerResponse) { if (failure != null) { LOG.debug("{}: AddServer request to {} for {} failed", persistenceId(), response.getPrimaryPath(), shardName, failure); - String msg = String.format("AddServer request to leader %s for shard %s failed", + final String msg = String.format("AddServer request to leader %s for shard %s failed", response.getPrimaryPath(), shardName); self().tell(new ForwardedAddServerFailure(shardName, msg, failure, removeShardOnFailure), sender); } else { @@ -1177,8 +1546,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client)); } - private void onAddServerFailure(String shardName, String message, Throwable failure, ActorRef sender, - boolean removeShardOnFailure) { + private void onAddServerFailure(final String shardName, final String message, final Throwable failure, + final ActorRef sender, final boolean removeShardOnFailure) { shardReplicaOperationsInProgress.remove(shardName); if (removeShardOnFailure) { @@ -1192,8 +1561,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { new RuntimeException(message, failure)), getSelf()); } - private void onAddServerReply(ShardInformation shardInfo, AddServerReply replyMsg, ActorRef sender, - String leaderPath, boolean removeShardOnFailure) { + private void onAddServerReply(final ShardInformation shardInfo, final AddServerReply replyMsg, + final ActorRef sender, final String leaderPath, final boolean removeShardOnFailure) { String shardName = shardInfo.getShardName(); shardReplicaOperationsInProgress.remove(shardName); @@ -1221,29 +1590,23 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } - private static Exception getServerChangeException(Class serverChange, ServerChangeStatus serverChangeStatus, - String leaderPath, ShardIdentifier shardId) { - Exception failure; + private static Exception getServerChangeException(final Class serverChange, + final ServerChangeStatus serverChangeStatus, final String leaderPath, final ShardIdentifier shardId) { switch (serverChangeStatus) { case TIMEOUT: - failure = new TimeoutException(String.format( + return new TimeoutException(String.format( "The shard leader %s timed out trying to replicate the initial data to the new shard %s." + "Possible causes - there was a problem replicating the data or shard leadership changed " + "while replicating the shard data", leaderPath, shardId.getShardName())); - break; case NO_LEADER: - failure = createNoShardLeaderException(shardId); - break; + return new NoShardLeaderException(shardId); case NOT_SUPPORTED: - failure = new UnsupportedOperationException(String.format("%s request is not supported for shard %s", + return new UnsupportedOperationException(String.format("%s request is not supported for shard %s", serverChange.getSimpleName(), shardId.getShardName())); - break; default : - failure = new RuntimeException(String.format( - "%s request to leader %s for shard %s failed with status %s", + return new RuntimeException(String.format("%s request to leader %s for shard %s failed with status %s", serverChange.getSimpleName(), leaderPath, shardId.getShardName(), serverChangeStatus)); } - return failure; } private void onRemoveShardReplica(final RemoveShardReplica shardReplicaMsg) { @@ -1252,12 +1615,12 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { findPrimary(shardReplicaMsg.getShardName(), new AutoFindPrimaryFailureResponseHandler(getSender(), shardReplicaMsg.getShardName(), persistenceId(), getSelf()) { @Override - public void onRemotePrimaryShardFound(RemotePrimaryShardFound response) { + public void onRemotePrimaryShardFound(final RemotePrimaryShardFound response) { doRemoveShardReplicaAsync(response.getPrimaryPath()); } @Override - public void onLocalPrimaryFound(LocalPrimaryShardFound response) { + public void onLocalPrimaryFound(final LocalPrimaryShardFound response) { doRemoveShardReplicaAsync(response.getPrimaryPath()); } @@ -1268,6 +1631,32 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { }); } + private void onRemovePrefixShardReplica(final RemovePrefixShardReplica message) { + LOG.debug("{}: onRemovePrefixShardReplica: {}", persistenceId(), message); + + final ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), + ClusterUtils.getCleanShardName(message.getShardPrefix())); + final String shardName = shardId.getShardName(); + + findPrimary(shardName, new AutoFindPrimaryFailureResponseHandler(getSender(), + shardName, persistenceId(), getSelf()) { + @Override + public void onRemotePrimaryShardFound(final RemotePrimaryShardFound response) { + doRemoveShardReplicaAsync(response.getPrimaryPath()); + } + + @Override + public void onLocalPrimaryFound(final LocalPrimaryShardFound response) { + doRemoveShardReplicaAsync(response.getPrimaryPath()); + } + + private void doRemoveShardReplicaAsync(final String primaryPath) { + getSelf().tell((RunnableMessage) () -> removePrefixShardReplica(message, getShardName(), + primaryPath, getSender()), getTargetActor()); + } + }); + } + private void persistShardList() { List shardList = new ArrayList<>(localShards.keySet()); for (ShardInformation shardInfo : localShards.values()) { @@ -1276,15 +1665,17 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } LOG.debug("{}: persisting the shard list {}", persistenceId(), shardList); - saveSnapshot(updateShardManagerSnapshot(shardList)); + saveSnapshot(updateShardManagerSnapshot(shardList, configuration.getAllPrefixShardConfigurations())); } - private ShardManagerSnapshot updateShardManagerSnapshot(List shardList) { - currentSnapshot = new ShardManagerSnapshot(shardList); + private ShardManagerSnapshot updateShardManagerSnapshot( + final List shardList, + final Map allPrefixShardConfigurations) { + currentSnapshot = new ShardManagerSnapshot(shardList, allPrefixShardConfigurations); return currentSnapshot; } - private void applyShardManagerSnapshot(ShardManagerSnapshot snapshot) { + private void applyShardManagerSnapshot(final ShardManagerSnapshot snapshot) { currentSnapshot = snapshot; LOG.debug("{}: onSnapshotOffer: {}", persistenceId(), currentSnapshot); @@ -1308,7 +1699,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } } - private void onSaveSnapshotSuccess(SaveSnapshotSuccess successMessage) { + private void onSaveSnapshotSuccess(final SaveSnapshotSuccess successMessage) { LOG.debug("{} saved ShardManager snapshot successfully. Deleting the prev snapshot if available", persistenceId()); deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), successMessage.metadata().timestamp() - 1, @@ -1332,7 +1723,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { localShardFound.getPath(), getSender())); } - private void onFlipShardMembersVotingStatus(FlipShardMembersVotingStatus flipMembersVotingStatus) { + private void onFlipShardMembersVotingStatus(final FlipShardMembersVotingStatus flipMembersVotingStatus) { LOG.debug("{}: onFlipShardMembersVotingStatus: {}", persistenceId(), flipMembersVotingStatus); ActorRef sender = getSender(); @@ -1341,9 +1732,9 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { Future future = ask(localShardFound.getPath(), GetOnDemandRaftState.INSTANCE, Timeout.apply(30, TimeUnit.SECONDS)); - future.onComplete(new OnComplete() { + future.onComplete(new OnComplete<>() { @Override - public void onComplete(Throwable failure, Object response) { + public void onComplete(final Throwable failure, final Object response) { if (failure != null) { sender.tell(new Status.Failure(new RuntimeException( String.format("Failed to access local shard %s", shardName), failure)), self()); @@ -1352,7 +1743,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { OnDemandRaftState raftState = (OnDemandRaftState) response; Map serverVotingStatusMap = new HashMap<>(); - for ( Entry e: raftState.getPeerVotingStates().entrySet()) { + for (Entry e: raftState.getPeerVotingStates().entrySet()) { serverVotingStatusMap.put(e.getKey(), !e.getValue()); } @@ -1367,10 +1758,15 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } - private void findLocalShard(FindLocalShard message) { + private void findLocalShard(final FindLocalShard message) { + LOG.debug("{}: findLocalShard : {}", persistenceId(), message.getShardName()); + final ShardInformation shardInformation = localShards.get(message.getShardName()); if (shardInformation == null) { + LOG.debug("{}: Local shard {} not found - shards present: {}", + persistenceId(), message.getShardName(), localShards.keySet()); + getSender().tell(new LocalShardNotFound(message.getShardName()), getSelf()); return; } @@ -1385,9 +1781,9 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { .getShardInitializationTimeout().duration().$times(2)); Future futureObj = ask(getSelf(), new FindLocalShard(shardName, true), findLocalTimeout); - futureObj.onComplete(new OnComplete() { + futureObj.onComplete(new OnComplete<>() { @Override - public void onComplete(Throwable failure, Object response) { + public void onComplete(final Throwable failure, final Object response) { if (failure != null) { LOG.debug("{}: Received failure from FindLocalShard for shard {}", persistenceId, shardName, failure); @@ -1398,22 +1794,23 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { getSelf().tell((RunnableMessage) () -> onLocalShardFound.accept((LocalShardFound) response), sender); } else if (response instanceof LocalShardNotFound) { - String msg = String.format("Local shard %s does not exist", shardName); - LOG.debug("{}: {}", persistenceId, msg); - sender.tell(new Status.Failure(new IllegalArgumentException(msg)), self()); + LOG.debug("{}: Local shard {} does not exist", persistenceId, shardName); + sender.tell(new Status.Failure(new IllegalArgumentException( + String.format("Local shard %s does not exist", shardName))), self()); } else { - String msg = String.format("Failed to find local shard %s: received response: %s", - shardName, response); - LOG.debug("{}: {}", persistenceId, msg); - sender.tell(new Status.Failure(response instanceof Throwable ? (Throwable) response : - new RuntimeException(msg)), self()); + LOG.debug("{}: Failed to find local shard {}: received response: {}", persistenceId, shardName, + response); + sender.tell(new Status.Failure(response instanceof Throwable ? (Throwable) response + : new RuntimeException( + String.format("Failed to find local shard %s: received response: %s", shardName, + response))), self()); } } } }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client)); } - private void changeShardMembersVotingStatus(ChangeServersVotingStatus changeServersVotingStatus, + private void changeShardMembersVotingStatus(final ChangeServersVotingStatus changeServersVotingStatus, final String shardName, final ActorRef shardActorRef, final ActorRef sender) { if (isShardReplicaOperationInProgress(shardName, sender)) { return; @@ -1430,15 +1827,16 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { Timeout timeout = new Timeout(datastoreContext.getShardLeaderElectionTimeout().duration().$times(2)); Future futureObj = ask(shardActorRef, changeServersVotingStatus, timeout); - futureObj.onComplete(new OnComplete() { + futureObj.onComplete(new OnComplete<>() { @Override - public void onComplete(Throwable failure, Object response) { + public void onComplete(final Throwable failure, final Object response) { shardReplicaOperationsInProgress.remove(shardName); if (failure != null) { - String msg = String.format("ChangeServersVotingStatus request to local shard %s failed", - shardActorRef.path()); - LOG.debug("{}: {}", persistenceId(), msg, failure); - sender.tell(new Status.Failure(new RuntimeException(msg, failure)), self()); + LOG.debug("{}: ChangeServersVotingStatus request to local shard {} failed", persistenceId(), + shardActorRef.path(), failure); + sender.tell(new Status.Failure(new RuntimeException( + String.format("ChangeServersVotingStatus request to local shard %s failed", + shardActorRef.path()), failure)), self()); } else { LOG.debug("{}: Received {} from local shard {}", persistenceId(), response, shardActorRef.path()); @@ -1469,8 +1867,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { String leaderPath; boolean removeShardOnFailure; - ForwardedAddServerReply(ShardInformation shardInfo, AddServerReply addServerReply, String leaderPath, - boolean removeShardOnFailure) { + ForwardedAddServerReply(final ShardInformation shardInfo, final AddServerReply addServerReply, + final String leaderPath, final boolean removeShardOnFailure) { this.shardInfo = shardInfo; this.addServerReply = addServerReply; this.leaderPath = leaderPath; @@ -1484,8 +1882,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { Throwable failure; boolean removeShardOnFailure; - ForwardedAddServerFailure(String shardName, String failureMessage, Throwable failure, - boolean removeShardOnFailure) { + ForwardedAddServerFailure(final String shardName, final String failureMessage, final Throwable failure, + final boolean removeShardOnFailure) { this.shardName = shardName; this.failureMessage = failureMessage; this.failure = failure; @@ -1497,7 +1895,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { private final Runnable replyRunnable; private Cancellable timeoutSchedule; - OnShardInitialized(Runnable replyRunnable) { + OnShardInitialized(final Runnable replyRunnable) { this.replyRunnable = replyRunnable; } @@ -1509,13 +1907,13 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { return timeoutSchedule; } - void setTimeoutSchedule(Cancellable timeoutSchedule) { + void setTimeoutSchedule(final Cancellable timeoutSchedule) { this.timeoutSchedule = timeoutSchedule; } } static class OnShardReady extends OnShardInitialized { - OnShardReady(Runnable replyRunnable) { + OnShardReady(final Runnable replyRunnable) { super(replyRunnable); } } @@ -1575,12 +1973,12 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { * @param persistenceId The persistenceId for the ShardManager * @param shardManagerActor The ShardManager actor which triggered the call to FindPrimary */ - protected AutoFindPrimaryFailureResponseHandler(ActorRef targetActor, String shardName, String persistenceId, - ActorRef shardManagerActor) { - this.targetActor = Preconditions.checkNotNull(targetActor); - this.shardName = Preconditions.checkNotNull(shardName); - this.persistenceId = Preconditions.checkNotNull(persistenceId); - this.shardManagerActor = Preconditions.checkNotNull(shardManagerActor); + protected AutoFindPrimaryFailureResponseHandler(final ActorRef targetActor, final String shardName, + final String persistenceId, final ActorRef shardManagerActor) { + this.targetActor = requireNonNull(targetActor); + this.shardName = requireNonNull(shardName); + this.persistenceId = requireNonNull(persistenceId); + this.shardManagerActor = requireNonNull(shardManagerActor); } public ActorRef getTargetActor() { @@ -1592,19 +1990,19 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { } @Override - public void onFailure(Throwable failure) { + public void onFailure(final Throwable failure) { LOG.debug("{}: Received failure from FindPrimary for shard {}", persistenceId, shardName, failure); targetActor.tell(new Status.Failure(new RuntimeException( String.format("Failed to find leader for shard %s", shardName), failure)), shardManagerActor); } @Override - public void onUnknownResponse(Object response) { - String msg = String.format("Failed to find leader for shard %s: received response: %s", - shardName, response); - LOG.debug("{}: {}", persistenceId, msg); - targetActor.tell(new Status.Failure(response instanceof Throwable ? (Throwable) response : - new RuntimeException(msg)), shardManagerActor); + public void onUnknownResponse(final Object response) { + LOG.debug("{}: Failed to find leader for shard {}: received response: {}", persistenceId, shardName, + response); + targetActor.tell(new Status.Failure(response instanceof Throwable ? (Throwable) response + : new RuntimeException(String.format("Failed to find leader for shard %s: received response: %s", + shardName, response))), shardManagerActor); } } @@ -1616,7 +2014,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { private final Object response; private final String leaderPath; - WrappedShardResponse(ShardIdentifier shardId, Object response, String leaderPath) { + WrappedShardResponse(final ShardIdentifier shardId, final Object response, final String leaderPath) { this.shardId = shardId; this.response = response; this.leaderPath = leaderPath; @@ -1640,7 +2038,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering { private final ShardInformation shardInfo; private final OnShardInitialized onShardInitialized; - ShardNotInitializedTimeout(ShardInformation shardInfo, OnShardInitialized onShardInitialized, ActorRef sender) { + ShardNotInitializedTimeout(final ShardInformation shardInfo, final OnShardInitialized onShardInitialized, + final ActorRef sender) { this.sender = sender; this.shardInfo = shardInfo; this.onShardInitialized = onShardInitialized;