Remove PeerUp/Down messages
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / shardmanager / ShardManager.java
index d3d8ce39c9a108f0b497b942f9e6e5368f3c6a21..85469b27e8b4fec4fec619c8cce8126bb4ca3c8a 100644 (file)
@@ -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;
@@ -34,7 +35,8 @@ 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 com.google.common.util.concurrent.SettableFuture;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -44,48 +46,44 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.function.Consumer;
 import java.util.function.Supplier;
 import org.opendaylight.controller.cluster.access.concepts.MemberName;
 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActorWithMetering;
-import org.opendaylight.controller.cluster.datastore.AbstractDataStore;
+import org.opendaylight.controller.cluster.common.actor.Dispatchers;
 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.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.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.Dispatchers;
+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;
@@ -105,21 +103,12 @@ 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.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.controller.md.sal.common.api.data.LogicalDatastoreType;
-import org.opendaylight.controller.md.sal.dom.api.DOMDataTreeChangeListener;
-import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
-import org.opendaylight.yangtools.concepts.ListenerRegistration;
-import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
-import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.concepts.Registration;
+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;
 
 /**
@@ -137,7 +126,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<String, ShardInformation> localShards = new HashMap<>();
+    @VisibleForTesting
+    final Map<String, ShardInformation> localShards = new HashMap<>();
 
     // The type of a ShardManager reflects the type of the datastore itself
     // A data store could be of type config/operational
@@ -147,19 +137,21 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
     private final Configuration configuration;
 
-    private final String shardDispatcherPath;
+    @VisibleForTesting
+    final String shardDispatcherPath;
 
     private final ShardManagerInfo shardManagerMBean;
 
     private DatastoreContextFactory datastoreContextFactory;
 
-    private final CountDownLatch waitTillReadyCountdownLatch;
+    private final SettableFuture<Void> readinessFuture;
 
     private final PrimaryShardInfoFutureCache primaryShardInfoCache;
 
-    private final ShardPeerAddressResolver peerAddressResolver;
+    @VisibleForTesting
+    final ShardPeerAddressResolver peerAddressResolver;
 
-    private SchemaContext schemaContext;
+    private EffectiveModelContext schemaContext;
 
     private DatastoreSnapshot restoreFromSnapshot;
 
@@ -167,22 +159,20 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
     private final Set<String> shardReplicaOperationsInProgress = new HashSet<>();
 
-    private final Map<String, Future<Boolean>> shardActorStoppingFutures = new HashMap<>();
+    private final Map<String, CompositeOnComplete<Boolean>> shardActorsStopping = new HashMap<>();
 
-    private final String persistenceId;
-    private final AbstractDataStore dataStore;
+    private final Set<Consumer<String>> shardAvailabilityCallbacks = new HashSet<>();
 
-    private ListenerRegistration<DOMDataTreeChangeListener> configListenerReg = null;
-    private PrefixedShardConfigUpdateHandler configUpdateHandler;
+    private final String persistenceId;
 
-    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.readinessFuture = builder.getReadinessFuture();
         this.primaryShardInfoCache = builder.getPrimaryShardInfoCache();
         this.restoreFromSnapshot = builder.getRestoreFromSnapshot();
 
@@ -198,8 +188,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                 "shard-manager-" + this.type,
                 datastoreContextFactory.getBaseDatastoreContext().getDataStoreMXBeanType());
         shardManagerMBean.registerMBean();
-
-        dataStore = builder.getDistributedDataStore();
     }
 
     @Override
@@ -212,15 +200,10 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         LOG.info("Stopping ShardManager {}", persistenceId());
 
         shardManagerMBean.unregisterMBean();
-
-        if (configListenerReg != null) {
-            configListenerReg.close();
-            configListenerReg = null;
-        }
     }
 
     @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) {
@@ -257,14 +240,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             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,
@@ -274,12 +249,10 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             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) {
@@ -295,13 +268,17 @@ 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(), message);
+            LOG.debug("{}: Successfully deleted prior snapshots", persistenceId());
         } else if (message instanceof RegisterRoleChangeListenerReply) {
             LOG.trace("{}: Received RegisterRoleChangeListenerReply", persistenceId());
         } else if (message instanceof ClusterEvent.MemberEvent) {
@@ -311,22 +288,34 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
-    private void onInitConfigListener() {
-        LOG.debug("{}: Initializing config listener on {}", persistenceId(), cluster.getCurrentMemberName());
+    private void onRegisterForShardAvailabilityChanges(final RegisterForShardAvailabilityChanges message) {
+        LOG.debug("{}: onRegisterForShardAvailabilityChanges: {}", persistenceId(), message);
+
+        final Consumer<String> 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 org.opendaylight.mdsal.common.api.LogicalDatastoreType type =
-                org.opendaylight.mdsal.common.api.LogicalDatastoreType
-                        .valueOf(datastoreContextFactory.getBaseDatastoreContext().getLogicalStoreType().name());
+        final String name = message.getName();
 
-        if (configUpdateHandler != null) {
-            configUpdateHandler.close();
+        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;
         }
 
-        configUpdateHandler = new PrefixedShardConfigUpdateHandler(self(), cluster.getCurrentMemberName());
-        configUpdateHandler.initListener(dataStore, type);
+        getSender().tell(new GetShardRoleReply(shardInformation.getRole()), ActorRef.noSender());
     }
 
-    private void onShutDown() {
+    void onShutDown() {
         List<Future<Boolean>> stopFutures = new ArrayList<>(localShards.size());
         for (ShardInformation info : localShards.values()) {
             if (info.getActor() != null) {
@@ -346,7 +335,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
         combinedFutures.onComplete(new OnComplete<Iterable<Boolean>>() {
             @Override
-            public void onComplete(Throwable failure, Iterable<Boolean> results) {
+            public void onComplete(final Throwable failure, final Iterable<Boolean> results) {
                 LOG.debug("{}: All shards shutdown - sending PoisonPill to self", persistenceId());
 
                 self().tell(PoisonPill.getInstance(), self());
@@ -369,15 +358,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());
@@ -395,48 +384,10 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
-    private void removePrefixShardReplica(final RemovePrefixShardReplica 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<Object> futureObj = ask(getContext().actorSelection(primaryPath),
-                new RemoveServer(shardId.toString()), removeServerTimeout);
-
-        futureObj.onComplete(new OnComplete<Object>() {
-            @Override
-            public void onComplete(Throwable failure, 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);
-
-                    // FAILURE
-                    sender.tell(new Status.Failure(new RuntimeException(msg, failure)), self());
-                } else {
-                    // SUCCESS
-                    self().tell(new WrappedShardResponse(shardId, response, primaryPath), sender);
-                }
-            }
-        }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
-    }
-
-    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 removeShardReplica(final RemoveShardReplica contextMessage, final String shardName,
+            final String primaryPath, final ActorRef sender) {
         if (isShardReplicaOperationInProgress(shardName, sender)) {
             return;
         }
@@ -455,18 +406,18 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         Future<Object> futureObj = ask(getContext().actorSelection(primaryPath),
                 new RemoveServer(shardId.toString()), removeServerTimeout);
 
-        futureObj.onComplete(new OnComplete<Object>() {
+        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);
@@ -475,7 +426,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
     }
 
-    private void onShardReplicaRemoved(ServerRemoved message) {
+    private void onShardReplicaRemoved(final ServerRemoved message) {
         removeShard(new ShardIdentifier.Builder().fromShardIdString(message.getServerId()).build());
     }
 
@@ -490,31 +441,44 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
         final ActorRef shardActor = shardInformation.getActor();
         if (shardActor != null) {
-            LOG.debug("{} : Sending Shutdown to Shard actor {}", persistenceId(), shardActor);
-            FiniteDuration duration = shardInformation.getDatastoreContext().getShardRaftConfig()
-                    .getElectionTimeOutInterval().$times(3);
-            final Future<Boolean> stopFuture = Patterns.gracefulStop(shardActor, duration, Shutdown.INSTANCE);
-            shardActorStoppingFutures.put(shardName, stopFuture);
-            stopFuture.onComplete(new OnComplete<Boolean>() {
+            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<Boolean> stopFuture = Patterns.gracefulStop(shardActor,
+                    FiniteDuration.apply(timeoutInMS, TimeUnit.MILLISECONDS), Shutdown.INSTANCE);
+
+            final CompositeOnComplete<Boolean> onComplete = new CompositeOnComplete<>() {
                 @Override
-                public void onComplete(Throwable failure, Boolean result) {
+                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) () -> shardActorStoppingFutures.remove(shardName),
-                            ActorRef.noSender());
+                    self().tell((RunnableMessage) () -> {
+                        // At any rate, invalidate primaryShardInfo cache
+                        primaryShardInfoCache.remove(shardName);
+
+                        shardActorsStopping.remove(shardName);
+                        notifyOnCompleteTasks(failure, result);
+                    }, ActorRef.noSender());
                 }
-            }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
+            };
+
+            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<String> notInitialized = null;
@@ -539,12 +503,12 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                 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;
@@ -567,87 +531,28 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
-    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);
-    }
-
+    @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
+        justification = "https://github.com/spotbugs/spotbugs/issues/811")
     private boolean isPreviousShardActorStopInProgress(final String shardName, final Object messageToDefer) {
-        final Future<Boolean> stopFuture = shardActorStoppingFutures.get(shardName);
-        if (stopFuture == null) {
+        final CompositeOnComplete<Boolean> stopOnComplete = shardActorsStopping.get(shardName);
+        if (stopOnComplete == null) {
             return false;
         }
 
-        LOG.debug("{} : Stop is in progress for shard {} - adding Future callback to defer {}", persistenceId(),
+        LOG.debug("{} : Stop is in progress for shard {} - adding OnComplete callback to defer {}", persistenceId(),
                 shardName, messageToDefer);
         final ActorRef sender = getSender();
-        stopFuture.onComplete(new OnComplete<Boolean>() {
+        stopOnComplete.addOnComplete(new OnComplete<Boolean>() {
             @Override
-            public void onComplete(Throwable failure, Boolean result) {
+            public void onComplete(final Throwable failure, final Boolean result) {
                 LOG.debug("{} : Stop complete for shard {} - re-queing {}", persistenceId(), shardName, messageToDefer);
                 self().tell(messageToDefer, sender);
             }
-        }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
+        });
 
         return true;
     }
 
-    private void doCreatePrefixShard(PrefixShardConfiguration config, ShardIdentifier shardId, String shardName) {
-        configuration.addPrefixShardConfiguration(config);
-
-        final Builder builder = newShardDatastoreContextBuilder(shardName);
-        builder.logicalStoreType(LogicalDatastoreType.valueOf(config.getPrefix().getDatastoreType().name()))
-                .storeRoot(config.getPrefix().getRootIdentifier());
-        DatastoreContext shardDatastoreContext = builder.build();
-
-        final Map<String, String> 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.setActor(newShardActor(schemaContext, 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();
@@ -694,29 +599,28 @@ 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();
     }
 
     private void checkReady() {
         if (isReadyWithLeaderId()) {
-            LOG.info("{}: All Shards are ready - data store {} is ready, available count is {}",
-                    persistenceId(), type, waitTillReadyCountdownLatch.getCount());
-
-            waitTillReadyCountdownLatch.countDown();
+            LOG.info("{}: All Shards are ready - data store {} is ready", persistenceId(), type);
+            readinessFuture.set(null);
         }
     }
 
-    private void onLeaderStateChanged(ShardLeaderStateChanged leaderStateChanged) {
+    private void onLeaderStateChanged(final ShardLeaderStateChanged leaderStateChanged) {
         LOG.info("{}: Received LeaderStateChanged message: {}", persistenceId(), leaderStateChanged);
 
         ShardInformation shardInformation = findShardInformation(leaderStateChanged.getMemberId());
@@ -725,6 +629,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             shardInformation.setLeaderVersion(leaderStateChanged.getLeaderPayloadVersion());
             if (shardInformation.setLeaderId(leaderStateChanged.getLeaderId())) {
                 primaryShardInfoCache.remove(shardInformation.getShardName());
+
+                notifyShardAvailabilityCallbacks(shardInformation);
             }
 
             checkReady();
@@ -733,7 +639,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(),
@@ -746,11 +656,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());
 
@@ -764,7 +674,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());
 
@@ -777,7 +687,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;
@@ -807,11 +717,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();
@@ -821,14 +732,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);
@@ -840,7 +751,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) {
@@ -864,8 +775,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         createLocalShards();
     }
 
-    private void sendResponse(ShardInformation shardInformation, boolean doWait,
-            boolean wantShardReady, final Supplier<Object> messageSupplier) {
+    private void sendResponse(final ShardInformation shardInformation, final boolean doWait,
+            final boolean wantShardReady, final Supplier<Object> messageSupplier) {
         if (!shardInformation.isShardInitialized() || wantShardReady && !shardInformation.isShardReadyWithLeaderId()) {
             if (doWait) {
                 final ActorRef sender = getSender();
@@ -888,7 +799,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(),
@@ -904,7 +815,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;
@@ -913,11 +824,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));
     }
@@ -927,33 +834,25 @@ 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,
                 message.member().address());
 
         peerAddressResolver.removePeerAddress(memberName);
-
-        for (ShardInformation info : localShards.values()) {
-            info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
-        }
     }
 
-    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,
                 message.member().address());
 
         peerAddressResolver.removePeerAddress(memberName);
-
-        for (ShardInformation info : localShards.values()) {
-            info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
-        }
     }
 
-    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,
@@ -962,12 +861,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,
@@ -976,19 +875,17 @@ 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()) {
             String shardName = info.getShardName();
             String peerId = getShardIdentifier(memberName, shardName).toString();
             info.updatePeerAddress(peerId, peerAddressResolver.getShardActorAddress(shardName, memberName), getSelf());
-
-            info.peerUp(memberName, peerId, getSelf());
         }
     }
 
-    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());
 
@@ -997,7 +894,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());
 
@@ -1005,36 +902,30 @@ 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());
-            }
 
-            info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
+                notifyShardAvailabilityCallbacks(info);
+            }
         }
     }
 
     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);
             }
-
-            info.peerUp(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
         }
     }
 
-    private void onDatastoreContextFactory(DatastoreContextFactory factory) {
+    private void onDatastoreContextFactory(final DatastoreContextFactory factory) {
         datastoreContextFactory = factory;
         for (ShardInformation info : localShards.values()) {
             info.setDatastoreContext(newShardDatastoreContext(info.getShardName()), getSelf());
@@ -1088,14 +979,26 @@ 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());
+                    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());
             }
@@ -1108,12 +1011,12 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     }
 
     @VisibleForTesting
-    protected ActorRef newShardActor(final SchemaContext shardSchemaContext, final ShardInformation info) {
-        return getContext().actorOf(info.newProps(shardSchemaContext).withDispatcher(shardDispatcherPath),
+    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();
@@ -1168,9 +1071,9 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                 .getShardInitializationTimeout().duration().$times(2));
 
         Future<Object> futureObj = ask(getSelf(), new FindPrimary(shardName, true), findPrimaryTimeout);
-        futureObj.onComplete(new OnComplete<Object>() {
+        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 {
@@ -1194,7 +1097,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);
     }
 
@@ -1212,7 +1115,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);
@@ -1220,18 +1124,27 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             LOG.debug("{}: Creating local shard: {}", persistenceId(), shardId);
 
             Map<String, String> 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<String, String> peerAddresses,
+                                        final DatastoreContext datastoreContext,
+                                        final Map<String, DatastoreSnapshot.ShardSnapshot> 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<String, String> getPeerAddresses(String shardName) {
+    Map<String, String> getPeerAddresses(final String shardName) {
         final Collection<MemberName> members = configuration.getMembersFromShardName(shardName);
         return getPeerAddresses(shardName, members);
     }
@@ -1253,7 +1166,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<Throwable, Directive>) t -> {
                     LOG.warn("Supervisor Strategy caught unexpected exception - resuming", t);
                     return SupervisorStrategy.resume();
@@ -1272,49 +1185,15 @@ 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) {
-            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());
-            return;
-        }
-
-        findPrimary(shardName, new AutoFindPrimaryFailureResponseHandler(getSender(), shardName, persistenceId(),
-                getSelf()) {
-            @Override
-            public void onRemotePrimaryShardFound(RemotePrimaryShardFound response) {
-                final RunnableMessage runnable = (RunnableMessage) () -> addPrefixShard(getShardName(),
-                        message.getShardPrefix(), response, getSender());
-                if (!isPreviousShardActorStopInProgress(getShardName(), runnable)) {
-                    getSelf().tell(runnable, getTargetActor());
-                }
-            }
-
-            @Override
-            public void onLocalPrimaryFound(LocalPrimaryShardFound message) {
-                sendLocalReplicaAlreadyExistsReply(getShardName(), getTargetActor());
-            }
-        });
-    }
-
     private void onAddShardReplica(final AddShardReplica shardReplicaMsg) {
         final String shardName = shardReplicaMsg.getShardName();
 
@@ -1322,25 +1201,26 @@ 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) {
+            public void onRemotePrimaryShardFound(final RemotePrimaryShardFound response) {
                 final RunnableMessage runnable = (RunnableMessage) () ->
                     addShard(getShardName(), response, getSender());
                 if (!isPreviousShardActorStopInProgress(getShardName(), runnable)) {
@@ -1349,51 +1229,22 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             }
 
             @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());
-    }
-
-    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);
-            localShards.put(shardName, shardInfo);
-            shardInfo.setActor(newShardActor(schemaContext, 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 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 addShard(final String shardName, final RemotePrimaryShardFound response, final ActorRef sender) {
         if (isShardReplicaOperationInProgress(shardName, sender)) {
             return;
@@ -1414,8 +1265,9 @@ 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;
@@ -1442,9 +1294,9 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         final Future<Object> futureObj = ask(getContext().actorSelection(response.getPrimaryPath()),
                 new AddServer(shardInfo.getShardId().toString(), localShardAddress, true), addServerTimeout);
 
-        futureObj.onComplete(new OnComplete<Object>() {
+        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);
@@ -1460,8 +1312,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) {
@@ -1475,8 +1327,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);
 
@@ -1504,29 +1356,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) {
@@ -1535,12 +1381,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());
             }
 
@@ -1551,32 +1397,6 @@ 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(RemotePrimaryShardFound response) {
-                doRemoveShardReplicaAsync(response.getPrimaryPath());
-            }
-
-            @Override
-            public void onLocalPrimaryFound(LocalPrimaryShardFound response) {
-                doRemoveShardReplicaAsync(response.getPrimaryPath());
-            }
-
-            private void doRemoveShardReplicaAsync(final String primaryPath) {
-                getSelf().tell((RunnableMessage) () -> removePrefixShardReplica(message, getShardName(),
-                        primaryPath, getSender()), getTargetActor());
-            }
-        });
-    }
-
     private void persistShardList() {
         List<String> shardList = new ArrayList<>(localShards.keySet());
         for (ShardInformation shardInfo : localShards.values()) {
@@ -1585,17 +1405,15 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             }
         }
         LOG.debug("{}: persisting the shard list {}", persistenceId(), shardList);
-        saveSnapshot(updateShardManagerSnapshot(shardList, configuration.getAllPrefixShardConfigurations()));
+        saveSnapshot(updateShardManagerSnapshot(shardList));
     }
 
-    private ShardManagerSnapshot updateShardManagerSnapshot(
-            final List<String> shardList,
-            final Map<DOMDataTreeIdentifier, PrefixShardConfiguration> allPrefixShardConfigurations) {
-        currentSnapshot = new ShardManagerSnapshot(shardList, allPrefixShardConfigurations);
+    private ShardManagerSnapshot updateShardManagerSnapshot(final List<String> shardList) {
+        currentSnapshot = new ShardManagerSnapshot(shardList);
         return currentSnapshot;
     }
 
-    private void applyShardManagerSnapshot(ShardManagerSnapshot snapshot) {
+    private void applyShardManagerSnapshot(final ShardManagerSnapshot snapshot) {
         currentSnapshot = snapshot;
 
         LOG.debug("{}: onSnapshotOffer: {}", persistenceId(), currentSnapshot);
@@ -1619,7 +1437,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,
@@ -1643,7 +1461,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();
@@ -1652,9 +1470,9 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             Future<Object> future = ask(localShardFound.getPath(), GetOnDemandRaftState.INSTANCE,
                     Timeout.apply(30, TimeUnit.SECONDS));
 
-            future.onComplete(new OnComplete<Object>() {
+            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());
@@ -1678,7 +1496,7 @@ 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());
@@ -1701,9 +1519,9 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                 .getShardInitializationTimeout().duration().$times(2));
 
         Future<Object> futureObj = ask(getSelf(), new FindLocalShard(shardName, true), findLocalTimeout);
-        futureObj.onComplete(new OnComplete<Object>() {
+        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);
@@ -1714,22 +1532,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;
@@ -1746,15 +1565,16 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         Timeout timeout = new Timeout(datastoreContext.getShardLeaderElectionTimeout().duration().$times(2));
         Future<Object> futureObj = ask(shardActorRef, changeServersVotingStatus, timeout);
 
-        futureObj.onComplete(new OnComplete<Object>() {
+        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());
 
@@ -1785,8 +1605,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;
@@ -1800,8 +1620,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;
@@ -1813,7 +1633,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         private final Runnable replyRunnable;
         private Cancellable timeoutSchedule;
 
-        OnShardInitialized(Runnable replyRunnable) {
+        OnShardInitialized(final Runnable replyRunnable) {
             this.replyRunnable = replyRunnable;
         }
 
@@ -1825,13 +1645,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);
         }
     }
@@ -1891,12 +1711,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() {
@@ -1908,19 +1728,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);
         }
     }
 
@@ -1932,7 +1752,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;
@@ -1956,7 +1776,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;