Adjust to yangtools-2.0.0 changes
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / shardmanager / ShardManager.java
index 7653aea3fb814b2c69e36684800902df013aab72..187d2522de9e7df810a878b8c4605a3e3a44e2eb 100644 (file)
@@ -25,6 +25,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;
@@ -33,8 +35,6 @@ 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 java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -49,39 +49,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,6 +108,15 @@ 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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -151,16 +170,22 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
     private final Set<String> shardReplicaOperationsInProgress = new HashSet<>();
 
+    private final Map<String, CompositeOnComplete<Boolean>> shardActorsStopping = new HashMap<>();
+
     private final String persistenceId;
+    private final AbstractDataStore dataStore;
+
+    private ListenerRegistration<DOMDataTreeChangeListener> configListenerReg = null;
+    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 +201,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
@@ -183,10 +215,15 @@ 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) {
@@ -220,17 +257,28 @@ 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) {
@@ -250,13 +298,56 @@ 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 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);
+        } 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 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);
+    }
+
     private void onShutDown() {
         List<Future<Boolean>> stopFutures = new ArrayList<>(localShards.size());
         for (ShardInformation info : localShards.values()) {
@@ -277,7 +368,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());
@@ -286,7 +377,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 +391,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 +417,48 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
-    private void removeShardReplica(RemoveShardReplica contextMessage, final String shardName, final String primaryPath,
-            final ActorRef sender) {
+    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(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);
+
+                    // 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(final RemoveShardReplica contextMessage, final String shardName,
+            final String primaryPath, final ActorRef sender) {
         if (isShardReplicaOperationInProgress(shardName, sender)) {
             return;
         }
@@ -348,7 +479,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
         futureObj.onComplete(new OnComplete<Object>() {
             @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",
@@ -366,17 +497,55 @@ 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());
+    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<Boolean> stopFuture = Patterns.gracefulStop(shardActor,
+                    FiniteDuration.apply(timeoutInMS, TimeUnit.MILLISECONDS), Shutdown.INSTANCE);
+
+            final CompositeOnComplete<Boolean> onComplete = new CompositeOnComplete<Boolean>() {
+                @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();
     }
 
@@ -400,13 +569,8 @@ 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()) {
@@ -415,7 +579,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     }
 
     @SuppressWarnings("checkstyle:IllegalCatch")
-    private void onCreateShard(CreateShard createShard) {
+    private void onCreateShard(final CreateShard createShard) {
         LOG.debug("{}: onCreateShard: {}", persistenceId(), createShard);
 
         Object reply;
@@ -438,9 +602,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<Boolean> 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<Boolean>() {
+            @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(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.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 +731,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 +754,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());
@@ -523,7 +771,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
-    private void onShardNotInitializedTimeout(ShardNotInitializedTimeout message) {
+    private void onShardNotInitializedTimeout(final ShardNotInitializedTimeout message) {
         ShardInformation shardInfo = message.getShardInfo();
 
         LOG.debug("{}: Received ShardNotInitializedTimeout message for shard {}", persistenceId(),
@@ -540,7 +788,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
-    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 +802,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 +815,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 +845,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();
@@ -618,7 +867,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         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 +879,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 +891,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<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();
@@ -712,11 +952,11 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         getSender().tell(messageSupplier.get(), getSelf());
     }
 
-    private static NoShardLeaderException createNoShardLeaderException(ShardIdentifier shardId) {
+    private static NoShardLeaderException createNoShardLeaderException(final 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 +966,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 +979,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 +992,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 +1001,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 +1015,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 +1027,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 +1036,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,11 +1044,9 @@ 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);
 
@@ -820,11 +1058,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 +1069,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());
@@ -889,12 +1125,25 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     private void updateSchemaContext(final Object message) {
         schemaContext = ((UpdateSchemaContext) message).getSchemaContext();
 
-        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 +1156,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();
@@ -969,7 +1218,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         Future<Object> futureObj = ask(getSelf(), new FindPrimary(shardName, true), findPrimaryTimeout);
         futureObj.onComplete(new OnComplete<Object>() {
             @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 +1242,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 +1260,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);
@@ -1030,10 +1280,13 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
      *
      * @param shardName the shard name
      */
-    private Map<String, String> getPeerAddresses(String shardName) {
-        Collection<MemberName> members = configuration.getMembersFromShardName(shardName);
-        Map<String, String> peerAddresses = new HashMap<>();
+    private Map<String, String> getPeerAddresses(final String shardName) {
+        final Collection<MemberName> members = configuration.getMembersFromShardName(shardName);
+        return getPeerAddresses(shardName, members);
+    }
 
+    private Map<String, String> getPeerAddresses(final String shardName, final Collection<MemberName> members) {
+        Map<String, String> peerAddresses = new HashMap<>();
         MemberName currentMemberName = this.cluster.getCurrentMemberName();
 
         for (MemberName memberName : members) {
@@ -1077,6 +1330,40 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         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(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();
 
@@ -1102,25 +1389,61 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         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) {
+    private void sendLocalReplicaAlreadyExistsReply(final String shardName, final 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);
+            shardInfo.setSchemaContext(schemaContext);
+            localShards.put(shardName, shardInfo);
+            shardInfo.setActor(newShardActor(shardInfo));
+        } else {
+            removeShardOnFailure = false;
+            shardInfo = existingShardInfo;
+        }
+
+        execAddShard(shardName, shardInfo, response, removeShardOnFailure, sender);
+    }
+
     private void addShard(final String shardName, final RemotePrimaryShardFound response, final ActorRef sender) {
         if (isShardReplicaOperationInProgress(shardName, sender)) {
             return;
@@ -1141,32 +1464,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<Object> futureObj = ask(getContext().actorSelection(response.getPrimaryPath()),
-            new AddServer(shardInfo.getShardId().toString(), localShardAddress, true), addServerTimeout);
+        final Future<Object> futureObj = ask(getContext().actorSelection(response.getPrimaryPath()),
+                new AddServer(shardInfo.getShardId().toString(), localShardAddress, true), addServerTimeout);
 
         futureObj.onComplete(new OnComplete<Object>() {
             @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 +1511,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 +1526,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 +1555,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 createNoShardLeaderException(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 +1580,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 +1596,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<String> shardList = new ArrayList<>(localShards.keySet());
         for (ShardInformation shardInfo : localShards.values()) {
@@ -1276,15 +1630,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<String> shardList) {
-        currentSnapshot = new ShardManagerSnapshot(shardList);
+    private ShardManagerSnapshot updateShardManagerSnapshot(
+            final List<String> shardList,
+            final Map<DOMDataTreeIdentifier, PrefixShardConfiguration> 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 +1664,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 +1688,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();
@@ -1343,7 +1699,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
             future.onComplete(new OnComplete<Object>() {
                 @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 +1708,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
                     OnDemandRaftState raftState = (OnDemandRaftState) response;
                     Map<String, Boolean> serverVotingStatusMap = new HashMap<>();
-                    for ( Entry<String, Boolean> e: raftState.getPeerVotingStates().entrySet()) {
+                    for (Entry<String, Boolean> e: raftState.getPeerVotingStates().entrySet()) {
                         serverVotingStatusMap.put(e.getKey(), !e.getValue());
                     }
 
@@ -1367,10 +1723,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;
         }
@@ -1387,7 +1748,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         Future<Object> futureObj = ask(getSelf(), new FindLocalShard(shardName, true), findLocalTimeout);
         futureObj.onComplete(new OnComplete<Object>() {
             @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);
@@ -1413,7 +1774,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }, 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;
@@ -1432,7 +1793,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
         futureObj.onComplete(new OnComplete<Object>() {
             @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",
@@ -1469,8 +1830,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 +1845,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 +1858,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         private final Runnable replyRunnable;
         private Cancellable timeoutSchedule;
 
-        OnShardInitialized(Runnable replyRunnable) {
+        OnShardInitialized(final Runnable replyRunnable) {
             this.replyRunnable = replyRunnable;
         }
 
@@ -1509,13 +1870,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,8 +1936,8 @@ 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) {
+        protected AutoFindPrimaryFailureResponseHandler(final ActorRef targetActor, final String shardName,
+                final String persistenceId, final ActorRef shardManagerActor) {
             this.targetActor = Preconditions.checkNotNull(targetActor);
             this.shardName = Preconditions.checkNotNull(shardName);
             this.persistenceId = Preconditions.checkNotNull(persistenceId);
@@ -1592,14 +1953,14 @@ 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) {
+        public void onUnknownResponse(final Object response) {
             String msg = String.format("Failed to find leader for shard %s: received response: %s",
                     shardName, response);
             LOG.debug("{}: {}", persistenceId, msg);
@@ -1616,7 +1977,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 +2001,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;