X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2Fentityownership%2FEntityOwnershipShard.java;h=6e1d1a855a5ff0c99ec2df73ab2847fa398b0412;hp=46743482928f92bc1e44f28c0df8c27923d8f721;hb=refs%2Fchanges%2F29%2F83829%2F3;hpb=d86f990976dcc2879b40dec7df1b3b5fba8cba78 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 4674348292..6e1d1a855a 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 @@ -7,6 +7,7 @@ */ package org.opendaylight.controller.cluster.datastore.entityownership; +import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.CANDIDATE_NAME_NODE_ID; import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.CANDIDATE_NODE_ID; import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_ID_NODE_ID; import static org.opendaylight.controller.cluster.datastore.entityownership.EntityOwnersModel.ENTITY_ID_QNAME; @@ -17,23 +18,29 @@ 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.cluster.Cluster; +import akka.cluster.ClusterEvent.CurrentClusterState; +import akka.cluster.Member; +import akka.cluster.MemberStatus; import akka.pattern.Patterns; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; import org.opendaylight.controller.cluster.access.concepts.MemberName; @@ -43,6 +50,7 @@ import org.opendaylight.controller.cluster.datastore.entityownership.messages.Ca 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; @@ -55,10 +63,13 @@ import org.opendaylight.controller.cluster.datastore.messages.PeerUp; 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.Modification; import org.opendaylight.controller.cluster.datastore.modification.WriteModification; import org.opendaylight.controller.cluster.raft.RaftState; +import org.opendaylight.controller.cluster.raft.VotingState; 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; import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode; import org.opendaylight.yangtools.yang.data.api.schema.MapNode; @@ -77,16 +88,12 @@ class EntityOwnershipShard extends Shard { private final EntityOwnershipShardCommitCoordinator commitCoordinator; private final EntityOwnershipListenerSupport listenerSupport; 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(); - } - - protected EntityOwnershipShard(Builder builder) { + protected EntityOwnershipShard(final Builder builder) { super(builder); this.localMemberName = builder.localMemberName; this.commitCoordinator = new EntityOwnershipShardCommitCoordinator(builder.localMemberName, LOG); @@ -94,15 +101,14 @@ class EntityOwnershipShard extends Shard { this.strategyConfig = builder.ownerSelectionStrategyConfig; this.entityOwnershipStatistics = new EntityOwnershipStatistics(); this.entityOwnershipStatistics.init(getDataStore()); + } - for(String peerId: getRaftActorContext().getPeerIds()) { - ShardIdentifier shardId = ShardIdentifier.fromShardIdString(peerId); - peerIdToMemberNames.put(peerId, shardId.getMemberName()); - } + private static DatastoreContext noPersistenceDatastoreContext(final DatastoreContext datastoreContext) { + return DatastoreContext.newBuilderFrom(datastoreContext).persistent(false).build(); } @Override - protected void onDatastoreContext(DatastoreContext context) { + protected void onDatastoreContext(final DatastoreContext context) { super.onDatastoreContext(noPersistenceDatastoreContext(context)); } @@ -116,42 +122,48 @@ class EntityOwnershipShard extends Shard { @Override public void handleNonRaftCommand(final Object message) { - if(message instanceof RegisterCandidateLocal) { + if (message instanceof RegisterCandidateLocal) { onRegisterCandidateLocal((RegisterCandidateLocal) message); - } else if(message instanceof UnregisterCandidateLocal) { - onUnregisterCandidateLocal((UnregisterCandidateLocal)message); - } else if(message instanceof CandidateAdded){ + } else if (message instanceof UnregisterCandidateLocal) { + onUnregisterCandidateLocal((UnregisterCandidateLocal) message); + } else if (message instanceof CandidateAdded) { onCandidateAdded((CandidateAdded) message); - } else if(message instanceof CandidateRemoved){ + } else if (message instanceof CandidateRemoved) { onCandidateRemoved((CandidateRemoved) message); - } else if(message instanceof PeerDown) { + } else if (message instanceof PeerDown) { onPeerDown((PeerDown) message); - } else if(message instanceof PeerUp) { + } else if (message instanceof PeerUp) { onPeerUp((PeerUp) message); - } else if(message instanceof RegisterListenerLocal) { - onRegisterListenerLocal((RegisterListenerLocal)message); - } else if(message instanceof UnregisterListenerLocal) { + } else if (message instanceof RegisterListenerLocal) { + onRegisterListenerLocal((RegisterListenerLocal) message); + } else if (message instanceof UnregisterListenerLocal) { onUnregisterListenerLocal((UnregisterListenerLocal) message); - } else if(message instanceof SelectOwner) { + } else if (message instanceof SelectOwner) { onSelectOwner((SelectOwner) message); - } else if(!commitCoordinator.handleMessage(message, this)) { + } else if (message instanceof RemoveAllCandidates) { + onRemoveAllCandidates((RemoveAllCandidates) message); + } else if (!commitCoordinator.handleMessage(message, this)) { super.handleNonRaftCommand(message); } } - private void onSelectOwner(SelectOwner selectOwner) { + private void onRemoveAllCandidates(final RemoveAllCandidates message) { + LOG.debug("{}: onRemoveAllCandidates: {}", persistenceId(), message); + + removeCandidateFromEntities(message.getMemberName()); + } + + private void onSelectOwner(final SelectOwner selectOwner) { LOG.debug("{}: onSelectOwner: {}", persistenceId(), selectOwner); String currentOwner = getCurrentOwner(selectOwner.getEntityPath()); - if(Strings.isNullOrEmpty(currentOwner)) { - String entityType = EntityOwnersModel.entityTypeFromEntityPath(selectOwner.getEntityPath()); + if (Strings.isNullOrEmpty(currentOwner)) { writeNewOwner(selectOwner.getEntityPath(), newOwner(currentOwner, selectOwner.getAllCandidates(), - entityOwnershipStatistics.byEntityType(entityType), selectOwner.getOwnerSelectionStrategy())); Cancellable cancellable = entityToScheduledOwnershipTask.get(selectOwner.getEntityPath()); - if(cancellable != null){ - if(!cancellable.isCancelled()){ + if (cancellable != null) { + if (!cancellable.isCancelled()) { cancellable.cancel(); } entityToScheduledOwnershipTask.remove(selectOwner.getEntityPath()); @@ -159,11 +171,9 @@ class EntityOwnershipShard extends Shard { } } - private void onRegisterCandidateLocal(RegisterCandidateLocal registerCandidate) { + private void onRegisterCandidateLocal(final RegisterCandidateLocal registerCandidate) { LOG.debug("{}: onRegisterCandidateLocal: {}", persistenceId(), registerCandidate); - listenerSupport.setHasCandidateForEntity(registerCandidate.getEntity()); - NormalizedNode entityOwners = entityOwnersWithCandidate(registerCandidate.getEntity().getType(), registerCandidate.getEntity().getIdentifier(), localMemberName.getName()); commitCoordinator.commitModification(new MergeModification(ENTITY_OWNERS_PATH, entityOwners), this); @@ -171,13 +181,12 @@ class EntityOwnershipShard extends Shard { getSender().tell(SuccessReply.INSTANCE, getSelf()); } - private void onUnregisterCandidateLocal(UnregisterCandidateLocal unregisterCandidate) { + private void onUnregisterCandidateLocal(final UnregisterCandidateLocal unregisterCandidate) { LOG.debug("{}: onUnregisterCandidateLocal: {}", persistenceId(), unregisterCandidate); DOMEntity entity = unregisterCandidate.getEntity(); - listenerSupport.unsetHasCandidateForEntity(entity); - - YangInstanceIdentifier candidatePath = candidatePath(entity.getType(), entity.getIdentifier(), localMemberName.getName()); + YangInstanceIdentifier candidatePath = candidatePath(entity.getType(), entity.getIdentifier(), + localMemberName.getName()); commitCoordinator.commitModification(new DeleteModification(candidatePath), this); getSender().tell(SuccessReply.INSTANCE, getSelf()); @@ -215,17 +224,19 @@ class EntityOwnershipShard extends Shard { }); } - private void onUnregisterListenerLocal(UnregisterListenerLocal unregisterListener) { + private void onUnregisterListenerLocal(final UnregisterListenerLocal unregisterListener) { LOG.debug("{}: onUnregisterListenerLocal: {}", persistenceId(), unregisterListener); - listenerSupport.removeEntityOwnershipListener(unregisterListener.getEntityType(), unregisterListener.getListener()); + listenerSupport.removeEntityOwnershipListener(unregisterListener.getEntityType(), + unregisterListener.getListener()); getSender().tell(SuccessReply.INSTANCE, getSelf()); } void tryCommitModifications(final BatchedModifications modifications) { - if(isLeader()) { - LOG.debug("{}: Committing BatchedModifications {} locally", persistenceId(), modifications.getTransactionID()); + if (isLeader()) { + LOG.debug("{}: Committing BatchedModifications {} locally", persistenceId(), + modifications.getTransactionId()); // Note that it's possible the commit won't get consensus and will timeout and not be applied // to the state. However we don't need to retry it in that case b/c it will be committed to @@ -235,10 +246,10 @@ class EntityOwnershipShard extends Shard { } else { final ActorSelection leader = getLeader(); if (leader != null) { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: Sending BatchedModifications {} to leader {}", persistenceId(), - modifications.getTransactionID(), leader); - } + possiblyRemoveAllInitialCandidates(leader); + + LOG.debug("{}: Sending BatchedModifications {} to leader {}", persistenceId(), + modifications.getTransactionId(), leader); Future future = Patterns.ask(leader, modifications, TimeUnit.SECONDS.toMillis( getDatastoreContext().getShardTransactionCommitTimeoutInSeconds())); @@ -248,8 +259,26 @@ class EntityOwnershipShard extends Shard { } } + void possiblyRemoveAllInitialCandidates(final 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()); } /** @@ -264,8 +293,9 @@ class EntityOwnershipShard extends Shard { return false; case IsolatedLeader: return true; + default: + throw new IllegalStateException("Unsupported RAFT state " + state); } - throw new IllegalStateException("Unsupported RAFT state " + state); } private void notifyAllListeners() { @@ -294,8 +324,6 @@ class EntityOwnershipShard extends Shard { @Override protected void onStateChanged() { - super.onStateChanged(); - boolean isLeader = isLeader(); LOG.debug("{}: onStateChanged: isLeader: {}, hasLeader: {}", persistenceId(), isLeader, hasLeader()); @@ -309,73 +337,118 @@ class EntityOwnershipShard extends Shard { } commitCoordinator.onStateChanged(this, isLeader); + + super.onStateChanged(); } @Override - protected void onLeaderChanged(String oldLeader, String newLeader) { - super.onLeaderChanged(oldLeader, newLeader); - + protected void onLeaderChanged(final String oldLeader, final String newLeader) { boolean isLeader = isLeader(); LOG.debug("{}: onLeaderChanged: oldLeader: {}, newLeader: {}, isLeader: {}", persistenceId(), oldLeader, newLeader, isLeader); if (isLeader) { + // Re-initialize the downPeerMemberNames from the current akka Cluster state. The previous leader, if any, + // is most likely down however it's possible we haven't received the PeerDown message yet. + initializeDownPeerMemberNamesFromClusterState(); + // 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(); - // Remove the candidates 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 remove all the candidates for those downed nodes + // 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 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. + Set ownedBy = new HashSet<>(downPeerMemberNames.size() + 1); for (MemberName downPeerName : downPeerMemberNames) { - removeCandidateFromEntities(downPeerName); + ownedBy.add(downPeerName.getName()); } + + // Also try to assign owners for entities that have no current owner. See explanation in onPeerUp. + ownedBy.add(""); + selectNewOwnerForEntitiesOwnedBy(ownedBy); } 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); } + + super.onLeaderChanged(oldLeader, newLeader); } - private void onCandidateRemoved(CandidateRemoved message) { + @Override + protected void onVotingStateChangeComplete() { + // Re-evaluate ownership for all entities - if a member changed from voting to non-voting it should lose + // ownership and vice versa it now is a candidate to become owner. + final List modifications = new ArrayList<>(); + searchForEntities((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(); + + Optional possibleOwner = + entityNode.getChild(ENTITY_OWNER_NODE_ID).map(node -> node.getValue().toString()); + String newOwner = newOwner(possibleOwner.orElse(null), getCandidateNames(entityNode), + getEntityOwnerElectionStrategy(entityPath)); + + if (!newOwner.equals(possibleOwner.orElse(""))) { + modifications.add(new WriteModification(entityPath, + ImmutableNodes.leafNode(ENTITY_OWNER_NODE_ID, newOwner))); + } + }); + + commitCoordinator.commitModifications(modifications, this); + } + + private void initializeDownPeerMemberNamesFromClusterState() { + Optional cluster = getRaftActorContext().getCluster(); + if (!cluster.isPresent()) { + return; + } + + CurrentClusterState state = cluster.get().state(); + Set unreachable = state.getUnreachable(); + + LOG.debug( + "{}: initializeDownPeerMemberNamesFromClusterState - current downPeerMemberNames: {}, unreachable: {}", + persistenceId(), downPeerMemberNames, unreachable); + + downPeerMemberNames.clear(); + for (Member m: unreachable) { + downPeerMemberNames.add(MemberName.forName(m.getRoles().iterator().next())); + } + + for (Member m: state.getMembers()) { + if (m.status() != MemberStatus.up() && m.status() != MemberStatus.weaklyUp()) { + LOG.debug("{}: Adding down member with status {}", persistenceId(), m.status()); + downPeerMemberNames.add(MemberName.forName(m.getRoles().iterator().next())); + } + } + + LOG.debug("{}: new downPeerMemberNames: {}", persistenceId(), downPeerMemberNames); + } + + private void onCandidateRemoved(final CandidateRemoved message) { LOG.debug("{}: onCandidateRemoved: {}", persistenceId(), message); - if(isLeader()) { + if (isLeader()) { String currentOwner = getCurrentOwner(message.getEntityPath()); - if(message.getRemovedCandidate().equals(currentOwner) || message.getRemainingCandidates().size() == 0){ - String entityType = EntityOwnersModel.entityTypeFromEntityPath(message.getEntityPath()); - writeNewOwner(message.getEntityPath(), - newOwner(currentOwner, message.getRemainingCandidates(), entityOwnershipStatistics.byEntityType(entityType), - getEntityOwnerElectionStrategy(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.getName().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.getName()), - candidateMapEntry(localMemberName.getName())), this); - } + writeNewOwner(message.getEntityPath(), + newOwner(currentOwner, message.getRemainingCandidates(), + getEntityOwnerElectionStrategy(message.getEntityPath()))); } } - private EntityOwnerSelectionStrategy getEntityOwnerElectionStrategy(YangInstanceIdentifier entityPath) { + private EntityOwnerSelectionStrategy getEntityOwnerElectionStrategy(final YangInstanceIdentifier entityPath) { final String entityType = EntityOwnersModel.entityTypeFromEntityPath(entityPath); return strategyConfig.createStrategy(entityType, entityOwnershipStatistics.byEntityType(entityType)); } - private void onCandidateAdded(CandidateAdded message) { - if(!isLeader()){ + private void onCandidateAdded(final CandidateAdded message) { + if (!isLeader()) { return; } @@ -383,101 +456,162 @@ 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()); + downPeerMemberNames.remove(MemberName.forName(message.getNewCandidate())); final String currentOwner = getCurrentOwner(message.getEntityPath()); final EntityOwnerSelectionStrategy strategy = getEntityOwnerElectionStrategy(message.getEntityPath()); - final String entityType = EntityOwnersModel.entityTypeFromEntityPath(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; + final int availableMembers = getRaftActorContext().getPeerIds().size() - downPeerMemberNames.size() + 1; LOG.debug("{}: Using strategy {} to select owner, currentOwner = {}", persistenceId(), strategy, currentOwner); - if(!message.getAllCandidates().contains(currentOwner)){ - if(strategy.getSelectionDelayInMillis() == 0L) { - writeNewOwner(message.getEntityPath(), newOwner(currentOwner, message.getAllCandidates(), - entityOwnershipStatistics.byEntityType(entityType), 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(), - entityOwnershipStatistics.byEntityType(entityType), strategy)); - } else { - scheduleOwnerSelection(message.getEntityPath(), message.getAllCandidates(), strategy); - } + 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) { + private void onPeerDown(final PeerDown peerDown) { LOG.info("{}: onPeerDown: {}", persistenceId(), peerDown); MemberName downMemberName = peerDown.getMemberName(); - if(downPeerMemberNames.add(downMemberName) && isLeader()) { - // Remove the down peer as a candidate from all entities. - removeCandidateFromEntities(downMemberName); + if (downPeerMemberNames.add(downMemberName) && isLeader()) { + // 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(ImmutableSet.of(downMemberName.getName())); } } - private void onPeerUp(PeerUp peerUp) { + private void selectNewOwnerForEntitiesOwnedBy(final Set ownedBy) { + final List modifications = new ArrayList<>(); + searchForEntitiesOwnedBy(ownedBy, (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.add(new WriteModification(entityPath, + ImmutableNodes.leafNode(ENTITY_OWNER_NODE_ID, newOwner))); + + } else { + LOG.debug("{}: Found entity {} but no other candidates - not clearing owner", persistenceId(), + entityPath); + } + }); + + commitCoordinator.commitModifications(modifications, this); + } + + private void onPeerUp(final PeerUp peerUp) { LOG.debug("{}: onPeerUp: {}", persistenceId(), peerUp); - peerIdToMemberNames.put(peerUp.getPeerId(), peerUp.getMemberName()); downPeerMemberNames.remove(peerUp.getMemberName()); // 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 now yet when a prior state or + // 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()); + + if (isLeader()) { + // Try to assign owners for entities that have no current owner. It's possible the peer that is now up + // had previously registered as a candidate and was the only candidate but the owner write tx couldn't be + // committed due to a leader change. Eg, the leader is able to successfully commit the candidate add tx but + // becomes isolated before it can commit the owner change and switches to follower. The majority partition + // with a new leader has the candidate but the entity has no owner. When the partition is healed and the + // previously isolated leader reconnects, we'll receive onPeerUp and, if there's still no owner, the + // previous leader will gain ownership. + selectNewOwnerForEntitiesOwnedBy(ImmutableSet.of("")); + } } - private void removeCandidateFromEntities(final MemberName owner) { - final BatchedModifications modifications = commitCoordinator.newBatchedModifications(); + private static Collection getCandidateNames(final MapEntryNode entity) { + return entity.getChild(CANDIDATE_NODE_ID).map(child -> { + Collection candidates = ((MapNode) child).getValue(); + Collection candidateNames = new ArrayList<>(candidates.size()); + for (MapEntryNode candidate: candidates) { + candidateNames.add(candidate.getChild(CANDIDATE_NAME_NODE_ID).get().getValue().toString()); + } + return candidateNames; + }).orElse(ImmutableList.of()); + } + + private void searchForEntitiesOwnedBy(final Set ownedBy, final EntityWalker walker) { + LOG.debug("{}: Searching for entities owned by {}", persistenceId(), ownedBy); + searchForEntities((entityTypeNode, 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.getName()); + Optional> possibleOwner = + entityNode.getChild(ENTITY_OWNER_NODE_ID); + String currentOwner = possibleOwner.isPresent() ? possibleOwner.get().getValue().toString() : ""; + if (ownedBy.contains(currentOwner)) { + walker.onEntity(entityTypeNode, entityNode); + } + }); + } + + private void removeCandidateFromEntities(final MemberName member) { + final List modifications = new ArrayList<>(); + searchForEntities((entityTypeNode, entityNode) -> { + if (hasCandidate(entityNode, member)) { + YangInstanceIdentifier entityId = (YangInstanceIdentifier) entityNode.getIdentifier() + .getValue(ENTITY_ID_QNAME); + YangInstanceIdentifier candidatePath = candidatePath(entityTypeNode.getIdentifier() + .getValue(ENTITY_TYPE_QNAME).toString(), entityId, member.getName()); LOG.info("{}: Found entity {}, removing candidate {}, path {}", persistenceId(), entityId, - owner, candidatePath); + member, candidatePath); - modifications.addModification(new DeleteModification(candidatePath)); + modifications.add(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())) + private static boolean hasCandidate(final MapEntryNode entity, final MemberName candidateName) { + return entity.getChild(CANDIDATE_NODE_ID) + .flatMap(child -> ((MapNode)child).getChild(candidateNodeKey(candidateName.getName()))) .isPresent(); } - private void searchForEntities(EntityWalker walker) { + private void searchForEntities(final EntityWalker walker) { Optional> possibleEntityTypes = getDataStore().readNode(ENTITY_TYPES_PATH); - if(!possibleEntityTypes.isPresent()) { + if (!possibleEntityTypes.isPresent()) { return; } - for(MapEntryNode entityType: ((MapNode) possibleEntityTypes.get()).getValue()) { + for (MapEntryNode entityType : ((MapNode) possibleEntityTypes.get()).getValue()) { Optional> possibleEntities = entityType.getChild(ENTITY_NODE_ID); - if(!possibleEntities.isPresent()) { + if (!possibleEntities.isPresent()) { // shouldn't happen but handle anyway continue; } - for(MapEntryNode entity: ((MapNode) possibleEntities.get()).getValue()) { + for (MapEntryNode entity: ((MapNode) possibleEntities.get()).getValue()) { walker.onEntity(entityType, entity); } } } - private void writeNewOwner(YangInstanceIdentifier entityPath, String newOwner) { + private void writeNewOwner(final YangInstanceIdentifier entityPath, final String newOwner) { LOG.debug("{}: Writing new owner {} for entity {}", persistenceId(), newOwner, entityPath); commitCoordinator.commitModification(new WriteModification(entityPath.node(ENTITY_OWNER_QNAME), @@ -486,59 +620,61 @@ class EntityOwnershipShard extends Shard { /** * 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){ + private void scheduleOwnerSelection(final YangInstanceIdentifier entityPath, final Collection allCandidates, + final 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()); + 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){ + private void cancelOwnerSelectionTask(final YangInstanceIdentifier entityPath) { final Cancellable lastScheduledTask = entityToScheduledOwnershipTask.get(entityPath); - if(lastScheduledTask != null && !lastScheduledTask.isCancelled()){ + if (lastScheduledTask != null && !lastScheduledTask.isCancelled()) { lastScheduledTask.cancel(); } } - private String newOwner(String currentOwner, Collection candidates, Map statistics, EntityOwnerSelectionStrategy ownerSelectionStrategy) { + private String newOwner(final String currentOwner, final Collection candidates, + final EntityOwnerSelectionStrategy ownerSelectionStrategy) { Collection viableCandidates = getViableCandidates(candidates); - if(viableCandidates.size() == 0){ + if (viableCandidates.isEmpty()) { return ""; } return ownerSelectionStrategy.newOwner(currentOwner, viableCandidates); } - private Collection getViableCandidates(Collection candidates) { + private Collection getViableCandidates(final Collection candidates) { + Map memberToVotingState = new HashMap<>(); + getRaftActorContext().getPeers().forEach(peerInfo -> memberToVotingState.put( + ShardIdentifier.fromShardIdString(peerInfo.getId()).getMemberName(), peerInfo.getVotingState())); + Collection viableCandidates = new ArrayList<>(); for (String candidate : candidates) { - if (!downPeerMemberNames.contains(candidate)) { + MemberName memberName = MemberName.forName(candidate); + if (memberToVotingState.get(memberName) != VotingState.NON_VOTING + && !downPeerMemberNames.contains(memberName)) { viableCandidates.add(candidate); } } return viableCandidates; } - private String getCurrentOwner(YangInstanceIdentifier entityId) { - Optional> optionalEntityOwner = getDataStore().readNode(entityId.node(ENTITY_OWNER_QNAME)); - if(optionalEntityOwner.isPresent()){ - return optionalEntityOwner.get().getValue().toString(); - } - return null; + private String getCurrentOwner(final YangInstanceIdentifier entityId) { + return getDataStore().readNode(entityId.node(ENTITY_OWNER_QNAME)) + .map(owner -> owner.getValue().toString()) + .orElse(null); } - private static interface EntityWalker { + @FunctionalInterface + private interface EntityWalker { void onEntity(MapEntryNode entityTypeNode, MapEntryNode entityNode); } @@ -554,15 +690,15 @@ class EntityOwnershipShard extends Shard { super(EntityOwnershipShard.class); } - Builder localMemberName(MemberName localMemberName) { + Builder localMemberName(final MemberName newLocalMemberName) { checkSealed(); - this.localMemberName = localMemberName; + this.localMemberName = newLocalMemberName; return this; } - Builder ownerSelectionStrategyConfig(EntityOwnerSelectionStrategyConfig ownerSelectionStrategyConfig){ + Builder ownerSelectionStrategyConfig(final EntityOwnerSelectionStrategyConfig newOwnerSelectionStrategyConfig) { checkSealed(); - this.ownerSelectionStrategyConfig = ownerSelectionStrategyConfig; + this.ownerSelectionStrategyConfig = newOwnerSelectionStrategyConfig; return this; }