svn merge -c 1334043. FIXES: MAPREDUCE-3958. RM: Remove RMNodeState and replace it with NodeState (Bikas Saha via bobby)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1334045 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Joseph Evans 2012-05-04 15:48:13 +00:00
parent 13acee63d3
commit 95a0505c60
17 changed files with 102 additions and 149 deletions

View File

@ -172,6 +172,9 @@ Release 2.0.0 - UNRELEASED
MAPREDUCE-3173. MRV2 UI doesn't work properly without internet (Devaraj K
via bobby)
MAPREDUCE-3958. RM: Remove RMNodeState and replace it with NodeState
(Bikas Saha via bobby)
Release 0.23.3 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -18,8 +18,6 @@
package org.apache.hadoop.yarn.api.records;
// TODO NodeState is a clone of RMNodeState made for MR-3353. In a subsequent
// patch RMNodeState should be replaced with NodeState
/**
* <p>State of a <code>Node</code>.</p>
*/

View File

@ -56,7 +56,6 @@ import org.apache.hadoop.yarn.ipc.RPCUtil;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@ -302,7 +301,7 @@ public class ApplicationMasterService extends AbstractService implements
numContainers = schedulerNodeReport.getNumContainers();
}
NodeReport report = BuilderUtils.newNodeReport(rmNode.getNodeID(),
RMNodeState.toNodeState(rmNode.getState()),
rmNode.getState(),
rmNode.getHttpAddress(), rmNode.getRackName(), used,
rmNode.getTotalCapability(), numContainers,
rmNode.getNodeHealthStatus());

View File

@ -79,7 +79,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
@ -414,7 +413,7 @@ public class ClientRMService extends AbstractService implements
}
NodeReport report = BuilderUtils.newNodeReport(rmNode.getNodeID(),
RMNodeState.toNodeState(rmNode.getState()),
rmNode.getState(),
rmNode.getHttpAddress(), rmNode.getRackName(), used,
rmNode.getTotalCapability(), numContainers,
rmNode.getNodeHealthStatus());

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
@ -56,7 +57,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
import org.apache.hadoop.yarn.state.MultipleArcTransition;
@ -498,7 +498,7 @@ public class RMAppImpl implements RMApp {
}
private void processNodeUpdate(RMAppNodeUpdateType type, RMNode node) {
RMNodeState nodeState = node.getState();
NodeState nodeState = node.getState();
updatedNodes.add(node);
LOG.debug("Received node update event:" + type + " for node:" + node
+ " with state:" + nodeState);

View File

@ -23,10 +23,10 @@ import java.util.List;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
/**
@ -99,7 +99,7 @@ public interface RMNode {
*/
public Node getNode();
public RMNodeState getState();
public NodeState getState();
public List<ContainerId> getContainersToCleanUp();

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
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.event.EventHandler;
import org.apache.hadoop.yarn.factories.RecordFactory;
@ -104,53 +105,53 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
.newRecordInstance(HeartbeatResponse.class);
private static final StateMachineFactory<RMNodeImpl,
RMNodeState,
NodeState,
RMNodeEventType,
RMNodeEvent> stateMachineFactory
= new StateMachineFactory<RMNodeImpl,
RMNodeState,
NodeState,
RMNodeEventType,
RMNodeEvent>(RMNodeState.NEW)
RMNodeEvent>(NodeState.NEW)
//Transitions from NEW state
.addTransition(RMNodeState.NEW, RMNodeState.RUNNING,
.addTransition(NodeState.NEW, NodeState.RUNNING,
RMNodeEventType.STARTED, new AddNodeTransition())
//Transitions from RUNNING state
.addTransition(RMNodeState.RUNNING,
EnumSet.of(RMNodeState.RUNNING, RMNodeState.UNHEALTHY),
.addTransition(NodeState.RUNNING,
EnumSet.of(NodeState.RUNNING, NodeState.UNHEALTHY),
RMNodeEventType.STATUS_UPDATE, new StatusUpdateWhenHealthyTransition())
.addTransition(RMNodeState.RUNNING, RMNodeState.DECOMMISSIONED,
.addTransition(NodeState.RUNNING, NodeState.DECOMMISSIONED,
RMNodeEventType.DECOMMISSION,
new DeactivateNodeTransition(RMNodeState.DECOMMISSIONED))
.addTransition(RMNodeState.RUNNING, RMNodeState.LOST,
new DeactivateNodeTransition(NodeState.DECOMMISSIONED))
.addTransition(NodeState.RUNNING, NodeState.LOST,
RMNodeEventType.EXPIRE,
new DeactivateNodeTransition(RMNodeState.LOST))
.addTransition(RMNodeState.RUNNING, RMNodeState.REBOOTED,
new DeactivateNodeTransition(NodeState.LOST))
.addTransition(NodeState.RUNNING, NodeState.REBOOTED,
RMNodeEventType.REBOOTING,
new DeactivateNodeTransition(RMNodeState.REBOOTED))
.addTransition(RMNodeState.RUNNING, RMNodeState.RUNNING,
new DeactivateNodeTransition(NodeState.REBOOTED))
.addTransition(NodeState.RUNNING, NodeState.RUNNING,
RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
.addTransition(RMNodeState.RUNNING, RMNodeState.RUNNING,
.addTransition(NodeState.RUNNING, NodeState.RUNNING,
RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
.addTransition(RMNodeState.RUNNING, RMNodeState.RUNNING,
.addTransition(NodeState.RUNNING, NodeState.RUNNING,
RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
//Transitions from UNHEALTHY state
.addTransition(RMNodeState.UNHEALTHY,
EnumSet.of(RMNodeState.UNHEALTHY, RMNodeState.RUNNING),
.addTransition(NodeState.UNHEALTHY,
EnumSet.of(NodeState.UNHEALTHY, NodeState.RUNNING),
RMNodeEventType.STATUS_UPDATE, new StatusUpdateWhenUnHealthyTransition())
.addTransition(RMNodeState.UNHEALTHY, RMNodeState.UNHEALTHY,
.addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
.addTransition(RMNodeState.UNHEALTHY, RMNodeState.UNHEALTHY,
.addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
.addTransition(RMNodeState.UNHEALTHY, RMNodeState.UNHEALTHY,
.addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
// create the topology tables
.installTopology();
private final StateMachine<RMNodeState, RMNodeEventType,
private final StateMachine<NodeState, RMNodeEventType,
RMNodeEvent> stateMachine;
public RMNodeImpl(NodeId nodeId, RMContext context, String hostName,
@ -252,7 +253,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
}
@Override
public RMNodeState getState() {
public NodeState getState() {
this.readLock.lock();
try {
@ -302,7 +303,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
LOG.debug("Processing " + event.getNodeId() + " of type " + event.getType());
try {
writeLock.lock();
RMNodeState oldState = getState();
NodeState oldState = getState();
try {
stateMachine.doTransition(event.getType(), event);
} catch (InvalidStateTransitonException e) {
@ -321,7 +322,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
}
}
private void updateMetricsForRejoinedNode(RMNodeState previousNodeState) {
private void updateMetricsForRejoinedNode(NodeState previousNodeState) {
ClusterMetrics metrics = ClusterMetrics.getMetrics();
metrics.incrNumActiveNodes();
@ -341,7 +342,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
}
}
private void updateMetricsForDeactivatedNode(RMNodeState finalState) {
private void updateMetricsForDeactivatedNode(NodeState finalState) {
ClusterMetrics metrics = ClusterMetrics.getMetrics();
metrics.decrNumActiveNodes();
@ -440,8 +441,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
public static class DeactivateNodeTransition
implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
private final RMNodeState finalState;
public DeactivateNodeTransition(RMNodeState finalState) {
private final NodeState finalState;
public DeactivateNodeTransition(NodeState finalState) {
this.finalState = finalState;
}
@ -466,9 +467,9 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
}
public static class StatusUpdateWhenHealthyTransition implements
MultipleArcTransition<RMNodeImpl, RMNodeEvent, RMNodeState> {
MultipleArcTransition<RMNodeImpl, RMNodeEvent, NodeState> {
@Override
public RMNodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
RMNodeStatusEvent statusEvent = (RMNodeStatusEvent) event;
@ -486,8 +487,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
new NodesListManagerEvent(
NodesListManagerEventType.NODE_UNUSABLE, rmNode));
// Update metrics
rmNode.updateMetricsForDeactivatedNode(RMNodeState.UNHEALTHY);
return RMNodeState.UNHEALTHY;
rmNode.updateMetricsForDeactivatedNode(NodeState.UNHEALTHY);
return NodeState.UNHEALTHY;
}
// Filter the map to only obtain just launched containers and finished
@ -541,15 +542,15 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
rmNode.containersToClean.clear();
rmNode.finishedApplications.clear();
return RMNodeState.RUNNING;
return NodeState.RUNNING;
}
}
public static class StatusUpdateWhenUnHealthyTransition implements
MultipleArcTransition<RMNodeImpl, RMNodeEvent, RMNodeState> {
MultipleArcTransition<RMNodeImpl, RMNodeEvent, NodeState> {
@Override
public RMNodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) {
RMNodeStatusEvent statusEvent = (RMNodeStatusEvent) event;
// Switch the last heartbeatresponse.
@ -566,11 +567,11 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
// notifiers get update metadata because they will very likely query it
// upon notification
// Update metrics
rmNode.updateMetricsForRejoinedNode(RMNodeState.UNHEALTHY);
return RMNodeState.RUNNING;
rmNode.updateMetricsForRejoinedNode(NodeState.UNHEALTHY);
return NodeState.RUNNING;
}
return RMNodeState.UNHEALTHY;
return NodeState.UNHEALTHY;
}
}
}

View File

@ -1,45 +0,0 @@
/**
* 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 org.apache.hadoop.yarn.api.records.NodeState;
//TODO yarn.api.records.NodeState is a clone of RMNodeState made for MR-3353.
// In a subsequent patch RMNodeState should be replaced with NodeState
public enum RMNodeState {
NEW, RUNNING, UNHEALTHY, DECOMMISSIONED, LOST, REBOOTED;
public static NodeState toNodeState(RMNodeState state) {
switch(state) {
case NEW:
return NodeState.NEW;
case RUNNING:
return NodeState.RUNNING;
case UNHEALTHY:
return NodeState.UNHEALTHY;
case DECOMMISSIONED:
return NodeState.DECOMMISSIONED;
case LOST:
return NodeState.LOST;
case REBOOTED:
return NodeState.REBOOTED;
}
return null;
}
};

View File

@ -27,10 +27,10 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit;
import java.util.Collection;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
import org.apache.hadoop.yarn.util.Times;
@ -78,9 +78,9 @@ class NodesPage extends RmView {
th(".mem", "Mem Avail").
_()._().
tbody();
RMNodeState stateFilter = null;
NodeState stateFilter = null;
if(type != null && !type.isEmpty()) {
stateFilter = RMNodeState.valueOf(type.toUpperCase());
stateFilter = NodeState.valueOf(type.toUpperCase());
}
Collection<RMNode> rmNodes = this.rmContext.getRMNodes().values();
boolean isInactive = false;
@ -96,14 +96,14 @@ class NodesPage extends RmView {
}
for (RMNode ni : rmNodes) {
if(stateFilter != null) {
RMNodeState state = ni.getState();
NodeState state = ni.getState();
if(!stateFilter.equals(state)) {
continue;
}
} else {
// No filter. User is asking for all nodes. Make sure you skip the
// unhealthy nodes.
if (ni.getState() == RMNodeState.UNHEALTHY) {
if (ni.getState() == NodeState.UNHEALTHY) {
continue;
}
}

View File

@ -39,13 +39,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
@ -162,7 +162,7 @@ public class RMWebServices {
Collection<RMNode> rmNodes = this.rm.getRMContext().getRMNodes().values();
boolean isInactive = false;
if (filterState != null && !filterState.isEmpty()) {
RMNodeState nodeState = RMNodeState.valueOf(filterState.toUpperCase());
NodeState nodeState = NodeState.valueOf(filterState.toUpperCase());
switch (nodeState) {
case DECOMMISSIONED:
case LOST:
@ -182,7 +182,7 @@ public class RMWebServices {
} else {
// No filter. User is asking for all nodes. Make sure you skip the
// unhealthy nodes.
if (ni.getState() == RMNodeState.UNHEALTHY) {
if (ni.getState() == NodeState.UNHEALTHY) {
continue;
}
}

View File

@ -25,8 +25,8 @@ import javax.xml.bind.annotation.XmlTransient;
import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeRepo
public class NodeInfo {
protected String rack;
protected RMNodeState state;
protected NodeState state;
protected String id;
protected String nodeHostName;
protected String nodeHTTPAddress;

View File

@ -26,11 +26,11 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState;
import com.google.common.collect.Lists;
@ -48,7 +48,7 @@ public class MockNodes {
for (int j = 0; j < nodesPerRack; ++j) {
if (j == (nodesPerRack - 1)) {
// One unhealthy node per rack.
list.add(nodeInfo(i, perNode, RMNodeState.UNHEALTHY));
list.add(nodeInfo(i, perNode, NodeState.UNHEALTHY));
}
list.add(newNodeInfo(i, perNode));
}
@ -61,7 +61,7 @@ public class MockNodes {
List<RMNode> list = Lists.newArrayList();
for (int i = 0; i < racks; ++i) {
for (int j = 0; j < nodesPerRack; ++j) {
RMNodeState[] allStates = RMNodeState.values();
NodeState[] allStates = NodeState.values();
list.add(nodeInfo(i, perNode, allStates[j % allStates.length]));
}
}
@ -102,11 +102,11 @@ public class MockNodes {
private Resource perNode;
private String rackName;
private NodeHealthStatus nodeHealthStatus;
private RMNodeState state;
private NodeState state;
public MockRMNodeImpl(NodeId nodeId, String nodeAddr, String httpAddress,
Resource perNode, String rackName, NodeHealthStatus nodeHealthStatus,
int cmdPort, String hostName, RMNodeState state) {
int cmdPort, String hostName, NodeState state) {
this.nodeId = nodeId;
this.nodeAddr = nodeAddr;
this.httpAddress = httpAddress;
@ -169,7 +169,7 @@ public class MockNodes {
}
@Override
public RMNodeState getState() {
public NodeState getState() {
return this.state;
}
@ -189,11 +189,11 @@ public class MockNodes {
}
};
private static RMNode buildRMNode(int rack, final Resource perNode, RMNodeState state, String httpAddr) {
private static RMNode buildRMNode(int rack, final Resource perNode, NodeState state, String httpAddr) {
return buildRMNode(rack, perNode, state, httpAddr, NODE_ID++);
}
private static RMNode buildRMNode(int rack, final Resource perNode, RMNodeState state, String httpAddr, int hostnum) {
private static RMNode buildRMNode(int rack, final Resource perNode, NodeState state, String httpAddr, int hostnum) {
final String rackName = "rack"+ rack;
final int nid = hostnum;
final String hostName = "host"+ nid;
@ -202,7 +202,7 @@ public class MockNodes {
final String httpAddress = httpAddr;
final NodeHealthStatus nodeHealthStatus =
recordFactory.newRecordInstance(NodeHealthStatus.class);
if (state != RMNodeState.UNHEALTHY) {
if (state != NodeState.UNHEALTHY) {
nodeHealthStatus.setIsNodeHealthy(true);
nodeHealthStatus.setHealthReport("HealthyMe");
}
@ -211,12 +211,12 @@ public class MockNodes {
}
public static RMNode nodeInfo(int rack, final Resource perNode,
RMNodeState state) {
NodeState state) {
return buildRMNode(rack, perNode, state, "N/A");
}
public static RMNode newNodeInfo(int rack, final Resource perNode) {
return buildRMNode(rack, perNode, RMNodeState.RUNNING, "localhost:0");
return buildRMNode(rack, perNode, NodeState.RUNNING, "localhost:0");
}
public static RMNode newNodeInfo(int rack, final Resource perNode, int hostnum) {

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory;
@ -48,7 +49,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState;
import org.apache.hadoop.yarn.util.Records;
import org.apache.log4j.Level;
import org.apache.log4j.LogManager;
@ -146,7 +146,7 @@ public class MockRM extends ResourceManager {
node.handle(new RMNodeEvent(nm.getNodeId(), RMNodeEventType.EXPIRE));
}
public void NMwaitForState(NodeId nodeid, RMNodeState finalState)
public void NMwaitForState(NodeId nodeid, NodeState finalState)
throws Exception {
RMNode node = getRMContext().getRMNodes().get(nodeid);
Assert.assertNotNull("node shouldn't be null", node);

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.util.BuilderUtils;
import org.junit.After;
@ -83,7 +82,7 @@ public class TestAMRMRPCNodeUpdates {
private void syncNodeLost(MockNM nm) throws Exception {
rm.sendNodeStarted(nm);
rm.NMwaitForState(nm.getNodeId(), RMNodeState.RUNNING);
rm.NMwaitForState(nm.getNodeId(), NodeState.RUNNING);
rm.sendNodeLost(nm);
dispatcher.await();
}

View File

@ -20,9 +20,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
import java.io.IOException;
import java.io.PrintWriter;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.NodesPage.NodesBlock;
import org.apache.hadoop.yarn.webapp.test.WebAppTests;
import org.junit.Before;
@ -44,7 +44,7 @@ public class TestNodesPage {
// The following is because of the way TestRMWebApp.mockRMContext creates
// nodes.
final int numberOfLostNodesPerRack = numberOfNodesPerRack
/ RMNodeState.values().length;
/ NodeState.values().length;
// Number of Actual Table Headers for NodesPage.NodesBlock might change in
// future. In that case this value should be adjusted to the new value.

View File

@ -31,6 +31,7 @@ import java.util.concurrent.ConcurrentMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@ -41,7 +42,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemStore;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
@ -124,13 +124,13 @@ public class TestRMWebApp {
// Unhealthy nodes
instance.moreParams().put(YarnWebParams.NODE_STATE,
RMNodeState.UNHEALTHY.toString());
NodeState.UNHEALTHY.toString());
instance.render();
WebAppTests.flushOutput(injector);
// Lost nodes
instance.moreParams().put(YarnWebParams.NODE_STATE,
RMNodeState.LOST.toString());
NodeState.LOST.toString());
instance.render();
WebAppTests.flushOutput(injector);

View File

@ -32,13 +32,13 @@ import javax.xml.parsers.DocumentBuilderFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
@ -54,7 +54,6 @@ import org.w3c.dom.Document;
import org.w3c.dom.Element;
import org.w3c.dom.NodeList;
import org.xml.sax.InputSource;
import com.google.inject.Guice;
import com.google.inject.Injector;
import com.google.inject.servlet.GuiceServletContextListener;
@ -131,15 +130,15 @@ public class TestRMWebServicesNodes extends JerseyTest {
MockNM nm1 = rm.registerNode("h1:1234", 5120);
MockNM nm2 = rm.registerNode("h2:1235", 5121);
rm.sendNodeStarted(nm1);
rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), RMNodeState.NEW);
rm.NMwaitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), NodeState.NEW);
// One unhealthy node which should not appear in the list after
// MAPREDUCE-3760.
MockNM nm3 = rm.registerNode("h3:1236", 5122);
rm.NMwaitForState(nm3.getNodeId(), RMNodeState.NEW);
rm.NMwaitForState(nm3.getNodeId(), NodeState.NEW);
rm.sendNodeStarted(nm3);
rm.NMwaitForState(nm3.getNodeId(), RMNodeState.RUNNING);
rm.NMwaitForState(nm3.getNodeId(), NodeState.RUNNING);
RMNodeImpl node = (RMNodeImpl) rm.getRMContext().getRMNodes()
.get(nm3.getNodeId());
NodeHealthStatus nodeHealth = node.getNodeHealthStatus();
@ -147,7 +146,7 @@ public class TestRMWebServicesNodes extends JerseyTest {
nodeHealth.setIsNodeHealthy(false);
node.handle(new RMNodeStatusEvent(nm3.getNodeId(), nodeHealth,
new ArrayList<ContainerStatus>(), null, null));
rm.NMwaitForState(nm3.getNodeId(), RMNodeState.UNHEALTHY);
rm.NMwaitForState(nm3.getNodeId(), NodeState.UNHEALTHY);
ClientResponse response =
r.path("ws").path("v1").path("cluster").path("nodes")
@ -169,11 +168,11 @@ public class TestRMWebServicesNodes extends JerseyTest {
MockNM nm1 = rm.registerNode("h1:1234", 5120);
MockNM nm2 = rm.registerNode("h2:1235", 5121);
rm.sendNodeStarted(nm1);
rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), RMNodeState.NEW);
rm.NMwaitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), NodeState.NEW);
ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").queryParam("state", RMNodeState.RUNNING.toString())
.path("nodes").queryParam("state", NodeState.RUNNING.toString())
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
@ -196,7 +195,7 @@ public class TestRMWebServicesNodes extends JerseyTest {
ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes")
.queryParam("state", RMNodeState.DECOMMISSIONED.toString())
.queryParam("state", NodeState.DECOMMISSIONED.toString())
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
JSONObject json = response.getEntity(JSONObject.class);
@ -231,7 +230,7 @@ public class TestRMWebServicesNodes extends JerseyTest {
WebServicesTestUtils
.checkStringMatch(
"exception message",
"No enum const class org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeState.BOGUSSTATE",
"No enum const class org.apache.hadoop.yarn.api.records.NodeState.BOGUSSTATE",
message);
WebServicesTestUtils.checkStringMatch("exception type",
"IllegalArgumentException", type);
@ -250,13 +249,13 @@ public class TestRMWebServicesNodes extends JerseyTest {
MockNM nm2 = rm.registerNode("h2:1234", 5120);
rm.sendNodeStarted(nm1);
rm.sendNodeStarted(nm2);
rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), RMNodeState.RUNNING);
rm.NMwaitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), NodeState.RUNNING);
rm.sendNodeLost(nm1);
rm.sendNodeLost(nm2);
ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").queryParam("state", RMNodeState.LOST.toString())
.path("nodes").queryParam("state", NodeState.LOST.toString())
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
@ -283,8 +282,8 @@ public class TestRMWebServicesNodes extends JerseyTest {
MockNM nm2 = rm.registerNode("h2:1234", 5120);
rm.sendNodeStarted(nm1);
rm.sendNodeStarted(nm2);
rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), RMNodeState.RUNNING);
rm.NMwaitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), NodeState.RUNNING);
rm.sendNodeLost(nm1);
rm.sendNodeLost(nm2);
@ -312,8 +311,8 @@ public class TestRMWebServicesNodes extends JerseyTest {
MockNM nm1 = rm.registerNode("h1:1234", 5120);
MockNM nm2 = rm.registerNode("h2:1235", 5121);
rm.sendNodeStarted(nm1);
rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), RMNodeState.NEW);
rm.NMwaitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), NodeState.NEW);
ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").queryParam("healthy", "true")
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
@ -332,8 +331,8 @@ public class TestRMWebServicesNodes extends JerseyTest {
MockNM nm1 = rm.registerNode("h1:1234", 5120);
MockNM nm2 = rm.registerNode("h2:1235", 5121);
rm.sendNodeStarted(nm1);
rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), RMNodeState.NEW);
rm.NMwaitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), NodeState.NEW);
ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").queryParam("healthy", "TRUe")
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
@ -353,8 +352,8 @@ public class TestRMWebServicesNodes extends JerseyTest {
MockNM nm1 = rm.registerNode("h1:1234", 5120);
MockNM nm2 = rm.registerNode("h2:1235", 5121);
rm.sendNodeStarted(nm1);
rm.NMwaitForState(nm2.getNodeId(), RMNodeState.NEW);
rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), NodeState.NEW);
rm.NMwaitForState(nm1.getNodeId(), NodeState.RUNNING);
RMNodeImpl node = (RMNodeImpl) rm.getRMContext().getRMNodes()
.get(nm1.getNodeId());
NodeHealthStatus nodeHealth = node.getNodeHealthStatus();
@ -362,11 +361,11 @@ public class TestRMWebServicesNodes extends JerseyTest {
nodeHealth.setIsNodeHealthy(false);
node.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeHealth,
new ArrayList<ContainerStatus>(), null, null));
rm.NMwaitForState(nm1.getNodeId(), RMNodeState.UNHEALTHY);
rm.NMwaitForState(nm1.getNodeId(), NodeState.UNHEALTHY);
ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").queryParam("healthy", "true")
.queryParam("state", RMNodeState.RUNNING.toString())
.queryParam("state", NodeState.RUNNING.toString())
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
JSONObject json = response.getEntity(JSONObject.class);
@ -380,8 +379,8 @@ public class TestRMWebServicesNodes extends JerseyTest {
MockNM nm1 = rm.registerNode("h1:1234", 5120);
MockNM nm2 = rm.registerNode("h2:1235", 5121);
rm.sendNodeStarted(nm1);
rm.NMwaitForState(nm1.getNodeId(), RMNodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), RMNodeState.NEW);
rm.NMwaitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.NMwaitForState(nm2.getNodeId(), NodeState.NEW);
ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").queryParam("healthy", "false")
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);