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