/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.hadoop.yarn.server.resourcemanager.rmnode;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceOption;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.NodesListManagerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.NodesListManagerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.NodesListManager;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRunningOnNodeEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
import org.apache.hadoop.yarn.server.utils.BuilderUtils.ContainerIdComparator;
import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
import org.apache.hadoop.yarn.state.MultipleArcTransition;
import org.apache.hadoop.yarn.state.SingleArcTransition;
import org.apache.hadoop.yarn.state.StateMachine;
import org.apache.hadoop.yarn.state.StateMachineFactory;
import com.google.common.annotations.VisibleForTesting;
import io.hops.util.DBUtility;
/**
* This class is used to keep track of all the applications/containers
* running on a node.
*
*/
@Private
@Unstable
@SuppressWarnings("unchecked")
public abstract class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
private static final Log LOG = LogFactory.getLog(RMNodeImpl.class);
private static final RecordFactory recordFactory = RecordFactoryProvider
.getRecordFactory(null);
private final ReadLock readLock;
protected final WriteLock writeLock;
protected final ConcurrentLinkedQueue<UpdatedContainerInfo> nodeUpdateQueue;
protected volatile boolean nextHeartBeat = true;
protected final NodeId nodeId;
protected final RMContext context;
protected final String hostName;
protected final int commandPort;
protected int httpPort;
private final String nodeAddress; // The containerManager address
protected String httpAddress;
protected volatile Resource totalCapability;
private final Node node;
private String healthReport;
private long lastHealthReportTime;
protected String nodeManagerVersion;
/* set of containers that have just launched */
protected final Set<ContainerId> launchedContainers =
new HashSet<ContainerId>();
/* set of containers that need to be cleaned */
protected final Set<ContainerId> containersToClean = new TreeSet<ContainerId>(
new ContainerIdComparator());
/*
* set of containers to notify NM to remove them from its context. Currently,
* this includes containers that were notified to AM about their completion
*/
protected final Set<ContainerId> containersToBeRemovedFromNM =
new HashSet<ContainerId>();
/* the list of applications that have finished and need to be purged */
protected final List<ApplicationId> finishedApplications = new ArrayList<ApplicationId>();
protected NodeHeartbeatResponse latestNodeHeartBeatResponse = recordFactory
.newRecordInstance(NodeHeartbeatResponse.class);
private static final StateMachineFactory<RMNodeImpl,
NodeState,
RMNodeEventType,
RMNodeEvent> stateMachineFactory
= new StateMachineFactory<RMNodeImpl,
NodeState,
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())
.addTransition(NodeState.NEW, NodeState.DECOMMISSIONED,
RMNodeEventType.DECOMMISSION,
new DeactivateNodeTransition(NodeState.DECOMMISSIONED))
//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())
//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())
// create the topology tables
.installTopology();
protected final StateMachine<NodeState, RMNodeEventType,
RMNodeEvent> stateMachine;
public RMNodeImpl(NodeId nodeId, RMContext context, String hostName,
int cmPort, int httpPort, Node node, Resource capability, String nodeManagerVersion) {
this.nodeId = nodeId;
this.context = context;
this.hostName = hostName;
this.commandPort = cmPort;
this.httpPort = httpPort;
this.totalCapability = capability;
this.nodeAddress = hostName + ":" + cmPort;
this.httpAddress = hostName + ":" + httpPort;
this.node = node;
this.healthReport = "Healthy";
this.lastHealthReportTime = System.currentTimeMillis();
this.nodeManagerVersion = nodeManagerVersion;
this.latestNodeHeartBeatResponse.setResponseId(0);
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
this.readLock = lock.readLock();
this.writeLock = lock.writeLock();
this.stateMachine = stateMachineFactory.make(this);
this.nodeUpdateQueue = new ConcurrentLinkedQueue<UpdatedContainerInfo>();
}
@Override
public String toString() {
return this.nodeId.toString();
}
@Override
public String getHostName() {
return hostName;
}
@Override
public int getCommandPort() {
return commandPort;
}
@Override
public int getHttpPort() {
return httpPort;
}
@Override
public NodeId getNodeID() {
return this.nodeId;
}
@Override
public String getNodeAddress() {
return this.nodeAddress;
}
@Override
public String getHttpAddress() {
return this.httpAddress;
}
@Override
public Resource getTotalCapability() {
return this.totalCapability;
}
@Override
public String getRackName() {
return node.getNetworkLocation();
}
@Override
public Node getNode() {
return this.node;
}
@Override
public String getHealthReport() {
this.readLock.lock();
try {
return this.healthReport;
} finally {
this.readLock.unlock();
}
}
public void setHealthReport(String healthReport) {
this.writeLock.lock();
try {
this.healthReport = healthReport;
} finally {
this.writeLock.unlock();
}
}
public void setLastHealthReportTime(long lastHealthReportTime) {
this.writeLock.lock();
try {
this.lastHealthReportTime = lastHealthReportTime;
} finally {
this.writeLock.unlock();
}
}
@Override
public long getLastHealthReportTime() {
this.readLock.lock();
try {
return this.lastHealthReportTime;
} finally {
this.readLock.unlock();
}
}
@Override
public String getNodeManagerVersion() {
return nodeManagerVersion;
}
@Override
public NodeState getState() {
this.readLock.lock();
try {
return this.stateMachine.getCurrentState();
} finally {
this.readLock.unlock();
}
}
@Override
public List<ApplicationId> getAppsToCleanup() {
this.readLock.lock();
try {
return new ArrayList<ApplicationId>(this.finishedApplications);
} finally {
this.readLock.unlock();
}
}
@Override
public List<ContainerId> getContainersToCleanUp() {
this.readLock.lock();
try {
return new ArrayList<ContainerId>(this.containersToClean);
} finally {
this.readLock.unlock();
}
};
@Override
public NodeHeartbeatResponse getLastNodeHeartBeatResponse() {
this.readLock.lock();
try {
return this.latestNodeHeartBeatResponse;
} finally {
this.readLock.unlock();
}
}
@Override
public void resetLastNodeHeartBeatResponse() {
this.writeLock.lock();
try {
latestNodeHeartBeatResponse.setResponseId(0);
} finally {
this.writeLock.unlock();
}
}
protected void updateMetricsForRejoinedNode(NodeState 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;
default:
LOG.debug("Unexpected previous node state");
}
}
protected 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");
}
switch (finalState) {
case DECOMMISSIONED:
metrics.incrDecommisionedNMs();
break;
case LOST:
metrics.incrNumLostNMs();
break;
case REBOOTED:
metrics.incrNumRebootedNMs();
break;
case UNHEALTHY:
metrics.incrNumUnhealthyNMs();
break;
case NEW:
break;
default:
LOG.debug("Unexpected final state");
}
}
abstract protected void handleRunningAppOnNode(RMNodeImpl rmNode,
RMContext context, ApplicationId appId, NodeId nodeId);
private static void updateNodeResourceFromEvent(RMNodeImpl rmNode,
RMNodeResourceUpdateEvent event){
ResourceOption resourceOption = event.getResourceOption();
// Set resource on RMNode
rmNode.totalCapability = resourceOption.getResource();
}
public static class AddNodeTransition implements
SingleArcTransition<RMNodeImpl, RMNodeEvent> {
@Override
public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
rmNode.addNodeTransitionInternal(rmNode,event);
}
}
protected abstract void addNodeTransitionInternal(RMNodeImpl rmNode, RMNodeEvent event);
protected static List<NMContainerStatus> updateNewNodeMetricsAndContainers(
RMNodeImpl rmNode, RMNodeStartedEvent startEvent) {
List<NMContainerStatus> containers;
ClusterMetrics.getMetrics().incrNumActiveNodes();
containers = startEvent.getNMContainerStatuses();
if (containers != null && !containers.isEmpty()) {
for (NMContainerStatus container : containers) {
if (container.getContainerState() == ContainerState.RUNNING) {
rmNode.launchedContainers.add(container.getContainerId());
}
}
}
return containers;
}
protected abstract void reconnectNodeTransitionInternal(RMNodeImpl rmNode, RMNodeEvent event);
protected void handleNMContainerStatus(
List<NMContainerStatus> nmContainerStatuses, RMNodeImpl rmnode) {
List<ContainerStatus> containerStatuses = new ArrayList<ContainerStatus>();
for (NMContainerStatus nmContainerStatus : nmContainerStatuses) {
containerStatuses.add(createContainerStatus(nmContainerStatus));
}
rmnode.handleContainerStatus(containerStatuses);
}
private ContainerStatus createContainerStatus(
NMContainerStatus remoteContainer) {
ContainerStatus cStatus = ContainerStatus.newInstance(remoteContainer.
getContainerId(),
remoteContainer.getContainerState(),
remoteContainer.getDiagnostics(),
remoteContainer.getContainerExitStatus());
return cStatus;
}
public static class ReconnectNodeTransition implements
SingleArcTransition<RMNodeImpl, RMNodeEvent> {
@Override
public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
rmNode.reconnectNodeTransitionInternal(rmNode, event);
}
}
public static class UpdateNodeResourceWhenRunningTransition
implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
@Override
public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
RMNodeResourceUpdateEvent updateEvent = (RMNodeResourceUpdateEvent)event;
updateNodeResourceFromEvent(rmNode, updateEvent);
// Notify new resourceOption to scheduler
rmNode.context.getDispatcher().getEventHandler().handle(
new NodeResourceUpdateSchedulerEvent(rmNode, updateEvent.getResourceOption()));
}
}
public static class UpdateNodeResourceWhenUnusableTransition
implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
@Override
public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
// The node is not usable, only log a warn message
LOG.warn("Try to update resource on a "+ rmNode.getState().toString() +
" node: "+rmNode.toString());
updateNodeResourceFromEvent(rmNode, (RMNodeResourceUpdateEvent)event);
// No need to notify scheduler as schedulerNode is not function now
// and can sync later from RMnode.
}
}
public static class CleanUpAppTransition
implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
@Override
public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
rmNode.cleanUpAppTransitionInternal(rmNode, event);
}
}
abstract protected void cleanUpAppTransitionInternal(RMNodeImpl rmNode,
RMNodeEvent event);
public static class CleanUpContainerTransition implements
SingleArcTransition<RMNodeImpl, RMNodeEvent> {
@Override
public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
rmNode.cleanUpContainerTransitionInternal(rmNode, event);
}
}
abstract protected void cleanUpContainerTransitionInternal(RMNodeImpl rmNode,
RMNodeEvent event);
public static class AddContainersToBeRemovedFromNMTransition implements
SingleArcTransition<RMNodeImpl, RMNodeEvent> {
@Override
public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
rmNode.containersToBeRemovedFromNM.addAll(((
RMNodeFinishedContainersPulledByAMEvent) event).getContainers());
}
}
public static class DeactivateNodeTransition
implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
private final NodeState finalState;
public DeactivateNodeTransition(NodeState finalState) {
this.finalState = finalState;
}
@Override
public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
rmNode.deactivateNodeTransitionInternal(rmNode, event, finalState);
}
}
abstract protected void deactivateNodeTransitionInternal(RMNodeImpl rmNode, RMNodeEvent event, final NodeState finalState);
public static class StatusUpdateWhenHealthyTransition implements
MultipleArcTransition<RMNodeImpl, RMNodeEvent, NodeState> {
@Override
public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
return rmNode.statusUpdateWhenHealthyTransitionInternal(rmNode, event);
}
}
abstract protected NodeState statusUpdateWhenHealthyTransitionInternal(
RMNodeImpl rmNode, RMNodeEvent event);
public static class StatusUpdateWhenUnHealthyTransition implements
MultipleArcTransition<RMNodeImpl, RMNodeEvent, NodeState> {
@Override
public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
return rmNode.statusUpdateWhenUnHealthyTransitionInternal(rmNode, event);
}
}
protected abstract NodeState statusUpdateWhenUnHealthyTransitionInternal(RMNodeImpl rmNode, RMNodeEvent event);
@VisibleForTesting
public void setNextHeartBeat(boolean nextHeartBeat) {
this.nextHeartBeat = nextHeartBeat;
}
@VisibleForTesting
public int getQueueSize() {
return nodeUpdateQueue.size();
}
// For test only.
@VisibleForTesting
public Set<ContainerId> getLaunchedContainers() {
return this.launchedContainers;
}
@Override
public Set<String> getNodeLabels() {
RMNodeLabelsManager nlm = context.getNodeLabelManager();
if (nlm == null || nlm.getLabelsOnNode(nodeId) == null) {
return CommonNodeLabelsManager.EMPTY_STRING_SET;
}
return nlm.getLabelsOnNode(nodeId);
}
abstract protected void handleContainerStatus(List<ContainerStatus> containerStatuses);
public boolean getNextHeartbeat(){
return nextHeartBeat;
}
}