Remove PeerUp/Down messages
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / shardmanager / ShardManager.java
index aa3cf3face4cd348500eb7d506774f8fecf8ac30..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,7 +46,6 @@ 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;
@@ -52,22 +53,18 @@ 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.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;
@@ -81,13 +78,11 @@ 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.CompositeOnComplete;
 import org.opendaylight.controller.cluster.datastore.utils.PrimaryShardInfoFutureCache;
 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener;
@@ -108,16 +103,8 @@ 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.dom.api.DOMDataTreeChangeListener;
-import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
-import org.opendaylight.yangtools.concepts.ListenerRegistration;
 import org.opendaylight.yangtools.concepts.Registration;
-import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
-import org.opendaylight.yangtools.yang.model.api.SchemaContext;
+import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.ExecutionContext;
@@ -139,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
@@ -149,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;
 
@@ -174,10 +164,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     private final Set<Consumer<String>> shardAvailabilityCallbacks = new HashSet<>();
 
     private final String persistenceId;
-    private final AbstractDataStore dataStore;
-
-    private ListenerRegistration<DOMDataTreeChangeListener> configListenerReg = null;
-    private PrefixedShardConfigUpdateHandler configUpdateHandler;
 
     ShardManager(final AbstractShardManagerCreator<?> builder) {
         this.cluster = builder.getCluster();
@@ -186,7 +172,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         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();
 
@@ -202,8 +188,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                 "shard-manager-" + this.type,
                 datastoreContextFactory.getBaseDatastoreContext().getDataStoreMXBeanType());
         shardManagerMBean.registerMBean();
-
-        dataStore = builder.getDistributedDataStore();
     }
 
     @Override
@@ -216,11 +200,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         LOG.info("Stopping ShardManager {}", persistenceId());
 
         shardManagerMBean.unregisterMBean();
-
-        if (configListenerReg != null) {
-            configListenerReg.close();
-            configListenerReg = null;
-        }
     }
 
     @Override
@@ -261,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,
@@ -278,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) {
@@ -346,22 +315,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         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() {
+    void onShutDown() {
         List<Future<Boolean>> stopFutures = new ArrayList<>(localShards.size());
         for (ShardInformation info : localShards.values()) {
             if (info.getActor() != null) {
@@ -430,47 +384,8 @@ 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(final Throwable failure, final Object response) {
-                if (failure != null) {
-                    shardReplicaOperationsInProgress.remove(shardName);
-
-                    LOG.debug("{}: RemoveServer request to leader {} for shard {} failed", persistenceId(), primaryPath,
-                        shardName, failure);
-
-                    // FAILURE
-                    sender.tell(new Status.Failure(new RuntimeException(
-                        String.format("RemoveServer request to leader %s for shard %s failed", primaryPath, shardName),
-                        failure)), self());
-                } else {
-                    // SUCCESS
-                    self().tell(new WrappedShardResponse(shardId, response, primaryPath), sender);
-                }
-            }
-        }, new Dispatchers(context().system().dispatchers()).getDispatcher(Dispatchers.DispatcherType.Client));
-    }
-
+    @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)) {
@@ -491,7 +406,7 @@ 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(final Throwable failure, final Object response) {
                 if (failure != null) {
@@ -535,7 +450,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             final Future<Boolean> stopFuture = Patterns.gracefulStop(shardActor,
                     FiniteDuration.apply(timeoutInMS, TimeUnit.MILLISECONDS), Shutdown.INSTANCE);
 
-            final CompositeOnComplete<Boolean> onComplete = new CompositeOnComplete<Boolean>() {
+            final CompositeOnComplete<Boolean> onComplete = new CompositeOnComplete<>() {
                 @Override
                 public void onComplete(final Throwable failure, final Boolean result) {
                     if (failure == null) {
@@ -563,7 +478,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         persistShardList();
     }
 
-    private void onGetSnapshot() {
+    private void onGetSnapshot(final GetSnapshot getSnapshot) {
         LOG.debug("{}: onGetSnapshot", persistenceId());
 
         List<String> notInitialized = null;
@@ -588,7 +503,7 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                 datastoreContextFactory.getBaseDatastoreContext().getShardInitializationTimeout().duration()));
 
         for (ShardInformation shardInfo: localShards.values()) {
-            shardInfo.getActor().tell(GetSnapshot.INSTANCE, replyActor);
+            shardInfo.getActor().tell(getSnapshot, replyActor);
         }
     }
 
@@ -616,32 +531,8 @@ 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 CompositeOnComplete<Boolean> stopOnComplete = shardActorsStopping.get(shardName);
         if (stopOnComplete == null) {
@@ -662,43 +553,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         return true;
     }
 
-    private void doCreatePrefixShard(final PrefixShardConfiguration config, final ShardIdentifier shardId,
-            final String shardName) {
-        configuration.addPrefixShardConfiguration(config);
-
-        final Builder builder = newShardDatastoreContextBuilder(shardName);
-        builder.logicalStoreType(config.getPrefix().getDatastoreType())
-                .storeRoot(config.getPrefix().getRootIdentifier());
-        DatastoreContext shardDatastoreContext = builder.build();
-
-        final Map<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();
@@ -761,10 +615,8 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
     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);
         }
     }
 
@@ -989,10 +841,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                 message.member().address());
 
         peerAddressResolver.removePeerAddress(memberName);
-
-        for (ShardInformation info : localShards.values()) {
-            info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
-        }
     }
 
     private void memberExited(final ClusterEvent.MemberExited message) {
@@ -1002,10 +850,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                 message.member().address());
 
         peerAddressResolver.removePeerAddress(memberName);
-
-        for (ShardInformation info : localShards.values()) {
-            info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
-        }
     }
 
     private void memberUp(final ClusterEvent.MemberUp message) {
@@ -1038,8 +882,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             String shardName = info.getShardName();
             String peerId = getShardIdentifier(memberName, shardName).toString();
             info.updatePeerAddress(peerId, peerAddressResolver.getShardActorAddress(shardName, memberName), getSelf());
-
-            info.peerUp(memberName, peerId, getSelf());
         }
     }
 
@@ -1070,8 +912,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
                 notifyShardAvailabilityCallbacks(info);
             }
-
-            info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
         }
     }
 
@@ -1082,8 +922,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                 LOG.debug("Marking Leader {} as available.", leaderId);
                 info.setLeaderAvailable(true);
             }
-
-            info.peerUp(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
         }
     }
 
@@ -1141,7 +979,7 @@ 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.getModules().size());
 
@@ -1158,7 +996,6 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                     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());
                 }
@@ -1234,7 +1071,7 @@ 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(final Throwable failure, final Object response) {
                 if (failure != null) {
@@ -1287,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(final String shardName) {
+    Map<String, String> getPeerAddresses(final String shardName) {
         final Collection<MemberName> members = configuration.getMembersFromShardName(shardName);
         return getPeerAddresses(shardName, members);
     }
@@ -1348,41 +1194,6 @@ 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) {
-            LOG.debug("{}: No SchemaContext is available in order to create a local shard instance for {}",
-                persistenceId(), shardName);
-            getSender().tell(new Status.Failure(new IllegalStateException(
-                "No SchemaContext is available in order to create a local shard instance for " + shardName)),
-                getSelf());
-            return;
-        }
-
-        findPrimary(shardName, new AutoFindPrimaryFailureResponseHandler(getSender(), shardName, persistenceId(),
-                getSelf()) {
-            @Override
-            public void onRemotePrimaryShardFound(final RemotePrimaryShardFound response) {
-                final RunnableMessage runnable = (RunnableMessage) () -> addPrefixShard(getShardName(),
-                        message.getShardPrefix(), response, getSender());
-                if (!isPreviousShardActorStopInProgress(getShardName(), runnable)) {
-                    getSelf().tell(runnable, getTargetActor());
-                }
-            }
-
-            @Override
-            public void onLocalPrimaryFound(final LocalPrimaryShardFound message) {
-                sendLocalReplicaAlreadyExistsReply(getShardName(), getTargetActor());
-            }
-        });
-    }
-
     private void onAddShardReplica(final AddShardReplica shardReplicaMsg) {
         final String shardName = shardReplicaMsg.getShardName();
 
@@ -1424,46 +1235,16 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         });
     }
 
+    @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());
     }
 
-    private void addPrefixShard(final String shardName, final YangInstanceIdentifier shardPrefix,
-                                final RemotePrimaryShardFound response, final ActorRef sender) {
-        if (isShardReplicaOperationInProgress(shardName, sender)) {
-            return;
-        }
-
-        shardReplicaOperationsInProgress.add(shardName);
-
-        final ShardInformation shardInfo;
-        final boolean removeShardOnFailure;
-        ShardInformation existingShardInfo = localShards.get(shardName);
-        if (existingShardInfo == null) {
-            removeShardOnFailure = true;
-            ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), shardName);
-
-            final Builder builder = newShardDatastoreContextBuilder(shardName);
-            builder.storeRoot(shardPrefix).customRaftPolicyImplementation(DisableElectionsRaftPolicy.class.getName());
-
-            DatastoreContext datastoreContext = builder.build();
-
-            shardInfo = new ShardInformation(shardName, shardId, getPeerAddresses(shardName), datastoreContext,
-                    Shard.builder(), peerAddressResolver);
-            shardInfo.setActiveMember(false);
-            shardInfo.setSchemaContext(schemaContext);
-            localShards.put(shardName, shardInfo);
-            shardInfo.setActor(newShardActor(shardInfo));
-        } else {
-            removeShardOnFailure = false;
-            shardInfo = existingShardInfo;
-        }
-
-        execAddShard(shardName, shardInfo, response, removeShardOnFailure, sender);
-    }
-
+    @SuppressFBWarnings(value = "UPM_UNCALLED_PRIVATE_METHOD",
+            justification = "https://github.com/spotbugs/spotbugs/issues/811")
     private void addShard(final String shardName, final RemotePrimaryShardFound response, final ActorRef sender) {
         if (isShardReplicaOperationInProgress(shardName, sender)) {
             return;
@@ -1513,7 +1294,7 @@ 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(final Throwable failure, final Object addServerResponse) {
                 if (failure != null) {
@@ -1616,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(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()) {
@@ -1650,13 +1405,11 @@ 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;
     }
 
@@ -1717,7 +1470,7 @@ 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(final Throwable failure, final Object response) {
                     if (failure != null) {
@@ -1766,7 +1519,7 @@ 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(final Throwable failure, final Object response) {
                 if (failure != null) {
@@ -1812,7 +1565,7 @@ 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(final Throwable failure, final Object response) {
                 shardReplicaOperationsInProgress.remove(shardName);
@@ -1960,10 +1713,10 @@ class ShardManager extends AbstractUntypedPersistentActorWithMetering {
          */
         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);
-            this.shardManagerActor = Preconditions.checkNotNull(shardManagerActor);
+            this.targetActor = requireNonNull(targetActor);
+            this.shardName = requireNonNull(shardName);
+            this.persistenceId = requireNonNull(persistenceId);
+            this.shardManagerActor = requireNonNull(shardManagerActor);
         }
 
         public ActorRef getTargetActor() {