-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Coordinator-driven graceful decommission/recommission of worker nodes #14876
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
87b8e6f
0312463
7eab69a
1e8887e
e65585d
c225b8f
be62233
dfe33c7
c056bc7
a46a510
0459735
1788829
3c1c1fa
58bd159
35da905
ce7b57f
a660133
cd8eac6
b9d26a7
f5b5fb8
ec8a8fd
7026432
c3ee15f
94e8eb5
bcef885
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,7 +18,6 @@ | |
| import com.google.common.collect.ImmutableSet; | ||
| import com.google.common.collect.ImmutableSetMultimap; | ||
| import com.google.common.collect.SetMultimap; | ||
| import com.google.common.collect.Sets; | ||
| import com.google.common.collect.Sets.SetView; | ||
| import io.airlift.discovery.client.ServiceDescriptor; | ||
| import io.airlift.discovery.client.ServiceSelector; | ||
|
|
@@ -43,6 +42,7 @@ | |
| import java.net.URI; | ||
| import java.net.URISyntaxException; | ||
| import java.util.ArrayList; | ||
| import java.util.HashSet; | ||
| import java.util.List; | ||
| import java.util.Optional; | ||
| import java.util.Set; | ||
|
|
@@ -59,6 +59,8 @@ | |
| import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; | ||
| import static io.trino.connector.system.GlobalSystemConnector.CATALOG_HANDLE; | ||
| import static io.trino.metadata.NodeState.ACTIVE; | ||
| import static io.trino.metadata.NodeState.DECOMMISSIONED; | ||
| import static io.trino.metadata.NodeState.DECOMMISSIONING; | ||
| import static io.trino.metadata.NodeState.INACTIVE; | ||
| import static io.trino.metadata.NodeState.SHUTTING_DOWN; | ||
| import static java.util.Locale.ENGLISH; | ||
|
|
@@ -93,6 +95,9 @@ public final class DiscoveryNodeManager | |
| @GuardedBy("this") | ||
| private Set<InternalNode> coordinators; | ||
|
|
||
| @GuardedBy("this") | ||
| private Set<String> nodesToBeDecommissioned = new HashSet<>(); | ||
|
|
||
| @GuardedBy("this") | ||
| private final List<Consumer<AllNodes>> listeners = new ArrayList<>(); | ||
|
|
||
|
|
@@ -169,10 +174,7 @@ public void destroy() | |
| private void pollWorkers() | ||
| { | ||
| AllNodes allNodes = getAllNodes(); | ||
| Set<InternalNode> aliveNodes = ImmutableSet.<InternalNode>builder() | ||
| .addAll(allNodes.getActiveNodes()) | ||
| .addAll(allNodes.getShuttingDownNodes()) | ||
| .build(); | ||
| Set<InternalNode> aliveNodes = allNodes.getAliveNodes(); | ||
|
|
||
| ImmutableSet<String> aliveNodeIds = aliveNodes.stream() | ||
| .map(InternalNode::getNodeIdentifier) | ||
|
|
@@ -215,9 +217,7 @@ private synchronized void refreshNodesInternal() | |
| .filter(service -> !failureDetector.getFailed().contains(service)) | ||
| .collect(toImmutableSet()); | ||
|
|
||
| ImmutableSet.Builder<InternalNode> activeNodesBuilder = ImmutableSet.builder(); | ||
| ImmutableSet.Builder<InternalNode> inactiveNodesBuilder = ImmutableSet.builder(); | ||
| ImmutableSet.Builder<InternalNode> shuttingDownNodesBuilder = ImmutableSet.builder(); | ||
| ImmutableSetMultimap.Builder<NodeState, InternalNode> nodeStateMapBuilder = ImmutableSetMultimap.builder(); | ||
| ImmutableSet.Builder<InternalNode> coordinatorsBuilder = ImmutableSet.builder(); | ||
| ImmutableSetMultimap.Builder<CatalogHandle, InternalNode> byCatalogHandleBuilder = ImmutableSetMultimap.builder(); | ||
|
|
||
|
|
@@ -229,55 +229,65 @@ private synchronized void refreshNodesInternal() | |
| InternalNode node = new InternalNode(service.getNodeId(), uri, nodeVersion, coordinator); | ||
| NodeState nodeState = getNodeState(node); | ||
|
|
||
| switch (nodeState) { | ||
| case ACTIVE: | ||
| activeNodesBuilder.add(node); | ||
| if (coordinator) { | ||
| coordinatorsBuilder.add(node); | ||
| } | ||
| // nodesToBeDecommissioned is the authoritative list of node to be decommissioned | ||
| // from coordinator perspective. Once a worker appears in the list, | ||
| // its state become DECOMMISSIONING even if worker has yet confirmed such | ||
| // so that no new tasks will be scheduled on it. | ||
| if (!coordinator && nodesToBeDecommissioned.contains(node.getNodeIdentifier()) | ||
| && nodeState == ACTIVE) { | ||
| log.debug("Treat " + node.getNodeIdentifier() + " as DECOMMISSIONING"); | ||
| nodeState = DECOMMISSIONING; | ||
| } | ||
|
|
||
| // record available active nodes organized by catalog handle | ||
| String catalogHandleIds = service.getProperties().get("catalogHandleIds"); | ||
| if (catalogHandleIds != null) { | ||
| catalogHandleIds = catalogHandleIds.toLowerCase(ENGLISH); | ||
| for (String catalogHandleId : CATALOG_HANDLE_ID_SPLITTER.split(catalogHandleIds)) { | ||
| byCatalogHandleBuilder.put(CatalogHandle.fromId(catalogHandleId), node); | ||
| } | ||
| // Add node to node state map. | ||
| nodeStateMapBuilder.put(nodeState, node); | ||
|
|
||
| if (nodeState == ACTIVE) { | ||
| if (coordinator) { | ||
| coordinatorsBuilder.add(node); | ||
| } | ||
|
|
||
| // record available active nodes organized by catalog handle | ||
| String catalogHandleIds = service.getProperties().get("catalogHandleIds"); | ||
| if (catalogHandleIds != null) { | ||
| catalogHandleIds = catalogHandleIds.toLowerCase(ENGLISH); | ||
| for (String catalogHandleId : CATALOG_HANDLE_ID_SPLITTER.split(catalogHandleIds)) { | ||
| byCatalogHandleBuilder.put(CatalogHandle.fromId(catalogHandleId), node); | ||
| } | ||
| } | ||
|
|
||
| // always add system connector | ||
| byCatalogHandleBuilder.put(CATALOG_HANDLE, node); | ||
| break; | ||
| case INACTIVE: | ||
| inactiveNodesBuilder.add(node); | ||
| break; | ||
| case SHUTTING_DOWN: | ||
| shuttingDownNodesBuilder.add(node); | ||
| break; | ||
| default: | ||
| log.error("Unknown state %s for node %s", nodeState, node); | ||
| // always add system connector | ||
| byCatalogHandleBuilder.put(CATALOG_HANDLE, node); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| if (allNodes != null) { | ||
| // nodes by catalog handle changes anytime a node adds or removes a catalog (note: this is not part of the listener system) | ||
| if (!allCatalogsOnAllNodes) { | ||
| activeNodesByCatalogHandle = Optional.of(byCatalogHandleBuilder.build()); | ||
| } | ||
|
|
||
| SetMultimap<NodeState, InternalNode> nodeStateMap = nodeStateMapBuilder.build(); | ||
| AllNodes currAllNodes = new AllNodes( | ||
| nodeStateMap.get(ACTIVE), | ||
| nodeStateMap.get(DECOMMISSIONED), | ||
| nodeStateMap.get(DECOMMISSIONING), | ||
| nodeStateMap.get(INACTIVE), | ||
| nodeStateMap.get(SHUTTING_DOWN), | ||
| coordinatorsBuilder.build()); | ||
|
|
||
| if (this.allNodes != null) { | ||
| // log node that are no longer active (but not shutting down) | ||
| SetView<InternalNode> missingNodes = difference(allNodes.getActiveNodes(), Sets.union(activeNodesBuilder.build(), shuttingDownNodesBuilder.build())); | ||
| SetView<InternalNode> missingNodes = difference(allNodes.getActiveNodes(), currAllNodes.getAliveNodes()); | ||
| for (InternalNode missingNode : missingNodes) { | ||
| log.info("Previously active node is missing: %s (last seen at %s)", missingNode.getNodeIdentifier(), missingNode.getHost()); | ||
| } | ||
| } | ||
|
|
||
| // nodes by catalog handle changes anytime a node adds or removes a catalog (note: this is not part of the listener system) | ||
| if (!allCatalogsOnAllNodes) { | ||
| activeNodesByCatalogHandle = Optional.of(byCatalogHandleBuilder.build()); | ||
| } | ||
|
|
||
| AllNodes allNodes = new AllNodes(activeNodesBuilder.build(), inactiveNodesBuilder.build(), shuttingDownNodesBuilder.build(), coordinatorsBuilder.build()); | ||
| // only update if all nodes actually changed (note: this does not include the connectors registered with the nodes) | ||
| if (!allNodes.equals(this.allNodes)) { | ||
| if (!currAllNodes.equals(this.allNodes)) { | ||
| // assign allNodes to a local variable for use in the callback below | ||
| this.allNodes = allNodes; | ||
| this.allNodes = currAllNodes; | ||
| coordinators = coordinatorsBuilder.build(); | ||
|
|
||
| // notify listeners | ||
|
|
@@ -289,22 +299,19 @@ private synchronized void refreshNodesInternal() | |
| private NodeState getNodeState(InternalNode node) | ||
| { | ||
| if (expectedNodeVersion.equals(node.getNodeVersion())) { | ||
| if (isNodeShuttingDown(node.getNodeIdentifier())) { | ||
| return SHUTTING_DOWN; | ||
| String nodeId = node.getNodeIdentifier(); | ||
| Optional<NodeState> remoteNodeState = nodeStates.containsKey(nodeId) | ||
| ? nodeStates.get(nodeId).getNodeState() | ||
| : Optional.empty(); | ||
| if (remoteNodeState.isPresent()) { | ||
| return remoteNodeState.get(); | ||
| } | ||
| // no remote node state | ||
| return ACTIVE; | ||
| } | ||
| return INACTIVE; | ||
| } | ||
|
|
||
| private boolean isNodeShuttingDown(String nodeId) | ||
| { | ||
| Optional<NodeState> remoteNodeState = nodeStates.containsKey(nodeId) | ||
| ? nodeStates.get(nodeId).getNodeState() | ||
| : Optional.empty(); | ||
| return remoteNodeState.isPresent() && remoteNodeState.get() == SHUTTING_DOWN; | ||
| } | ||
|
|
||
| @Override | ||
| public synchronized AllNodes getAllNodes() | ||
| { | ||
|
|
@@ -317,6 +324,18 @@ public int getActiveNodeCount() | |
| return getAllNodes().getActiveNodes().size(); | ||
| } | ||
|
|
||
| @Managed | ||
| public int getDecommissionedNodeCount() | ||
| { | ||
| return getAllNodes().getDecommissionedNodes().size(); | ||
| } | ||
|
|
||
| @Managed | ||
| public int getDecommissioningNodeCount() | ||
| { | ||
| return getAllNodes().getDecommissioningNodes().size(); | ||
| } | ||
|
|
||
|
Comment on lines
327
to
338
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. how are the docomissioned/decomissioning nodes represented in Web UI? Will they be accounted for in the number of active nodes in the cluster?
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It is currently not reflected in the Web UI. I figured three files to touch to add them to UI.
I tried "babel" to generate 3 from 2, but got error "Support for the experimental syntax 'jsx' isn't currently enabled". So this one is pending on how to generate the file in 3 (assuming it was generated by tool but not edited by hand).
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I have tried to manually update No 3 but cluster-hud UI stuck in loading and I am not sure how to debug it (not familiar with React nor js in general). So 2 & 3 is not included for now, and may need tip on how to do No 3. |
||
| @Managed | ||
| public int getInactiveNodeCount() | ||
| { | ||
|
|
@@ -335,6 +354,10 @@ public Set<InternalNode> getNodes(NodeState state) | |
| switch (state) { | ||
| case ACTIVE: | ||
| return getAllNodes().getActiveNodes(); | ||
| case DECOMMISSIONED: | ||
| return getAllNodes().getDecommissionedNodes(); | ||
| case DECOMMISSIONING: | ||
| return getAllNodes().getDecommissioningNodes(); | ||
| case INACTIVE: | ||
| return getAllNodes().getInactiveNodes(); | ||
| case SHUTTING_DOWN: | ||
|
|
@@ -407,4 +430,9 @@ private static boolean isCoordinator(ServiceDescriptor service) | |
| { | ||
| return Boolean.parseBoolean(service.getProperties().get("coordinator")); | ||
| } | ||
|
|
||
| public synchronized void setNodesToExclude(Set<String> nodesToExclude) | ||
| { | ||
| this.nodesToBeDecommissioned = nodesToExclude; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,6 +16,8 @@ | |
| public enum NodeState | ||
| { | ||
| ACTIVE, | ||
| DECOMMISSIONED, | ||
| DECOMMISSIONING, | ||
| INACTIVE, | ||
| SHUTTING_DOWN | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,31 @@ | ||
| /* | ||
| * Licensed under the Apache License, Version 2.0 (the "License"); | ||
| * you may not use this file except in compliance with the License. | ||
| * You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package io.trino.server; | ||
|
|
||
| import javax.inject.Qualifier; | ||
|
|
||
| import java.lang.annotation.Retention; | ||
| import java.lang.annotation.Target; | ||
|
|
||
| import static java.lang.annotation.ElementType.FIELD; | ||
| import static java.lang.annotation.ElementType.METHOD; | ||
| import static java.lang.annotation.ElementType.PARAMETER; | ||
| import static java.lang.annotation.RetentionPolicy.RUNTIME; | ||
|
|
||
| @Retention(RUNTIME) | ||
| @Target({FIELD, PARAMETER, METHOD}) | ||
| @Qualifier | ||
| public @interface ForNodes | ||
| { | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Honestly I am not a big fan of this dichotomy. I would rather keep what workers report as a source of truth, and report that.
Also the quesion here may become obsolete if we would decide to drop
NodesResource.refreshNodesas discused in https://github.com/trinodb/trino/pull/14876/files#r1051663508