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