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:
parent
13acee63d3
commit
95a0505c60
|
@ -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
|
||||
|
|
|
@ -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>
|
||||
*/
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
Loading…
Reference in New Issue