BUG-5280: use MemberName instead of String
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / entityownership / EntityOwnershipShard.java
index 334e093d7f4babfafef4174275d7c32092decbe1..9e3de6ba27e7ab1927872f612874036b666f1aea 100644 (file)
@@ -7,7 +7,6 @@
  */
 package org.opendaylight.controller.cluster.datastore.entityownership;
 
-import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.CANDIDATE_NAME_NODE_ID;
 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.CANDIDATE_NODE_ID;
 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_ID_NODE_ID;
 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_ID_QNAME;
@@ -25,8 +24,8 @@ import static org.opendaylight.controller.cluster.datastore.entityownership.Enti
 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.entityOwnersWithCandidate;
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
+import akka.actor.Cancellable;
 import akka.pattern.Patterns;
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
@@ -37,6 +36,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import org.opendaylight.controller.cluster.access.concepts.MemberName;
 import org.opendaylight.controller.cluster.datastore.DatastoreContext;
 import org.opendaylight.controller.cluster.datastore.Shard;
 import org.opendaylight.controller.cluster.datastore.entityownership.messages.CandidateAdded;
@@ -47,8 +47,7 @@ import org.opendaylight.controller.cluster.datastore.entityownership.messages.Se
 import org.opendaylight.controller.cluster.datastore.entityownership.messages.UnregisterCandidateLocal;
 import org.opendaylight.controller.cluster.datastore.entityownership.messages.UnregisterListenerLocal;
 import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.EntityOwnerSelectionStrategy;
-import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.EntityOwnerSelectionStrategyWrapper;
-import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.FirstCandidateSelectionStrategy;
+import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.EntityOwnerSelectionStrategyConfig;
 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
 import org.opendaylight.controller.cluster.datastore.messages.PeerDown;
@@ -57,15 +56,16 @@ import org.opendaylight.controller.cluster.datastore.messages.SuccessReply;
 import org.opendaylight.controller.cluster.datastore.modification.DeleteModification;
 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
+import org.opendaylight.controller.cluster.raft.RaftState;
 import org.opendaylight.controller.md.sal.common.api.clustering.Entity;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
-import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
 import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild;
 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
 import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
 
 /**
  * Special Shard for EntityOwnership.
@@ -73,13 +73,14 @@ import scala.concurrent.Future;
  * @author Thomas Pantelis
  */
 class EntityOwnershipShard extends Shard {
-    private final String localMemberName;
+    private final MemberName localMemberName;
     private final EntityOwnershipShardCommitCoordinator commitCoordinator;
     private final EntityOwnershipListenerSupport listenerSupport;
-    private final Set<String> downPeerMemberNames = new HashSet<>();
-    private final Map<String, String> peerIdToMemberNames = new HashMap<>();
-    private final Map<String, EntityOwnerSelectionStrategyWrapper> ownerSelectionStrategies = new HashMap<>();
-    private final EntityOwnerSelectionStrategyWrapper defaultEntityOwnerSelectionStrategy;
+    private final Set<MemberName> downPeerMemberNames = new HashSet<>();
+    private final Map<String, MemberName> peerIdToMemberNames = new HashMap<>();
+    private final EntityOwnerSelectionStrategyConfig strategyConfig;
+    private final Map<YangInstanceIdentifier, Cancellable> entityToScheduledOwnershipTask = new HashMap<>();
+    private final EntityOwnershipStatistics entityOwnershipStatistics;
 
     private static DatastoreContext noPersistenceDatastoreContext(DatastoreContext datastoreContext) {
         return DatastoreContext.newBuilderFrom(datastoreContext).persistent(false).build();
@@ -90,8 +91,9 @@ class EntityOwnershipShard extends Shard {
         this.localMemberName = builder.localMemberName;
         this.commitCoordinator = new EntityOwnershipShardCommitCoordinator(builder.localMemberName, LOG);
         this.listenerSupport = new EntityOwnershipListenerSupport(getContext(), persistenceId());
-        this.defaultEntityOwnerSelectionStrategy =
-                createEntityOwnerSelectionStrategyWrapper(FirstCandidateSelectionStrategy.INSTANCE);
+        this.strategyConfig = builder.ownerSelectionStrategyConfig;
+        this.entityOwnershipStatistics = new EntityOwnershipStatistics();
+        this.entityOwnershipStatistics.init(getDataStore());
 
         for(String peerId: getRaftActorContext().getPeerIds()) {
             ShardIdentifier shardId = ShardIdentifier.builder().fromShardIdString(peerId).build();
@@ -113,7 +115,7 @@ class EntityOwnershipShard extends Shard {
     }
 
     @Override
-    public void onReceiveCommand(final Object message) throws Exception {
+    public void handleNonRaftCommand(final Object message) {
         if(message instanceof RegisterCandidateLocal) {
             onRegisterCandidateLocal((RegisterCandidateLocal) message);
         } else if(message instanceof UnregisterCandidateLocal) {
@@ -133,15 +135,27 @@ class EntityOwnershipShard extends Shard {
         } else if(message instanceof SelectOwner) {
             onSelectOwner((SelectOwner) message);
         } else if(!commitCoordinator.handleMessage(message, this)) {
-            super.onReceiveCommand(message);
+            super.handleNonRaftCommand(message);
         }
     }
 
     private void onSelectOwner(SelectOwner selectOwner) {
+        LOG.debug("{}: onSelectOwner: {}", persistenceId(), selectOwner);
+
         String currentOwner = getCurrentOwner(selectOwner.getEntityPath());
         if(Strings.isNullOrEmpty(currentOwner)) {
-            writeNewOwner(selectOwner.getEntityPath(), newOwner(selectOwner.getAllCandidates(),
+            String entityType = EntityOwnersModel.entityTypeFromEntityPath(selectOwner.getEntityPath());
+            writeNewOwner(selectOwner.getEntityPath(), newOwner(currentOwner, selectOwner.getAllCandidates(),
+                    entityOwnershipStatistics.byEntityType(entityType),
                     selectOwner.getOwnerSelectionStrategy()));
+
+            Cancellable cancellable = entityToScheduledOwnershipTask.get(selectOwner.getEntityPath());
+            if(cancellable != null){
+                if(!cancellable.isCancelled()){
+                    cancellable.cancel();
+                }
+                entityToScheduledOwnershipTask.remove(selectOwner.getEntityPath());
+            }
         }
     }
 
@@ -151,7 +165,7 @@ class EntityOwnershipShard extends Shard {
         listenerSupport.setHasCandidateForEntity(registerCandidate.getEntity());
 
         NormalizedNode<?, ?> entityOwners = entityOwnersWithCandidate(registerCandidate.getEntity().getType(),
-                registerCandidate.getEntity().getId(), localMemberName);
+                registerCandidate.getEntity().getId(), localMemberName.getName());
         commitCoordinator.commitModification(new MergeModification(ENTITY_OWNERS_PATH, entityOwners), this);
 
         getSender().tell(SuccessReply.INSTANCE, getSelf());
@@ -163,7 +177,7 @@ class EntityOwnershipShard extends Shard {
         Entity entity = unregisterCandidate.getEntity();
         listenerSupport.unsetHasCandidateForEntity(entity);
 
-        YangInstanceIdentifier candidatePath = candidatePath(entity.getType(), entity.getId(), localMemberName);
+        YangInstanceIdentifier candidatePath = candidatePath(entity.getType(), entity.getId(), localMemberName.getName());
         commitCoordinator.commitModification(new DeleteModification(candidatePath), this);
 
         getSender().tell(SuccessReply.INSTANCE, getSelf());
@@ -176,17 +190,27 @@ class EntityOwnershipShard extends Shard {
 
         getSender().tell(SuccessReply.INSTANCE, getSelf());
 
-        searchForEntitiesOwnedBy(localMemberName, new EntityWalker() {
-            @Override
-            public void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode) {
-                Optional<DataContainerChild<? extends PathArgument, ?>> possibleType =
-                        entityTypeNode.getChild(ENTITY_TYPE_NODE_ID);
-                String entityType = possibleType.isPresent() ? possibleType.get().getValue().toString() : null;
-                if (registerListener.getEntityType().equals(entityType)) {
-                    Entity entity = new Entity(entityType,
-                            (YangInstanceIdentifier) entityNode.getChild(ENTITY_ID_NODE_ID).get().getValue());
-                    listenerSupport.notifyEntityOwnershipListener(entity, false, true, true, registerListener.getListener());
+        searchForEntities((entityTypeNode, entityNode) -> {
+            Optional<DataContainerChild<?, ?>> possibleType = entityTypeNode.getChild(ENTITY_TYPE_NODE_ID);
+            String entityType = possibleType.isPresent() ? possibleType.get().getValue().toString() : null;
+            if (registerListener.getEntityType().equals(entityType)) {
+                final boolean hasOwner;
+                final boolean isOwner;
+
+                Optional<DataContainerChild<?, ?>> possibleOwner = entityNode.getChild(ENTITY_OWNER_NODE_ID);
+                if (possibleOwner.isPresent()) {
+                    isOwner = localMemberName.getName().equals(possibleOwner.get().getValue().toString());
+                    hasOwner = true;
+                } else {
+                    isOwner = false;
+                    hasOwner = false;
                 }
+
+                Entity entity = new Entity(entityType,
+                    (YangInstanceIdentifier) entityNode.getChild(ENTITY_ID_NODE_ID).get().getValue());
+
+                listenerSupport.notifyEntityOwnershipListener(entity, false, isOwner, hasOwner,
+                    registerListener.getListener());
             }
         });
     }
@@ -228,31 +252,90 @@ class EntityOwnershipShard extends Shard {
         return getLeader() != null && !isIsolatedLeader();
     }
 
+    /**
+     * Determine if we are in jeopardy based on observed RAFT state.
+     */
+    private static boolean inJeopardy(final RaftState state) {
+        switch (state) {
+            case Candidate:
+            case Follower:
+            case Leader:
+                return false;
+            case IsolatedLeader:
+                return true;
+        }
+        throw new IllegalStateException("Unsupported RAFT state " + state);
+    }
+
+    private void notifyAllListeners() {
+        searchForEntities((entityTypeNode, entityNode) -> {
+            Optional<DataContainerChild<?, ?>> possibleType = entityTypeNode.getChild(ENTITY_TYPE_NODE_ID);
+            if (possibleType.isPresent()) {
+                final boolean hasOwner;
+                final boolean isOwner;
+
+                Optional<DataContainerChild<?, ?>> possibleOwner = entityNode.getChild(ENTITY_OWNER_NODE_ID);
+                if (possibleOwner.isPresent()) {
+                    isOwner = localMemberName.getName().equals(possibleOwner.get().getValue().toString());
+                    hasOwner = true;
+                } else {
+                    isOwner = false;
+                    hasOwner = false;
+                }
+
+                Entity entity = new Entity(possibleType.get().getValue().toString(),
+                    (YangInstanceIdentifier) entityNode.getChild(ENTITY_ID_NODE_ID).get().getValue());
+
+                listenerSupport.notifyEntityOwnershipListeners(entity, isOwner, isOwner, hasOwner);
+            }
+        });
+    }
+
     @Override
     protected void onStateChanged() {
         super.onStateChanged();
 
-        commitCoordinator.onStateChanged(this, isLeader());
+        boolean isLeader = isLeader();
+        LOG.debug("{}: onStateChanged: isLeader: {}, hasLeader: {}", persistenceId(), isLeader, hasLeader());
+
+        // Examine current RAFT state to see if we are in jeopardy, potentially notifying all listeners
+        final boolean inJeopardy = inJeopardy(getRaftState());
+        final boolean wasInJeopardy = listenerSupport.setInJeopardy(inJeopardy);
+        if (inJeopardy != wasInJeopardy) {
+            LOG.debug("{}: {} jeopardy state, notifying all listeners", persistenceId(),
+                inJeopardy ? "entered" : "left");
+            notifyAllListeners();
+        }
+
+        commitCoordinator.onStateChanged(this, isLeader);
     }
 
     @Override
     protected void onLeaderChanged(String oldLeader, String newLeader) {
         super.onLeaderChanged(oldLeader, newLeader);
 
+        boolean isLeader = isLeader();
         LOG.debug("{}: onLeaderChanged: oldLeader: {}, newLeader: {}, isLeader: {}", persistenceId(), oldLeader,
-                newLeader, isLeader());
-
-        if(isLeader()) {
-            // We were just elected leader. If the old leader is down, select new owners for the entities
-            // owned by the down leader.
+                newLeader, isLeader);
 
-            String oldLeaderMemberName = peerIdToMemberNames.get(oldLeader);
+        if (isLeader) {
 
-            LOG.debug("{}: oldLeaderMemberName: {}", persistenceId(), oldLeaderMemberName);
+            // Clear all existing strategies so that they get re-created when we call createStrategy again
+            // This allows the strategies to be re-initialized with existing statistics maintained by
+            // EntityOwnershipStatistics
+            strategyConfig.clearStrategies();
 
-            if(downPeerMemberNames.contains(oldLeaderMemberName)) {
-                selectNewOwnerForEntitiesOwnedBy(oldLeaderMemberName);
+            // Remove the candidates for all members that are known to be down. In a cluster which has greater than
+            // 3 nodes it is possible for a some node beside the leader being down when the leadership transitions
+            // it makes sense to use this event to remove all the candidates for those downed nodes
+            for (MemberName downPeerName : downPeerMemberNames) {
+                removeCandidateFromEntities(downPeerName);
             }
+        } else {
+            // The leader changed - notify the coordinator to check if pending modifications need to be sent.
+            // While onStateChanged also does this, this method handles the case where the shard hears from a
+            // leader and stays in the follower state. In that case no behavior state change occurs.
+            commitCoordinator.onStateChanged(this, isLeader);
         }
     }
 
@@ -261,9 +344,11 @@ class EntityOwnershipShard extends Shard {
 
         if(isLeader()) {
             String currentOwner = getCurrentOwner(message.getEntityPath());
-            if(message.getRemovedCandidate().equals(currentOwner)){
+            if(message.getRemovedCandidate().equals(currentOwner) || message.getRemainingCandidates().size() == 0){
+                String entityType = EntityOwnersModel.entityTypeFromEntityPath(message.getEntityPath());
                 writeNewOwner(message.getEntityPath(),
-                        newOwner(message.getRemainingCandidates(), getEntityOwnerElectionStrategyWrapper(message.getEntityPath())));
+                        newOwner(currentOwner, message.getRemainingCandidates(), entityOwnershipStatistics.byEntityType(entityType),
+                                getEntityOwnerElectionStrategy(message.getEntityPath())));
             }
         } else {
             // We're not the leader. If the removed candidate is our local member then check if we actually
@@ -271,28 +356,21 @@ class EntityOwnershipShard extends Shard {
             // and the leader removed our candidate since the leader can't tell the difference between a
             // temporary network partition and a node's process actually restarted. So, in that case, re-add
             // our candidate.
-            if(localMemberName.equals(message.getRemovedCandidate()) &&
+            if(localMemberName.getName().equals(message.getRemovedCandidate()) &&
                     listenerSupport.hasCandidateForEntity(createEntity(message.getEntityPath()))) {
                 LOG.debug("Local candidate member was removed but a local candidate is registered for {}" +
                     " - adding back local candidate", message.getEntityPath());
 
                 commitCoordinator.commitModification(new MergeModification(
-                        candidatePath(message.getEntityPath(), localMemberName),
-                        candidateMapEntry(localMemberName)), this);
+                        candidatePath(message.getEntityPath(), localMemberName.getName()),
+                        candidateMapEntry(localMemberName.getName())), this);
             }
         }
     }
 
-    private EntityOwnerSelectionStrategyWrapper getEntityOwnerElectionStrategyWrapper(YangInstanceIdentifier entityPath) {
-        String entityType = EntityOwnersModel.entityTypeFromEntityPath(entityPath);
-        EntityOwnerSelectionStrategyWrapper entityOwnerSelectionStrategy = ownerSelectionStrategies.get(entityType);
-
-        if(entityOwnerSelectionStrategy == null){
-            entityOwnerSelectionStrategy = defaultEntityOwnerSelectionStrategy;
-            ownerSelectionStrategies.put(entityType, entityOwnerSelectionStrategy);
-        }
-
-        return entityOwnerSelectionStrategy;
+    private EntityOwnerSelectionStrategy getEntityOwnerElectionStrategy(YangInstanceIdentifier entityPath) {
+        final String entityType = EntityOwnersModel.entityTypeFromEntityPath(entityPath);
+        return strategyConfig.createStrategy(entityType, entityOwnershipStatistics.byEntityType(entityType));
     }
 
     private void onCandidateAdded(CandidateAdded message) {
@@ -306,13 +384,28 @@ class EntityOwnershipShard extends Shard {
         // remove it from the downPeerMemberNames.
         downPeerMemberNames.remove(message.getNewCandidate());
 
-        String currentOwner = getCurrentOwner(message.getEntityPath());
-        if(Strings.isNullOrEmpty(currentOwner)){
-            EntityOwnerSelectionStrategyWrapper strategy = getEntityOwnerElectionStrategyWrapper(message.getEntityPath());
-            if(strategy.selectionDelayInMillis() == 0L) {
-                writeNewOwner(message.getEntityPath(), newOwner(message.getAllCandidates(), strategy));
+        final String currentOwner = getCurrentOwner(message.getEntityPath());
+        final EntityOwnerSelectionStrategy strategy = getEntityOwnerElectionStrategy(message.getEntityPath());
+        final String entityType = EntityOwnersModel.entityTypeFromEntityPath(message.getEntityPath());
+
+        // Available members is all the known peers - the number of peers that are down + self
+        // So if there are 2 peers and 1 is down then availableMembers will be 2
+        final int availableMembers = (peerIdToMemberNames.size() - downPeerMemberNames.size()) + 1;
+
+        LOG.debug("{}: Using strategy {} to select owner, currentOwner = {}", persistenceId(), strategy, currentOwner);
+
+        if(!message.getAllCandidates().contains(currentOwner)){
+            if(strategy.getSelectionDelayInMillis() == 0L) {
+                writeNewOwner(message.getEntityPath(), newOwner(currentOwner, message.getAllCandidates(),
+                        entityOwnershipStatistics.byEntityType(entityType), strategy));
+            } else if(message.getAllCandidates().size() == availableMembers) {
+                LOG.debug("{}: Received the maximum candidates requests : {} writing new owner",
+                        persistenceId(), availableMembers);
+                cancelOwnerSelectionTask(message.getEntityPath());
+                writeNewOwner(message.getEntityPath(), newOwner(currentOwner, message.getAllCandidates(),
+                        entityOwnershipStatistics.byEntityType(entityType), strategy));
             } else {
-                strategy.scheduleOwnerSelection(message.getEntityPath(), message.getAllCandidates());
+                scheduleOwnerSelection(message.getEntityPath(), message.getAllCandidates(), strategy);
             }
         }
     }
@@ -320,7 +413,7 @@ class EntityOwnershipShard extends Shard {
     private void onPeerDown(PeerDown peerDown) {
         LOG.info("{}: onPeerDown: {}", persistenceId(), peerDown);
 
-        String downMemberName = peerDown.getMemberName();
+        MemberName downMemberName = peerDown.getMemberName();
         if(downPeerMemberNames.add(downMemberName) && isLeader()) {
             // Remove the down peer as a candidate from all entities.
             removeCandidateFromEntities(downMemberName);
@@ -332,75 +425,36 @@ class EntityOwnershipShard extends Shard {
 
         peerIdToMemberNames.put(peerUp.getPeerId(), peerUp.getMemberName());
         downPeerMemberNames.remove(peerUp.getMemberName());
-    }
 
-    private void selectNewOwnerForEntitiesOwnedBy(String owner) {
-        final BatchedModifications modifications = commitCoordinator.newBatchedModifications();
-        searchForEntitiesOwnedBy(owner, new EntityWalker() {
-            @Override
-            public void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode) {
-
-                YangInstanceIdentifier entityPath = YangInstanceIdentifier.builder(ENTITY_TYPES_PATH).
-                        node(entityTypeNode.getIdentifier()).node(ENTITY_NODE_ID).node(entityNode.getIdentifier()).
-                        node(ENTITY_OWNER_NODE_ID).build();
-
-                Object newOwner = newOwner(getCandidateNames(entityNode), getEntityOwnerElectionStrategyWrapper(entityPath));
-
-                LOG.debug("{}: Found entity {}, writing new owner {}", persistenceId(), entityPath, newOwner);
-
-                modifications.addModification(new WriteModification(entityPath,
-                        ImmutableNodes.leafNode(ENTITY_OWNER_NODE_ID, newOwner)));
-            }
-        });
-
-        commitCoordinator.commitModifications(modifications, this);
+        // Notify the coordinator to check if pending modifications need to be sent. We do this here
+        // to handle the case where the leader's peer address isn't now yet when a prior state or
+        // leader change occurred.
+        commitCoordinator.onStateChanged(this, isLeader());
     }
 
-    private void removeCandidateFromEntities(final String owner) {
+    private void removeCandidateFromEntities(final MemberName owner) {
         final BatchedModifications modifications = commitCoordinator.newBatchedModifications();
-        searchForEntities(new EntityWalker() {
-            @Override
-            public void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode) {
-                if (hasCandidate(entityNode, owner)) {
-                    YangInstanceIdentifier entityId =
-                            (YangInstanceIdentifier) entityNode.getIdentifier().getKeyValues().get(ENTITY_ID_QNAME);
-                    YangInstanceIdentifier candidatePath = candidatePath(
-                            entityTypeNode.getIdentifier().getKeyValues().get(ENTITY_TYPE_QNAME).toString(),
-                            entityId, owner);
-
-                    LOG.info("{}: Found entity {}, removing candidate {}, path {}", persistenceId(), entityId,
-                            owner, candidatePath);
-
-                    modifications.addModification(new DeleteModification(candidatePath));
-                }
+        searchForEntities((entityTypeNode, entityNode) -> {
+            if (hasCandidate(entityNode, owner)) {
+                YangInstanceIdentifier entityId =
+                        (YangInstanceIdentifier) entityNode.getIdentifier().getKeyValues().get(ENTITY_ID_QNAME);
+                YangInstanceIdentifier candidatePath = candidatePath(
+                        entityTypeNode.getIdentifier().getKeyValues().get(ENTITY_TYPE_QNAME).toString(),
+                        entityId, owner.getName());
+
+                LOG.info("{}: Found entity {}, removing candidate {}, path {}", persistenceId(), entityId,
+                        owner, candidatePath);
+
+                modifications.addModification(new DeleteModification(candidatePath));
             }
         });
 
         commitCoordinator.commitModifications(modifications, this);
     }
 
-    private static boolean hasCandidate(MapEntryNode entity, String candidateName) {
-        return ((MapNode)entity.getChild(CANDIDATE_NODE_ID).get()).getChild(candidateNodeKey(candidateName)).isPresent();
-    }
-
-    private void searchForEntitiesOwnedBy(final String owner, final EntityWalker walker) {
-        Optional<NormalizedNode<?, ?>> possibleEntityTypes = getDataStore().readNode(ENTITY_TYPES_PATH);
-        if(!possibleEntityTypes.isPresent()) {
-            return;
-        }
-
-        LOG.debug("{}: Searching for entities owned by {}", persistenceId(), owner);
-
-        searchForEntities(new EntityWalker() {
-            @Override
-            public void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode) {
-                Optional<DataContainerChild<? extends PathArgument, ?>> possibleOwner =
-                        entityNode.getChild(ENTITY_OWNER_NODE_ID);
-                if(possibleOwner.isPresent() && owner.equals(possibleOwner.get().getValue().toString())) {
-                    walker.onEntity(entityTypeNode, entityNode);
-                }
-            }
-        });
+    private static boolean hasCandidate(MapEntryNode entity, MemberName candidateName) {
+        return ((MapNode)entity.getChild(CANDIDATE_NODE_ID).get()).getChild(candidateNodeKey(candidateName.getName()))
+                .isPresent();
     }
 
     private void searchForEntities(EntityWalker walker) {
@@ -410,10 +464,10 @@ class EntityOwnershipShard extends Shard {
         }
 
         for(MapEntryNode entityType:  ((MapNode) possibleEntityTypes.get()).getValue()) {
-            Optional<DataContainerChild<? extends PathArgument, ?>> possibleEntities =
-                    entityType.getChild(ENTITY_NODE_ID);
+            Optional<DataContainerChild<?, ?>> possibleEntities = entityType.getChild(ENTITY_NODE_ID);
             if(!possibleEntities.isPresent()) {
-                continue; // shouldn't happen but handle anyway
+                // shouldn't happen but handle anyway
+                continue;
             }
 
             for(MapEntryNode entity:  ((MapNode) possibleEntities.get()).getValue()) {
@@ -422,16 +476,6 @@ class EntityOwnershipShard extends Shard {
         }
     }
 
-    private static Collection<String> getCandidateNames(MapEntryNode entity) {
-        Collection<MapEntryNode> candidates = ((MapNode)entity.getChild(CANDIDATE_NODE_ID).get()).getValue();
-        Collection<String> candidateNames = new ArrayList<>(candidates.size());
-        for(MapEntryNode candidate: candidates) {
-            candidateNames.add(candidate.getChild(CANDIDATE_NAME_NODE_ID).get().getValue().toString());
-        }
-
-        return candidateNames;
-    }
-
     private void writeNewOwner(YangInstanceIdentifier entityPath, String newOwner) {
         LOG.debug("{}: Writing new owner {} for entity {}", persistenceId(), newOwner, entityPath);
 
@@ -439,12 +483,39 @@ class EntityOwnershipShard extends Shard {
                 ImmutableNodes.leafNode(ENTITY_OWNER_NODE_ID, newOwner)), this);
     }
 
-    private String newOwner(Collection<String> candidates, EntityOwnerSelectionStrategy ownerSelectionStrategy) {
+    /**
+     * Schedule a new owner selection job. Cancelling any outstanding job if it has not been cancelled.
+     *
+     * @param entityPath
+     * @param allCandidates
+     */
+    public void scheduleOwnerSelection(YangInstanceIdentifier entityPath, Collection<String> allCandidates,
+                                       EntityOwnerSelectionStrategy strategy){
+        cancelOwnerSelectionTask(entityPath);
+
+        LOG.debug("{}: Scheduling owner selection after {} ms", persistenceId(), strategy.getSelectionDelayInMillis());
+
+        final Cancellable lastScheduledTask = context().system().scheduler().scheduleOnce(
+                FiniteDuration.apply(strategy.getSelectionDelayInMillis(), TimeUnit.MILLISECONDS)
+                , self(), new SelectOwner(entityPath, allCandidates, strategy)
+                , context().system().dispatcher(), self());
+
+        entityToScheduledOwnershipTask.put(entityPath, lastScheduledTask);
+    }
+
+    private void cancelOwnerSelectionTask(YangInstanceIdentifier entityPath){
+        final Cancellable lastScheduledTask = entityToScheduledOwnershipTask.get(entityPath);
+        if(lastScheduledTask != null && !lastScheduledTask.isCancelled()){
+            lastScheduledTask.cancel();
+        }
+    }
+
+    private String newOwner(String currentOwner, Collection<String> candidates, Map<String, Long> statistics, EntityOwnerSelectionStrategy ownerSelectionStrategy) {
         Collection<String> viableCandidates = getViableCandidates(candidates);
         if(viableCandidates.size() == 0){
             return "";
         }
-        return ownerSelectionStrategy.newOwner(viableCandidates);
+        return ownerSelectionStrategy.newOwner(currentOwner, viableCandidates);
     }
 
     private Collection<String> getViableCandidates(Collection<String> candidates) {
@@ -470,43 +541,35 @@ class EntityOwnershipShard extends Shard {
         void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode);
     }
 
-    private EntityOwnerSelectionStrategyWrapper createEntityOwnerSelectionStrategyWrapper(EntityOwnerSelectionStrategy entityOwnerSelectionStrategy){
-        return new EntityOwnerSelectionStrategyWrapper(context().system().scheduler(), self(),
-                context().system().dispatcher(), entityOwnerSelectionStrategy);
-    }
-
-    @VisibleForTesting
-    void addEntityOwnerSelectionStrategy(String entityType, Class<? extends EntityOwnerSelectionStrategy> ownerSelectionStrategyClass){
-        try {
-            EntityOwnerSelectionStrategyWrapper strategy =
-                    createEntityOwnerSelectionStrategyWrapper(ownerSelectionStrategyClass.newInstance());
-            ownerSelectionStrategies.put(entityType, strategy);
-        } catch (InstantiationException | IllegalAccessException e) {
-            LOG.error("Exception occurred when adding election strategy", e);
-        }
-    }
-
     public static Builder newBuilder() {
         return new Builder();
     }
 
     static class Builder extends Shard.AbstractBuilder<Builder, EntityOwnershipShard> {
-        private String localMemberName;
+        private MemberName localMemberName;
+        private EntityOwnerSelectionStrategyConfig ownerSelectionStrategyConfig;
 
         protected Builder() {
             super(EntityOwnershipShard.class);
         }
 
-        Builder localMemberName(String localMemberName) {
+        Builder localMemberName(MemberName localMemberName) {
             checkSealed();
             this.localMemberName = localMemberName;
             return this;
         }
 
+        Builder ownerSelectionStrategyConfig(EntityOwnerSelectionStrategyConfig ownerSelectionStrategyConfig){
+            checkSealed();
+            this.ownerSelectionStrategyConfig = ownerSelectionStrategyConfig;
+            return this;
+        }
+
         @Override
         protected void verify() {
             super.verify();
             Preconditions.checkNotNull(localMemberName, "localMemberName should not be null");
+            Preconditions.checkNotNull(ownerSelectionStrategyConfig, "ownerSelectionStrategyConfig should not be null");
         }
     }
 }