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