BUG-5414 introduce EOS inJeopardy flag
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / entityownership / EntityOwnershipShard.java
1 /*
2  * Copyright (c) 2015 Brocade Communications Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.datastore.entityownership;
9
10 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.CANDIDATE_NODE_ID;
11 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_ID_NODE_ID;
12 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_ID_QNAME;
13 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_NODE_ID;
14 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_OWNERS_PATH;
15 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_OWNER_NODE_ID;
16 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_OWNER_QNAME;
17 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_TYPES_PATH;
18 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_TYPE_NODE_ID;
19 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_TYPE_QNAME;
20 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.candidateMapEntry;
21 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.candidateNodeKey;
22 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.candidatePath;
23 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.createEntity;
24 import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.entityOwnersWithCandidate;
25 import akka.actor.ActorRef;
26 import akka.actor.ActorSelection;
27 import akka.actor.Cancellable;
28 import akka.pattern.Patterns;
29 import com.google.common.base.Optional;
30 import com.google.common.base.Preconditions;
31 import com.google.common.base.Strings;
32 import java.util.ArrayList;
33 import java.util.Collection;
34 import java.util.HashMap;
35 import java.util.HashSet;
36 import java.util.Map;
37 import java.util.Set;
38 import java.util.concurrent.TimeUnit;
39 import org.opendaylight.controller.cluster.datastore.DatastoreContext;
40 import org.opendaylight.controller.cluster.datastore.Shard;
41 import org.opendaylight.controller.cluster.datastore.entityownership.messages.CandidateAdded;
42 import org.opendaylight.controller.cluster.datastore.entityownership.messages.CandidateRemoved;
43 import org.opendaylight.controller.cluster.datastore.entityownership.messages.RegisterCandidateLocal;
44 import org.opendaylight.controller.cluster.datastore.entityownership.messages.RegisterListenerLocal;
45 import org.opendaylight.controller.cluster.datastore.entityownership.messages.SelectOwner;
46 import org.opendaylight.controller.cluster.datastore.entityownership.messages.UnregisterCandidateLocal;
47 import org.opendaylight.controller.cluster.datastore.entityownership.messages.UnregisterListenerLocal;
48 import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.EntityOwnerSelectionStrategy;
49 import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.EntityOwnerSelectionStrategyConfig;
50 import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier;
51 import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications;
52 import org.opendaylight.controller.cluster.datastore.messages.PeerDown;
53 import org.opendaylight.controller.cluster.datastore.messages.PeerUp;
54 import org.opendaylight.controller.cluster.datastore.messages.SuccessReply;
55 import org.opendaylight.controller.cluster.datastore.modification.DeleteModification;
56 import org.opendaylight.controller.cluster.datastore.modification.MergeModification;
57 import org.opendaylight.controller.cluster.datastore.modification.WriteModification;
58 import org.opendaylight.controller.cluster.raft.RaftState;
59 import org.opendaylight.controller.md.sal.common.api.clustering.Entity;
60 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
61 import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild;
62 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
63 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
64 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
65 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
66 import scala.concurrent.Future;
67 import scala.concurrent.duration.FiniteDuration;
68
69 /**
70  * Special Shard for EntityOwnership.
71  *
72  * @author Thomas Pantelis
73  */
74 class EntityOwnershipShard extends Shard {
75     private final String localMemberName;
76     private final EntityOwnershipShardCommitCoordinator commitCoordinator;
77     private final EntityOwnershipListenerSupport listenerSupport;
78     private final Set<String> downPeerMemberNames = new HashSet<>();
79     private final Map<String, String> peerIdToMemberNames = new HashMap<>();
80     private final EntityOwnerSelectionStrategyConfig strategyConfig;
81     private final Map<YangInstanceIdentifier, Cancellable> entityToScheduledOwnershipTask = new HashMap<>();
82     private final EntityOwnershipStatistics entityOwnershipStatistics;
83
84     private static DatastoreContext noPersistenceDatastoreContext(DatastoreContext datastoreContext) {
85         return DatastoreContext.newBuilderFrom(datastoreContext).persistent(false).build();
86     }
87
88     protected EntityOwnershipShard(Builder builder) {
89         super(builder);
90         this.localMemberName = builder.localMemberName;
91         this.commitCoordinator = new EntityOwnershipShardCommitCoordinator(builder.localMemberName, LOG);
92         this.listenerSupport = new EntityOwnershipListenerSupport(getContext(), persistenceId());
93         this.strategyConfig = builder.ownerSelectionStrategyConfig;
94         this.entityOwnershipStatistics = new EntityOwnershipStatistics();
95         this.entityOwnershipStatistics.init(getDataStore());
96
97         for(String peerId: getRaftActorContext().getPeerIds()) {
98             ShardIdentifier shardId = ShardIdentifier.builder().fromShardIdString(peerId).build();
99             peerIdToMemberNames.put(peerId, shardId.getMemberName());
100         }
101     }
102
103     @Override
104     protected void onDatastoreContext(DatastoreContext context) {
105         super.onDatastoreContext(noPersistenceDatastoreContext(context));
106     }
107
108     @Override
109     protected void onRecoveryComplete() {
110         super.onRecoveryComplete();
111
112         new CandidateListChangeListener(getSelf(), persistenceId()).init(getDataStore());
113         new EntityOwnerChangeListener(localMemberName, listenerSupport).init(getDataStore());
114     }
115
116     @Override
117     public void handleNonRaftCommand(final Object message) {
118         if(message instanceof RegisterCandidateLocal) {
119             onRegisterCandidateLocal((RegisterCandidateLocal) message);
120         } else if(message instanceof UnregisterCandidateLocal) {
121             onUnregisterCandidateLocal((UnregisterCandidateLocal)message);
122         } else if(message instanceof CandidateAdded){
123             onCandidateAdded((CandidateAdded) message);
124         } else if(message instanceof CandidateRemoved){
125             onCandidateRemoved((CandidateRemoved) message);
126         } else if(message instanceof PeerDown) {
127             onPeerDown((PeerDown) message);
128         } else if(message instanceof PeerUp) {
129             onPeerUp((PeerUp) message);
130         } else if(message instanceof RegisterListenerLocal) {
131             onRegisterListenerLocal((RegisterListenerLocal)message);
132         } else if(message instanceof UnregisterListenerLocal) {
133             onUnregisterListenerLocal((UnregisterListenerLocal) message);
134         } else if(message instanceof SelectOwner) {
135             onSelectOwner((SelectOwner) message);
136         } else if(!commitCoordinator.handleMessage(message, this)) {
137             super.handleNonRaftCommand(message);
138         }
139     }
140
141     private void onSelectOwner(SelectOwner selectOwner) {
142         LOG.debug("{}: onSelectOwner: {}", persistenceId(), selectOwner);
143
144         String currentOwner = getCurrentOwner(selectOwner.getEntityPath());
145         if(Strings.isNullOrEmpty(currentOwner)) {
146             String entityType = EntityOwnersModel.entityTypeFromEntityPath(selectOwner.getEntityPath());
147             writeNewOwner(selectOwner.getEntityPath(), newOwner(currentOwner, selectOwner.getAllCandidates(),
148                     entityOwnershipStatistics.byEntityType(entityType),
149                     selectOwner.getOwnerSelectionStrategy()));
150
151             Cancellable cancellable = entityToScheduledOwnershipTask.get(selectOwner.getEntityPath());
152             if(cancellable != null){
153                 if(!cancellable.isCancelled()){
154                     cancellable.cancel();
155                 }
156                 entityToScheduledOwnershipTask.remove(selectOwner.getEntityPath());
157             }
158         }
159     }
160
161     private void onRegisterCandidateLocal(RegisterCandidateLocal registerCandidate) {
162         LOG.debug("{}: onRegisterCandidateLocal: {}", persistenceId(), registerCandidate);
163
164         listenerSupport.setHasCandidateForEntity(registerCandidate.getEntity());
165
166         NormalizedNode<?, ?> entityOwners = entityOwnersWithCandidate(registerCandidate.getEntity().getType(),
167                 registerCandidate.getEntity().getId(), localMemberName);
168         commitCoordinator.commitModification(new MergeModification(ENTITY_OWNERS_PATH, entityOwners), this);
169
170         getSender().tell(SuccessReply.INSTANCE, getSelf());
171     }
172
173     private void onUnregisterCandidateLocal(UnregisterCandidateLocal unregisterCandidate) {
174         LOG.debug("{}: onUnregisterCandidateLocal: {}", persistenceId(), unregisterCandidate);
175
176         Entity entity = unregisterCandidate.getEntity();
177         listenerSupport.unsetHasCandidateForEntity(entity);
178
179         YangInstanceIdentifier candidatePath = candidatePath(entity.getType(), entity.getId(), localMemberName);
180         commitCoordinator.commitModification(new DeleteModification(candidatePath), this);
181
182         getSender().tell(SuccessReply.INSTANCE, getSelf());
183     }
184
185     private void onRegisterListenerLocal(final RegisterListenerLocal registerListener) {
186         LOG.debug("{}: onRegisterListenerLocal: {}", persistenceId(), registerListener);
187
188         listenerSupport.addEntityOwnershipListener(registerListener.getEntityType(), registerListener.getListener());
189
190         getSender().tell(SuccessReply.INSTANCE, getSelf());
191
192         searchForEntities(new EntityWalker() {
193             @Override
194             public void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode) {
195                 Optional<DataContainerChild<?, ?>> possibleType = entityTypeNode.getChild(ENTITY_TYPE_NODE_ID);
196                 String entityType = possibleType.isPresent() ? possibleType.get().getValue().toString() : null;
197                 if (registerListener.getEntityType().equals(entityType)) {
198                     final boolean hasOwner;
199                     final boolean isOwner;
200
201                     Optional<DataContainerChild<?, ?>> possibleOwner = entityNode.getChild(ENTITY_OWNER_NODE_ID);
202                     if (possibleOwner.isPresent()) {
203                         isOwner = localMemberName.equals(possibleOwner.get().getValue().toString());
204                         hasOwner = true;
205                     } else {
206                         isOwner = false;
207                         hasOwner = false;
208                     }
209
210                     Entity entity = new Entity(entityType,
211                         (YangInstanceIdentifier) entityNode.getChild(ENTITY_ID_NODE_ID).get().getValue());
212
213                     listenerSupport.notifyEntityOwnershipListener(entity, false, isOwner, hasOwner,
214                         registerListener.getListener());
215                 }
216             }
217         });
218     }
219
220     private void onUnregisterListenerLocal(UnregisterListenerLocal unregisterListener) {
221         LOG.debug("{}: onUnregisterListenerLocal: {}", persistenceId(), unregisterListener);
222
223         listenerSupport.removeEntityOwnershipListener(unregisterListener.getEntityType(), unregisterListener.getListener());
224
225         getSender().tell(SuccessReply.INSTANCE, getSelf());
226     }
227
228     void tryCommitModifications(final BatchedModifications modifications) {
229         if(isLeader()) {
230             LOG.debug("{}: Committing BatchedModifications {} locally", persistenceId(), modifications.getTransactionID());
231
232             // Note that it's possible the commit won't get consensus and will timeout and not be applied
233             // to the state. However we don't need to retry it in that case b/c it will be committed to
234             // the journal first and, once a majority of followers come back on line and it is replicated,
235             // it will be applied at that point.
236             handleBatchedModificationsLocal(modifications, self());
237         } else {
238             final ActorSelection leader = getLeader();
239             if (leader != null) {
240                 if(LOG.isDebugEnabled()) {
241                     LOG.debug("{}: Sending BatchedModifications {} to leader {}", persistenceId(),
242                             modifications.getTransactionID(), leader);
243                 }
244
245                 Future<Object> future = Patterns.ask(leader, modifications, TimeUnit.SECONDS.toMillis(
246                         getDatastoreContext().getShardTransactionCommitTimeoutInSeconds()));
247
248                 Patterns.pipe(future, getContext().dispatcher()).pipeTo(getSelf(), ActorRef.noSender());
249             }
250         }
251     }
252
253     boolean hasLeader() {
254         return getLeader() != null && !isIsolatedLeader();
255     }
256
257     /**
258      * Determine if we are in jeopardy based on observed RAFT state.
259      */
260     private static boolean inJeopardy(final RaftState state) {
261         switch (state) {
262             case Candidate:
263             case Follower:
264             case Leader:
265                 return false;
266             case IsolatedLeader:
267                 return true;
268         }
269         throw new IllegalStateException("Unsupported RAFT state " + state);
270     }
271
272     private void notifyAllListeners() {
273         searchForEntities(new EntityWalker() {
274             @Override
275             public void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode) {
276                 Optional<DataContainerChild<?, ?>> possibleType = entityTypeNode.getChild(ENTITY_TYPE_NODE_ID);
277                 if (possibleType.isPresent()) {
278                     final boolean hasOwner;
279                     final boolean isOwner;
280
281                     Optional<DataContainerChild<?, ?>> possibleOwner = entityNode.getChild(ENTITY_OWNER_NODE_ID);
282                     if (possibleOwner.isPresent()) {
283                         isOwner = localMemberName.equals(possibleOwner.get().getValue().toString());
284                         hasOwner = true;
285                     } else {
286                         isOwner = false;
287                         hasOwner = false;
288                     }
289
290                     Entity entity = new Entity(possibleType.get().getValue().toString(),
291                         (YangInstanceIdentifier) entityNode.getChild(ENTITY_ID_NODE_ID).get().getValue());
292
293                     listenerSupport.notifyEntityOwnershipListeners(entity, isOwner, isOwner, hasOwner);
294                 }
295             }
296         });
297     }
298
299     @Override
300     protected void onStateChanged() {
301         super.onStateChanged();
302
303         boolean isLeader = isLeader();
304         LOG.debug("{}: onStateChanged: isLeader: {}, hasLeader: {}", persistenceId(), isLeader, hasLeader());
305
306         // Examine current RAFT state to see if we are in jeopardy, potentially notifying all listeners
307         final boolean inJeopardy = inJeopardy(getRaftState());
308         final boolean wasInJeopardy = listenerSupport.setInJeopardy(inJeopardy);
309         if (inJeopardy != wasInJeopardy) {
310             LOG.debug("{}: {} jeopardy state, notifying all listeners", persistenceId(),
311                 inJeopardy ? "entered" : "left");
312             notifyAllListeners();
313         }
314
315         commitCoordinator.onStateChanged(this, isLeader);
316     }
317
318     @Override
319     protected void onLeaderChanged(String oldLeader, String newLeader) {
320         super.onLeaderChanged(oldLeader, newLeader);
321
322         boolean isLeader = isLeader();
323         LOG.debug("{}: onLeaderChanged: oldLeader: {}, newLeader: {}, isLeader: {}", persistenceId(), oldLeader,
324                 newLeader, isLeader);
325
326         if(isLeader) {
327
328             // Clear all existing strategies so that they get re-created when we call createStrategy again
329             // This allows the strategies to be re-initialized with existing statistics maintained by
330             // EntityOwnershipStatistics
331             strategyConfig.clearStrategies();
332
333             // Remove the candidates for all members that are known to be down. In a cluster which has greater than
334             // 3 nodes it is possible for a some node beside the leader being down when the leadership transitions
335             // it makes sense to use this event to remove all the candidates for those downed nodes
336             for(String downPeerName : downPeerMemberNames){
337                 removeCandidateFromEntities(downPeerName);
338             }
339         } else {
340             // The leader changed - notify the coordinator to check if pending modifications need to be sent.
341             // While onStateChanged also does this, this method handles the case where the shard hears from a
342             // leader and stays in the follower state. In that case no behavior state change occurs.
343             commitCoordinator.onStateChanged(this, isLeader);
344         }
345     }
346
347     private void onCandidateRemoved(CandidateRemoved message) {
348         LOG.debug("{}: onCandidateRemoved: {}", persistenceId(), message);
349
350         if(isLeader()) {
351             String currentOwner = getCurrentOwner(message.getEntityPath());
352             if(message.getRemovedCandidate().equals(currentOwner) || message.getRemainingCandidates().size() == 0){
353                 String entityType = EntityOwnersModel.entityTypeFromEntityPath(message.getEntityPath());
354                 writeNewOwner(message.getEntityPath(),
355                         newOwner(currentOwner, message.getRemainingCandidates(), entityOwnershipStatistics.byEntityType(entityType),
356                                 getEntityOwnerElectionStrategy(message.getEntityPath())));
357             }
358         } else {
359             // We're not the leader. If the removed candidate is our local member then check if we actually
360             // have a local candidate registered. If we do then we must have been partitioned from the leader
361             // and the leader removed our candidate since the leader can't tell the difference between a
362             // temporary network partition and a node's process actually restarted. So, in that case, re-add
363             // our candidate.
364             if(localMemberName.equals(message.getRemovedCandidate()) &&
365                     listenerSupport.hasCandidateForEntity(createEntity(message.getEntityPath()))) {
366                 LOG.debug("Local candidate member was removed but a local candidate is registered for {}" +
367                     " - adding back local candidate", message.getEntityPath());
368
369                 commitCoordinator.commitModification(new MergeModification(
370                         candidatePath(message.getEntityPath(), localMemberName),
371                         candidateMapEntry(localMemberName)), this);
372             }
373         }
374     }
375
376     private EntityOwnerSelectionStrategy getEntityOwnerElectionStrategy(YangInstanceIdentifier entityPath) {
377         final String entityType = EntityOwnersModel.entityTypeFromEntityPath(entityPath);
378         return strategyConfig.createStrategy(entityType, entityOwnershipStatistics.byEntityType(entityType));
379     }
380
381     private void onCandidateAdded(CandidateAdded message) {
382         if(!isLeader()){
383             return;
384         }
385
386         LOG.debug("{}: onCandidateAdded: {}", persistenceId(), message);
387
388         // Since a node's candidate member is only added by the node itself, we can assume the node is up so
389         // remove it from the downPeerMemberNames.
390         downPeerMemberNames.remove(message.getNewCandidate());
391
392         final String currentOwner = getCurrentOwner(message.getEntityPath());
393         final EntityOwnerSelectionStrategy strategy = getEntityOwnerElectionStrategy(message.getEntityPath());
394         final String entityType = EntityOwnersModel.entityTypeFromEntityPath(message.getEntityPath());
395
396         // Available members is all the known peers - the number of peers that are down + self
397         // So if there are 2 peers and 1 is down then availableMembers will be 2
398         final int availableMembers = (peerIdToMemberNames.size() - downPeerMemberNames.size()) + 1;
399
400         LOG.debug("{}: Using strategy {} to select owner, currentOwner = {}", persistenceId(), strategy, currentOwner);
401
402         if(!message.getAllCandidates().contains(currentOwner)){
403             if(strategy.getSelectionDelayInMillis() == 0L) {
404                 writeNewOwner(message.getEntityPath(), newOwner(currentOwner, message.getAllCandidates(),
405                         entityOwnershipStatistics.byEntityType(entityType), strategy));
406             } else if(message.getAllCandidates().size() == availableMembers) {
407                 LOG.debug("{}: Received the maximum candidates requests : {} writing new owner",
408                         persistenceId(), availableMembers);
409                 cancelOwnerSelectionTask(message.getEntityPath());
410                 writeNewOwner(message.getEntityPath(), newOwner(currentOwner, message.getAllCandidates(),
411                         entityOwnershipStatistics.byEntityType(entityType), strategy));
412             } else {
413                 scheduleOwnerSelection(message.getEntityPath(), message.getAllCandidates(), strategy);
414             }
415         }
416     }
417
418     private void onPeerDown(PeerDown peerDown) {
419         LOG.info("{}: onPeerDown: {}", persistenceId(), peerDown);
420
421         String downMemberName = peerDown.getMemberName();
422         if(downPeerMemberNames.add(downMemberName) && isLeader()) {
423             // Remove the down peer as a candidate from all entities.
424             removeCandidateFromEntities(downMemberName);
425         }
426     }
427
428     private void onPeerUp(PeerUp peerUp) {
429         LOG.debug("{}: onPeerUp: {}", persistenceId(), peerUp);
430
431         peerIdToMemberNames.put(peerUp.getPeerId(), peerUp.getMemberName());
432         downPeerMemberNames.remove(peerUp.getMemberName());
433
434         // Notify the coordinator to check if pending modifications need to be sent. We do this here
435         // to handle the case where the leader's peer address isn't now yet when a prior state or
436         // leader change occurred.
437         commitCoordinator.onStateChanged(this, isLeader());
438     }
439
440     private void removeCandidateFromEntities(final String owner) {
441         final BatchedModifications modifications = commitCoordinator.newBatchedModifications();
442         searchForEntities(new EntityWalker() {
443             @Override
444             public void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode) {
445                 if (hasCandidate(entityNode, owner)) {
446                     YangInstanceIdentifier entityId =
447                             (YangInstanceIdentifier) entityNode.getIdentifier().getKeyValues().get(ENTITY_ID_QNAME);
448                     YangInstanceIdentifier candidatePath = candidatePath(
449                             entityTypeNode.getIdentifier().getKeyValues().get(ENTITY_TYPE_QNAME).toString(),
450                             entityId, owner);
451
452                     LOG.info("{}: Found entity {}, removing candidate {}, path {}", persistenceId(), entityId,
453                             owner, candidatePath);
454
455                     modifications.addModification(new DeleteModification(candidatePath));
456                 }
457             }
458         });
459
460         commitCoordinator.commitModifications(modifications, this);
461     }
462
463     private static boolean hasCandidate(MapEntryNode entity, String candidateName) {
464         return ((MapNode)entity.getChild(CANDIDATE_NODE_ID).get()).getChild(candidateNodeKey(candidateName)).isPresent();
465     }
466
467     private void searchForEntities(EntityWalker walker) {
468         Optional<NormalizedNode<?, ?>> possibleEntityTypes = getDataStore().readNode(ENTITY_TYPES_PATH);
469         if(!possibleEntityTypes.isPresent()) {
470             return;
471         }
472
473         for(MapEntryNode entityType:  ((MapNode) possibleEntityTypes.get()).getValue()) {
474             Optional<DataContainerChild<?, ?>> possibleEntities = entityType.getChild(ENTITY_NODE_ID);
475             if(!possibleEntities.isPresent()) {
476                 // shouldn't happen but handle anyway
477                 continue;
478             }
479
480             for(MapEntryNode entity:  ((MapNode) possibleEntities.get()).getValue()) {
481                 walker.onEntity(entityType, entity);
482             }
483         }
484     }
485
486     private void writeNewOwner(YangInstanceIdentifier entityPath, String newOwner) {
487         LOG.debug("{}: Writing new owner {} for entity {}", persistenceId(), newOwner, entityPath);
488
489         commitCoordinator.commitModification(new WriteModification(entityPath.node(ENTITY_OWNER_QNAME),
490                 ImmutableNodes.leafNode(ENTITY_OWNER_NODE_ID, newOwner)), this);
491     }
492
493     /**
494      * Schedule a new owner selection job. Cancelling any outstanding job if it has not been cancelled.
495      *
496      * @param entityPath
497      * @param allCandidates
498      */
499     public void scheduleOwnerSelection(YangInstanceIdentifier entityPath, Collection<String> allCandidates,
500                                        EntityOwnerSelectionStrategy strategy){
501         cancelOwnerSelectionTask(entityPath);
502
503         LOG.debug("{}: Scheduling owner selection after {} ms", persistenceId(), strategy.getSelectionDelayInMillis());
504
505         final Cancellable lastScheduledTask = context().system().scheduler().scheduleOnce(
506                 FiniteDuration.apply(strategy.getSelectionDelayInMillis(), TimeUnit.MILLISECONDS)
507                 , self(), new SelectOwner(entityPath, allCandidates, strategy)
508                 , context().system().dispatcher(), self());
509
510         entityToScheduledOwnershipTask.put(entityPath, lastScheduledTask);
511     }
512
513     private void cancelOwnerSelectionTask(YangInstanceIdentifier entityPath){
514         final Cancellable lastScheduledTask = entityToScheduledOwnershipTask.get(entityPath);
515         if(lastScheduledTask != null && !lastScheduledTask.isCancelled()){
516             lastScheduledTask.cancel();
517         }
518     }
519
520     private String newOwner(String currentOwner, Collection<String> candidates, Map<String, Long> statistics, EntityOwnerSelectionStrategy ownerSelectionStrategy) {
521         Collection<String> viableCandidates = getViableCandidates(candidates);
522         if(viableCandidates.size() == 0){
523             return "";
524         }
525         return ownerSelectionStrategy.newOwner(currentOwner, viableCandidates);
526     }
527
528     private Collection<String> getViableCandidates(Collection<String> candidates) {
529         Collection<String> viableCandidates = new ArrayList<>();
530
531         for (String candidate : candidates) {
532             if (!downPeerMemberNames.contains(candidate)) {
533                 viableCandidates.add(candidate);
534             }
535         }
536         return viableCandidates;
537     }
538
539     private String getCurrentOwner(YangInstanceIdentifier entityId) {
540         Optional<NormalizedNode<?, ?>> optionalEntityOwner = getDataStore().readNode(entityId.node(ENTITY_OWNER_QNAME));
541         if(optionalEntityOwner.isPresent()){
542             return optionalEntityOwner.get().getValue().toString();
543         }
544         return null;
545     }
546
547     private static interface EntityWalker {
548         void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode);
549     }
550
551     public static Builder newBuilder() {
552         return new Builder();
553     }
554
555     static class Builder extends Shard.AbstractBuilder<Builder, EntityOwnershipShard> {
556         private String localMemberName;
557         private EntityOwnerSelectionStrategyConfig ownerSelectionStrategyConfig;
558
559         protected Builder() {
560             super(EntityOwnershipShard.class);
561         }
562
563         Builder localMemberName(String localMemberName) {
564             checkSealed();
565             this.localMemberName = localMemberName;
566             return this;
567         }
568
569         Builder ownerSelectionStrategyConfig(EntityOwnerSelectionStrategyConfig ownerSelectionStrategyConfig){
570             checkSealed();
571             this.ownerSelectionStrategyConfig = ownerSelectionStrategyConfig;
572             return this;
573         }
574
575         @Override
576         protected void verify() {
577             super.verify();
578             Preconditions.checkNotNull(localMemberName, "localMemberName should not be null");
579             Preconditions.checkNotNull(ownerSelectionStrategyConfig, "ownerSelectionStrategyConfig should not be null");
580         }
581     }
582 }