Bug 4105: Change ownership on member down/up
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ShardManager.java
index cff44b13cb3edb1756ed6159e95d8fe259f91fde..f8237b56a72cf3e1cdc0ef8c497b3f0d9c2896bc 100644 (file)
@@ -18,32 +18,27 @@ import akka.actor.SupervisorStrategy;
 import akka.cluster.ClusterEvent;
 import akka.japi.Creator;
 import akka.japi.Function;
-import akka.japi.Procedure;
 import akka.persistence.RecoveryCompleted;
-import akka.persistence.RecoveryFailure;
 import akka.serialization.Serialization;
 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.ImmutableSet;
 import com.google.common.collect.Sets;
 import java.io.Serializable;
 import java.util.ArrayList;
 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 org.opendaylight.controller.cluster.DataPersistenceProvider;
-import org.opendaylight.controller.cluster.NonPersistentDataProvider;
-import org.opendaylight.controller.cluster.PersistentDataProvider;
+import java.util.concurrent.TimeUnit;
 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActorWithMetering;
+import org.opendaylight.controller.cluster.datastore.config.Configuration;
 import org.opendaylight.controller.cluster.datastore.exceptions.NoShardLeaderException;
 import org.opendaylight.controller.cluster.datastore.exceptions.NotInitializedException;
 import org.opendaylight.controller.cluster.datastore.exceptions.PrimaryNotFoundException;
@@ -52,24 +47,34 @@ import org.opendaylight.controller.cluster.datastore.identifiers.ShardManagerIde
 import org.opendaylight.controller.cluster.datastore.jmx.mbeans.shardmanager.ShardManagerInfo;
 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.CreateShard;
+import org.opendaylight.controller.cluster.datastore.messages.CreateShardReply;
 import org.opendaylight.controller.cluster.datastore.messages.FindLocalShard;
 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.PrimaryFound;
+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.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.notifications.LeaderStateChanged;
+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.yangtools.yang.model.api.ModuleIdentifier;
+import org.opendaylight.controller.cluster.raft.base.messages.SwitchBehavior;
+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;
 import scala.concurrent.duration.Duration;
+import scala.concurrent.duration.FiniteDuration;
 
 /**
  * The ShardManager has the following jobs,
@@ -110,26 +115,27 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
     private DatastoreContext datastoreContext;
 
-    private Collection<String> knownModules = Collections.emptySet();
+    private final CountDownLatch waitTillReadyCountdownLatch;
 
-    private final DataPersistenceProvider dataPersistenceProvider;
+    private final PrimaryShardInfoFutureCache primaryShardInfoCache;
 
-    private final CountDownLatch waitTillReadyCountdownLatch;
+    private SchemaContext schemaContext;
 
     /**
      */
     protected ShardManager(ClusterWrapper cluster, Configuration configuration,
-            DatastoreContext datastoreContext, CountDownLatch waitTillReadyCountdownLatch) {
+            DatastoreContext datastoreContext, CountDownLatch waitTillReadyCountdownLatch,
+            PrimaryShardInfoFutureCache primaryShardInfoCache) {
 
         this.cluster = Preconditions.checkNotNull(cluster, "cluster should not be null");
         this.configuration = Preconditions.checkNotNull(configuration, "configuration should not be null");
         this.datastoreContext = datastoreContext;
-        this.dataPersistenceProvider = createDataPersistenceProvider(datastoreContext.isPersistent());
         this.type = datastoreContext.getDataStoreType();
         this.shardManagerIdentifierString = ShardManagerIdentifier.builder().type(type).build().toString();
         this.shardDispatcherPath =
                 new Dispatchers(context().system().dispatchers()).getDispatcherPath(Dispatchers.DispatcherType.Shard);
         this.waitTillReadyCountdownLatch = waitTillReadyCountdownLatch;
+        this.primaryShardInfoCache = primaryShardInfoCache;
 
         // Subscribe this actor to cluster member events
         cluster.subscribeToMemberEvents(getSelf());
@@ -137,21 +143,20 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         createLocalShards();
     }
 
-    protected DataPersistenceProvider createDataPersistenceProvider(boolean persistent) {
-        return (persistent) ? new PersistentDataProvider(this) : new NonPersistentDataProvider();
-    }
-
     public static Props props(
         final ClusterWrapper cluster,
         final Configuration configuration,
         final DatastoreContext datastoreContext,
-        final CountDownLatch waitTillReadyCountdownLatch) {
+        final CountDownLatch waitTillReadyCountdownLatch,
+        final PrimaryShardInfoFutureCache primaryShardInfoCache) {
 
         Preconditions.checkNotNull(cluster, "cluster should not be null");
         Preconditions.checkNotNull(configuration, "configuration should not be null");
         Preconditions.checkNotNull(waitTillReadyCountdownLatch, "waitTillReadyCountdownLatch should not be null");
+        Preconditions.checkNotNull(primaryShardInfoCache, "primaryShardInfoCache should not be null");
 
-        return Props.create(new ShardManagerCreator(cluster, configuration, datastoreContext, waitTillReadyCountdownLatch));
+        return Props.create(new ShardManagerCreator(cluster, configuration, datastoreContext,
+                waitTillReadyCountdownLatch, primaryShardInfoCache));
     }
 
     @Override
@@ -173,10 +178,14 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             onActorInitialized(message);
         } else if (message instanceof ClusterEvent.MemberUp){
             memberUp((ClusterEvent.MemberUp) message);
+        } else if (message instanceof ClusterEvent.MemberExited){
+            memberExited((ClusterEvent.MemberExited) message);
         } else if(message instanceof ClusterEvent.MemberRemoved) {
             memberRemoved((ClusterEvent.MemberRemoved) message);
         } else if(message instanceof ClusterEvent.UnreachableMember) {
-            ignoreMessage(message);
+            memberUnreachable((ClusterEvent.UnreachableMember)message);
+        } else if(message instanceof ClusterEvent.ReachableMember) {
+            memberReachable((ClusterEvent.ReachableMember) message);
         } else if(message instanceof DatastoreContext) {
             onDatastoreContext((DatastoreContext)message);
         } else if(message instanceof RoleChangeNotification) {
@@ -185,27 +194,79 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             onFollowerInitialSyncStatus((FollowerInitialSyncUpStatus) message);
         } else if(message instanceof ShardNotInitializedTimeout) {
             onShardNotInitializedTimeout((ShardNotInitializedTimeout)message);
-        } else if(message instanceof LeaderStateChanged) {
-            onLeaderStateChanged((LeaderStateChanged)message);
+        } else if(message instanceof ShardLeaderStateChanged) {
+            onLeaderStateChanged((ShardLeaderStateChanged) message);
+        } else if(message instanceof SwitchShardBehavior){
+            onSwitchShardBehavior((SwitchShardBehavior) message);
+        } else if(message instanceof CreateShard) {
+            onCreateShard((CreateShard)message);
         } else {
             unknownMessage(message);
         }
 
     }
 
-    private void onLeaderStateChanged(LeaderStateChanged leaderStateChanged) {
+    private void onCreateShard(CreateShard createShard) {
+        Object reply;
+        try {
+            if(localShards.containsKey(createShard.getShardName())) {
+                throw new IllegalStateException(String.format("Shard with name %s already exists",
+                        createShard.getShardName()));
+            }
+
+            ShardIdentifier shardId = getShardIdentifier(cluster.getCurrentMemberName(), createShard.getShardName());
+            Map<String, String> peerAddresses = getPeerAddresses(createShard.getShardName(), createShard.getMemberNames());
+
+            LOG.debug("onCreateShard: shardId: {}, memberNames: {}. peerAddresses: {}", shardId,
+                    createShard.getMemberNames(), peerAddresses);
+
+            DatastoreContext shardDatastoreContext = createShard.getDatastoreContext();
+            if(shardDatastoreContext == null) {
+                shardDatastoreContext = datastoreContext;
+            }
+
+            ShardInformation info = new ShardInformation(createShard.getShardName(), shardId, peerAddresses,
+                    shardDatastoreContext, createShard.getShardPropsCreator());
+            localShards.put(createShard.getShardName(), info);
+
+            mBean.addLocalShard(shardId.toString());
+
+            if(schemaContext != null) {
+                info.setActor(newShardActor(schemaContext, info));
+            }
+
+            reply = new CreateShardReply();
+        } catch (Exception e) {
+            LOG.error("onCreateShard failed", e);
+            reply = new akka.actor.Status.Failure(e);
+        }
+
+        if(getSender() != null && !getContext().system().deadLetters().equals(getSender())) {
+            getSender().tell(reply, getSelf());
+        }
+    }
+
+    private void checkReady(){
+        if (isReadyWithLeaderId()) {
+            LOG.info("{}: All Shards are ready - data store {} is ready, available count is {}",
+                    persistenceId(), type, waitTillReadyCountdownLatch.getCount());
+
+            waitTillReadyCountdownLatch.countDown();
+        }
+    }
+
+    private void onLeaderStateChanged(ShardLeaderStateChanged leaderStateChanged) {
         LOG.info("{}: Received LeaderStateChanged message: {}", persistenceId(), leaderStateChanged);
 
         ShardInformation shardInformation = findShardInformation(leaderStateChanged.getMemberId());
         if(shardInformation != null) {
-            shardInformation.setLeaderId(leaderStateChanged.getLeaderId());
-            if (isReadyWithLeaderId()) {
-                LOG.info("{}: All Shards are ready - data store {} is ready, available count is {}",
-                        persistenceId(), type, waitTillReadyCountdownLatch.getCount());
-
-                waitTillReadyCountdownLatch.countDown();
+            shardInformation.setLocalDataTree(leaderStateChanged.getLocalShardDataTree());
+            shardInformation.setLeaderVersion(leaderStateChanged.getLeaderPayloadVersion());
+            if(shardInformation.setLeaderId(leaderStateChanged.getLeaderId())) {
+                primaryShardInfoCache.remove(shardInformation.getShardName());
             }
 
+            checkReady();
         } else {
             LOG.debug("No shard found with member Id {}", leaderStateChanged.getMemberId());
         }
@@ -249,14 +310,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         ShardInformation shardInformation = findShardInformation(roleChanged.getMemberId());
         if(shardInformation != null) {
             shardInformation.setRole(roleChanged.getNewRole());
-
-            if (isReadyWithLeaderId()) {
-                LOG.info("{}: All Shards are ready - data store {} is ready, available count is {}",
-                        persistenceId(), type, waitTillReadyCountdownLatch.getCount());
-
-                waitTillReadyCountdownLatch.countDown();
-            }
-
+            checkReady();
             mBean.setSyncStatus(isInSync());
         }
     }
@@ -323,26 +377,12 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
     @Override
     protected void handleRecover(Object message) throws Exception {
-        if(dataPersistenceProvider.isRecoveryApplicable()) {
-            if (message instanceof SchemaContextModules) {
-                SchemaContextModules msg = (SchemaContextModules) message;
-                knownModules = ImmutableSet.copyOf(msg.getModules());
-            } else if (message instanceof RecoveryFailure) {
-                RecoveryFailure failure = (RecoveryFailure) message;
-                LOG.error("Recovery failed", failure.cause());
-            } else if (message instanceof RecoveryCompleted) {
-                LOG.info("Recovery complete : {}", persistenceId());
-
-                // Delete all the messages from the akka journal except the last one
-                deleteMessages(lastSequenceNr() - 1);
-            }
-        } else {
-            if (message instanceof RecoveryCompleted) {
-                LOG.info("Recovery complete : {}", persistenceId());
+        if (message instanceof RecoveryCompleted) {
+            LOG.info("Recovery complete : {}", persistenceId());
 
-                // Delete all the messages from the akka journal
-                deleteMessages(lastSequenceNr());
-            }
+            // We no longer persist SchemaContext modules so delete all the prior messages from the akka
+            // journal on upgrade from Helium.
+            deleteMessages(lastSequenceNr());
         }
     }
 
@@ -383,8 +423,16 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
                 LOG.debug("{}: Scheduling timer to wait for shard {}", persistenceId(), shardInformation.getShardName());
 
+                FiniteDuration timeout = datastoreContext.getShardInitializationTimeout().duration();
+                if(shardInformation.isShardInitialized()) {
+                    // If the shard is already initialized then we'll wait enough time for the shard to
+                    // elect a leader, ie 2 times the election timeout.
+                    timeout = FiniteDuration.create(datastoreContext.getShardRaftConfig()
+                            .getElectionTimeOutInterval().toMillis() * 2, TimeUnit.MILLISECONDS);
+                }
+
                 Cancellable timeoutSchedule = getContext().system().scheduler().scheduleOnce(
-                        datastoreContext.getShardInitializationTimeout().duration(), getSelf(),
+                        timeout, getSelf(),
                         new ShardNotInitializedTimeout(shardInformation, onShardInitialized, sender),
                         getContext().dispatcher(), getSelf());
 
@@ -407,9 +455,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     }
 
     private NoShardLeaderException createNoShardLeaderException(ShardIdentifier shardId) {
-        return new NoShardLeaderException(String.format(
-                "Could not find a leader for shard %s. This typically happens when the system is coming up or " +
-                "recovering and a leader is being elected. Try again later.", shardId));
+        return new NoShardLeaderException(null, shardId.toString());
     }
 
     private NotInitializedException createNotInitializedException(ShardIdentifier shardId) {
@@ -423,7 +469,24 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         LOG.debug("{}: Received MemberRemoved: memberName: {}, address: {}", persistenceId(), memberName,
                 message.member().address());
 
-        memberNameToAddress.remove(message.member().roles().head());
+        memberNameToAddress.remove(memberName);
+
+        for(ShardInformation info : localShards.values()){
+            info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
+        }
+    }
+
+    private void memberExited(ClusterEvent.MemberExited message) {
+        String memberName = message.member().roles().head();
+
+        LOG.debug("{}: Received MemberExited: memberName: {}, address: {}", persistenceId(), memberName,
+                message.member().address());
+
+        memberNameToAddress.remove(memberName);
+
+        for(ShardInformation info : localShards.values()){
+            info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
+        }
     }
 
     private void memberUp(ClusterEvent.MemberUp message) {
@@ -436,8 +499,52 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
         for(ShardInformation info : localShards.values()){
             String shardName = info.getShardName();
-            info.updatePeerAddress(getShardIdentifier(memberName, shardName).toString(),
-                getShardActorPath(shardName, memberName), getSelf());
+            String peerId = getShardIdentifier(memberName, shardName).toString();
+            info.updatePeerAddress(peerId, getShardActorPath(shardName, memberName), getSelf());
+
+            info.peerUp(memberName, peerId, getSelf());
+        }
+
+        checkReady();
+    }
+
+    private void memberReachable(ClusterEvent.ReachableMember message) {
+        String memberName = message.member().roles().head();
+        LOG.debug("Received ReachableMember: memberName {}, address: {}", memberName, message.member().address());
+
+        markMemberAvailable(memberName);
+    }
+
+    private void memberUnreachable(ClusterEvent.UnreachableMember message) {
+        String memberName = message.member().roles().head();
+        LOG.debug("Received UnreachableMember: memberName {}, address: {}", memberName, message.member().address());
+
+        markMemberUnavailable(memberName);
+    }
+
+    private void markMemberUnavailable(final String memberName) {
+        for(ShardInformation info : localShards.values()){
+            String leaderId = info.getLeaderId();
+            if(leaderId != null && leaderId.contains(memberName)) {
+                LOG.debug("Marking Leader {} as unavailable.", leaderId);
+                info.setLeaderAvailable(false);
+
+                primaryShardInfoCache.remove(info.getShardName());
+            }
+
+            info.peerDown(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
+        }
+    }
+
+    private void markMemberAvailable(final String memberName) {
+        for(ShardInformation info : localShards.values()){
+            String leaderId = info.getLeaderId();
+            if(leaderId != null && leaderId.contains(memberName)) {
+                LOG.debug("Marking Leader {} as available.", leaderId);
+                info.setLeaderAvailable(true);
+            }
+
+            info.peerUp(memberName, getShardIdentifier(memberName, info.getShardName()).toString(), getSelf());
         }
     }
 
@@ -450,48 +557,38 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
+    private void onSwitchShardBehavior(SwitchShardBehavior message) {
+        ShardIdentifier identifier = ShardIdentifier.builder().fromShardIdString(message.getShardName()).build();
+
+        ShardInformation shardInformation = localShards.get(identifier.getShardName());
+
+        if(shardInformation != null && shardInformation.getActor() != null) {
+            shardInformation.getActor().tell(
+                    new SwitchBehavior(RaftState.valueOf(message.getNewState()), message.getTerm()), getSelf());
+        } else {
+            LOG.warn("Could not switch the behavior of shard {} to {} - shard is not yet available",
+                    message.getShardName(), message.getNewState());
+        }
+    }
+
     /**
      * Notifies all the local shards of a change in the schema context
      *
      * @param message
      */
     private void updateSchemaContext(final Object message) {
-        final SchemaContext schemaContext = ((UpdateSchemaContext) message).getSchemaContext();
-
-        Set<ModuleIdentifier> allModuleIdentifiers = schemaContext.getAllModuleIdentifiers();
-        Set<String> newModules = new HashSet<>(128);
-
-        for(ModuleIdentifier moduleIdentifier : allModuleIdentifiers){
-            String s = moduleIdentifier.getNamespace().toString();
-            newModules.add(s);
-        }
-
-        if(newModules.containsAll(knownModules)) {
-
-            LOG.debug("New SchemaContext has a super set of current knownModules - persisting info");
-
-            knownModules = ImmutableSet.copyOf(newModules);
+        schemaContext = ((UpdateSchemaContext) message).getSchemaContext();
 
-            dataPersistenceProvider.persist(new SchemaContextModules(newModules), new Procedure<SchemaContextModules>() {
+        LOG.debug("Got updated SchemaContext: # of modules {}", schemaContext.getAllModuleIdentifiers().size());
 
-                @Override
-                public void apply(SchemaContextModules param) throws Exception {
-                    LOG.debug("Sending new SchemaContext to Shards");
-                    for (ShardInformation info : localShards.values()) {
-                        if (info.getActor() == null) {
-                            info.setActor(newShardActor(schemaContext, info));
-                        } else {
-                            info.getActor().tell(message, getSelf());
-                        }
-                    }
-                }
-
-            });
-        } else {
-            LOG.debug("Rejecting schema context update - not a super set of previously known modules:\nUPDATE: {}\nKNOWN: {}",
-                    newModules, knownModules);
+        for (ShardInformation info : localShards.values()) {
+            if (info.getActor() == null) {
+                LOG.debug("Creating Shard {}", info.getShardId());
+                info.setActor(newShardActor(schemaContext, info));
+            } else {
+                info.getActor().tell(message, getSelf());
+            }
         }
-
     }
 
     @VisibleForTesting
@@ -501,8 +598,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
     @VisibleForTesting
     protected ActorRef newShardActor(final SchemaContext schemaContext, ShardInformation info) {
-        return getContext().actorOf(Shard.props(info.getShardId(),
-                info.getPeerAddresses(), datastoreContext, schemaContext)
+        return getContext().actorOf(info.newProps(schemaContext)
                         .withDispatcher(shardDispatcherPath), info.getShardId().toString());
     }
 
@@ -510,6 +606,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         LOG.debug("{}: In findPrimary: {}", persistenceId(), message);
 
         final String shardName = message.getShardName();
+        final boolean canReturnLocalShardState = !(message instanceof RemoteFindPrimary);
 
         // First see if the there is a local replica for the shard
         final ShardInformation info = localShards.get(shardName);
@@ -517,7 +614,10 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             sendResponse(info, message.isWaitUntilReady(), true, new Supplier<Object>() {
                 @Override
                 public Object get() {
-                    Object found = new PrimaryFound(info.getSerializedLeaderActor());
+                    String primaryPath = info.getSerializedLeaderActor();
+                    Object found = canReturnLocalShardState && info.isLeader() ?
+                            new LocalPrimaryShardFound(primaryPath, info.getLocalShardDataTree().get()) :
+                                new RemotePrimaryShardFound(primaryPath, info.getLeaderVersion());
 
                     if(LOG.isDebugEnabled()) {
                         LOG.debug("{}: Found primary for {}: {}", persistenceId(), shardName, found);
@@ -537,7 +637,8 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                 LOG.debug("{}: findPrimary for {} forwarding to remote ShardManager {}", persistenceId(),
                         shardName, path);
 
-                getContext().actorSelection(path).forward(message, getContext());
+                getContext().actorSelection(path).forward(new RemoteFindPrimary(shardName,
+                        message.isWaitUntilReady()), getContext());
                 return;
             }
         }
@@ -584,19 +685,22 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
      */
     private void createLocalShards() {
         String memberName = this.cluster.getCurrentMemberName();
-        List<String> memberShardNames =
-            this.configuration.getMemberShardNames(memberName);
+        Collection<String> memberShardNames = this.configuration.getMemberShardNames(memberName);
 
+        ShardPropsCreator shardPropsCreator = new DefaultShardPropsCreator();
         List<String> localShardActorNames = new ArrayList<>();
         for(String shardName : memberShardNames){
             ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
             Map<String, String> peerAddresses = getPeerAddresses(shardName);
             localShardActorNames.add(shardId.toString());
-            localShards.put(shardName, new ShardInformation(shardName, shardId, peerAddresses));
+            localShards.put(shardName, new ShardInformation(shardName, shardId, peerAddresses, datastoreContext,
+                    shardPropsCreator));
         }
 
-        mBean = ShardManagerInfo.createShardManagerMBean("shard-manager-" + this.type,
+        mBean = ShardManagerInfo.createShardManagerMBean(memberName, "shard-manager-" + this.type,
                     datastoreContext.getDataStoreMXBeanType(), localShardActorNames);
+
+        mBean.setShardManager(this);
     }
 
     /**
@@ -605,18 +709,20 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
      * @param shardName
      * @return
      */
-    private Map<String, String> getPeerAddresses(String shardName){
+    private Map<String, String> getPeerAddresses(String shardName) {
+        return getPeerAddresses(shardName, configuration.getMembersFromShardName(shardName));
+    }
 
-        Map<String, String> peerAddresses = new HashMap<>();
+    private Map<String, String> getPeerAddresses(String shardName, Collection<String> members) {
 
-        List<String> members = this.configuration.getMembersFromShardName(shardName);
+        Map<String, String> peerAddresses = new HashMap<>();
 
         String currentMemberName = this.cluster.getCurrentMemberName();
 
-        for(String memberName : members){
-            if(!currentMemberName.equals(memberName)){
+        for(String memberName : members) {
+            if(!currentMemberName.equals(memberName)) {
                 ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
-                String path = getShardActorPath(shardName, currentMemberName);
+                String path = getShardActorPath(shardName, memberName);
                 peerAddresses.put(shardId.toString(), path);
             }
         }
@@ -643,16 +749,6 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         return "shard-manager-" + type;
     }
 
-    @VisibleForTesting
-    Collection<String> getKnownModules() {
-        return knownModules;
-    }
-
-    @VisibleForTesting
-    DataPersistenceProvider getDataPersistenceProvider() {
-        return dataPersistenceProvider;
-    }
-
     @VisibleForTesting
     ShardManagerInfoMBean getMBean(){
         return mBean;
@@ -665,6 +761,8 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         private ActorRef actor;
         private ActorPath actorPath;
         private final Map<String, String> peerAddresses;
+        private Optional<DataTree> localShardDataTree;
+        private boolean leaderAvailable = false;
 
         // flag that determines if the actor is ready for business
         private boolean actorInitialized = false;
@@ -674,12 +772,23 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         private final Set<OnShardInitialized> onShardInitializedSet = Sets.newHashSet();
         private String role ;
         private String leaderId;
+        private short leaderVersion;
+
+        private final DatastoreContext datastoreContext;
+        private final ShardPropsCreator shardPropsCreator;
 
         private ShardInformation(String shardName, ShardIdentifier shardId,
-                Map<String, String> peerAddresses) {
+                Map<String, String> peerAddresses, DatastoreContext datastoreContext,
+                ShardPropsCreator shardPropsCreator) {
             this.shardName = shardName;
             this.shardId = shardId;
             this.peerAddresses = peerAddresses;
+            this.datastoreContext = datastoreContext;
+            this.shardPropsCreator = shardPropsCreator;
+        }
+
+        Props newProps(SchemaContext schemaContext) {
+            return shardPropsCreator.newProps(shardId, peerAddresses, datastoreContext, schemaContext);
         }
 
         String getShardName() {
@@ -703,6 +812,14 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             return shardId;
         }
 
+        void setLocalDataTree(Optional<DataTree> localShardDataTree) {
+            this.localShardDataTree = localShardDataTree;
+        }
+
+        Optional<DataTree> getLocalShardDataTree() {
+            return localShardDataTree;
+        }
+
         Map<String, String> getPeerAddresses() {
             return peerAddresses;
         }
@@ -719,19 +836,32 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                                 peerId, peerAddress, actor.path());
                     }
 
-                    actor.tell(new PeerAddressResolved(peerId.toString(), peerAddress), sender);
+                    actor.tell(new PeerAddressResolved(peerId, peerAddress), sender);
                 }
 
                 notifyOnShardInitializedCallbacks();
             }
         }
 
+        void peerDown(String memberName, String peerId, ActorRef sender) {
+            if(peerAddresses.containsKey(peerId) && actor != null) {
+                actor.tell(new PeerDown(memberName, peerId), sender);
+            }
+        }
+
+        void peerUp(String memberName, String peerId, ActorRef sender) {
+            if(peerAddresses.containsKey(peerId) && actor != null) {
+                actor.tell(new PeerUp(memberName, peerId), sender);
+            }
+        }
+
         boolean isShardReady() {
             return !RaftState.Candidate.name().equals(role) && !Strings.isNullOrEmpty(role);
         }
 
         boolean isShardReadyWithLeaderId() {
-            return isShardReady() && (isLeader() || peerAddresses.containsKey(leaderId));
+            return leaderAvailable && isShardReady() && !RaftState.IsolatedLeader.name().equals(role) &&
+                    (isLeader() || peerAddresses.get(leaderId) != null);
         }
 
         boolean isShardInitialized() {
@@ -809,10 +939,31 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             return false;
         }
 
-        void setLeaderId(String leaderId) {
+        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;
+        }
+
+        short getLeaderVersion() {
+            return leaderVersion;
+        }
+
+        void setLeaderVersion(short leaderVersion) {
+            this.leaderVersion = leaderVersion;
         }
     }
 
@@ -823,18 +974,21 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         final Configuration configuration;
         final DatastoreContext datastoreContext;
         private final CountDownLatch waitTillReadyCountdownLatch;
+        private final PrimaryShardInfoFutureCache primaryShardInfoCache;
 
-        ShardManagerCreator(ClusterWrapper cluster,
-                            Configuration configuration, DatastoreContext datastoreContext, CountDownLatch waitTillReadyCountdownLatch) {
+        ShardManagerCreator(ClusterWrapper cluster, Configuration configuration, DatastoreContext datastoreContext,
+                CountDownLatch waitTillReadyCountdownLatch, PrimaryShardInfoFutureCache primaryShardInfoCache) {
             this.cluster = cluster;
             this.configuration = configuration;
             this.datastoreContext = datastoreContext;
             this.waitTillReadyCountdownLatch = waitTillReadyCountdownLatch;
+            this.primaryShardInfoCache = primaryShardInfoCache;
         }
 
         @Override
         public ShardManager create() throws Exception {
-            return new ShardManager(cluster, configuration, datastoreContext, waitTillReadyCountdownLatch);
+            return new ShardManager(cluster, configuration, datastoreContext, waitTillReadyCountdownLatch,
+                    primaryShardInfoCache);
         }
     }
 
@@ -889,6 +1043,11 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         }
     }
 
+    /**
+     * We no longer persist SchemaContextModules but keep this class around for now for backwards
+     * compatibility so we don't get de-serialization failures on upgrade from Helium.
+     */
+    @Deprecated
     static class SchemaContextModules implements Serializable {
         private static final long serialVersionUID = -8884620101025936590L;