CDS: Implement front-end support for local transactions
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ShardManager.java
index 426a2e0934f173560647a13569b22d1e06f632b2..f4fa7b3a97e8617f8ac474b494a08deb330a0b6d 100644 (file)
@@ -11,6 +11,7 @@ package org.opendaylight.controller.cluster.datastore;
 import akka.actor.ActorPath;
 import akka.actor.ActorRef;
 import akka.actor.Address;
+import akka.actor.Cancellable;
 import akka.actor.OneForOneStrategy;
 import akka.actor.Props;
 import akka.actor.SupervisorStrategy;
@@ -20,37 +21,54 @@ 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.Lists;
+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 org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActorWithMetering;
+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.identifiers.ShardManagerIdentifier;
 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.ActorNotInitialized;
 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.PrimaryNotFound;
+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.UpdateSchemaContext;
 import org.opendaylight.controller.cluster.datastore.utils.Dispatchers;
+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.data.api.schema.tree.DataTree;
 import org.opendaylight.yangtools.yang.model.api.ModuleIdentifier;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.slf4j.Logger;
@@ -68,7 +86,7 @@ import scala.concurrent.duration.Duration;
  */
 public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
-    private final Logger LOG = LoggerFactory.getLogger(getClass());
+    private static final Logger LOG = LoggerFactory.getLogger(ShardManager.class);
 
     // Stores a mapping between a member name and the address of the member
     // Member names look like "member-1", "member-2" etc and are as specified
@@ -84,32 +102,38 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     // A data store could be of type config/operational
     private final String type;
 
+    private final String shardManagerIdentifierString;
+
     private final ClusterWrapper cluster;
 
     private final Configuration configuration;
 
     private final String shardDispatcherPath;
 
-    private ShardManagerInfoMBean mBean;
+    private ShardManagerInfo mBean;
 
-    private final DatastoreContext datastoreContext;
+    private DatastoreContext datastoreContext;
 
     private Collection<String> knownModules = Collections.emptySet();
 
     private final DataPersistenceProvider dataPersistenceProvider;
 
+    private final CountDownLatch waitTillReadyCountdownLatch;
+
     /**
      */
     protected ShardManager(ClusterWrapper cluster, Configuration configuration,
-            DatastoreContext datastoreContext) {
+            DatastoreContext datastoreContext, CountDownLatch waitTillReadyCountdownLatch) {
 
         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;
 
         // Subscribe this actor to cluster member events
         cluster.subscribeToMemberEvents(getSelf());
@@ -118,24 +142,33 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
     }
 
     protected DataPersistenceProvider createDataPersistenceProvider(boolean persistent) {
-        return (persistent) ? new PersistentDataProvider() : new NonPersistentDataProvider();
+        return (persistent) ? new PersistentDataProvider(this) : new NonPersistentDataProvider();
     }
 
     public static Props props(
         final ClusterWrapper cluster,
         final Configuration configuration,
-        final DatastoreContext datastoreContext) {
+        final DatastoreContext datastoreContext,
+        final CountDownLatch waitTillReadyCountdownLatch) {
 
         Preconditions.checkNotNull(cluster, "cluster should not be null");
         Preconditions.checkNotNull(configuration, "configuration should not be null");
+        Preconditions.checkNotNull(waitTillReadyCountdownLatch, "waitTillReadyCountdownLatch should not be null");
+
+        return Props.create(new ShardManagerCreator(cluster, configuration, datastoreContext, waitTillReadyCountdownLatch));
+    }
+
+    @Override
+    public void postStop() {
+        LOG.info("Stopping ShardManager");
 
-        return Props.create(new ShardManagerCreator(cluster, configuration, datastoreContext));
+        mBean.unregisterMBean();
     }
 
     @Override
     public void handleCommand(Object message) throws Exception {
-        if (message.getClass().equals(FindPrimary.SERIALIZABLE_CLASS)) {
-            findPrimary(FindPrimary.fromSerializable(message));
+        if (message  instanceof FindPrimary) {
+            findPrimary((FindPrimary)message);
         } else if(message instanceof FindLocalShard){
             findLocalShard((FindLocalShard) message);
         } else if (message instanceof UpdateSchemaContext) {
@@ -148,12 +181,118 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             memberRemoved((ClusterEvent.MemberRemoved) message);
         } else if(message instanceof ClusterEvent.UnreachableMember) {
             ignoreMessage(message);
-        } else{
+        } else if(message instanceof DatastoreContext) {
+            onDatastoreContext((DatastoreContext)message);
+        } else if(message instanceof RoleChangeNotification) {
+            onRoleChangeNotification((RoleChangeNotification) message);
+        } else if(message instanceof FollowerInitialSyncUpStatus){
+            onFollowerInitialSyncStatus((FollowerInitialSyncUpStatus) message);
+        } else if(message instanceof ShardNotInitializedTimeout) {
+            onShardNotInitializedTimeout((ShardNotInitializedTimeout)message);
+        } else if(message instanceof ShardLeaderStateChanged) {
+            onLeaderStateChanged((ShardLeaderStateChanged)message);
+        } else {
             unknownMessage(message);
         }
 
     }
 
+    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.setLocalDataTree(leaderStateChanged.getLocalShardDataTree());
+            shardInformation.setLeaderId(leaderStateChanged.getLeaderId());
+            checkReady();
+        } else {
+            LOG.debug("No shard found with member Id {}", leaderStateChanged.getMemberId());
+        }
+    }
+
+    private void onShardNotInitializedTimeout(ShardNotInitializedTimeout message) {
+        ShardInformation shardInfo = message.getShardInfo();
+
+        LOG.debug("{}: Received ShardNotInitializedTimeout message for shard {}", persistenceId(),
+                shardInfo.getShardName());
+
+        shardInfo.removeOnShardInitialized(message.getOnShardInitialized());
+
+        if(!shardInfo.isShardInitialized()) {
+            LOG.debug("{}: Returning NotInitializedException for shard {}", persistenceId(), shardInfo.getShardName());
+            message.getSender().tell(createNotInitializedException(shardInfo.shardId), getSelf());
+        } else {
+            LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(), shardInfo.getShardName());
+            message.getSender().tell(createNoShardLeaderException(shardInfo.shardId), getSelf());
+        }
+    }
+
+    private void onFollowerInitialSyncStatus(FollowerInitialSyncUpStatus status) {
+        LOG.info("{} Received follower initial sync status for {} status sync done {}", persistenceId(),
+                status.getName(), status.isInitialSyncDone());
+
+        ShardInformation shardInformation = findShardInformation(status.getName());
+
+        if(shardInformation != null) {
+            shardInformation.setFollowerSyncStatus(status.isInitialSyncDone());
+
+            mBean.setSyncStatus(isInSync());
+        }
+
+    }
+
+    private void onRoleChangeNotification(RoleChangeNotification roleChanged) {
+        LOG.info("{}: Received role changed for {} from {} to {}", persistenceId(), roleChanged.getMemberId(),
+                roleChanged.getOldRole(), roleChanged.getNewRole());
+
+        ShardInformation shardInformation = findShardInformation(roleChanged.getMemberId());
+        if(shardInformation != null) {
+            shardInformation.setRole(roleChanged.getNewRole());
+            checkReady();
+            mBean.setSyncStatus(isInSync());
+        }
+    }
+
+
+    private ShardInformation findShardInformation(String memberId) {
+        for(ShardInformation info : localShards.values()){
+            if(info.getShardId().toString().equals(memberId)){
+                return info;
+            }
+        }
+
+        return null;
+    }
+
+    private boolean isReadyWithLeaderId() {
+        boolean isReady = true;
+        for (ShardInformation info : localShards.values()) {
+            if(!info.isShardReadyWithLeaderId()){
+                isReady = false;
+                break;
+            }
+        }
+        return isReady;
+    }
+
+    private boolean isInSync(){
+        for (ShardInformation info : localShards.values()) {
+            if(!info.isInSync()){
+                return false;
+            }
+        }
+        return true;
+    }
+
     private void onActorInitialized(Object message) {
         final ActorRef sender = getSender();
 
@@ -168,14 +307,18 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         if (shardId.getShardName() == null) {
             return;
         }
+
         markShardAsInitialized(shardId.getShardName());
     }
 
     private void markShardAsInitialized(String shardName) {
-        LOG.debug("Initializing shard [{}]", shardName);
+        LOG.debug("{}: Initializing shard [{}]", persistenceId(), shardName);
+
         ShardInformation shardInformation = localShards.get(shardName);
         if (shardInformation != null) {
             shardInformation.setActorInitialized();
+
+            shardInformation.getActor().tell(new RegisterRoleChangeListener(), self());
         }
     }
 
@@ -212,7 +355,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             return;
         }
 
-        sendResponse(shardInformation, message.isWaitUntilInitialized(), new Supplier<Object>() {
+        sendResponse(shardInformation, message.isWaitUntilInitialized(), false, new Supplier<Object>() {
             @Override
             public Object get() {
                 return new LocalShardFound(shardInformation.getActor());
@@ -220,20 +363,42 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         });
     }
 
-    private void sendResponse(ShardInformation shardInformation, boolean waitUntilInitialized,
-            final Supplier<Object> messageSupplier) {
-        if (!shardInformation.isShardInitialized()) {
-            if(waitUntilInitialized) {
+    private void sendResponse(ShardInformation shardInformation, boolean doWait,
+            boolean wantShardReady, final Supplier<Object> messageSupplier) {
+        if (!shardInformation.isShardInitialized() || (wantShardReady && !shardInformation.isShardReadyWithLeaderId())) {
+            if(doWait) {
                 final ActorRef sender = getSender();
                 final ActorRef self = self();
-                shardInformation.addRunnableOnInitialized(new Runnable() {
+
+                Runnable replyRunnable = new Runnable() {
                     @Override
                     public void run() {
                         sender.tell(messageSupplier.get(), self);
                     }
-                });
+                };
+
+                OnShardInitialized onShardInitialized = wantShardReady ? new OnShardReady(replyRunnable) :
+                    new OnShardInitialized(replyRunnable);
+
+                shardInformation.addOnShardInitialized(onShardInitialized);
+
+                LOG.debug("{}: Scheduling timer to wait for shard {}", persistenceId(), shardInformation.getShardName());
+
+                Cancellable timeoutSchedule = getContext().system().scheduler().scheduleOnce(
+                        datastoreContext.getShardInitializationTimeout().duration(), getSelf(),
+                        new ShardNotInitializedTimeout(shardInformation, onShardInitialized, sender),
+                        getContext().dispatcher(), getSelf());
+
+                onShardInitialized.setTimeoutSchedule(timeoutSchedule);
+
+            } else if (!shardInformation.isShardInitialized()) {
+                LOG.debug("{}: Returning NotInitializedException for shard {}", persistenceId(),
+                        shardInformation.getShardName());
+                getSender().tell(createNotInitializedException(shardInformation.shardId), getSelf());
             } else {
-                getSender().tell(new ActorNotInitialized(), getSelf());
+                LOG.debug("{}: Returning NoShardLeaderException for shard {}", persistenceId(),
+                        shardInformation.getShardName());
+                getSender().tell(createNoShardLeaderException(shardInformation.shardId), getSelf());
             }
 
             return;
@@ -242,19 +407,49 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         getSender().tell(messageSupplier.get(), getSelf());
     }
 
+    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));
+    }
+
+    private NotInitializedException createNotInitializedException(ShardIdentifier shardId) {
+        return new NotInitializedException(String.format(
+                "Found primary shard %s but it's not initialized yet. Please try again later", shardId));
+    }
+
     private void memberRemoved(ClusterEvent.MemberRemoved message) {
+        String memberName = message.member().roles().head();
+
+        LOG.debug("{}: Received MemberRemoved: memberName: {}, address: {}", persistenceId(), memberName,
+                message.member().address());
+
         memberNameToAddress.remove(message.member().roles().head());
     }
 
     private void memberUp(ClusterEvent.MemberUp message) {
         String memberName = message.member().roles().head();
 
+        LOG.debug("{}: Received MemberUp: memberName: {}, address: {}", persistenceId(), memberName,
+                message.member().address());
+
         memberNameToAddress.put(memberName, message.member().address());
 
         for(ShardInformation info : localShards.values()){
             String shardName = info.getShardName();
-            info.updatePeerAddress(getShardIdentifier(memberName, shardName),
-                getShardActorPath(shardName, memberName));
+            info.updatePeerAddress(getShardIdentifier(memberName, shardName).toString(),
+                getShardActorPath(shardName, memberName), getSelf());
+        }
+
+        checkReady();
+    }
+
+    private void onDatastoreContext(DatastoreContext context) {
+        datastoreContext = context;
+        for (ShardInformation info : localShards.values()) {
+            if (info.getActor() != null) {
+                info.getActor().tell(datastoreContext, getSelf());
+            }
         }
     }
 
@@ -287,9 +482,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                     LOG.debug("Sending new SchemaContext to Shards");
                     for (ShardInformation info : localShards.values()) {
                         if (info.getActor() == null) {
-                            info.setActor(getContext().actorOf(Shard.props(info.getShardId(),
-                                            info.getPeerAddresses(), datastoreContext, schemaContext)
-                                            .withDispatcher(shardDispatcherPath), info.getShardId().toString()));
+                            info.setActor(newShardActor(schemaContext, info));
                         } else {
                             info.getActor().tell(message, getSelf());
                         }
@@ -304,54 +497,76 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
 
     }
 
+    @VisibleForTesting
+    protected ClusterWrapper getCluster() {
+        return cluster;
+    }
+
+    @VisibleForTesting
+    protected ActorRef newShardActor(final SchemaContext schemaContext, ShardInformation info) {
+        return getContext().actorOf(Shard.props(info.getShardId(),
+                info.getPeerAddresses(), datastoreContext, schemaContext)
+                        .withDispatcher(shardDispatcherPath), info.getShardId().toString());
+    }
+
     private void findPrimary(FindPrimary message) {
-        String shardName = message.getShardName();
+        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);
         if (info != null) {
-            sendResponse(info, message.isWaitUntilInitialized(), new Supplier<Object>() {
+            sendResponse(info, message.isWaitUntilReady(), true, new Supplier<Object>() {
                 @Override
                 public Object get() {
-                    return new PrimaryFound(info.getActorPath().toString()).toSerializable();
+                    String primaryPath = info.getSerializedLeaderActor();
+                    Object found = canReturnLocalShardState && info.isLeader() ?
+                            new LocalPrimaryShardFound(primaryPath, info.getLocalShardDataTree().get()) :
+                                new RemotePrimaryShardFound(primaryPath);
+
+                    if(LOG.isDebugEnabled()) {
+                        LOG.debug("{}: Found primary for {}: {}", persistenceId(), shardName, found);
+                    }
+
+                    return found;
                 }
             });
 
             return;
         }
 
-        List<String> members = configuration.getMembersFromShardName(shardName);
+        for(Map.Entry<String, Address> entry: memberNameToAddress.entrySet()) {
+            if(!cluster.getCurrentMemberName().equals(entry.getKey())) {
+                String path = getShardManagerActorPathBuilder(entry.getValue()).toString();
 
-        if(cluster.getCurrentMemberName() != null) {
-            members.remove(cluster.getCurrentMemberName());
-        }
+                LOG.debug("{}: findPrimary for {} forwarding to remote ShardManager {}", persistenceId(),
+                        shardName, path);
 
-        /**
-         * FIXME: Instead of sending remote shard actor path back to sender,
-         * forward FindPrimary message to remote shard manager
-         */
-        // There is no way for us to figure out the primary (for now) so assume
-        // that one of the remote nodes is a primary
-        for(String memberName : members) {
-            Address address = memberNameToAddress.get(memberName);
-            if(address != null){
-                String path =
-                    getShardActorPath(shardName, memberName);
-                getSender().tell(new PrimaryFound(path).toSerializable(), getSelf());
+                getContext().actorSelection(path).forward(new RemoteFindPrimary(shardName,
+                        message.isWaitUntilReady()), getContext());
                 return;
             }
         }
-        getSender().tell(new PrimaryNotFound(shardName).toSerializable(), getSelf());
+
+        LOG.debug("{}: No shard found for {}", persistenceId(), shardName);
+
+        getSender().tell(new PrimaryNotFoundException(
+                String.format("No primary shard found for %s.", shardName)), getSelf());
+    }
+
+    private StringBuilder getShardManagerActorPathBuilder(Address address) {
+        StringBuilder builder = new StringBuilder();
+        builder.append(address.toString()).append("/user/").append(shardManagerIdentifierString);
+        return builder;
     }
 
     private String getShardActorPath(String shardName, String memberName) {
         Address address = memberNameToAddress.get(memberName);
         if(address != null) {
-            StringBuilder builder = new StringBuilder();
-            builder.append(address.toString())
-                .append("/user/")
-                .append(ShardManagerIdentifier.builder().type(type).build().toString())
-                .append("/")
+            StringBuilder builder = getShardManagerActorPathBuilder(address);
+            builder.append("/")
                 .append(getShardIdentifier(memberName, shardName));
             return builder.toString();
         }
@@ -383,7 +598,7 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         List<String> localShardActorNames = new ArrayList<>();
         for(String shardName : memberShardNames){
             ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
-            Map<ShardIdentifier, String> peerAddresses = getPeerAddresses(shardName);
+            Map<String, String> peerAddresses = getPeerAddresses(shardName);
             localShardActorNames.add(shardId.toString());
             localShards.put(shardName, new ShardInformation(shardName, shardId, peerAddresses));
         }
@@ -398,22 +613,19 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
      * @param shardName
      * @return
      */
-    private Map<ShardIdentifier, String> getPeerAddresses(String shardName){
+    private Map<String, String> getPeerAddresses(String shardName){
 
-        Map<ShardIdentifier, String> peerAddresses = new HashMap<>();
+        Map<String, String> peerAddresses = new HashMap<>();
 
-        List<String> members =
-            this.configuration.getMembersFromShardName(shardName);
+        List<String> members = this.configuration.getMembersFromShardName(shardName);
 
         String currentMemberName = this.cluster.getCurrentMemberName();
 
         for(String memberName : members){
             if(!currentMemberName.equals(memberName)){
-                ShardIdentifier shardId = getShardIdentifier(memberName,
-                    shardName);
-                String path =
-                    getShardActorPath(shardName, currentMemberName);
-                peerAddresses.put(shardId, path);
+                ShardIdentifier shardId = getShardIdentifier(memberName, shardName);
+                String path = getShardActorPath(shardName, currentMemberName);
+                peerAddresses.put(shardId.toString(), path);
             }
         }
         return peerAddresses;
@@ -449,20 +661,31 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         return dataPersistenceProvider;
     }
 
-    private class ShardInformation {
+    @VisibleForTesting
+    ShardManagerInfoMBean getMBean(){
+        return mBean;
+    }
+
+    @VisibleForTesting
+    protected static class ShardInformation {
         private final ShardIdentifier shardId;
         private final String shardName;
         private ActorRef actor;
         private ActorPath actorPath;
-        private final Map<ShardIdentifier, String> peerAddresses;
+        private final Map<String, String> peerAddresses;
+        private Optional<DataTree> localShardDataTree;
 
         // flag that determines if the actor is ready for business
         private boolean actorInitialized = false;
 
-        private final List<Runnable> runnablesOnInitialized = Lists.newArrayList();
+        private boolean followerSyncStatus = false;
+
+        private final Set<OnShardInitialized> onShardInitializedSet = Sets.newHashSet();
+        private String role ;
+        private String leaderId;
 
         private ShardInformation(String shardName, ShardIdentifier shardId,
-                Map<ShardIdentifier, String> peerAddresses) {
+                Map<String, String> peerAddresses) {
             this.shardName = shardName;
             this.shardId = shardId;
             this.peerAddresses = peerAddresses;
@@ -489,11 +712,19 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
             return shardId;
         }
 
-        Map<ShardIdentifier, String> getPeerAddresses() {
+        void setLocalDataTree(Optional<DataTree> localShardDataTree) {
+            this.localShardDataTree = localShardDataTree;
+        }
+
+        Optional<DataTree> getLocalShardDataTree() {
+            return localShardDataTree;
+        }
+
+        Map<String, String> getPeerAddresses() {
             return peerAddresses;
         }
 
-        void updatePeerAddress(ShardIdentifier peerId, String peerAddress){
+        void updatePeerAddress(String peerId, String peerAddress, ActorRef sender){
             LOG.info("updatePeerAddress for peer {} with address {}", peerId,
                 peerAddress);
             if(peerAddresses.containsKey(peerId)){
@@ -505,27 +736,100 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
                                 peerId, peerAddress, actor.path());
                     }
 
-                    actor.tell(new PeerAddressResolved(peerId, peerAddress), getSelf());
+                    actor.tell(new PeerAddressResolved(peerId.toString(), peerAddress), sender);
                 }
+
+                notifyOnShardInitializedCallbacks();
             }
         }
 
+        boolean isShardReady() {
+            return !RaftState.Candidate.name().equals(role) && !Strings.isNullOrEmpty(role);
+        }
+
+        boolean isShardReadyWithLeaderId() {
+            return isShardReady() && (isLeader() || peerAddresses.get(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 peerAddresses.get(leaderId);
+            }
+        }
+
         void setActorInitialized() {
+            LOG.debug("Shard {} is initialized", shardId);
+
             this.actorInitialized = true;
 
-            for(Runnable runnable: runnablesOnInitialized) {
-                runnable.run();
+            notifyOnShardInitializedCallbacks();
+        }
+
+        private void notifyOnShardInitializedCallbacks() {
+            if(onShardInitializedSet.isEmpty()) {
+                return;
             }
 
-            runnablesOnInitialized.clear();
+            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 addRunnableOnInitialized(Runnable runnable) {
-            runnablesOnInitialized.add(runnable);
+        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;
+        }
+
+        void setLeaderId(String leaderId) {
+            this.leaderId = leaderId;
+
+            notifyOnShardInitializedCallbacks();
         }
     }
 
@@ -535,17 +839,70 @@ public class ShardManager extends AbstractUntypedPersistentActorWithMetering {
         final ClusterWrapper cluster;
         final Configuration configuration;
         final DatastoreContext datastoreContext;
+        private final CountDownLatch waitTillReadyCountdownLatch;
 
         ShardManagerCreator(ClusterWrapper cluster,
-                Configuration configuration, DatastoreContext datastoreContext) {
+                            Configuration configuration, DatastoreContext datastoreContext, CountDownLatch waitTillReadyCountdownLatch) {
             this.cluster = cluster;
             this.configuration = configuration;
             this.datastoreContext = datastoreContext;
+            this.waitTillReadyCountdownLatch = waitTillReadyCountdownLatch;
         }
 
         @Override
         public ShardManager create() throws Exception {
-            return new ShardManager(cluster, configuration, datastoreContext);
+            return new ShardManager(cluster, configuration, datastoreContext, waitTillReadyCountdownLatch);
+        }
+    }
+
+    private static class OnShardInitialized {
+        private final Runnable replyRunnable;
+        private Cancellable timeoutSchedule;
+
+        OnShardInitialized(Runnable replyRunnable) {
+            this.replyRunnable = replyRunnable;
+        }
+
+        Runnable getReplyRunnable() {
+            return replyRunnable;
+        }
+
+        Cancellable getTimeoutSchedule() {
+            return timeoutSchedule;
+        }
+
+        void setTimeoutSchedule(Cancellable timeoutSchedule) {
+            this.timeoutSchedule = timeoutSchedule;
+        }
+    }
+
+    private 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;
         }
     }