X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2Fentityownership%2FEntityOwnershipShard.java;h=4a6a37aabe77e43e7b77ba6ba7e7a117caf75328;hb=8119659681a6814d257314178e759a6ef1b49766;hp=334e093d7f4babfafef4174275d7c32092decbe1;hpb=e9cceae4b6689c7c3880b441c13d35cbe4b13ce0;p=controller.git diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/EntityOwnershipShard.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/EntityOwnershipShard.java index 334e093d7f..4a6a37aabe 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/EntityOwnershipShard.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/entityownership/EntityOwnershipShard.java @@ -18,15 +18,13 @@ import static org.opendaylight.controller.cluster.datastore.entityownership.Enti import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_TYPES_PATH; import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_TYPE_NODE_ID; import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_TYPE_QNAME; -import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.candidateMapEntry; import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.candidateNodeKey; import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.candidatePath; -import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.createEntity; import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.entityOwnersWithCandidate; import akka.actor.ActorRef; import akka.actor.ActorSelection; +import akka.actor.Cancellable; import akka.pattern.Patterns; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Strings; @@ -37,18 +35,19 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; +import org.opendaylight.controller.cluster.access.concepts.MemberName; import org.opendaylight.controller.cluster.datastore.DatastoreContext; import org.opendaylight.controller.cluster.datastore.Shard; import org.opendaylight.controller.cluster.datastore.entityownership.messages.CandidateAdded; import org.opendaylight.controller.cluster.datastore.entityownership.messages.CandidateRemoved; import org.opendaylight.controller.cluster.datastore.entityownership.messages.RegisterCandidateLocal; import org.opendaylight.controller.cluster.datastore.entityownership.messages.RegisterListenerLocal; +import org.opendaylight.controller.cluster.datastore.entityownership.messages.RemoveAllCandidates; import org.opendaylight.controller.cluster.datastore.entityownership.messages.SelectOwner; import org.opendaylight.controller.cluster.datastore.entityownership.messages.UnregisterCandidateLocal; import org.opendaylight.controller.cluster.datastore.entityownership.messages.UnregisterListenerLocal; import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.EntityOwnerSelectionStrategy; -import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.EntityOwnerSelectionStrategyWrapper; -import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.FirstCandidateSelectionStrategy; +import org.opendaylight.controller.cluster.datastore.entityownership.selectionstrategy.EntityOwnerSelectionStrategyConfig; import org.opendaylight.controller.cluster.datastore.identifiers.ShardIdentifier; import org.opendaylight.controller.cluster.datastore.messages.BatchedModifications; import org.opendaylight.controller.cluster.datastore.messages.PeerDown; @@ -57,7 +56,8 @@ import org.opendaylight.controller.cluster.datastore.messages.SuccessReply; import org.opendaylight.controller.cluster.datastore.modification.DeleteModification; import org.opendaylight.controller.cluster.datastore.modification.MergeModification; import org.opendaylight.controller.cluster.datastore.modification.WriteModification; -import org.opendaylight.controller.md.sal.common.api.clustering.Entity; +import org.opendaylight.controller.cluster.raft.RaftState; +import org.opendaylight.mdsal.eos.dom.api.DOMEntity; import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier; import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument; import org.opendaylight.yangtools.yang.data.api.schema.DataContainerChild; @@ -66,6 +66,7 @@ import org.opendaylight.yangtools.yang.data.api.schema.MapNode; import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode; import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes; import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; /** * Special Shard for EntityOwnership. @@ -73,13 +74,15 @@ import scala.concurrent.Future; * @author Thomas Pantelis */ class EntityOwnershipShard extends Shard { - private final String localMemberName; + private final MemberName localMemberName; private final EntityOwnershipShardCommitCoordinator commitCoordinator; private final EntityOwnershipListenerSupport listenerSupport; - private final Set downPeerMemberNames = new HashSet<>(); - private final Map peerIdToMemberNames = new HashMap<>(); - private final Map ownerSelectionStrategies = new HashMap<>(); - private final EntityOwnerSelectionStrategyWrapper defaultEntityOwnerSelectionStrategy; + private final Set downPeerMemberNames = new HashSet<>(); + private final Map peerIdToMemberNames = new HashMap<>(); + private final EntityOwnerSelectionStrategyConfig strategyConfig; + private final Map entityToScheduledOwnershipTask = new HashMap<>(); + private final EntityOwnershipStatistics entityOwnershipStatistics; + private boolean removeAllInitialCandidates = true; private static DatastoreContext noPersistenceDatastoreContext(DatastoreContext datastoreContext) { return DatastoreContext.newBuilderFrom(datastoreContext).persistent(false).build(); @@ -90,11 +93,12 @@ class EntityOwnershipShard extends Shard { this.localMemberName = builder.localMemberName; this.commitCoordinator = new EntityOwnershipShardCommitCoordinator(builder.localMemberName, LOG); this.listenerSupport = new EntityOwnershipListenerSupport(getContext(), persistenceId()); - this.defaultEntityOwnerSelectionStrategy = - createEntityOwnerSelectionStrategyWrapper(FirstCandidateSelectionStrategy.INSTANCE); + this.strategyConfig = builder.ownerSelectionStrategyConfig; + this.entityOwnershipStatistics = new EntityOwnershipStatistics(); + this.entityOwnershipStatistics.init(getDataStore()); for(String peerId: getRaftActorContext().getPeerIds()) { - ShardIdentifier shardId = ShardIdentifier.builder().fromShardIdString(peerId).build(); + ShardIdentifier shardId = ShardIdentifier.fromShardIdString(peerId); peerIdToMemberNames.put(peerId, shardId.getMemberName()); } } @@ -113,7 +117,7 @@ class EntityOwnershipShard extends Shard { } @Override - public void onReceiveCommand(final Object message) throws Exception { + public void handleNonRaftCommand(final Object message) { if(message instanceof RegisterCandidateLocal) { onRegisterCandidateLocal((RegisterCandidateLocal) message); } else if(message instanceof UnregisterCandidateLocal) { @@ -132,16 +136,34 @@ class EntityOwnershipShard extends Shard { onUnregisterListenerLocal((UnregisterListenerLocal) message); } else if(message instanceof SelectOwner) { onSelectOwner((SelectOwner) message); + } else if(message instanceof RemoveAllCandidates) { + onRemoveAllCandidates((RemoveAllCandidates) message); } else if(!commitCoordinator.handleMessage(message, this)) { - super.onReceiveCommand(message); + super.handleNonRaftCommand(message); } } + private void onRemoveAllCandidates(RemoveAllCandidates message) { + LOG.debug("{}: onRemoveAllCandidates: {}", persistenceId(), message); + + removeCandidateFromEntities(message.getMemberName()); + } + private void onSelectOwner(SelectOwner selectOwner) { + LOG.debug("{}: onSelectOwner: {}", persistenceId(), selectOwner); + String currentOwner = getCurrentOwner(selectOwner.getEntityPath()); if(Strings.isNullOrEmpty(currentOwner)) { - writeNewOwner(selectOwner.getEntityPath(), newOwner(selectOwner.getAllCandidates(), + writeNewOwner(selectOwner.getEntityPath(), newOwner(currentOwner, selectOwner.getAllCandidates(), selectOwner.getOwnerSelectionStrategy())); + + Cancellable cancellable = entityToScheduledOwnershipTask.get(selectOwner.getEntityPath()); + if(cancellable != null){ + if(!cancellable.isCancelled()){ + cancellable.cancel(); + } + entityToScheduledOwnershipTask.remove(selectOwner.getEntityPath()); + } } } @@ -151,7 +173,7 @@ class EntityOwnershipShard extends Shard { listenerSupport.setHasCandidateForEntity(registerCandidate.getEntity()); NormalizedNode entityOwners = entityOwnersWithCandidate(registerCandidate.getEntity().getType(), - registerCandidate.getEntity().getId(), localMemberName); + registerCandidate.getEntity().getIdentifier(), localMemberName.getName()); commitCoordinator.commitModification(new MergeModification(ENTITY_OWNERS_PATH, entityOwners), this); getSender().tell(SuccessReply.INSTANCE, getSelf()); @@ -160,10 +182,10 @@ class EntityOwnershipShard extends Shard { private void onUnregisterCandidateLocal(UnregisterCandidateLocal unregisterCandidate) { LOG.debug("{}: onUnregisterCandidateLocal: {}", persistenceId(), unregisterCandidate); - Entity entity = unregisterCandidate.getEntity(); + DOMEntity entity = unregisterCandidate.getEntity(); listenerSupport.unsetHasCandidateForEntity(entity); - YangInstanceIdentifier candidatePath = candidatePath(entity.getType(), entity.getId(), localMemberName); + YangInstanceIdentifier candidatePath = candidatePath(entity.getType(), entity.getIdentifier(), localMemberName.getName()); commitCoordinator.commitModification(new DeleteModification(candidatePath), this); getSender().tell(SuccessReply.INSTANCE, getSelf()); @@ -176,17 +198,27 @@ class EntityOwnershipShard extends Shard { getSender().tell(SuccessReply.INSTANCE, getSelf()); - searchForEntitiesOwnedBy(localMemberName, new EntityWalker() { - @Override - public void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode) { - Optional> possibleType = - entityTypeNode.getChild(ENTITY_TYPE_NODE_ID); - String entityType = possibleType.isPresent() ? possibleType.get().getValue().toString() : null; - if (registerListener.getEntityType().equals(entityType)) { - Entity entity = new Entity(entityType, - (YangInstanceIdentifier) entityNode.getChild(ENTITY_ID_NODE_ID).get().getValue()); - listenerSupport.notifyEntityOwnershipListener(entity, false, true, true, registerListener.getListener()); + searchForEntities((entityTypeNode, entityNode) -> { + Optional> possibleType = entityTypeNode.getChild(ENTITY_TYPE_NODE_ID); + String entityType = possibleType.isPresent() ? possibleType.get().getValue().toString() : null; + if (registerListener.getEntityType().equals(entityType)) { + final boolean hasOwner; + final boolean isOwner; + + Optional> possibleOwner = entityNode.getChild(ENTITY_OWNER_NODE_ID); + if (possibleOwner.isPresent()) { + isOwner = localMemberName.getName().equals(possibleOwner.get().getValue().toString()); + hasOwner = true; + } else { + isOwner = false; + hasOwner = false; } + + DOMEntity entity = new DOMEntity(entityType, + (YangInstanceIdentifier) entityNode.getChild(ENTITY_ID_NODE_ID).get().getValue()); + + listenerSupport.notifyEntityOwnershipListener(entity, false, isOwner, hasOwner, + registerListener.getListener()); } }); } @@ -211,6 +243,8 @@ class EntityOwnershipShard extends Shard { } else { final ActorSelection leader = getLeader(); if (leader != null) { + possiblyRemoveAllInitialCandidates(leader); + if(LOG.isDebugEnabled()) { LOG.debug("{}: Sending BatchedModifications {} to leader {}", persistenceId(), modifications.getTransactionID(), leader); @@ -224,35 +258,113 @@ class EntityOwnershipShard extends Shard { } } + void possiblyRemoveAllInitialCandidates(ActorSelection leader) { + // The following handles removing all candidates on startup when re-joining with a remote leader. When a + // follower is detected as down, the leader will re-assign new owners to entities that were owned by the + // down member but doesn't remove the down member as a candidate, as the down node may actually be isolated + // and still running. Therefore on startup we send an initial message to the remote leader to remove any + // potential stale candidates we had previously registered, as it's possible a candidate may not be + // registered by a client in the new incarnation. We have to send the RemoveAllCandidates message prior to any + // pending registrations. + if(removeAllInitialCandidates && leader != null) { + removeAllInitialCandidates = false; + if(!isLeader()) { + LOG.debug("{} - got new leader {} on startup - sending RemoveAllCandidates", persistenceId(), leader); + + leader.tell(new RemoveAllCandidates(localMemberName), ActorRef.noSender()); + } + } + } + boolean hasLeader() { - return getLeader() != null && !isIsolatedLeader(); + return getLeader() != null && (!isLeader() || isLeaderActive()); + } + + /** + * Determine if we are in jeopardy based on observed RAFT state. + */ + private static boolean inJeopardy(final RaftState state) { + switch (state) { + case Candidate: + case Follower: + case Leader: + case PreLeader: + return false; + case IsolatedLeader: + return true; + } + throw new IllegalStateException("Unsupported RAFT state " + state); + } + + private void notifyAllListeners() { + searchForEntities((entityTypeNode, entityNode) -> { + Optional> possibleType = entityTypeNode.getChild(ENTITY_TYPE_NODE_ID); + if (possibleType.isPresent()) { + final boolean hasOwner; + final boolean isOwner; + + Optional> possibleOwner = entityNode.getChild(ENTITY_OWNER_NODE_ID); + if (possibleOwner.isPresent()) { + isOwner = localMemberName.getName().equals(possibleOwner.get().getValue().toString()); + hasOwner = true; + } else { + isOwner = false; + hasOwner = false; + } + + DOMEntity entity = new DOMEntity(possibleType.get().getValue().toString(), + (YangInstanceIdentifier) entityNode.getChild(ENTITY_ID_NODE_ID).get().getValue()); + + listenerSupport.notifyEntityOwnershipListeners(entity, isOwner, isOwner, hasOwner); + } + }); } @Override protected void onStateChanged() { super.onStateChanged(); - commitCoordinator.onStateChanged(this, isLeader()); + boolean isLeader = isLeader(); + LOG.debug("{}: onStateChanged: isLeader: {}, hasLeader: {}", persistenceId(), isLeader, hasLeader()); + + // Examine current RAFT state to see if we are in jeopardy, potentially notifying all listeners + final boolean inJeopardy = inJeopardy(getRaftState()); + final boolean wasInJeopardy = listenerSupport.setInJeopardy(inJeopardy); + if (inJeopardy != wasInJeopardy) { + LOG.debug("{}: {} jeopardy state, notifying all listeners", persistenceId(), + inJeopardy ? "entered" : "left"); + notifyAllListeners(); + } + + commitCoordinator.onStateChanged(this, isLeader); } @Override protected void onLeaderChanged(String oldLeader, String newLeader) { super.onLeaderChanged(oldLeader, newLeader); + boolean isLeader = isLeader(); LOG.debug("{}: onLeaderChanged: oldLeader: {}, newLeader: {}, isLeader: {}", persistenceId(), oldLeader, - newLeader, isLeader()); + newLeader, isLeader); - if(isLeader()) { - // We were just elected leader. If the old leader is down, select new owners for the entities - // owned by the down leader. - - String oldLeaderMemberName = peerIdToMemberNames.get(oldLeader); + if (isLeader) { - LOG.debug("{}: oldLeaderMemberName: {}", persistenceId(), oldLeaderMemberName); + // Clear all existing strategies so that they get re-created when we call createStrategy again + // This allows the strategies to be re-initialized with existing statistics maintained by + // EntityOwnershipStatistics + strategyConfig.clearStrategies(); - if(downPeerMemberNames.contains(oldLeaderMemberName)) { - selectNewOwnerForEntitiesOwnedBy(oldLeaderMemberName); + // Re-assign owners for all members that are known to be down. In a cluster which has greater than + // 3 nodes it is possible for a some node beside the leader being down when the leadership transitions + // it makes sense to use this event to re-assign owners for those downed nodes + for (MemberName downPeerName : downPeerMemberNames) { + selectNewOwnerForEntitiesOwnedBy(downPeerName); } + } else { + // The leader changed - notify the coordinator to check if pending modifications need to be sent. + // While onStateChanged also does this, this method handles the case where the shard hears from a + // leader and stays in the follower state. In that case no behavior state change occurs. + commitCoordinator.onStateChanged(this, isLeader); } } @@ -261,38 +373,16 @@ class EntityOwnershipShard extends Shard { if(isLeader()) { String currentOwner = getCurrentOwner(message.getEntityPath()); - if(message.getRemovedCandidate().equals(currentOwner)){ + if(message.getRemovedCandidate().equals(currentOwner) || message.getRemainingCandidates().isEmpty()){ writeNewOwner(message.getEntityPath(), - newOwner(message.getRemainingCandidates(), getEntityOwnerElectionStrategyWrapper(message.getEntityPath()))); - } - } else { - // We're not the leader. If the removed candidate is our local member then check if we actually - // have a local candidate registered. If we do then we must have been partitioned from the leader - // and the leader removed our candidate since the leader can't tell the difference between a - // temporary network partition and a node's process actually restarted. So, in that case, re-add - // our candidate. - if(localMemberName.equals(message.getRemovedCandidate()) && - listenerSupport.hasCandidateForEntity(createEntity(message.getEntityPath()))) { - LOG.debug("Local candidate member was removed but a local candidate is registered for {}" + - " - adding back local candidate", message.getEntityPath()); - - commitCoordinator.commitModification(new MergeModification( - candidatePath(message.getEntityPath(), localMemberName), - candidateMapEntry(localMemberName)), this); + newOwner(currentOwner, message.getRemainingCandidates(), getEntityOwnerElectionStrategy(message.getEntityPath()))); } } } - private EntityOwnerSelectionStrategyWrapper getEntityOwnerElectionStrategyWrapper(YangInstanceIdentifier entityPath) { - String entityType = EntityOwnersModel.entityTypeFromEntityPath(entityPath); - EntityOwnerSelectionStrategyWrapper entityOwnerSelectionStrategy = ownerSelectionStrategies.get(entityType); - - if(entityOwnerSelectionStrategy == null){ - entityOwnerSelectionStrategy = defaultEntityOwnerSelectionStrategy; - ownerSelectionStrategies.put(entityType, entityOwnerSelectionStrategy); - } - - return entityOwnerSelectionStrategy; + private EntityOwnerSelectionStrategy getEntityOwnerElectionStrategy(YangInstanceIdentifier entityPath) { + final String entityType = EntityOwnersModel.entityTypeFromEntityPath(entityPath); + return strategyConfig.createStrategy(entityType, entityOwnershipStatistics.byEntityType(entityType)); } private void onCandidateAdded(CandidateAdded message) { @@ -304,105 +394,130 @@ class EntityOwnershipShard extends Shard { // Since a node's candidate member is only added by the node itself, we can assume the node is up so // remove it from the downPeerMemberNames. - downPeerMemberNames.remove(message.getNewCandidate()); - - String currentOwner = getCurrentOwner(message.getEntityPath()); - if(Strings.isNullOrEmpty(currentOwner)){ - EntityOwnerSelectionStrategyWrapper strategy = getEntityOwnerElectionStrategyWrapper(message.getEntityPath()); - if(strategy.selectionDelayInMillis() == 0L) { - writeNewOwner(message.getEntityPath(), newOwner(message.getAllCandidates(), strategy)); - } else { - strategy.scheduleOwnerSelection(message.getEntityPath(), message.getAllCandidates()); - } + downPeerMemberNames.remove(MemberName.forName(message.getNewCandidate())); + + final String currentOwner = getCurrentOwner(message.getEntityPath()); + final EntityOwnerSelectionStrategy strategy = getEntityOwnerElectionStrategy(message.getEntityPath()); + + // Available members is all the known peers - the number of peers that are down + self + // So if there are 2 peers and 1 is down then availableMembers will be 2 + final int availableMembers = peerIdToMemberNames.size() - downPeerMemberNames.size() + 1; + + LOG.debug("{}: Using strategy {} to select owner, currentOwner = {}", persistenceId(), strategy, currentOwner); + + if(strategy.getSelectionDelayInMillis() == 0L) { + writeNewOwner(message.getEntityPath(), newOwner(currentOwner, message.getAllCandidates(), + strategy)); + } else if(message.getAllCandidates().size() == availableMembers) { + LOG.debug("{}: Received the maximum candidates requests : {} writing new owner", + persistenceId(), availableMembers); + cancelOwnerSelectionTask(message.getEntityPath()); + writeNewOwner(message.getEntityPath(), newOwner(currentOwner, message.getAllCandidates(), + strategy)); + } else { + scheduleOwnerSelection(message.getEntityPath(), message.getAllCandidates(), strategy); } } private void onPeerDown(PeerDown peerDown) { LOG.info("{}: onPeerDown: {}", persistenceId(), peerDown); - String downMemberName = peerDown.getMemberName(); + MemberName downMemberName = peerDown.getMemberName(); if(downPeerMemberNames.add(downMemberName) && isLeader()) { - // Remove the down peer as a candidate from all entities. - removeCandidateFromEntities(downMemberName); + // Select new owners for entities owned by the down peer and which have other candidates. For an entity for + // which the down peer is the only candidate, we leave it as the owner and don't clear it. This is done to + // handle the case where the peer member process is actually still running but the node is partitioned. + // When the partition is healed, the peer just remains as the owner. If the peer process actually restarted, + // it will first remove all its candidates on startup. If another candidate is registered during the time + // the peer is down, the new candidate will be selected as the new owner. + + selectNewOwnerForEntitiesOwnedBy(downMemberName); } } - private void onPeerUp(PeerUp peerUp) { - LOG.debug("{}: onPeerUp: {}", persistenceId(), peerUp); - - peerIdToMemberNames.put(peerUp.getPeerId(), peerUp.getMemberName()); - downPeerMemberNames.remove(peerUp.getMemberName()); - } - - private void selectNewOwnerForEntitiesOwnedBy(String owner) { + private void selectNewOwnerForEntitiesOwnedBy(MemberName downMemberName) { final BatchedModifications modifications = commitCoordinator.newBatchedModifications(); - searchForEntitiesOwnedBy(owner, new EntityWalker() { - @Override - public void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode) { - - YangInstanceIdentifier entityPath = YangInstanceIdentifier.builder(ENTITY_TYPES_PATH). - node(entityTypeNode.getIdentifier()).node(ENTITY_NODE_ID).node(entityNode.getIdentifier()). - node(ENTITY_OWNER_NODE_ID).build(); - - Object newOwner = newOwner(getCandidateNames(entityNode), getEntityOwnerElectionStrategyWrapper(entityPath)); - + searchForEntitiesOwnedBy(downMemberName.getName(), (entityTypeNode, entityNode) -> { + YangInstanceIdentifier entityPath = YangInstanceIdentifier.builder(ENTITY_TYPES_PATH). + node(entityTypeNode.getIdentifier()).node(ENTITY_NODE_ID).node(entityNode.getIdentifier()). + node(ENTITY_OWNER_NODE_ID).build(); + String newOwner = newOwner(getCurrentOwner(entityPath), getCandidateNames(entityNode), + getEntityOwnerElectionStrategy(entityPath)); + + if(!newOwner.isEmpty()) { LOG.debug("{}: Found entity {}, writing new owner {}", persistenceId(), entityPath, newOwner); modifications.addModification(new WriteModification(entityPath, ImmutableNodes.leafNode(ENTITY_OWNER_NODE_ID, newOwner))); + + } else { + LOG.debug("{}: Found entity {} but no other candidates - not clearing owner", persistenceId(), + entityPath, newOwner); } }); commitCoordinator.commitModifications(modifications, this); } - private void removeCandidateFromEntities(final String owner) { - final BatchedModifications modifications = commitCoordinator.newBatchedModifications(); - searchForEntities(new EntityWalker() { - @Override - public void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode) { - if (hasCandidate(entityNode, owner)) { - YangInstanceIdentifier entityId = - (YangInstanceIdentifier) entityNode.getIdentifier().getKeyValues().get(ENTITY_ID_QNAME); - YangInstanceIdentifier candidatePath = candidatePath( - entityTypeNode.getIdentifier().getKeyValues().get(ENTITY_TYPE_QNAME).toString(), - entityId, owner); - - LOG.info("{}: Found entity {}, removing candidate {}, path {}", persistenceId(), entityId, - owner, candidatePath); - - modifications.addModification(new DeleteModification(candidatePath)); - } - } - }); + private void onPeerUp(PeerUp peerUp) { + LOG.debug("{}: onPeerUp: {}", persistenceId(), peerUp); - commitCoordinator.commitModifications(modifications, this); - } + peerIdToMemberNames.put(peerUp.getPeerId(), peerUp.getMemberName()); + downPeerMemberNames.remove(peerUp.getMemberName()); - private static boolean hasCandidate(MapEntryNode entity, String candidateName) { - return ((MapNode)entity.getChild(CANDIDATE_NODE_ID).get()).getChild(candidateNodeKey(candidateName)).isPresent(); + // Notify the coordinator to check if pending modifications need to be sent. We do this here + // to handle the case where the leader's peer address isn't known yet when a prior state or + // leader change occurred. + commitCoordinator.onStateChanged(this, isLeader()); } - private void searchForEntitiesOwnedBy(final String owner, final EntityWalker walker) { - Optional> possibleEntityTypes = getDataStore().readNode(ENTITY_TYPES_PATH); - if(!possibleEntityTypes.isPresent()) { - return; + private Collection getCandidateNames(MapEntryNode entity) { + Collection candidates = ((MapNode)entity.getChild(CANDIDATE_NODE_ID).get()).getValue(); + Collection candidateNames = new ArrayList<>(candidates.size()); + for(MapEntryNode candidate: candidates) { + candidateNames.add(candidate.getChild(CANDIDATE_NAME_NODE_ID).get().getValue().toString()); } + return candidateNames; + } + + private void searchForEntitiesOwnedBy(final String owner, final EntityWalker walker) { LOG.debug("{}: Searching for entities owned by {}", persistenceId(), owner); - searchForEntities(new EntityWalker() { - @Override - public void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode) { - Optional> possibleOwner = - entityNode.getChild(ENTITY_OWNER_NODE_ID); - if(possibleOwner.isPresent() && owner.equals(possibleOwner.get().getValue().toString())) { - walker.onEntity(entityTypeNode, entityNode); - } + searchForEntities((entityTypeNode, entityNode) -> { + Optional> possibleOwner = + entityNode.getChild(ENTITY_OWNER_NODE_ID); + if(possibleOwner.isPresent() && owner.equals(possibleOwner.get().getValue().toString())) { + walker.onEntity(entityTypeNode, entityNode); } }); } + private void removeCandidateFromEntities(final MemberName member) { + final BatchedModifications modifications = commitCoordinator.newBatchedModifications(); + searchForEntities((entityTypeNode, entityNode) -> { + if (hasCandidate(entityNode, member)) { + YangInstanceIdentifier entityId = + (YangInstanceIdentifier) entityNode.getIdentifier().getKeyValues().get(ENTITY_ID_QNAME); + YangInstanceIdentifier candidatePath = candidatePath( + entityTypeNode.getIdentifier().getKeyValues().get(ENTITY_TYPE_QNAME).toString(), + entityId, member.getName()); + + LOG.info("{}: Found entity {}, removing candidate {}, path {}", persistenceId(), entityId, + member, candidatePath); + + modifications.addModification(new DeleteModification(candidatePath)); + } + }); + + commitCoordinator.commitModifications(modifications, this); + } + + private static boolean hasCandidate(MapEntryNode entity, MemberName candidateName) { + return ((MapNode)entity.getChild(CANDIDATE_NODE_ID).get()).getChild(candidateNodeKey(candidateName.getName())) + .isPresent(); + } + private void searchForEntities(EntityWalker walker) { Optional> possibleEntityTypes = getDataStore().readNode(ENTITY_TYPES_PATH); if(!possibleEntityTypes.isPresent()) { @@ -410,10 +525,10 @@ class EntityOwnershipShard extends Shard { } for(MapEntryNode entityType: ((MapNode) possibleEntityTypes.get()).getValue()) { - Optional> possibleEntities = - entityType.getChild(ENTITY_NODE_ID); + Optional> possibleEntities = entityType.getChild(ENTITY_NODE_ID); if(!possibleEntities.isPresent()) { - continue; // shouldn't happen but handle anyway + // shouldn't happen but handle anyway + continue; } for(MapEntryNode entity: ((MapNode) possibleEntities.get()).getValue()) { @@ -422,16 +537,6 @@ class EntityOwnershipShard extends Shard { } } - private static Collection getCandidateNames(MapEntryNode entity) { - Collection candidates = ((MapNode)entity.getChild(CANDIDATE_NODE_ID).get()).getValue(); - Collection candidateNames = new ArrayList<>(candidates.size()); - for(MapEntryNode candidate: candidates) { - candidateNames.add(candidate.getChild(CANDIDATE_NAME_NODE_ID).get().getValue().toString()); - } - - return candidateNames; - } - private void writeNewOwner(YangInstanceIdentifier entityPath, String newOwner) { LOG.debug("{}: Writing new owner {} for entity {}", persistenceId(), newOwner, entityPath); @@ -439,19 +544,46 @@ class EntityOwnershipShard extends Shard { ImmutableNodes.leafNode(ENTITY_OWNER_NODE_ID, newOwner)), this); } - private String newOwner(Collection candidates, EntityOwnerSelectionStrategy ownerSelectionStrategy) { + /** + * Schedule a new owner selection job. Cancelling any outstanding job if it has not been cancelled. + * + * @param entityPath + * @param allCandidates + */ + public void scheduleOwnerSelection(YangInstanceIdentifier entityPath, Collection allCandidates, + EntityOwnerSelectionStrategy strategy){ + cancelOwnerSelectionTask(entityPath); + + LOG.debug("{}: Scheduling owner selection after {} ms", persistenceId(), strategy.getSelectionDelayInMillis()); + + final Cancellable lastScheduledTask = context().system().scheduler().scheduleOnce( + FiniteDuration.apply(strategy.getSelectionDelayInMillis(), TimeUnit.MILLISECONDS) + , self(), new SelectOwner(entityPath, allCandidates, strategy) + , context().system().dispatcher(), self()); + + entityToScheduledOwnershipTask.put(entityPath, lastScheduledTask); + } + + private void cancelOwnerSelectionTask(YangInstanceIdentifier entityPath){ + final Cancellable lastScheduledTask = entityToScheduledOwnershipTask.get(entityPath); + if(lastScheduledTask != null && !lastScheduledTask.isCancelled()){ + lastScheduledTask.cancel(); + } + } + + private String newOwner(String currentOwner, Collection candidates, EntityOwnerSelectionStrategy ownerSelectionStrategy) { Collection viableCandidates = getViableCandidates(candidates); - if(viableCandidates.size() == 0){ + if(viableCandidates.isEmpty()){ return ""; } - return ownerSelectionStrategy.newOwner(viableCandidates); + return ownerSelectionStrategy.newOwner(currentOwner, viableCandidates); } private Collection getViableCandidates(Collection candidates) { Collection viableCandidates = new ArrayList<>(); for (String candidate : candidates) { - if (!downPeerMemberNames.contains(candidate)) { + if (!downPeerMemberNames.contains(MemberName.forName(candidate))) { viableCandidates.add(candidate); } } @@ -466,47 +598,40 @@ class EntityOwnershipShard extends Shard { return null; } + @FunctionalInterface private static interface EntityWalker { void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode); } - private EntityOwnerSelectionStrategyWrapper createEntityOwnerSelectionStrategyWrapper(EntityOwnerSelectionStrategy entityOwnerSelectionStrategy){ - return new EntityOwnerSelectionStrategyWrapper(context().system().scheduler(), self(), - context().system().dispatcher(), entityOwnerSelectionStrategy); - } - - @VisibleForTesting - void addEntityOwnerSelectionStrategy(String entityType, Class ownerSelectionStrategyClass){ - try { - EntityOwnerSelectionStrategyWrapper strategy = - createEntityOwnerSelectionStrategyWrapper(ownerSelectionStrategyClass.newInstance()); - ownerSelectionStrategies.put(entityType, strategy); - } catch (InstantiationException | IllegalAccessException e) { - LOG.error("Exception occurred when adding election strategy", e); - } - } - public static Builder newBuilder() { return new Builder(); } static class Builder extends Shard.AbstractBuilder { - private String localMemberName; + private MemberName localMemberName; + private EntityOwnerSelectionStrategyConfig ownerSelectionStrategyConfig; protected Builder() { super(EntityOwnershipShard.class); } - Builder localMemberName(String localMemberName) { + Builder localMemberName(MemberName localMemberName) { checkSealed(); this.localMemberName = localMemberName; return this; } + Builder ownerSelectionStrategyConfig(EntityOwnerSelectionStrategyConfig ownerSelectionStrategyConfig){ + checkSealed(); + this.ownerSelectionStrategyConfig = ownerSelectionStrategyConfig; + return this; + } + @Override protected void verify() { super.verify(); Preconditions.checkNotNull(localMemberName, "localMemberName should not be null"); + Preconditions.checkNotNull(ownerSelectionStrategyConfig, "ownerSelectionStrategyConfig should not be null"); } } }