|
@@ -64,6 +64,7 @@ import org.apache.hadoop.yarn.util.BuilderUtils.ContainerIdComparator;
|
|
*/
|
|
*/
|
|
@Private
|
|
@Private
|
|
@Unstable
|
|
@Unstable
|
|
|
|
+@SuppressWarnings("unchecked")
|
|
public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
|
|
|
private static final Log LOG = LogFactory.getLog(RMNodeImpl.class);
|
|
private static final Log LOG = LogFactory.getLog(RMNodeImpl.class);
|
|
@@ -116,11 +117,14 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
EnumSet.of(RMNodeState.RUNNING, RMNodeState.UNHEALTHY),
|
|
EnumSet.of(RMNodeState.RUNNING, RMNodeState.UNHEALTHY),
|
|
RMNodeEventType.STATUS_UPDATE, new StatusUpdateWhenHealthyTransition())
|
|
RMNodeEventType.STATUS_UPDATE, new StatusUpdateWhenHealthyTransition())
|
|
.addTransition(RMNodeState.RUNNING, RMNodeState.DECOMMISSIONED,
|
|
.addTransition(RMNodeState.RUNNING, RMNodeState.DECOMMISSIONED,
|
|
- RMNodeEventType.DECOMMISSION, new RemoveNodeTransition())
|
|
|
|
|
|
+ RMNodeEventType.DECOMMISSION,
|
|
|
|
+ new DeactivateNodeTransition(RMNodeState.DECOMMISSIONED))
|
|
.addTransition(RMNodeState.RUNNING, RMNodeState.LOST,
|
|
.addTransition(RMNodeState.RUNNING, RMNodeState.LOST,
|
|
- RMNodeEventType.EXPIRE, new RemoveNodeTransition())
|
|
|
|
|
|
+ RMNodeEventType.EXPIRE,
|
|
|
|
+ new DeactivateNodeTransition(RMNodeState.LOST))
|
|
.addTransition(RMNodeState.RUNNING, RMNodeState.REBOOTED,
|
|
.addTransition(RMNodeState.RUNNING, RMNodeState.REBOOTED,
|
|
- RMNodeEventType.REBOOTING, new RemoveNodeTransition())
|
|
|
|
|
|
+ RMNodeEventType.REBOOTING,
|
|
|
|
+ new DeactivateNodeTransition(RMNodeState.REBOOTED))
|
|
.addTransition(RMNodeState.RUNNING, RMNodeState.RUNNING,
|
|
.addTransition(RMNodeState.RUNNING, RMNodeState.RUNNING,
|
|
RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
|
|
RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
|
|
.addTransition(RMNodeState.RUNNING, RMNodeState.RUNNING,
|
|
.addTransition(RMNodeState.RUNNING, RMNodeState.RUNNING,
|
|
@@ -304,26 +308,50 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
writeLock.unlock();
|
|
writeLock.unlock();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ private void updateMetricsForRejoinedNode(RMNodeState previousNodeState) {
|
|
|
|
+ ClusterMetrics metrics = ClusterMetrics.getMetrics();
|
|
|
|
+ metrics.incrNumActiveNodes();
|
|
|
|
+
|
|
|
|
+ switch (previousNodeState) {
|
|
|
|
+ case LOST:
|
|
|
|
+ metrics.decrNumLostNMs();
|
|
|
|
+ break;
|
|
|
|
+ case REBOOTED:
|
|
|
|
+ metrics.decrNumRebootedNMs();
|
|
|
|
+ break;
|
|
|
|
+ case DECOMMISSIONED:
|
|
|
|
+ metrics.decrDecommisionedNMs();
|
|
|
|
+ break;
|
|
|
|
+ case UNHEALTHY:
|
|
|
|
+ metrics.decrNumUnhealthyNMs();
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void updateMetricsForDeactivatedNode(RMNodeState finalState) {
|
|
|
|
+ ClusterMetrics metrics = ClusterMetrics.getMetrics();
|
|
|
|
+ metrics.decrNumActiveNodes();
|
|
|
|
+
|
|
|
|
+ switch (finalState) {
|
|
|
|
+ case DECOMMISSIONED:
|
|
|
|
+ metrics.incrDecommisionedNMs();
|
|
|
|
+ break;
|
|
|
|
+ case LOST:
|
|
|
|
+ metrics.incrNumLostNMs();
|
|
|
|
+ break;
|
|
|
|
+ case REBOOTED:
|
|
|
|
+ metrics.incrNumRebootedNMs();
|
|
|
|
+ break;
|
|
|
|
+ case UNHEALTHY:
|
|
|
|
+ metrics.incrNumUnhealthyNMs();
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
public static class AddNodeTransition implements
|
|
public static class AddNodeTransition implements
|
|
SingleArcTransition<RMNodeImpl, RMNodeEvent> {
|
|
SingleArcTransition<RMNodeImpl, RMNodeEvent> {
|
|
-
|
|
|
|
- private void updateMetrics(RMNodeState nodeState) {
|
|
|
|
- ClusterMetrics metrics = ClusterMetrics.getMetrics();
|
|
|
|
- switch (nodeState) {
|
|
|
|
- case LOST:
|
|
|
|
- metrics.decrNumLostNMs();
|
|
|
|
- break;
|
|
|
|
- case REBOOTED:
|
|
|
|
- metrics.decrNumRebootedNMs();
|
|
|
|
- break;
|
|
|
|
- case DECOMMISSIONED:
|
|
|
|
- metrics.decrDecommisionedNMs();
|
|
|
|
- break;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
|
@Override
|
|
@Override
|
|
public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
// Inform the scheduler
|
|
// Inform the scheduler
|
|
@@ -333,12 +361,14 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
|
|
|
String host = rmNode.nodeId.getHost();
|
|
String host = rmNode.nodeId.getHost();
|
|
if (rmNode.context.getInactiveRMNodes().containsKey(host)) {
|
|
if (rmNode.context.getInactiveRMNodes().containsKey(host)) {
|
|
- RMNode node = rmNode.context.getInactiveRMNodes().get(host);
|
|
|
|
|
|
+ // Old node rejoining
|
|
|
|
+ RMNode previouRMNode = rmNode.context.getInactiveRMNodes().get(host);
|
|
rmNode.context.getInactiveRMNodes().remove(host);
|
|
rmNode.context.getInactiveRMNodes().remove(host);
|
|
- updateMetrics(node.getState());
|
|
|
|
|
|
+ rmNode.updateMetricsForRejoinedNode(previouRMNode.getState());
|
|
|
|
+ } else {
|
|
|
|
+ // Increment activeNodes explicitly because this is a new node.
|
|
|
|
+ ClusterMetrics.getMetrics().incrNumActiveNodes();
|
|
}
|
|
}
|
|
-
|
|
|
|
- ClusterMetrics.getMetrics().addNode();
|
|
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -362,28 +392,33 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- public static class RemoveNodeTransition
|
|
|
|
|
|
+ public static class DeactivateNodeTransition
|
|
implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
|
|
implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
|
|
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
|
|
|
+ private final RMNodeState finalState;
|
|
|
|
+ public DeactivateNodeTransition(RMNodeState finalState) {
|
|
|
|
+ this.finalState = finalState;
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
// Inform the scheduler
|
|
// Inform the scheduler
|
|
rmNode.context.getDispatcher().getEventHandler().handle(
|
|
rmNode.context.getDispatcher().getEventHandler().handle(
|
|
new NodeRemovedSchedulerEvent(rmNode));
|
|
new NodeRemovedSchedulerEvent(rmNode));
|
|
|
|
|
|
- // Remove the node from the system.
|
|
|
|
|
|
+ // Deactivate the node
|
|
rmNode.context.getRMNodes().remove(rmNode.nodeId);
|
|
rmNode.context.getRMNodes().remove(rmNode.nodeId);
|
|
- LOG.info("Removed Node " + rmNode.nodeId);
|
|
|
|
|
|
+ LOG.info("Deactivating Node " + rmNode.nodeId + " as it is now "
|
|
|
|
+ + finalState);
|
|
rmNode.context.getInactiveRMNodes().put(rmNode.nodeId.getHost(), rmNode);
|
|
rmNode.context.getInactiveRMNodes().put(rmNode.nodeId.getHost(), rmNode);
|
|
- //Update the metrics
|
|
|
|
- ClusterMetrics.getMetrics().removeNode(event.getType());
|
|
|
|
|
|
+
|
|
|
|
+ //Update the metrics
|
|
|
|
+ rmNode.updateMetricsForDeactivatedNode(finalState);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
public static class StatusUpdateWhenHealthyTransition implements
|
|
public static class StatusUpdateWhenHealthyTransition implements
|
|
MultipleArcTransition<RMNodeImpl, RMNodeEvent, RMNodeState> {
|
|
MultipleArcTransition<RMNodeImpl, RMNodeEvent, RMNodeState> {
|
|
- @SuppressWarnings("unchecked")
|
|
|
|
@Override
|
|
@Override
|
|
public RMNodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
public RMNodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
|
|
|
|
@@ -399,7 +434,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
// Inform the scheduler
|
|
// Inform the scheduler
|
|
rmNode.context.getDispatcher().getEventHandler().handle(
|
|
rmNode.context.getDispatcher().getEventHandler().handle(
|
|
new NodeRemovedSchedulerEvent(rmNode));
|
|
new NodeRemovedSchedulerEvent(rmNode));
|
|
- ClusterMetrics.getMetrics().incrNumUnhealthyNMs();
|
|
|
|
|
|
+ // Update metrics
|
|
|
|
+ rmNode.updateMetricsForDeactivatedNode(RMNodeState.UNHEALTHY);
|
|
return RMNodeState.UNHEALTHY;
|
|
return RMNodeState.UNHEALTHY;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -458,11 +494,9 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- public static class StatusUpdateWhenUnHealthyTransition
|
|
|
|
- implements
|
|
|
|
|
|
+ public static class StatusUpdateWhenUnHealthyTransition implements
|
|
MultipleArcTransition<RMNodeImpl, RMNodeEvent, RMNodeState> {
|
|
MultipleArcTransition<RMNodeImpl, RMNodeEvent, RMNodeState> {
|
|
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
|
@Override
|
|
@Override
|
|
public RMNodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
public RMNodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
|
|
RMNodeStatusEvent statusEvent = (RMNodeStatusEvent) event;
|
|
RMNodeStatusEvent statusEvent = (RMNodeStatusEvent) event;
|
|
@@ -474,7 +508,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
if (remoteNodeHealthStatus.getIsNodeHealthy()) {
|
|
if (remoteNodeHealthStatus.getIsNodeHealthy()) {
|
|
rmNode.context.getDispatcher().getEventHandler().handle(
|
|
rmNode.context.getDispatcher().getEventHandler().handle(
|
|
new NodeAddedSchedulerEvent(rmNode));
|
|
new NodeAddedSchedulerEvent(rmNode));
|
|
- ClusterMetrics.getMetrics().decrNumUnhealthyNMs();
|
|
|
|
|
|
+ // Update metrics
|
|
|
|
+ rmNode.updateMetricsForRejoinedNode(RMNodeState.UNHEALTHY);
|
|
return RMNodeState.RUNNING;
|
|
return RMNodeState.RUNNING;
|
|
}
|
|
}
|
|
|
|
|