mirror of
https://github.com/opennetworkinglab/onos.git
synced 2025-10-27 22:31:18 +01:00
ONOS-1965: Evict inactive nodes from candidates map + Rely on cluster events to trigger stale lock purge
Change-Id: Ib7cfea397f98d6271beb78e3b88041bb84550506
This commit is contained in:
parent
cd38ba5f56
commit
cc58675360
@ -13,6 +13,9 @@ import org.apache.felix.scr.annotations.Reference;
|
|||||||
import org.apache.felix.scr.annotations.ReferenceCardinality;
|
import org.apache.felix.scr.annotations.ReferenceCardinality;
|
||||||
import org.apache.felix.scr.annotations.Service;
|
import org.apache.felix.scr.annotations.Service;
|
||||||
import org.onlab.util.KryoNamespace;
|
import org.onlab.util.KryoNamespace;
|
||||||
|
import org.onosproject.cluster.ClusterEvent;
|
||||||
|
import org.onosproject.cluster.ClusterEvent.Type;
|
||||||
|
import org.onosproject.cluster.ClusterEventListener;
|
||||||
import org.onosproject.cluster.ClusterService;
|
import org.onosproject.cluster.ClusterService;
|
||||||
import org.onosproject.cluster.Leadership;
|
import org.onosproject.cluster.Leadership;
|
||||||
import org.onosproject.cluster.LeadershipEvent;
|
import org.onosproject.cluster.LeadershipEvent;
|
||||||
@ -32,6 +35,7 @@ import org.onosproject.store.service.Versioned;
|
|||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Map.Entry;
|
import java.util.Map.Entry;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
@ -55,7 +59,7 @@ import static org.onosproject.cluster.ControllerNode.State.INACTIVE;
|
|||||||
/**
|
/**
|
||||||
* Distributed Lock Manager implemented on top of ConsistentMap.
|
* Distributed Lock Manager implemented on top of ConsistentMap.
|
||||||
* <p>
|
* <p>
|
||||||
* This implementation makes use of cluster manager's failure
|
* This implementation makes use of ClusterService's failure
|
||||||
* detection capabilities to detect and purge stale locks.
|
* detection capabilities to detect and purge stale locks.
|
||||||
* TODO: Ensure lock safety and liveness.
|
* TODO: Ensure lock safety and liveness.
|
||||||
*/
|
*/
|
||||||
@ -81,27 +85,28 @@ public class DistributedLeadershipManager implements LeadershipService {
|
|||||||
private final Logger log = getLogger(getClass());
|
private final Logger log = getLogger(getClass());
|
||||||
private ExecutorService messageHandlingExecutor;
|
private ExecutorService messageHandlingExecutor;
|
||||||
private ScheduledExecutorService retryLeaderLockExecutor;
|
private ScheduledExecutorService retryLeaderLockExecutor;
|
||||||
private ScheduledExecutorService deadLockDetectionExecutor;
|
private ScheduledExecutorService staleLeadershipPurgeExecutor;
|
||||||
private ScheduledExecutorService leadershipStatusBroadcaster;
|
private ScheduledExecutorService leadershipStatusBroadcaster;
|
||||||
|
|
||||||
private ConsistentMap<String, NodeId> leaderMap;
|
private ConsistentMap<String, NodeId> leaderMap;
|
||||||
private ConsistentMap<String, List<NodeId>> candidateMap;
|
private ConsistentMap<String, List<NodeId>> candidateMap;
|
||||||
|
|
||||||
private ListenerRegistry<LeadershipEvent, LeadershipEventListener>
|
private ListenerRegistry<LeadershipEvent, LeadershipEventListener> listenerRegistry;
|
||||||
listenerRegistry;
|
|
||||||
private final Map<String, Leadership> leaderBoard = Maps.newConcurrentMap();
|
private final Map<String, Leadership> leaderBoard = Maps.newConcurrentMap();
|
||||||
private final Map<String, Leadership> candidateBoard = Maps.newConcurrentMap();
|
private final Map<String, Leadership> candidateBoard = Maps.newConcurrentMap();
|
||||||
private NodeId localNodeId;
|
private final ClusterEventListener clusterEventListener = new InternalClusterEventListener();
|
||||||
|
|
||||||
|
private NodeId localNodeId;
|
||||||
private Set<String> activeTopics = Sets.newConcurrentHashSet();
|
private Set<String> activeTopics = Sets.newConcurrentHashSet();
|
||||||
|
|
||||||
private static final int ELECTION_JOIN_ATTEMPT_INTERVAL_SEC = 2;
|
private static final int ELECTION_JOIN_ATTEMPT_INTERVAL_SEC = 2;
|
||||||
private static final int DELAY_BETWEEN_LEADER_LOCK_ATTEMPTS_SEC = 2;
|
private static final int DELAY_BETWEEN_LEADER_LOCK_ATTEMPTS_SEC = 2;
|
||||||
private static final int DEADLOCK_DETECTION_INTERVAL_SEC = 2;
|
|
||||||
private static final int LEADERSHIP_STATUS_UPDATE_INTERVAL_SEC = 2;
|
private static final int LEADERSHIP_STATUS_UPDATE_INTERVAL_SEC = 2;
|
||||||
|
private static final int DELAY_BETWEEN_STALE_LEADERSHIP_PURGE_ATTEMPTS_SEC = 2;
|
||||||
private static final int LEADER_CANDIDATE_POS = 0;
|
private static final int LEADER_CANDIDATE_POS = 0;
|
||||||
|
|
||||||
|
private final AtomicBoolean staleLeadershipPurgeScheduled = new AtomicBoolean(false);
|
||||||
|
|
||||||
private static final Serializer SERIALIZER = Serializer.using(
|
private static final Serializer SERIALIZER = Serializer.using(
|
||||||
new KryoNamespace.Builder().register(KryoNamespaces.API).build());
|
new KryoNamespace.Builder().register(KryoNamespaces.API).build());
|
||||||
|
|
||||||
@ -122,8 +127,8 @@ public class DistributedLeadershipManager implements LeadershipService {
|
|||||||
groupedThreads("onos/store/leadership", "message-handler"));
|
groupedThreads("onos/store/leadership", "message-handler"));
|
||||||
retryLeaderLockExecutor = Executors.newScheduledThreadPool(
|
retryLeaderLockExecutor = Executors.newScheduledThreadPool(
|
||||||
4, groupedThreads("onos/store/leadership", "election-thread-%d"));
|
4, groupedThreads("onos/store/leadership", "election-thread-%d"));
|
||||||
deadLockDetectionExecutor = Executors.newSingleThreadScheduledExecutor(
|
staleLeadershipPurgeExecutor = Executors.newSingleThreadScheduledExecutor(
|
||||||
groupedThreads("onos/store/leadership", "dead-lock-detector"));
|
groupedThreads("onos/store/leadership", "stale-leadership-evictor"));
|
||||||
leadershipStatusBroadcaster = Executors.newSingleThreadScheduledExecutor(
|
leadershipStatusBroadcaster = Executors.newSingleThreadScheduledExecutor(
|
||||||
groupedThreads("onos/store/leadership", "peer-updater"));
|
groupedThreads("onos/store/leadership", "peer-updater"));
|
||||||
clusterCommunicator.addSubscriber(
|
clusterCommunicator.addSubscriber(
|
||||||
@ -132,8 +137,8 @@ public class DistributedLeadershipManager implements LeadershipService {
|
|||||||
this::onLeadershipEvent,
|
this::onLeadershipEvent,
|
||||||
messageHandlingExecutor);
|
messageHandlingExecutor);
|
||||||
|
|
||||||
deadLockDetectionExecutor.scheduleWithFixedDelay(
|
clusterService.addListener(clusterEventListener);
|
||||||
this::purgeStaleLocks, 0, DEADLOCK_DETECTION_INTERVAL_SEC, TimeUnit.SECONDS);
|
|
||||||
leadershipStatusBroadcaster.scheduleWithFixedDelay(
|
leadershipStatusBroadcaster.scheduleWithFixedDelay(
|
||||||
this::sendLeadershipStatus, 0, LEADERSHIP_STATUS_UPDATE_INTERVAL_SEC, TimeUnit.SECONDS);
|
this::sendLeadershipStatus, 0, LEADERSHIP_STATUS_UPDATE_INTERVAL_SEC, TimeUnit.SECONDS);
|
||||||
|
|
||||||
@ -151,12 +156,13 @@ public class DistributedLeadershipManager implements LeadershipService {
|
|||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
|
clusterService.removeListener(clusterEventListener);
|
||||||
eventDispatcher.removeSink(LeadershipEvent.class);
|
eventDispatcher.removeSink(LeadershipEvent.class);
|
||||||
clusterCommunicator.removeSubscriber(LEADERSHIP_EVENT_MESSAGE_SUBJECT);
|
clusterCommunicator.removeSubscriber(LEADERSHIP_EVENT_MESSAGE_SUBJECT);
|
||||||
|
|
||||||
messageHandlingExecutor.shutdown();
|
messageHandlingExecutor.shutdown();
|
||||||
retryLeaderLockExecutor.shutdown();
|
retryLeaderLockExecutor.shutdown();
|
||||||
deadLockDetectionExecutor.shutdown();
|
staleLeadershipPurgeExecutor.shutdown();
|
||||||
leadershipStatusBroadcaster.shutdown();
|
leadershipStatusBroadcaster.shutdown();
|
||||||
|
|
||||||
log.info("Stopped");
|
log.info("Stopped");
|
||||||
@ -508,12 +514,25 @@ public class DistributedLeadershipManager implements LeadershipService {
|
|||||||
TimeUnit.SECONDS);
|
TimeUnit.SECONDS);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void purgeStaleLocks() {
|
private void scheduleStaleLeadershipPurge(int afterDelaySec) {
|
||||||
|
if (staleLeadershipPurgeScheduled.compareAndSet(false, true)) {
|
||||||
|
staleLeadershipPurgeExecutor.schedule(
|
||||||
|
this::purgeStaleLeadership,
|
||||||
|
afterDelaySec,
|
||||||
|
TimeUnit.SECONDS);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Purges locks held by inactive nodes and evicts inactive nodes from candidacy.
|
||||||
|
*/
|
||||||
|
private void purgeStaleLeadership() {
|
||||||
|
AtomicBoolean rerunPurge = new AtomicBoolean(false);
|
||||||
try {
|
try {
|
||||||
|
staleLeadershipPurgeScheduled.set(false);
|
||||||
leaderMap.entrySet()
|
leaderMap.entrySet()
|
||||||
.stream()
|
.stream()
|
||||||
.filter(e -> clusterService.getState(e.getValue().value()) == INACTIVE)
|
.filter(e -> clusterService.getState(e.getValue().value()) == INACTIVE)
|
||||||
.filter(e -> activeTopics.contains(e.getKey()))
|
|
||||||
.forEach(entry -> {
|
.forEach(entry -> {
|
||||||
String path = entry.getKey();
|
String path = entry.getKey();
|
||||||
NodeId nodeId = entry.getValue().value();
|
NodeId nodeId = entry.getValue().value();
|
||||||
@ -528,10 +547,52 @@ public class DistributedLeadershipManager implements LeadershipService {
|
|||||||
}
|
}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
log.warn("Failed to purge stale lock held by {} for {}", nodeId, path, e);
|
log.warn("Failed to purge stale lock held by {} for {}", nodeId, path, e);
|
||||||
|
rerunPurge.set(true);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
candidateMap.entrySet()
|
||||||
|
.forEach(entry -> {
|
||||||
|
String path = entry.getKey();
|
||||||
|
Versioned<List<NodeId>> candidates = entry.getValue();
|
||||||
|
List<NodeId> candidatesList = candidates != null
|
||||||
|
? candidates.value() : Collections.emptyList();
|
||||||
|
List<NodeId> activeCandidatesList =
|
||||||
|
candidatesList.stream()
|
||||||
|
.filter(n -> clusterService.getState(n) == ACTIVE)
|
||||||
|
.filter(n -> !localNodeId.equals(n) || activeTopics.contains(path))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
if (activeCandidatesList.size() < candidatesList.size()) {
|
||||||
|
Set<NodeId> removedCandidates =
|
||||||
|
Sets.difference(Sets.newHashSet(candidatesList),
|
||||||
|
Sets.newHashSet(activeCandidatesList));
|
||||||
|
try {
|
||||||
|
if (candidateMap.replace(path, entry.getValue().version(), activeCandidatesList)) {
|
||||||
|
log.info("Evicted inactive candidates {} from "
|
||||||
|
+ "candidate list for {}", removedCandidates, path);
|
||||||
|
Versioned<List<NodeId>> updatedCandidates = candidateMap.get(path);
|
||||||
|
publish(new LeadershipEvent(
|
||||||
|
LeadershipEvent.Type.CANDIDATES_CHANGED,
|
||||||
|
new Leadership(path,
|
||||||
|
updatedCandidates.value(),
|
||||||
|
updatedCandidates.version(),
|
||||||
|
updatedCandidates.creationTime())));
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
log.warn("Failed to evict inactive candidates {} from "
|
||||||
|
+ "candidate list for {}", removedCandidates, path, e);
|
||||||
|
rerunPurge.set(true);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
log.debug("Failed cleaning up stale locks", e);
|
log.warn("Failure purging state leadership.", e);
|
||||||
|
rerunPurge.set(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (rerunPurge.get()) {
|
||||||
|
log.info("Rescheduling stale leadership purge due to errors encountered in previous run");
|
||||||
|
scheduleStaleLeadershipPurge(DELAY_BETWEEN_STALE_LEADERSHIP_PURGE_ATTEMPTS_SEC);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -555,4 +616,14 @@ public class DistributedLeadershipManager implements LeadershipService {
|
|||||||
log.debug("Failed to send leadership updates", e);
|
log.debug("Failed to send leadership updates", e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private class InternalClusterEventListener implements ClusterEventListener {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void event(ClusterEvent event) {
|
||||||
|
if (event.type() == Type.INSTANCE_DEACTIVATED || event.type() == Type.INSTANCE_REMOVED) {
|
||||||
|
scheduleStaleLeadershipPurge(0);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user