|
@@ -144,101 +144,150 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
RMNodeEventType,
|
|
|
RMNodeEvent>(NodeState.NEW)
|
|
|
|
|
|
- //Transitions from NEW state
|
|
|
- .addTransition(NodeState.NEW, NodeState.RUNNING,
|
|
|
- RMNodeEventType.STARTED, new AddNodeTransition())
|
|
|
- .addTransition(NodeState.NEW, NodeState.NEW,
|
|
|
- RMNodeEventType.RESOURCE_UPDATE,
|
|
|
- new UpdateNodeResourceWhenUnusableTransition())
|
|
|
-
|
|
|
- //Transitions from RUNNING state
|
|
|
- .addTransition(NodeState.RUNNING,
|
|
|
- EnumSet.of(NodeState.RUNNING, NodeState.UNHEALTHY),
|
|
|
- RMNodeEventType.STATUS_UPDATE, new StatusUpdateWhenHealthyTransition())
|
|
|
- .addTransition(NodeState.RUNNING, NodeState.DECOMMISSIONED,
|
|
|
- RMNodeEventType.DECOMMISSION,
|
|
|
- new DeactivateNodeTransition(NodeState.DECOMMISSIONED))
|
|
|
- .addTransition(NodeState.RUNNING, NodeState.LOST,
|
|
|
- RMNodeEventType.EXPIRE,
|
|
|
- new DeactivateNodeTransition(NodeState.LOST))
|
|
|
- .addTransition(NodeState.RUNNING, NodeState.REBOOTED,
|
|
|
- RMNodeEventType.REBOOTING,
|
|
|
- new DeactivateNodeTransition(NodeState.REBOOTED))
|
|
|
- .addTransition(NodeState.RUNNING, NodeState.RUNNING,
|
|
|
- RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
|
|
|
- .addTransition(NodeState.RUNNING, NodeState.RUNNING,
|
|
|
- RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
|
|
|
- .addTransition(NodeState.RUNNING, NodeState.RUNNING,
|
|
|
- RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
|
|
|
- new AddContainersToBeRemovedFromNMTransition())
|
|
|
- .addTransition(NodeState.RUNNING, NodeState.RUNNING,
|
|
|
- RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
|
|
|
- .addTransition(NodeState.RUNNING, NodeState.RUNNING,
|
|
|
- RMNodeEventType.RESOURCE_UPDATE, new UpdateNodeResourceWhenRunningTransition())
|
|
|
- .addTransition(NodeState.RUNNING, NodeState.SHUTDOWN,
|
|
|
- RMNodeEventType.SHUTDOWN,
|
|
|
- new DeactivateNodeTransition(NodeState.SHUTDOWN))
|
|
|
-
|
|
|
- //Transitions from REBOOTED state
|
|
|
- .addTransition(NodeState.REBOOTED, NodeState.REBOOTED,
|
|
|
- RMNodeEventType.RESOURCE_UPDATE,
|
|
|
- new UpdateNodeResourceWhenUnusableTransition())
|
|
|
-
|
|
|
- //Transitions from DECOMMISSIONED state
|
|
|
- .addTransition(NodeState.DECOMMISSIONED, NodeState.DECOMMISSIONED,
|
|
|
- RMNodeEventType.RESOURCE_UPDATE,
|
|
|
- new UpdateNodeResourceWhenUnusableTransition())
|
|
|
- .addTransition(NodeState.DECOMMISSIONED, NodeState.DECOMMISSIONED,
|
|
|
- RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
|
|
|
- new AddContainersToBeRemovedFromNMTransition())
|
|
|
-
|
|
|
- //Transitions from LOST state
|
|
|
- .addTransition(NodeState.LOST, NodeState.LOST,
|
|
|
- RMNodeEventType.RESOURCE_UPDATE,
|
|
|
- new UpdateNodeResourceWhenUnusableTransition())
|
|
|
- .addTransition(NodeState.LOST, NodeState.LOST,
|
|
|
- RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
|
|
|
- new AddContainersToBeRemovedFromNMTransition())
|
|
|
-
|
|
|
- //Transitions from UNHEALTHY state
|
|
|
- .addTransition(NodeState.UNHEALTHY,
|
|
|
- EnumSet.of(NodeState.UNHEALTHY, NodeState.RUNNING),
|
|
|
- RMNodeEventType.STATUS_UPDATE,
|
|
|
- new StatusUpdateWhenUnHealthyTransition())
|
|
|
- .addTransition(NodeState.UNHEALTHY, NodeState.DECOMMISSIONED,
|
|
|
- RMNodeEventType.DECOMMISSION,
|
|
|
- new DeactivateNodeTransition(NodeState.DECOMMISSIONED))
|
|
|
- .addTransition(NodeState.UNHEALTHY, NodeState.LOST,
|
|
|
- RMNodeEventType.EXPIRE,
|
|
|
- new DeactivateNodeTransition(NodeState.LOST))
|
|
|
- .addTransition(NodeState.UNHEALTHY, NodeState.REBOOTED,
|
|
|
- RMNodeEventType.REBOOTING,
|
|
|
- new DeactivateNodeTransition(NodeState.REBOOTED))
|
|
|
- .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
|
|
|
- RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
|
|
|
- .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
|
|
|
- RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
|
|
|
- .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
|
|
|
- RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
|
|
|
- .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
|
|
|
- RMNodeEventType.RESOURCE_UPDATE, new UpdateNodeResourceWhenUnusableTransition())
|
|
|
- .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
|
|
|
- RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
|
|
|
- new AddContainersToBeRemovedFromNMTransition())
|
|
|
- .addTransition(NodeState.UNHEALTHY, NodeState.SHUTDOWN,
|
|
|
- RMNodeEventType.SHUTDOWN,
|
|
|
- new DeactivateNodeTransition(NodeState.SHUTDOWN))
|
|
|
-
|
|
|
- //Transitions from SHUTDOWN state
|
|
|
- .addTransition(NodeState.SHUTDOWN, NodeState.SHUTDOWN,
|
|
|
- RMNodeEventType.RESOURCE_UPDATE,
|
|
|
- new UpdateNodeResourceWhenUnusableTransition())
|
|
|
- .addTransition(NodeState.SHUTDOWN, NodeState.SHUTDOWN,
|
|
|
- RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
|
|
|
- new AddContainersToBeRemovedFromNMTransition())
|
|
|
-
|
|
|
- // create the topology tables
|
|
|
- .installTopology();
|
|
|
+ //Transitions from NEW state
|
|
|
+ .addTransition(NodeState.NEW, NodeState.RUNNING,
|
|
|
+ RMNodeEventType.STARTED, new AddNodeTransition())
|
|
|
+ .addTransition(NodeState.NEW, NodeState.NEW,
|
|
|
+ RMNodeEventType.RESOURCE_UPDATE,
|
|
|
+ new UpdateNodeResourceWhenUnusableTransition())
|
|
|
+
|
|
|
+ //Transitions from RUNNING state
|
|
|
+ .addTransition(NodeState.RUNNING,
|
|
|
+ EnumSet.of(NodeState.RUNNING, NodeState.UNHEALTHY),
|
|
|
+ RMNodeEventType.STATUS_UPDATE,
|
|
|
+ new StatusUpdateWhenHealthyTransition())
|
|
|
+ .addTransition(NodeState.RUNNING, NodeState.DECOMMISSIONED,
|
|
|
+ RMNodeEventType.DECOMMISSION,
|
|
|
+ new DeactivateNodeTransition(NodeState.DECOMMISSIONED))
|
|
|
+ .addTransition(NodeState.RUNNING, NodeState.DECOMMISSIONING,
|
|
|
+ RMNodeEventType.GRACEFUL_DECOMMISSION,
|
|
|
+ new DecommissioningNodeTransition(NodeState.RUNNING,
|
|
|
+ NodeState.DECOMMISSIONING))
|
|
|
+ .addTransition(NodeState.RUNNING, NodeState.LOST,
|
|
|
+ RMNodeEventType.EXPIRE,
|
|
|
+ new DeactivateNodeTransition(NodeState.LOST))
|
|
|
+ .addTransition(NodeState.RUNNING, NodeState.REBOOTED,
|
|
|
+ RMNodeEventType.REBOOTING,
|
|
|
+ new DeactivateNodeTransition(NodeState.REBOOTED))
|
|
|
+ .addTransition(NodeState.RUNNING, NodeState.RUNNING,
|
|
|
+ RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
|
|
|
+ .addTransition(NodeState.RUNNING, NodeState.RUNNING,
|
|
|
+ RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
|
|
|
+ .addTransition(NodeState.RUNNING, NodeState.RUNNING,
|
|
|
+ RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
|
|
|
+ new AddContainersToBeRemovedFromNMTransition())
|
|
|
+ .addTransition(NodeState.RUNNING, EnumSet.of(NodeState.RUNNING),
|
|
|
+ RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
|
|
|
+ .addTransition(NodeState.RUNNING, NodeState.RUNNING,
|
|
|
+ RMNodeEventType.RESOURCE_UPDATE, new UpdateNodeResourceWhenRunningTransition())
|
|
|
+ .addTransition(NodeState.RUNNING, NodeState.SHUTDOWN,
|
|
|
+ RMNodeEventType.SHUTDOWN,
|
|
|
+ new DeactivateNodeTransition(NodeState.SHUTDOWN))
|
|
|
+
|
|
|
+ //Transitions from REBOOTED state
|
|
|
+ .addTransition(NodeState.REBOOTED, NodeState.REBOOTED,
|
|
|
+ RMNodeEventType.RESOURCE_UPDATE,
|
|
|
+ new UpdateNodeResourceWhenUnusableTransition())
|
|
|
+
|
|
|
+ //Transitions from DECOMMISSIONED state
|
|
|
+ .addTransition(NodeState.DECOMMISSIONED, NodeState.DECOMMISSIONED,
|
|
|
+ RMNodeEventType.RESOURCE_UPDATE,
|
|
|
+ new UpdateNodeResourceWhenUnusableTransition())
|
|
|
+ .addTransition(NodeState.DECOMMISSIONED, NodeState.DECOMMISSIONED,
|
|
|
+ RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
|
|
|
+ new AddContainersToBeRemovedFromNMTransition())
|
|
|
+
|
|
|
+ //Transitions from DECOMMISSIONING state
|
|
|
+ .addTransition(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONED,
|
|
|
+ RMNodeEventType.DECOMMISSION,
|
|
|
+ new DeactivateNodeTransition(NodeState.DECOMMISSIONED))
|
|
|
+ .addTransition(NodeState.DECOMMISSIONING, NodeState.RUNNING,
|
|
|
+ RMNodeEventType.RECOMMISSION,
|
|
|
+ new RecommissionNodeTransition(NodeState.RUNNING))
|
|
|
+ .addTransition(NodeState.DECOMMISSIONING,
|
|
|
+ EnumSet.of(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONED),
|
|
|
+ RMNodeEventType.STATUS_UPDATE,
|
|
|
+ new StatusUpdateWhenHealthyTransition())
|
|
|
+ .addTransition(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONING,
|
|
|
+ RMNodeEventType.GRACEFUL_DECOMMISSION,
|
|
|
+ new DecommissioningNodeTransition(NodeState.DECOMMISSIONING,
|
|
|
+ NodeState.DECOMMISSIONING))
|
|
|
+ .addTransition(NodeState.DECOMMISSIONING, NodeState.LOST,
|
|
|
+ RMNodeEventType.EXPIRE,
|
|
|
+ new DeactivateNodeTransition(NodeState.LOST))
|
|
|
+ .addTransition(NodeState.DECOMMISSIONING, NodeState.REBOOTED,
|
|
|
+ RMNodeEventType.REBOOTING,
|
|
|
+ new DeactivateNodeTransition(NodeState.REBOOTED))
|
|
|
+
|
|
|
+ .addTransition(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONING,
|
|
|
+ RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
|
|
|
+
|
|
|
+ // TODO (in YARN-3223) update resource when container finished.
|
|
|
+ .addTransition(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONING,
|
|
|
+ RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
|
|
|
+ // TODO (in YARN-3223) update resource when container finished.
|
|
|
+ .addTransition(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONING,
|
|
|
+ RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
|
|
|
+ new AddContainersToBeRemovedFromNMTransition())
|
|
|
+ .addTransition(NodeState.DECOMMISSIONING, EnumSet.of(
|
|
|
+ NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONED),
|
|
|
+ RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
|
|
|
+ .addTransition(NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONING,
|
|
|
+ RMNodeEventType.RESOURCE_UPDATE,
|
|
|
+ new UpdateNodeResourceWhenRunningTransition())
|
|
|
+
|
|
|
+ //Transitions from LOST state
|
|
|
+ .addTransition(NodeState.LOST, NodeState.LOST,
|
|
|
+ RMNodeEventType.RESOURCE_UPDATE,
|
|
|
+ new UpdateNodeResourceWhenUnusableTransition())
|
|
|
+ .addTransition(NodeState.LOST, NodeState.LOST,
|
|
|
+ RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
|
|
|
+ new AddContainersToBeRemovedFromNMTransition())
|
|
|
+
|
|
|
+ //Transitions from UNHEALTHY state
|
|
|
+ .addTransition(NodeState.UNHEALTHY,
|
|
|
+ EnumSet.of(NodeState.UNHEALTHY, NodeState.RUNNING),
|
|
|
+ RMNodeEventType.STATUS_UPDATE,
|
|
|
+ new StatusUpdateWhenUnHealthyTransition())
|
|
|
+ .addTransition(NodeState.UNHEALTHY, NodeState.DECOMMISSIONED,
|
|
|
+ RMNodeEventType.DECOMMISSION,
|
|
|
+ new DeactivateNodeTransition(NodeState.DECOMMISSIONED))
|
|
|
+ .addTransition(NodeState.UNHEALTHY, NodeState.DECOMMISSIONING,
|
|
|
+ RMNodeEventType.GRACEFUL_DECOMMISSION,
|
|
|
+ new DecommissioningNodeTransition(NodeState.UNHEALTHY,
|
|
|
+ NodeState.DECOMMISSIONING))
|
|
|
+ .addTransition(NodeState.UNHEALTHY, NodeState.LOST,
|
|
|
+ RMNodeEventType.EXPIRE,
|
|
|
+ new DeactivateNodeTransition(NodeState.LOST))
|
|
|
+ .addTransition(NodeState.UNHEALTHY, NodeState.REBOOTED,
|
|
|
+ RMNodeEventType.REBOOTING,
|
|
|
+ new DeactivateNodeTransition(NodeState.REBOOTED))
|
|
|
+ .addTransition(NodeState.UNHEALTHY, EnumSet.of(NodeState.UNHEALTHY),
|
|
|
+ RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
|
|
|
+ .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
|
|
|
+ RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
|
|
|
+ .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
|
|
|
+ RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
|
|
|
+ .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
|
|
|
+ RMNodeEventType.RESOURCE_UPDATE,
|
|
|
+ new UpdateNodeResourceWhenUnusableTransition())
|
|
|
+ .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
|
|
|
+ RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
|
|
|
+ new AddContainersToBeRemovedFromNMTransition())
|
|
|
+ .addTransition(NodeState.UNHEALTHY, NodeState.SHUTDOWN,
|
|
|
+ RMNodeEventType.SHUTDOWN,
|
|
|
+ new DeactivateNodeTransition(NodeState.SHUTDOWN))
|
|
|
+
|
|
|
+ //Transitions from SHUTDOWN state
|
|
|
+ .addTransition(NodeState.SHUTDOWN, NodeState.SHUTDOWN,
|
|
|
+ RMNodeEventType.RESOURCE_UPDATE,
|
|
|
+ new UpdateNodeResourceWhenUnusableTransition())
|
|
|
+ .addTransition(NodeState.SHUTDOWN, NodeState.SHUTDOWN,
|
|
|
+ RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
|
|
|
+ new AddContainersToBeRemovedFromNMTransition())
|
|
|
+
|
|
|
+ // create the topology tables
|
|
|
+ .installTopology();
|
|
|
|
|
|
private final StateMachine<NodeState, RMNodeEventType,
|
|
|
RMNodeEvent> stateMachine;
|
|
@@ -265,7 +314,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
this.writeLock = lock.writeLock();
|
|
|
|
|
|
this.stateMachine = stateMachineFactory.make(this);
|
|
|
-
|
|
|
+
|
|
|
this.nodeUpdateQueue = new ConcurrentLinkedQueue<UpdatedContainerInfo>();
|
|
|
|
|
|
this.containerAllocationExpirer = context.getContainerAllocationExpirer();
|
|
@@ -291,6 +340,11 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
return httpPort;
|
|
|
}
|
|
|
|
|
|
+ // Test only
|
|
|
+ public void setHttpPort(int port) {
|
|
|
+ this.httpPort = port;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public NodeId getNodeID() {
|
|
|
return this.nodeId;
|
|
@@ -497,23 +551,35 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
metrics.decrNumShutdownNMs();
|
|
|
break;
|
|
|
default:
|
|
|
- LOG.debug("Unexpected previous node state");
|
|
|
+ LOG.debug("Unexpected previous node state");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ // Treats nodes in decommissioning as active nodes
|
|
|
+ // TODO we may want to differentiate active nodes and decommissioning node in
|
|
|
+ // metrics later.
|
|
|
+ private void updateMetricsForGracefulDecommissionOnUnhealthyNode() {
|
|
|
+ ClusterMetrics metrics = ClusterMetrics.getMetrics();
|
|
|
+ metrics.incrNumActiveNodes();
|
|
|
+ metrics.decrNumUnhealthyNMs();
|
|
|
+ }
|
|
|
+
|
|
|
private void updateMetricsForDeactivatedNode(NodeState initialState,
|
|
|
NodeState finalState) {
|
|
|
ClusterMetrics metrics = ClusterMetrics.getMetrics();
|
|
|
|
|
|
switch (initialState) {
|
|
|
- case RUNNING:
|
|
|
- metrics.decrNumActiveNodes();
|
|
|
- break;
|
|
|
- case UNHEALTHY:
|
|
|
- metrics.decrNumUnhealthyNMs();
|
|
|
- break;
|
|
|
- default:
|
|
|
- LOG.debug("Unexpected inital state");
|
|
|
+ case RUNNING:
|
|
|
+ metrics.decrNumActiveNodes();
|
|
|
+ break;
|
|
|
+ case DECOMMISSIONING:
|
|
|
+ metrics.decrNumActiveNodes();
|
|
|
+ break;
|
|
|
+ case UNHEALTHY:
|
|
|
+ metrics.decrNumUnhealthyNMs();
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ LOG.debug("Unexpected inital state");
|
|
|
}
|
|
|
|
|
|
switch (finalState) {
|
|
@@ -608,10 +674,10 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
}
|
|
|
|
|
|
public static class ReconnectNodeTransition implements
|
|
|
- SingleArcTransition<RMNodeImpl, RMNodeEvent> {
|
|
|
+ MultipleArcTransition<RMNodeImpl, RMNodeEvent, NodeState> {
|
|
|
|
|
|
@Override
|
|
|
- public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
|
+ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
|
RMNodeReconnectEvent reconnectEvent = (RMNodeReconnectEvent) event;
|
|
|
RMNode newNode = reconnectEvent.getReconnectedNode();
|
|
|
rmNode.nodeManagerVersion = newNode.getNodeManagerVersion();
|
|
@@ -622,6 +688,12 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
// No application running on the node, so send node-removal event with
|
|
|
// cleaning up old container info.
|
|
|
if (noRunningApps) {
|
|
|
+ if (rmNode.getState() == NodeState.DECOMMISSIONING) {
|
|
|
+ // When node in decommissioning, and no running apps on this node,
|
|
|
+ // it will return as decommissioned state.
|
|
|
+ deactivateNode(rmNode, NodeState.DECOMMISSIONED);
|
|
|
+ return NodeState.DECOMMISSIONED;
|
|
|
+ }
|
|
|
rmNode.nodeUpdateQueue.clear();
|
|
|
rmNode.context.getDispatcher().getEventHandler().handle(
|
|
|
new NodeRemovedSchedulerEvent(rmNode));
|
|
@@ -652,6 +724,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
rmNode.context.getDispatcher().getEventHandler().handle(
|
|
|
new RMNodeStartedEvent(newNode.getNodeID(), null, null));
|
|
|
}
|
|
|
+
|
|
|
} else {
|
|
|
rmNode.httpPort = newNode.getHttpPort();
|
|
|
rmNode.httpAddress = newNode.getHttpAddress();
|
|
@@ -678,17 +751,21 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
new NodeResourceUpdateSchedulerEvent(rmNode, ResourceOption
|
|
|
.newInstance(newNode.getTotalCapability(), -1)));
|
|
|
}
|
|
|
+
|
|
|
}
|
|
|
+ return rmNode.getState();
|
|
|
}
|
|
|
|
|
|
private void handleNMContainerStatus(
|
|
|
List<NMContainerStatus> nmContainerStatuses, RMNodeImpl rmnode) {
|
|
|
- List<ContainerStatus> containerStatuses =
|
|
|
- new ArrayList<ContainerStatus>();
|
|
|
- for (NMContainerStatus nmContainerStatus : nmContainerStatuses) {
|
|
|
- containerStatuses.add(createContainerStatus(nmContainerStatus));
|
|
|
+ if (nmContainerStatuses != null) {
|
|
|
+ List<ContainerStatus> containerStatuses =
|
|
|
+ new ArrayList<ContainerStatus>();
|
|
|
+ for (NMContainerStatus nmContainerStatus : nmContainerStatuses) {
|
|
|
+ containerStatuses.add(createContainerStatus(nmContainerStatus));
|
|
|
+ }
|
|
|
+ rmnode.handleContainerStatus(containerStatuses);
|
|
|
}
|
|
|
- rmnode.handleContainerStatus(containerStatuses);
|
|
|
}
|
|
|
|
|
|
private ContainerStatus createContainerStatus(
|
|
@@ -770,31 +847,94 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
|
|
|
@Override
|
|
|
public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
|
- // Inform the scheduler
|
|
|
- rmNode.nodeUpdateQueue.clear();
|
|
|
- // If the current state is NodeState.UNHEALTHY
|
|
|
- // Then node is already been removed from the
|
|
|
- // Scheduler
|
|
|
- NodeState initialState = rmNode.getState();
|
|
|
- if (!initialState.equals(NodeState.UNHEALTHY)) {
|
|
|
- rmNode.context.getDispatcher().getEventHandler()
|
|
|
- .handle(new NodeRemovedSchedulerEvent(rmNode));
|
|
|
+ RMNodeImpl.deactivateNode(rmNode, finalState);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Put a node in deactivated (decommissioned) status.
|
|
|
+ * @param rmNode
|
|
|
+ * @param finalState
|
|
|
+ */
|
|
|
+ public static void deactivateNode(RMNodeImpl rmNode, NodeState finalState) {
|
|
|
+
|
|
|
+ reportNodeUnusable(rmNode, finalState);
|
|
|
+
|
|
|
+ // Deactivate the node
|
|
|
+ rmNode.context.getRMNodes().remove(rmNode.nodeId);
|
|
|
+ LOG.info("Deactivating Node " + rmNode.nodeId + " as it is now "
|
|
|
+ + finalState);
|
|
|
+ rmNode.context.getInactiveRMNodes().put(rmNode.nodeId, rmNode);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Report node is UNUSABLE and update metrics.
|
|
|
+ * @param rmNode
|
|
|
+ * @param finalState
|
|
|
+ */
|
|
|
+ public static void reportNodeUnusable(RMNodeImpl rmNode,
|
|
|
+ NodeState finalState) {
|
|
|
+ // Inform the scheduler
|
|
|
+ rmNode.nodeUpdateQueue.clear();
|
|
|
+ // If the current state is NodeState.UNHEALTHY
|
|
|
+ // Then node is already been removed from the
|
|
|
+ // Scheduler
|
|
|
+ NodeState initialState = rmNode.getState();
|
|
|
+ if (!initialState.equals(NodeState.UNHEALTHY)) {
|
|
|
+ rmNode.context.getDispatcher().getEventHandler()
|
|
|
+ .handle(new NodeRemovedSchedulerEvent(rmNode));
|
|
|
+ }
|
|
|
+ rmNode.context.getDispatcher().getEventHandler().handle(
|
|
|
+ new NodesListManagerEvent(
|
|
|
+ NodesListManagerEventType.NODE_UNUSABLE, rmNode));
|
|
|
+
|
|
|
+ //Update the metrics
|
|
|
+ rmNode.updateMetricsForDeactivatedNode(initialState, finalState);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The transition to put node in decommissioning state.
|
|
|
+ */
|
|
|
+ public static class DecommissioningNodeTransition
|
|
|
+ implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
|
|
|
+ private final NodeState initState;
|
|
|
+ private final NodeState finalState;
|
|
|
+
|
|
|
+ public DecommissioningNodeTransition(NodeState initState,
|
|
|
+ NodeState finalState) {
|
|
|
+ this.initState = initState;
|
|
|
+ this.finalState = finalState;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
|
+ LOG.info("Put Node " + rmNode.nodeId + " in DECOMMISSIONING.");
|
|
|
+ if (initState.equals(NodeState.UNHEALTHY)) {
|
|
|
+ rmNode.updateMetricsForGracefulDecommissionOnUnhealthyNode();
|
|
|
}
|
|
|
- rmNode.context.getDispatcher().getEventHandler().handle(
|
|
|
- new NodesListManagerEvent(
|
|
|
- NodesListManagerEventType.NODE_UNUSABLE, rmNode));
|
|
|
+ // TODO (in YARN-3223) Keep NM's available resource to be 0
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public static class RecommissionNodeTransition
|
|
|
+ implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
|
|
|
|
|
|
- // Deactivate the node
|
|
|
- rmNode.context.getRMNodes().remove(rmNode.nodeId);
|
|
|
- LOG.info("Deactivating Node " + rmNode.nodeId + " as it is now "
|
|
|
- + finalState);
|
|
|
- rmNode.context.getInactiveRMNodes().put(rmNode.nodeId, rmNode);
|
|
|
+ private final NodeState finalState;
|
|
|
+ public RecommissionNodeTransition(NodeState finalState) {
|
|
|
+ this.finalState = finalState;
|
|
|
+ }
|
|
|
|
|
|
- //Update the metrics
|
|
|
- rmNode.updateMetricsForDeactivatedNode(initialState, finalState);
|
|
|
+ @Override
|
|
|
+ public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
|
+ LOG.info("Node " + rmNode.nodeId + " in DECOMMISSIONING is " +
|
|
|
+ "recommissioned back to RUNNING.");
|
|
|
+ // TODO handle NM resource resume in YARN-3223.
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Status update transition when node is healthy.
|
|
|
+ */
|
|
|
public static class StatusUpdateWhenHealthyTransition implements
|
|
|
MultipleArcTransition<RMNodeImpl, RMNodeEvent, NodeState> {
|
|
|
@Override
|
|
@@ -805,25 +945,44 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
// Switch the last heartbeatresponse.
|
|
|
rmNode.latestNodeHeartBeatResponse = statusEvent.getLatestResponse();
|
|
|
|
|
|
- NodeHealthStatus remoteNodeHealthStatus =
|
|
|
+ NodeHealthStatus remoteNodeHealthStatus =
|
|
|
statusEvent.getNodeHealthStatus();
|
|
|
rmNode.setHealthReport(remoteNodeHealthStatus.getHealthReport());
|
|
|
rmNode.setLastHealthReportTime(
|
|
|
remoteNodeHealthStatus.getLastHealthReportTime());
|
|
|
+ NodeState initialState = rmNode.getState();
|
|
|
+ boolean isNodeDecommissioning =
|
|
|
+ initialState.equals(NodeState.DECOMMISSIONING);
|
|
|
if (!remoteNodeHealthStatus.getIsNodeHealthy()) {
|
|
|
- LOG.info("Node " + rmNode.nodeId + " reported UNHEALTHY with details: "
|
|
|
- + remoteNodeHealthStatus.getHealthReport());
|
|
|
- rmNode.nodeUpdateQueue.clear();
|
|
|
- // Inform the scheduler
|
|
|
- rmNode.context.getDispatcher().getEventHandler().handle(
|
|
|
- new NodeRemovedSchedulerEvent(rmNode));
|
|
|
- rmNode.context.getDispatcher().getEventHandler().handle(
|
|
|
- new NodesListManagerEvent(
|
|
|
- NodesListManagerEventType.NODE_UNUSABLE, rmNode));
|
|
|
- // Update metrics
|
|
|
- rmNode.updateMetricsForDeactivatedNode(rmNode.getState(),
|
|
|
- NodeState.UNHEALTHY);
|
|
|
- return NodeState.UNHEALTHY;
|
|
|
+ LOG.info("Node " + rmNode.nodeId +
|
|
|
+ " reported UNHEALTHY with details: " +
|
|
|
+ remoteNodeHealthStatus.getHealthReport());
|
|
|
+ // if a node in decommissioning receives an unhealthy report,
|
|
|
+ // it will keep decommissioning.
|
|
|
+ if (isNodeDecommissioning) {
|
|
|
+ return NodeState.DECOMMISSIONING;
|
|
|
+ } else {
|
|
|
+ reportNodeUnusable(rmNode, NodeState.UNHEALTHY);
|
|
|
+ return NodeState.UNHEALTHY;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (isNodeDecommissioning) {
|
|
|
+ List<ApplicationId> runningApps = rmNode.getRunningApps();
|
|
|
+
|
|
|
+ List<ApplicationId> keepAliveApps = statusEvent.getKeepAliveAppIds();
|
|
|
+
|
|
|
+ // no running (and keeping alive) app on this node, get it
|
|
|
+ // decommissioned.
|
|
|
+ // TODO may need to check no container is being scheduled on this node
|
|
|
+ // as well.
|
|
|
+ if ((runningApps == null || runningApps.size() == 0)
|
|
|
+ && (keepAliveApps == null || keepAliveApps.size() == 0)) {
|
|
|
+ RMNodeImpl.deactivateNode(rmNode, NodeState.DECOMMISSIONED);
|
|
|
+ return NodeState.DECOMMISSIONED;
|
|
|
+ }
|
|
|
+
|
|
|
+ // TODO (in YARN-3223) if node in decommissioning, get node resource
|
|
|
+ // updated if container get finished (keep available resource to be 0)
|
|
|
}
|
|
|
|
|
|
rmNode.handleContainerStatus(statusEvent.getContainers());
|
|
@@ -848,7 +1007,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
statusEvent.getKeepAliveAppIds());
|
|
|
}
|
|
|
|
|
|
- return NodeState.RUNNING;
|
|
|
+ return initialState;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -857,11 +1016,12 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
|
|
|
@Override
|
|
|
public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
|
- RMNodeStatusEvent statusEvent = (RMNodeStatusEvent) event;
|
|
|
+ RMNodeStatusEvent statusEvent = (RMNodeStatusEvent)event;
|
|
|
|
|
|
// Switch the last heartbeatresponse.
|
|
|
rmNode.latestNodeHeartBeatResponse = statusEvent.getLatestResponse();
|
|
|
- NodeHealthStatus remoteNodeHealthStatus = statusEvent.getNodeHealthStatus();
|
|
|
+ NodeHealthStatus remoteNodeHealthStatus =
|
|
|
+ statusEvent.getNodeHealthStatus();
|
|
|
rmNode.setHealthReport(remoteNodeHealthStatus.getHealthReport());
|
|
|
rmNode.setLastHealthReportTime(
|
|
|
remoteNodeHealthStatus.getLastHealthReportTime());
|