Use java.util.function.Supplier instead of Guava
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / shardmanager / ShardManager.java
index 4eb6dd01dc9316f7dbc1d11c0447cb530e5ada58..fa8b7854dae2160245b282f097dab8b4329ecb93 100644 (file)
@@ -14,7 +14,6 @@ import akka.actor.Address;
 import akka.actor.Cancellable;
 import akka.actor.OneForOneStrategy;
 import akka.actor.PoisonPill;
-import akka.actor.Props;
 import akka.actor.Status;
 import akka.actor.SupervisorStrategy;
 import akka.cluster.ClusterEvent;
@@ -27,15 +26,9 @@ import akka.persistence.SaveSnapshotFailure;
 import akka.persistence.SaveSnapshotSuccess;
 import akka.persistence.SnapshotOffer;
 import akka.persistence.SnapshotSelectionCriteria;
-import akka.serialization.Serialization;
 import akka.util.Timeout;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Objects;
-import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
-import com.google.common.base.Supplier;
-import com.google.common.collect.Sets;
 import java.io.ByteArrayInputStream;
 import java.io.ObjectInputStream;
 import java.util.ArrayList;
@@ -43,13 +36,13 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 import org.apache.commons.lang3.SerializationUtils;
@@ -58,7 +51,6 @@ import org.opendaylight.controller.cluster.datastore.ClusterWrapper;
 import org.opendaylight.controller.cluster.datastore.DatastoreContext;
 import org.opendaylight.controller.cluster.datastore.DatastoreContextFactory;
 import org.opendaylight.controller.cluster.datastore.Shard;
-import org.opendaylight.controller.cluster.datastore.ShardManagerSnapshot;
 import org.opendaylight.controller.cluster.datastore.config.Configuration;
 import org.opendaylight.controller.cluster.datastore.config.ModuleShardConfiguration;
 import org.opendaylight.controller.cluster.datastore.exceptions.AlreadyExistsException;
@@ -66,7 +58,6 @@ import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderExc
 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.jmx.mbeans.shardmanager.ShardManagerInfoMBean;
 import org.opendaylight.controller.cluster.datastore.messages.ActorInitialized;
 import org.opendaylight.controller.cluster.datastore.messages.AddShardReplica;
 import org.opendaylight.controller.cluster.datastore.messages.CreateShard;
@@ -76,20 +67,15 @@ import org.opendaylight.controller.cluster.datastore.messages.FindPrimary;
 import org.opendaylight.controller.cluster.datastore.messages.LocalPrimaryShardFound;
 import org.opendaylight.controller.cluster.datastore.messages.LocalShardFound;
 import org.opendaylight.controller.cluster.datastore.messages.LocalShardNotFound;
-import org.opendaylight.controller.cluster.datastore.messages.PeerAddressResolved;
-import org.opendaylight.controller.cluster.datastore.messages.PeerDown;
-import org.opendaylight.controller.cluster.datastore.messages.PeerUp;
 import org.opendaylight.controller.cluster.datastore.messages.RemoteFindPrimary;
 import org.opendaylight.controller.cluster.datastore.messages.RemotePrimaryShardFound;
 import org.opendaylight.controller.cluster.datastore.messages.RemoveShardReplica;
 import org.opendaylight.controller.cluster.datastore.messages.ShardLeaderStateChanged;
-import org.opendaylight.controller.cluster.datastore.messages.SwitchShardBehavior;
 import org.opendaylight.controller.cluster.datastore.messages.UpdateSchemaContext;
 import org.opendaylight.controller.cluster.datastore.utils.Dispatchers;
 import org.opendaylight.controller.cluster.datastore.utils.PrimaryShardInfoFutureCache;
 import org.opendaylight.controller.cluster.notifications.RegisterRoleChangeListener;
 import org.opendaylight.controller.cluster.notifications.RoleChangeNotification;
-import org.opendaylight.controller.cluster.raft.RaftState;
 import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus;
 import org.opendaylight.controller.cluster.raft.base.messages.SwitchBehavior;
 import org.opendaylight.controller.cluster.raft.client.messages.GetSnapshot;
@@ -101,7 +87,6 @@ import org.opendaylight.controller.cluster.raft.messages.RemoveServerReply;
 import org.opendaylight.controller.cluster.raft.messages.ServerChangeStatus;
 import org.opendaylight.controller.cluster.raft.messages.ServerRemoved;
 import org.opendaylight.controller.cluster.raft.policy.DisableElectionsRaftPolicy;
-import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTree;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -119,7 +104,7 @@ import scala.concurrent.duration.FiniteDuration;
  * <li> Monitor the cluster members and store their addresses
  * <ul>
  */
-public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
+class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
     private static final Logger LOG = LoggerFactory.getLogger(ShardManager.class);
 
@@ -158,19 +143,16 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
     private final String persistenceId;
 
-    /**
-     */
-    protected ShardManager(AbstractBuilder<?> builder) {
-
-        this.cluster = builder.cluster;
-        this.configuration = builder.configuration;
-        this.datastoreContextFactory = builder.datastoreContextFactory;
-        this.type = builder.datastoreContextFactory.getBaseDatastoreContext().getDataStoreName();
+    ShardManager(AbstractShardManagerCreator<?> builder) {
+        this.cluster = builder.getCluster();
+        this.configuration = builder.getConfiguration();
+        this.datastoreContextFactory = builder.getDdatastoreContextFactory();
+        this.type = datastoreContextFactory.getBaseDatastoreContext().getDataStoreName();
         this.shardDispatcherPath =
                 new Dispatchers(context().system().dispatchers()).getDispatcherPath(Dispatchers.DispatcherType.Shard);
-        this.waitTillReadyCountdownLatch = builder.waitTillReadyCountdownLatch;
-        this.primaryShardInfoCache = builder.primaryShardInfoCache;
-        this.restoreFromSnapshot = builder.restoreFromSnapshot;
+        this.waitTillReadyCountdownLatch = builder.getWaitTillReadyCountdownLatch();
+        this.primaryShardInfoCache = builder.getPrimaryShardInfoCache();
+        this.restoreFromSnapshot = builder.getRestoreFromSnapshot();
 
         String possiblePersistenceId = datastoreContextFactory.getBaseDatastoreContext().getShardManagerPersistenceId();
         persistenceId = possiblePersistenceId != null ? possiblePersistenceId : "shard-manager-" + type;
@@ -180,12 +162,9 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         // Subscribe this actor to cluster member events
         cluster.subscribeToMemberEvents(getSelf());
 
-        List<String> localShardActorNames = new ArrayList<>();
-        mBean = ShardManagerInfo.createShardManagerMBean(cluster.getCurrentMemberName(),
-                "shard-manager-" + this.type,
-                datastoreContextFactory.getBaseDatastoreContext().getDataStoreMXBeanType(),
-                localShardActorNames);
-        mBean.setShardManager(this);
+        mBean = new ShardManagerInfo(getSelf(), cluster.getCurrentMemberName(), "shard-manager-" + this.type,
+                datastoreContextFactory.getBaseDatastoreContext().getDataStoreMXBeanType());
+        mBean.registerMBean();
     }
 
     @Override
@@ -256,6 +235,8 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                     persistenceId(), ((SaveSnapshotFailure) message).cause());
         } else if(message instanceof Shutdown) {
             onShutDown();
+        } else if (message instanceof GetLocalShardIds) {
+            onGetLocalShardIds();
         } else {
             unknownMessage(message);
         }
@@ -318,14 +299,13 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         if (replyMsg.getStatus() == ServerChangeStatus.OK) {
             LOG.debug ("{}: Leader shard successfully removed the replica shard {}", persistenceId(),
                     shardId.getShardName());
-            originalSender.tell(new akka.actor.Status.Success(null), getSelf());
+            originalSender.tell(new Status.Success(null), getSelf());
         } else {
             LOG.warn ("{}: Leader failed to remove shard replica {} with status {}",
                     persistenceId(), shardId, replyMsg.getStatus());
 
-            Exception failure = getServerChangeException(RemoveServer.class, replyMsg.getStatus(),
-                    leaderPath, shardId);
-            originalSender.tell(new akka.actor.Status.Failure(failure), getSelf());
+            Exception failure = getServerChangeException(RemoveServer.class, replyMsg.getStatus(), leaderPath, shardId);
+            originalSender.tell(new Status.Failure(failure), getSelf());
         }
     }
 
@@ -408,7 +388,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
 
         if(notInitialized != null) {
-            getSender().tell(new akka.actor.Status.Failure(new IllegalStateException(String.format(
+            getSender().tell(new Status.Failure(new IllegalStateException(String.format(
                     "%d shard(s) %s are not initialized", notInitialized.size(), notInitialized))), getSelf());
             return;
         }
@@ -435,14 +415,14 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             String shardName = createShard.getModuleShardConfig().getShardName();
             if(localShards.containsKey(shardName)) {
                 LOG.debug("{}: Shard {} already exists", persistenceId(), shardName);
-                reply = new akka.actor.Status.Success(String.format("Shard with name %s already exists", shardName));
+                reply = new Status.Success(String.format("Shard with name %s already exists", shardName));
             } else {
                 doCreateShard(createShard);
-                reply = new akka.actor.Status.Success(null);
+                reply = new Status.Success(null);
             }
         } catch (Exception e) {
             LOG.error("{}: onCreateShard failed", persistenceId(), e);
-            reply = new akka.actor.Status.Failure(e);
+            reply = new Status.Failure(e);
         }
 
         if(getSender() != null && !getContext().system().deadLetters().equals(getSender())) {
@@ -495,8 +475,6 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         info.setActiveMember(isActiveMember);
         localShards.put(info.getShardName(), info);
 
-        mBean.addLocalShard(shardId.toString());
-
         if(schemaContext != null) {
             info.setActor(newShardActor(schemaContext, info));
         }
@@ -547,10 +525,10 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
         if(!shardInfo.isShardInitialized()) {
             LOG.debug("{}: Returning NotInitializedException for shard {}", persistenceId(), shardInfo.getShardName());
-            message.getSender().tell(createNotInitializedException(shardInfo.shardId), getSelf());
+            message.getSender().tell(createNotInitializedException(shardInfo.getShardId()), getSelf());
         } else {
             LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(), shardInfo.getShardName());
-            message.getSender().tell(createNoShardLeaderException(shardInfo.shardId), getSelf());
+            message.getSender().tell(createNoShardLeaderException(shardInfo.getShardId()), getSelf());
         }
     }
 
@@ -729,11 +707,11 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             } else if (!shardInformation.isShardInitialized()) {
                 LOG.debug("{}: Returning NotInitializedException for shard {}", persistenceId(),
                         shardInformation.getShardName());
-                getSender().tell(createNotInitializedException(shardInformation.shardId), getSelf());
+                getSender().tell(createNotInitializedException(shardInformation.getShardId()), getSelf());
             } else {
                 LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(),
                         shardInformation.getShardName());
-                getSender().tell(createNoShardLeaderException(shardInformation.shardId), getSelf());
+                getSender().tell(createNoShardLeaderException(shardInformation.getShardId()), getSelf());
             }
 
             return;
@@ -849,17 +827,44 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
-    private void onSwitchShardBehavior(SwitchShardBehavior message) {
-        ShardIdentifier identifier = ShardIdentifier.builder().fromShardIdString(message.getShardName()).build();
+    private void onGetLocalShardIds() {
+        final List<String> response = new ArrayList<>(localShards.size());
 
-        ShardInformation shardInformation = localShards.get(identifier.getShardName());
+        for (ShardInformation info : localShards.values()) {
+            response.add(info.getShardId().toString());
+        }
 
-        if(shardInformation != null && shardInformation.getActor() != null) {
-            shardInformation.getActor().tell(
-                    new SwitchBehavior(message.getNewState(), message.getTerm()), getSelf());
+        getSender().tell(new Status.Success(response), getSelf());
+    }
+
+    private void onSwitchShardBehavior(final SwitchShardBehavior message) {
+        final ShardIdentifier identifier = message.getShardId();
+
+        if (identifier != null) {
+            final ShardInformation info = localShards.get(identifier.getShardName());
+            if (info == null) {
+                getSender().tell(new Status.Failure(
+                    new IllegalArgumentException("Shard " + identifier + " is not local")), getSelf());
+                return;
+            }
+
+            switchShardBehavior(info, new SwitchBehavior(message.getNewState(), message.getTerm()));
         } else {
+            for (ShardInformation info : localShards.values()) {
+                switchShardBehavior(info, new SwitchBehavior(message.getNewState(), message.getTerm()));
+            }
+        }
+
+        getSender().tell(new Status.Success(null), getSelf());
+    }
+
+    private void switchShardBehavior(final ShardInformation info, final SwitchBehavior switchBehavior) {
+        final ActorRef actor = info.getActor();
+        if (actor != null) {
+            actor.tell(switchBehavior, getSelf());
+          } else {
             LOG.warn("Could not switch the behavior of shard {} to {} - shard is not yet available",
-                    message.getShardName(), message.getNewState());
+                info.getShardName(), switchBehavior.getNewState());
         }
     }
 
@@ -922,11 +927,11 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             return;
         }
 
-        Collection<String> visitedAddresses;
+        final Collection<String> visitedAddresses;
         if(message instanceof RemoteFindPrimary) {
             visitedAddresses = ((RemoteFindPrimary)message).getVisitedAddresses();
         } else {
-            visitedAddresses = new ArrayList<>();
+            visitedAddresses = new ArrayList<>(1);
         }
 
         visitedAddresses.add(peerAddressResolver.getShardManagerActorPathBuilder(cluster.getSelfAddress()).toString());
@@ -990,7 +995,6 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             localShards.put(shardName, new ShardInformation(shardName, shardId, peerAddresses,
                     newShardDatastoreContext(shardName), Shard.builder().restoreFromSnapshot(
                         shardSnapshots.get(shardName)), peerAddressResolver));
-            mBean.addLocalShard(shardId.toString());
         }
     }
 
@@ -1044,7 +1048,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         if (shardReplicaOperationsInProgress.contains(shardName)) {
             String msg = String.format("A shard replica operation for %s is already in progress", shardName);
             LOG.debug ("{}: {}", persistenceId(), msg);
-            sender.tell(new akka.actor.Status.Failure(new IllegalStateException(msg)), getSelf());
+            sender.tell(new Status.Failure(new IllegalStateException(msg)), getSelf());
             return true;
         }
 
@@ -1060,7 +1064,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         if (!(this.configuration.isShardConfigured(shardName))) {
             String msg = String.format("No module configuration exists for shard %s", shardName);
             LOG.debug ("{}: {}", persistenceId(), msg);
-            getSender().tell(new akka.actor.Status.Failure(new IllegalArgumentException(msg)), getSelf());
+            getSender().tell(new Status.Failure(new IllegalArgumentException(msg)), getSelf());
             return;
         }
 
@@ -1069,7 +1073,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             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 akka.actor.Status.Failure(new IllegalStateException(msg)), getSelf());
+            getSender().tell(new Status.Failure(new IllegalStateException(msg)), getSelf());
             return;
         }
 
@@ -1090,7 +1094,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     private void sendLocalReplicaAlreadyExistsReply(String shardName, ActorRef sender) {
         String msg = String.format("Local shard %s already exists", shardName);
         LOG.debug ("{}: {}", persistenceId(), msg);
-        sender.tell(new akka.actor.Status.Failure(new AlreadyExistsException(msg)), getSelf());
+        sender.tell(new Status.Failure(new AlreadyExistsException(msg)), getSelf());
     }
 
     private void addShard(final String shardName, final RemotePrimaryShardFound response, final ActorRef sender) {
@@ -1160,7 +1164,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             }
         }
 
-        sender.tell(new akka.actor.Status.Failure(message == null ? failure :
+        sender.tell(new Status.Failure(message == null ? failure :
             new RuntimeException(message, failure)), getSelf());
     }
 
@@ -1179,8 +1183,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             shardInfo.setActiveMember(true);
             persistShardList();
 
-            mBean.addLocalShard(shardInfo.getShardId().toString());
-            sender.tell(new akka.actor.Status.Success(null), getSelf());
+            sender.tell(new Status.Success(null), getSelf());
         } else if(replyMsg.getStatus() == ServerChangeStatus.ALREADY_EXISTS) {
             sendLocalReplicaAlreadyExistsReply(shardName, sender);
         } else {
@@ -1282,7 +1285,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             0, 0));
     }
 
-    private static class ForwardedAddServerReply {
+    private static final class ForwardedAddServerReply {
         ShardInformation shardInfo;
         AddServerReply addServerReply;
         String leaderPath;
@@ -1297,7 +1300,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
-    private static class ForwardedAddServerFailure {
+    private static final class ForwardedAddServerFailure {
         String shardName;
         String failureMessage;
         Throwable failure;
@@ -1312,238 +1315,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
-    @VisibleForTesting
-    protected static class ShardInformation {
-        private final ShardIdentifier shardId;
-        private final String shardName;
-        private ActorRef actor;
-        private final Map<String, String> initialPeerAddresses;
-        private Optional<DataTree> localShardDataTree;
-        private boolean leaderAvailable = false;
-
-        // flag that determines if the actor is ready for business
-        private boolean actorInitialized = false;
-
-        private boolean followerSyncStatus = false;
-
-        private final Set<OnShardInitialized> onShardInitializedSet = Sets.newHashSet();
-        private String role ;
-        private String leaderId;
-        private short leaderVersion;
-
-        private DatastoreContext datastoreContext;
-        private Shard.AbstractBuilder<?, ?> builder;
-        private final ShardPeerAddressResolver addressResolver;
-        private boolean isActiveMember = true;
-
-        private ShardInformation(String shardName, ShardIdentifier shardId,
-                Map<String, String> initialPeerAddresses, DatastoreContext datastoreContext,
-                Shard.AbstractBuilder<?, ?> builder, ShardPeerAddressResolver addressResolver) {
-            this.shardName = shardName;
-            this.shardId = shardId;
-            this.initialPeerAddresses = initialPeerAddresses;
-            this.datastoreContext = datastoreContext;
-            this.builder = builder;
-            this.addressResolver = addressResolver;
-        }
-
-        Props newProps(SchemaContext schemaContext) {
-            Preconditions.checkNotNull(builder);
-            Props props = builder.id(shardId).peerAddresses(initialPeerAddresses).datastoreContext(datastoreContext).
-                    schemaContext(schemaContext).props();
-            builder = null;
-            return props;
-        }
-
-        String getShardName() {
-            return shardName;
-        }
-
-        @Nullable
-        ActorRef getActor(){
-            return actor;
-        }
-
-        void setActor(ActorRef actor) {
-            this.actor = actor;
-        }
-
-        ShardIdentifier getShardId() {
-            return shardId;
-        }
-
-        void setLocalDataTree(Optional<DataTree> localShardDataTree) {
-            this.localShardDataTree = localShardDataTree;
-        }
-
-        Optional<DataTree> getLocalShardDataTree() {
-            return localShardDataTree;
-        }
-
-        DatastoreContext getDatastoreContext() {
-            return datastoreContext;
-        }
-
-        void setDatastoreContext(DatastoreContext datastoreContext, ActorRef sender) {
-            this.datastoreContext = datastoreContext;
-            if (actor != null) {
-                LOG.debug ("Sending new DatastoreContext to {}", shardId);
-                actor.tell(this.datastoreContext, sender);
-            }
-        }
-
-        void updatePeerAddress(String peerId, String peerAddress, ActorRef sender){
-            LOG.info("updatePeerAddress for peer {} with address {}", peerId, peerAddress);
-
-            if(actor != null) {
-                if(LOG.isDebugEnabled()) {
-                    LOG.debug("Sending PeerAddressResolved for peer {} with address {} to {}",
-                            peerId, peerAddress, actor.path());
-                }
-
-                actor.tell(new PeerAddressResolved(peerId, peerAddress), sender);
-            }
-
-            notifyOnShardInitializedCallbacks();
-        }
-
-        void peerDown(String memberName, String peerId, ActorRef sender) {
-            if(actor != null) {
-                actor.tell(new PeerDown(memberName, peerId), sender);
-            }
-        }
-
-        void peerUp(String memberName, String peerId, ActorRef sender) {
-            if(actor != null) {
-                actor.tell(new PeerUp(memberName, peerId), sender);
-            }
-        }
-
-        boolean isShardReady() {
-            return !RaftState.Candidate.name().equals(role) && !Strings.isNullOrEmpty(role);
-        }
-
-        boolean isShardReadyWithLeaderId() {
-            return leaderAvailable && isShardReady() && !RaftState.IsolatedLeader.name().equals(role) &&
-                    (isLeader() || addressResolver.resolve(leaderId) != null);
-        }
-
-        boolean isShardInitialized() {
-            return getActor() != null && actorInitialized;
-        }
-
-        boolean isLeader() {
-            return Objects.equal(leaderId, shardId.toString());
-        }
-
-        String getSerializedLeaderActor() {
-            if(isLeader()) {
-                return Serialization.serializedActorPath(getActor());
-            } else {
-                return addressResolver.resolve(leaderId);
-            }
-        }
-
-        void setActorInitialized() {
-            LOG.debug("Shard {} is initialized", shardId);
-
-            this.actorInitialized = true;
-
-            notifyOnShardInitializedCallbacks();
-        }
-
-        private void notifyOnShardInitializedCallbacks() {
-            if(onShardInitializedSet.isEmpty()) {
-                return;
-            }
-
-            boolean ready = isShardReadyWithLeaderId();
-
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("Shard {} is {} - notifying {} OnShardInitialized callbacks", shardId,
-                        ready ? "ready" : "initialized", onShardInitializedSet.size());
-            }
-
-            Iterator<OnShardInitialized> iter = onShardInitializedSet.iterator();
-            while(iter.hasNext()) {
-                OnShardInitialized onShardInitialized = iter.next();
-                if(!(onShardInitialized instanceof OnShardReady) || ready) {
-                    iter.remove();
-                    onShardInitialized.getTimeoutSchedule().cancel();
-                    onShardInitialized.getReplyRunnable().run();
-                }
-            }
-        }
-
-        void addOnShardInitialized(OnShardInitialized onShardInitialized) {
-            onShardInitializedSet.add(onShardInitialized);
-        }
-
-        void removeOnShardInitialized(OnShardInitialized onShardInitialized) {
-            onShardInitializedSet.remove(onShardInitialized);
-        }
-
-        void setRole(String newRole) {
-            this.role = newRole;
-
-            notifyOnShardInitializedCallbacks();
-        }
-
-        void setFollowerSyncStatus(boolean syncStatus){
-            this.followerSyncStatus = syncStatus;
-        }
-
-        boolean isInSync(){
-            if(RaftState.Follower.name().equals(this.role)){
-                return followerSyncStatus;
-            } else if(RaftState.Leader.name().equals(this.role)){
-                return true;
-            }
-
-            return false;
-        }
-
-        boolean setLeaderId(String leaderId) {
-            boolean changed = !Objects.equal(this.leaderId, leaderId);
-            this.leaderId = leaderId;
-            if(leaderId != null) {
-                this.leaderAvailable = true;
-            }
-            notifyOnShardInitializedCallbacks();
-
-            return changed;
-        }
-
-        String getLeaderId() {
-            return leaderId;
-        }
-
-        void setLeaderAvailable(boolean leaderAvailable) {
-            this.leaderAvailable = leaderAvailable;
-
-            if(leaderAvailable) {
-                notifyOnShardInitializedCallbacks();
-            }
-        }
-
-        short getLeaderVersion() {
-            return leaderVersion;
-        }
-
-        void setLeaderVersion(short leaderVersion) {
-            this.leaderVersion = leaderVersion;
-        }
-
-        boolean isActiveMember() {
-            return isActiveMember;
-        }
-
-        void setActiveMember(boolean isActiveMember) {
-            this.isActiveMember = isActiveMember;
-        }
-    }
-
-    private static class OnShardInitialized {
+    static class OnShardInitialized {
         private final Runnable replyRunnable;
         private Cancellable timeoutSchedule;
 
@@ -1564,112 +1336,12 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
-    private static class OnShardReady extends OnShardInitialized {
+    static class OnShardReady extends OnShardInitialized {
         OnShardReady(Runnable replyRunnable) {
             super(replyRunnable);
         }
     }
 
-    private static class ShardNotInitializedTimeout {
-        private final ActorRef sender;
-        private final ShardInformation shardInfo;
-        private final OnShardInitialized onShardInitialized;
-
-        ShardNotInitializedTimeout(ShardInformation shardInfo, OnShardInitialized onShardInitialized, ActorRef sender) {
-            this.sender = sender;
-            this.shardInfo = shardInfo;
-            this.onShardInitialized = onShardInitialized;
-        }
-
-        ActorRef getSender() {
-            return sender;
-        }
-
-        ShardInformation getShardInfo() {
-            return shardInfo;
-        }
-
-        OnShardInitialized getOnShardInitialized() {
-            return onShardInitialized;
-        }
-    }
-
-    public static Builder builder() {
-        return new Builder();
-    }
-
-    public static abstract class AbstractBuilder<T extends AbstractBuilder<T>> {
-        private ClusterWrapper cluster;
-        private Configuration configuration;
-        private DatastoreContextFactory datastoreContextFactory;
-        private CountDownLatch waitTillReadyCountdownLatch;
-        private PrimaryShardInfoFutureCache primaryShardInfoCache;
-        private DatastoreSnapshot restoreFromSnapshot;
-        private volatile boolean sealed;
-
-        @SuppressWarnings("unchecked")
-        private T self() {
-            return (T) this;
-        }
-
-        protected void checkSealed() {
-            Preconditions.checkState(!sealed, "Builder is already sealed - further modifications are not allowed");
-        }
-
-        public T cluster(ClusterWrapper cluster) {
-            checkSealed();
-            this.cluster = cluster;
-            return self();
-        }
-
-        public T configuration(Configuration configuration) {
-            checkSealed();
-            this.configuration = configuration;
-            return self();
-        }
-
-        public T datastoreContextFactory(DatastoreContextFactory datastoreContextFactory) {
-            checkSealed();
-            this.datastoreContextFactory = datastoreContextFactory;
-            return self();
-        }
-
-        public T waitTillReadyCountdownLatch(CountDownLatch waitTillReadyCountdownLatch) {
-            checkSealed();
-            this.waitTillReadyCountdownLatch = waitTillReadyCountdownLatch;
-            return self();
-        }
-
-        public T primaryShardInfoCache(PrimaryShardInfoFutureCache primaryShardInfoCache) {
-            checkSealed();
-            this.primaryShardInfoCache = primaryShardInfoCache;
-            return self();
-        }
-
-        public T restoreFromSnapshot(DatastoreSnapshot restoreFromSnapshot) {
-            checkSealed();
-            this.restoreFromSnapshot = restoreFromSnapshot;
-            return self();
-        }
-
-        protected void verify() {
-            sealed = true;
-            Preconditions.checkNotNull(cluster, "cluster should not be null");
-            Preconditions.checkNotNull(configuration, "configuration should not be null");
-            Preconditions.checkNotNull(datastoreContextFactory, "datastoreContextFactory should not be null");
-            Preconditions.checkNotNull(waitTillReadyCountdownLatch, "waitTillReadyCountdownLatch should not be null");
-            Preconditions.checkNotNull(primaryShardInfoCache, "primaryShardInfoCache should not be null");
-        }
-
-        public Props props() {
-            verify();
-            return Props.create(ShardManager.class, this);
-        }
-    }
-
-    public static class Builder extends AbstractBuilder<Builder> {
-    }
-
     private void findPrimary(final String shardName, final FindPrimaryResponseHandler handler) {
         Timeout findPrimaryTimeout = new Timeout(datastoreContextFactory.getBaseDatastoreContext().
                 getShardInitializationTimeout().duration().$times(2));
@@ -1761,7 +1433,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         @Override
         public void onFailure(Throwable failure) {
             LOG.debug ("{}: Received failure from FindPrimary for shard {}", persistenceId, shardName, failure);
-            targetActor.tell(new akka.actor.Status.Failure(new RuntimeException(
+            targetActor.tell(new Status.Failure(new RuntimeException(
                     String.format("Failed to find leader for shard %s", shardName), failure)), shardManagerActor);
         }
 
@@ -1770,12 +1442,11 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             String msg = String.format("Failed to find leader for shard %s: received response: %s",
                     shardName, response);
             LOG.debug ("{}: {}", persistenceId, msg);
-            targetActor.tell(new akka.actor.Status.Failure(response instanceof Throwable ? (Throwable) response :
+            targetActor.tell(new Status.Failure(response instanceof Throwable ? (Throwable) response :
                     new RuntimeException(msg)), shardManagerActor);
         }
     }
 
-
     /**
      * The PrimaryShardFoundForContext is a DTO which puts together a message (aka 'Context' message) which needs to be
      * forwarded to the primary replica of a shard and the message (aka 'PrimaryShardFound' message) that is received
@@ -1825,12 +1496,12 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     /**
      * The WrappedShardResponse class wraps a response from a Shard.
      */
-    private static class WrappedShardResponse {
+    private static final class WrappedShardResponse {
         private final ShardIdentifier shardId;
         private final Object response;
         private final String leaderPath;
 
-        private WrappedShardResponse(ShardIdentifier shardId, Object response, String leaderPath) {
+        WrappedShardResponse(ShardIdentifier shardId, Object response, String leaderPath) {
             this.shardId = shardId;
             this.response = response;
             this.leaderPath = leaderPath;
@@ -1848,6 +1519,30 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             return leaderPath;
         }
     }
+
+    private static final class ShardNotInitializedTimeout {
+        private final ActorRef sender;
+        private final ShardInformation shardInfo;
+        private final OnShardInitialized onShardInitialized;
+
+        ShardNotInitializedTimeout(ShardInformation shardInfo, OnShardInitialized onShardInitialized, ActorRef sender) {
+            this.sender = sender;
+            this.shardInfo = shardInfo;
+            this.onShardInitialized = onShardInitialized;
+        }
+
+        ActorRef getSender() {
+            return sender;
+        }
+
+        ShardInformation getShardInfo() {
+            return shardInfo;
+        }
+
+        OnShardInitialized getOnShardInitialized() {
+            return onShardInitialized;
+        }
+    }
 }