YARN-4947. Test timeout is happening for TestRMWebServicesNodes. Contributed by Bibin A Chundatt

This commit is contained in:
Rohith Sharma K S 2016-05-04 09:58:26 +05:30
parent 6d77d6eab7
commit 75e0450593

View File

@ -34,17 +34,22 @@
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState; import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceUtilization; import org.apache.hadoop.yarn.api.records.ResourceUtilization;
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus; import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.server.api.records.NodeStatus; import org.apache.hadoop.yarn.server.api.records.NodeStatus;
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; 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.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.RMNodeImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStartedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent; 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.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
import org.apache.hadoop.yarn.util.RackResolver;
import org.apache.hadoop.yarn.util.YarnVersionInfo;
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
import org.apache.hadoop.yarn.webapp.JerseyTestBase; import org.apache.hadoop.yarn.webapp.JerseyTestBase;
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils; import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
@ -130,24 +135,21 @@ public void testNodesDefaultWithUnHealthyNode() throws JSONException,
Exception { Exception {
WebResource r = resource(); WebResource r = resource();
MockNM nm1 = rm.registerNode("h1:1234", 5120); getRunningRMNode("h1", 1234, 5120);
MockNM nm2 = rm.registerNode("h2:1235", 5121); // h2 will be in NEW state
rm.sendNodeStarted(nm1); getNewRMNode("h2", 1235, 5121);
rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.waitForState(nm2.getNodeId(), NodeState.NEW);
MockNM nm3 = rm.registerNode("h3:1236", 5122); RMNode node3 = getRunningRMNode("h3", 1236, 5122);
rm.waitForState(nm3.getNodeId(), NodeState.NEW); NodeId nodeId3 = node3.getNodeID();
rm.sendNodeStarted(nm3);
rm.waitForState(nm3.getNodeId(), NodeState.RUNNING); RMNode node = rm.getRMContext().getRMNodes().get(nodeId3);
RMNodeImpl node = (RMNodeImpl) rm.getRMContext().getRMNodes()
.get(nm3.getNodeId());
NodeHealthStatus nodeHealth = NodeHealthStatus.newInstance(false, NodeHealthStatus nodeHealth = NodeHealthStatus.newInstance(false,
"test health report", System.currentTimeMillis()); "test health report", System.currentTimeMillis());
NodeStatus nodeStatus = NodeStatus.newInstance(nm3.getNodeId(), 1, NodeStatus nodeStatus = NodeStatus.newInstance(nodeId3, 1,
new ArrayList<ContainerStatus>(), null, nodeHealth, null, null, null); new ArrayList<ContainerStatus>(), null, nodeHealth, null, null, null);
node.handle(new RMNodeStatusEvent(nm3.getNodeId(), nodeStatus, null)); ((RMNodeImpl) node)
rm.waitForState(nm3.getNodeId(), NodeState.UNHEALTHY); .handle(new RMNodeStatusEvent(nodeId3, nodeStatus, null));
rm.waitForState(nodeId3, NodeState.UNHEALTHY);
ClientResponse response = ClientResponse response =
r.path("ws").path("v1").path("cluster").path("nodes") r.path("ws").path("v1").path("cluster").path("nodes")
@ -163,14 +165,38 @@ public void testNodesDefaultWithUnHealthyNode() throws JSONException,
assertEquals("incorrect number of elements", 3, nodeArray.length()); assertEquals("incorrect number of elements", 3, nodeArray.length());
} }
private RMNode getRunningRMNode(String host, int port, int memory) {
RMNodeImpl rmnode1 = getNewRMNode(host, port, memory);
sendStartedEvent(rmnode1);
return rmnode1;
}
private void sendStartedEvent(RMNode node) {
((RMNodeImpl) node)
.handle(new RMNodeStartedEvent(node.getNodeID(), null, null));
}
private void sendLostEvent(RMNode node) {
((RMNodeImpl) node)
.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.EXPIRE));
}
private RMNodeImpl getNewRMNode(String host, int port, int memory) {
NodeId nodeId = NodeId.newInstance(host, port);
RMNodeImpl nodeImpl = new RMNodeImpl(nodeId, rm.getRMContext(),
nodeId.getHost(), nodeId.getPort(), nodeId.getPort() + 1,
RackResolver.resolve(nodeId.getHost()), Resource.newInstance(memory, 4),
YarnVersionInfo.getVersion());
rm.getRMContext().getRMNodes().put(nodeId, nodeImpl);
return nodeImpl;
}
@Test @Test
public void testNodesQueryNew() throws JSONException, Exception { public void testNodesQueryNew() throws JSONException, Exception {
WebResource r = resource(); WebResource r = resource();
MockNM nm1 = rm.registerNode("h1:1234", 5120); getRunningRMNode("h1", 1234, 5120);
MockNM nm2 = rm.registerNode("h2:1235", 5121); // h2 will be in NEW state
rm.sendNodeStarted(nm1); RMNode rmnode2 = getNewRMNode("h2", 1235, 5121);
rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.waitForState(nm2.getNodeId(), NodeState.NEW);
ClientResponse response = r.path("ws").path("v1").path("cluster") ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").queryParam("states", NodeState.NEW.toString()) .path("nodes").queryParam("states", NodeState.NEW.toString())
@ -185,14 +211,14 @@ public void testNodesQueryNew() throws JSONException, Exception {
assertEquals("incorrect number of elements", 1, nodeArray.length()); assertEquals("incorrect number of elements", 1, nodeArray.length());
JSONObject info = nodeArray.getJSONObject(0); JSONObject info = nodeArray.getJSONObject(0);
verifyNodeInfo(info, nm2); verifyNodeInfo(info, rmnode2);
} }
@Test @Test
public void testNodesQueryStateNone() throws JSONException, Exception { public void testNodesQueryStateNone() throws JSONException, Exception {
WebResource r = resource(); WebResource r = resource();
rm.registerNode("h1:1234", 5120); getNewRMNode("h1", 1234, 5120);
rm.registerNode("h2:1235", 5121); getNewRMNode("h2", 1235, 5121);
ClientResponse response = r.path("ws").path("v1").path("cluster") ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes") .path("nodes")
@ -207,8 +233,8 @@ public void testNodesQueryStateNone() throws JSONException, Exception {
@Test @Test
public void testNodesQueryStateInvalid() throws JSONException, Exception { public void testNodesQueryStateInvalid() throws JSONException, Exception {
WebResource r = resource(); WebResource r = resource();
rm.registerNode("h1:1234", 5120); getNewRMNode("h1", 1234, 5120);
rm.registerNode("h2:1235", 5121); getNewRMNode("h2", 1235, 5121);
try { try {
r.path("ws").path("v1").path("cluster").path("nodes") r.path("ws").path("v1").path("cluster").path("nodes")
@ -238,22 +264,16 @@ public void testNodesQueryStateInvalid() throws JSONException, Exception {
WebServicesTestUtils.checkStringMatch("exception classname", WebServicesTestUtils.checkStringMatch("exception classname",
"java.lang.IllegalArgumentException", classname); "java.lang.IllegalArgumentException", classname);
} finally {
rm.stop();
} }
} }
@Test @Test
public void testNodesQueryStateLost() throws JSONException, Exception { public void testNodesQueryStateLost() throws JSONException, Exception {
WebResource r = resource(); WebResource r = resource();
MockNM nm1 = rm.registerNode("h1:1234", 5120); RMNode rmnode1 = getRunningRMNode("h1", 1234, 5120);
MockNM nm2 = rm.registerNode("h2:1234", 5120); sendLostEvent(rmnode1);
rm.sendNodeStarted(nm1); RMNode rmnode2 = getRunningRMNode("h2", 1235, 5121);
rm.sendNodeStarted(nm2); sendLostEvent(rmnode2);
rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.waitForState(nm2.getNodeId(), NodeState.RUNNING);
rm.sendNodeLost(nm1);
rm.sendNodeLost(nm2);
ClientResponse response = r.path("ws").path("v1").path("cluster") ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").queryParam("states", NodeState.LOST.toString()) .path("nodes").queryParam("states", NodeState.LOST.toString())
@ -280,14 +300,9 @@ public void testNodesQueryStateLost() throws JSONException, Exception {
@Test @Test
public void testSingleNodeQueryStateLost() throws JSONException, Exception { public void testSingleNodeQueryStateLost() throws JSONException, Exception {
WebResource r = resource(); WebResource r = resource();
MockNM nm1 = rm.registerNode("h1:1234", 5120); getRunningRMNode("h1", 1234, 5120);
MockNM nm2 = rm.registerNode("h2:1234", 5120); RMNode rmnode2 = getRunningRMNode("h2", 1234, 5121);
rm.sendNodeStarted(nm1); sendLostEvent(rmnode2);
rm.sendNodeStarted(nm2);
rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.waitForState(nm2.getNodeId(), NodeState.RUNNING);
rm.sendNodeLost(nm1);
rm.sendNodeLost(nm2);
ClientResponse response = r.path("ws").path("v1").path("cluster") ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").path("h2:1234").accept(MediaType.APPLICATION_JSON) .path("nodes").path("h2:1234").accept(MediaType.APPLICATION_JSON)
@ -300,8 +315,8 @@ public void testSingleNodeQueryStateLost() throws JSONException, Exception {
assertEquals("Incorrect Node Information.", "h2:1234", id); assertEquals("Incorrect Node Information.", "h2:1234", id);
NodeId nodeId = NodeId.newInstance("h2", 1234); RMNode rmNode =
RMNode rmNode = rm.getRMContext().getInactiveRMNodes().get(nodeId); rm.getRMContext().getInactiveRMNodes().get(rmnode2.getNodeID());
WebServicesTestUtils.checkStringMatch("nodeHTTPAddress", "", WebServicesTestUtils.checkStringMatch("nodeHTTPAddress", "",
info.getString("nodeHTTPAddress")); info.getString("nodeHTTPAddress"));
WebServicesTestUtils.checkStringMatch("state", WebServicesTestUtils.checkStringMatch("state",
@ -311,11 +326,9 @@ public void testSingleNodeQueryStateLost() throws JSONException, Exception {
@Test @Test
public void testNodesQueryRunning() throws JSONException, Exception { public void testNodesQueryRunning() throws JSONException, Exception {
WebResource r = resource(); WebResource r = resource();
MockNM nm1 = rm.registerNode("h1:1234", 5120); getRunningRMNode("h1", 1234, 5120);
MockNM nm2 = rm.registerNode("h2:1235", 5121); // h2 will be in NEW state
rm.sendNodeStarted(nm1); getNewRMNode("h2", 1235, 5121);
rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.waitForState(nm2.getNodeId(), NodeState.NEW);
ClientResponse response = r.path("ws").path("v1").path("cluster") ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").queryParam("states", "running") .path("nodes").queryParam("states", "running")
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
@ -331,11 +344,9 @@ public void testNodesQueryRunning() throws JSONException, Exception {
@Test @Test
public void testNodesQueryHealthyFalse() throws JSONException, Exception { public void testNodesQueryHealthyFalse() throws JSONException, Exception {
WebResource r = resource(); WebResource r = resource();
MockNM nm1 = rm.registerNode("h1:1234", 5120); getRunningRMNode("h1", 1234, 5120);
MockNM nm2 = rm.registerNode("h2:1235", 5121); // h2 will be in NEW state
rm.sendNodeStarted(nm1); getNewRMNode("h2", 1235, 5121);
rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.waitForState(nm2.getNodeId(), NodeState.NEW);
ClientResponse response = r.path("ws").path("v1").path("cluster") ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").queryParam("states", "UNHEALTHY") .path("nodes").queryParam("states", "UNHEALTHY")
.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
@ -348,12 +359,8 @@ public void testNodesQueryHealthyFalse() throws JSONException, Exception {
public void testNodesHelper(String path, String media) throws JSONException, public void testNodesHelper(String path, String media) throws JSONException,
Exception { Exception {
WebResource r = resource(); WebResource r = resource();
MockNM nm1 = rm.registerNode("h1:1234", 5120); RMNode rmnode1 = getRunningRMNode("h1", 1234, 5120);
MockNM nm2 = rm.registerNode("h2:1235", 5121); RMNode rmnode2 = getRunningRMNode("h2", 1235, 5121);
rm.sendNodeStarted(nm1);
rm.sendNodeStarted(nm2);
rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
rm.waitForState(nm2.getNodeId(), NodeState.RUNNING);
ClientResponse response = r.path("ws").path("v1").path("cluster") ClientResponse response = r.path("ws").path("v1").path("cluster")
.path(path).accept(media).get(ClientResponse.class); .path(path).accept(media).get(ClientResponse.class);
@ -368,36 +375,36 @@ public void testNodesHelper(String path, String media) throws JSONException,
String id = info.get("id").toString(); String id = info.get("id").toString();
if (id.matches("h1:1234")) { if (id.matches("h1:1234")) {
verifyNodeInfo(info, nm1); verifyNodeInfo(info, rmnode1);
verifyNodeInfo(nodeArray.getJSONObject(1), nm2); verifyNodeInfo(nodeArray.getJSONObject(1), rmnode2);
} else { } else {
verifyNodeInfo(info, nm2); verifyNodeInfo(info, rmnode2);
verifyNodeInfo(nodeArray.getJSONObject(1), nm1); verifyNodeInfo(nodeArray.getJSONObject(1), rmnode1);
} }
} }
@Test @Test
public void testSingleNode() throws JSONException, Exception { public void testSingleNode() throws JSONException, Exception {
rm.registerNode("h1:1234", 5120); getRunningRMNode("h1", 1234, 5120);
MockNM nm2 = rm.registerNode("h2:1235", 5121); RMNode rmnode2 = getRunningRMNode("h2", 1235, 5121);
testSingleNodeHelper("h2:1235", nm2, MediaType.APPLICATION_JSON); testSingleNodeHelper("h2:1235", rmnode2, MediaType.APPLICATION_JSON);
} }
@Test @Test
public void testSingleNodeSlash() throws JSONException, Exception { public void testSingleNodeSlash() throws JSONException, Exception {
MockNM nm1 = rm.registerNode("h1:1234", 5120); RMNode rmnode1 = getRunningRMNode("h1", 1234, 5120);
rm.registerNode("h2:1235", 5121); getRunningRMNode("h2", 1235, 5121);
testSingleNodeHelper("h1:1234/", nm1, MediaType.APPLICATION_JSON); testSingleNodeHelper("h1:1234/", rmnode1, MediaType.APPLICATION_JSON);
} }
@Test @Test
public void testSingleNodeDefault() throws JSONException, Exception { public void testSingleNodeDefault() throws JSONException, Exception {
MockNM nm1 = rm.registerNode("h1:1234", 5120); RMNode rmnode1 = getRunningRMNode("h1", 1234, 5120);
rm.registerNode("h2:1235", 5121); getRunningRMNode("h2", 1235, 5121);
testSingleNodeHelper("h1:1234/", nm1, ""); testSingleNodeHelper("h1:1234/", rmnode1, "");
} }
public void testSingleNodeHelper(String nodeid, MockNM nm, String media) public void testSingleNodeHelper(String nodeid, RMNode nm, String media)
throws JSONException, Exception { throws JSONException, Exception {
WebResource r = resource(); WebResource r = resource();
ClientResponse response = r.path("ws").path("v1").path("cluster") ClientResponse response = r.path("ws").path("v1").path("cluster")
@ -412,8 +419,10 @@ public void testSingleNodeHelper(String nodeid, MockNM nm, String media)
@Test @Test
public void testNonexistNode() throws JSONException, Exception { public void testNonexistNode() throws JSONException, Exception {
rm.registerNode("h1:1234", 5120); // add h1 node in NEW state
rm.registerNode("h2:1235", 5121); getNewRMNode("h1", 1234, 5120);
// add h2 node in NEW state
getNewRMNode("h2", 1235, 5121);
WebResource r = resource(); WebResource r = resource();
try { try {
r.path("ws").path("v1").path("cluster").path("nodes") r.path("ws").path("v1").path("cluster").path("nodes")
@ -433,16 +442,14 @@ public void testNonexistNode() throws JSONException, Exception {
String classname = exception.getString("javaClassName"); String classname = exception.getString("javaClassName");
verifyNonexistNodeException(message, type, classname); verifyNonexistNodeException(message, type, classname);
} finally {
rm.stop();
} }
} }
// test that the exception output defaults to JSON // test that the exception output defaults to JSON
@Test @Test
public void testNonexistNodeDefault() throws JSONException, Exception { public void testNonexistNodeDefault() throws JSONException, Exception {
rm.registerNode("h1:1234", 5120); getNewRMNode("h1", 1234, 5120);
rm.registerNode("h2:1235", 5121); getNewRMNode("h2", 1235, 5121);
WebResource r = resource(); WebResource r = resource();
try { try {
r.path("ws").path("v1").path("cluster").path("nodes") r.path("ws").path("v1").path("cluster").path("nodes")
@ -460,16 +467,14 @@ public void testNonexistNodeDefault() throws JSONException, Exception {
String type = exception.getString("exception"); String type = exception.getString("exception");
String classname = exception.getString("javaClassName"); String classname = exception.getString("javaClassName");
verifyNonexistNodeException(message, type, classname); verifyNonexistNodeException(message, type, classname);
} finally {
rm.stop();
} }
} }
// test that the exception output works in XML // test that the exception output works in XML
@Test @Test
public void testNonexistNodeXML() throws JSONException, Exception { public void testNonexistNodeXML() throws JSONException, Exception {
rm.registerNode("h1:1234", 5120); getNewRMNode("h1", 1234, 5120);
rm.registerNode("h2:1235", 5121); getNewRMNode("h2", 1235, 5121);
WebResource r = resource(); WebResource r = resource();
try { try {
r.path("ws").path("v1").path("cluster").path("nodes") r.path("ws").path("v1").path("cluster").path("nodes")
@ -495,8 +500,6 @@ public void testNonexistNodeXML() throws JSONException, Exception {
String classname = WebServicesTestUtils.getXmlString(element, String classname = WebServicesTestUtils.getXmlString(element,
"javaClassName"); "javaClassName");
verifyNonexistNodeException(message, type, classname); verifyNonexistNodeException(message, type, classname);
} finally {
rm.stop();
} }
} }
@ -511,8 +514,8 @@ private void verifyNonexistNodeException(String message, String type, String cla
@Test @Test
public void testInvalidNode() throws JSONException, Exception { public void testInvalidNode() throws JSONException, Exception {
rm.registerNode("h1:1234", 5120); getNewRMNode("h1", 1234, 5120);
rm.registerNode("h2:1235", 5121); getNewRMNode("h2", 1235, 5121);
WebResource r = resource(); WebResource r = resource();
try { try {
@ -538,16 +541,13 @@ public void testInvalidNode() throws JSONException, Exception {
"IllegalArgumentException", type); "IllegalArgumentException", type);
WebServicesTestUtils.checkStringMatch("exception classname", WebServicesTestUtils.checkStringMatch("exception classname",
"java.lang.IllegalArgumentException", classname); "java.lang.IllegalArgumentException", classname);
} finally {
rm.stop();
} }
} }
@Test @Test
public void testNodesXML() throws JSONException, Exception { public void testNodesXML() throws JSONException, Exception {
rm.start();
WebResource r = resource(); WebResource r = resource();
MockNM nm1 = rm.registerNode("h1:1234", 5120); RMNodeImpl rmnode1 = getNewRMNode("h1", 1234, 5120);
// MockNM nm2 = rm.registerNode("h2:1235", 5121); // MockNM nm2 = rm.registerNode("h2:1235", 5121);
ClientResponse response = r.path("ws").path("v1").path("cluster") ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").accept(MediaType.APPLICATION_XML) .path("nodes").accept(MediaType.APPLICATION_XML)
@ -563,15 +563,14 @@ public void testNodesXML() throws JSONException, Exception {
assertEquals("incorrect number of elements", 1, nodesApps.getLength()); assertEquals("incorrect number of elements", 1, nodesApps.getLength());
NodeList nodes = dom.getElementsByTagName("node"); NodeList nodes = dom.getElementsByTagName("node");
assertEquals("incorrect number of elements", 1, nodes.getLength()); assertEquals("incorrect number of elements", 1, nodes.getLength());
verifyNodesXML(nodes, nm1); verifyNodesXML(nodes, rmnode1);
rm.stop();
} }
@Test @Test
public void testSingleNodesXML() throws JSONException, Exception { public void testSingleNodesXML() throws JSONException, Exception {
rm.start();
WebResource r = resource(); WebResource r = resource();
MockNM nm1 = rm.registerNode("h1:1234", 5120); // add h2 node in NEW state
RMNodeImpl rmnode1 = getNewRMNode("h1", 1234, 5120);
// MockNM nm2 = rm.registerNode("h2:1235", 5121); // MockNM nm2 = rm.registerNode("h2:1235", 5121);
ClientResponse response = r.path("ws").path("v1").path("cluster") ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").path("h1:1234").accept(MediaType.APPLICATION_XML) .path("nodes").path("h1:1234").accept(MediaType.APPLICATION_XML)
@ -587,16 +586,14 @@ public void testSingleNodesXML() throws JSONException, Exception {
Document dom = db.parse(is); Document dom = db.parse(is);
NodeList nodes = dom.getElementsByTagName("node"); NodeList nodes = dom.getElementsByTagName("node");
assertEquals("incorrect number of elements", 1, nodes.getLength()); assertEquals("incorrect number of elements", 1, nodes.getLength());
verifyNodesXML(nodes, nm1); verifyNodesXML(nodes, rmnode1);
rm.stop();
} }
@Test @Test
public void testNodes2XML() throws JSONException, Exception { public void testNodes2XML() throws JSONException, Exception {
rm.start();
WebResource r = resource(); WebResource r = resource();
rm.registerNode("h1:1234", 5120); getNewRMNode("h1", 1234, 5120);
rm.registerNode("h2:1235", 5121); getNewRMNode("h2", 1235, 5121);
ClientResponse response = r.path("ws").path("v1").path("cluster") ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").accept(MediaType.APPLICATION_XML) .path("nodes").accept(MediaType.APPLICATION_XML)
.get(ClientResponse.class); .get(ClientResponse.class);
@ -612,20 +609,17 @@ public void testNodes2XML() throws JSONException, Exception {
assertEquals("incorrect number of elements", 1, nodesApps.getLength()); assertEquals("incorrect number of elements", 1, nodesApps.getLength());
NodeList nodes = dom.getElementsByTagName("node"); NodeList nodes = dom.getElementsByTagName("node");
assertEquals("incorrect number of elements", 2, nodes.getLength()); assertEquals("incorrect number of elements", 2, nodes.getLength());
rm.stop();
} }
@Test @Test
public void testQueryAll() throws Exception { public void testQueryAll() throws Exception {
WebResource r = resource(); WebResource r = resource();
MockNM nm1 = rm.registerNode("h1:1234", 5120); getRunningRMNode("h1", 1234, 5120);
MockNM nm2 = rm.registerNode("h2:1235", 5121); // add h2 node in NEW state
MockNM nm3 = rm.registerNode("h3:1236", 5122); getNewRMNode("h2", 1235, 5121);
rm.sendNodeStarted(nm1); // add lost node
rm.sendNodeStarted(nm3); RMNode nm3 = getRunningRMNode("h3", 1236, 5122);
rm.waitForState(nm1.getNodeId(), NodeState.RUNNING); sendLostEvent(nm3);
rm.waitForState(nm2.getNodeId(), NodeState.NEW);
rm.sendNodeLost(nm3);
ClientResponse response = r.path("ws").path("v1").path("cluster") ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes") .path("nodes")
@ -643,23 +637,22 @@ public void testQueryAll() throws Exception {
@Test @Test
public void testNodesResourceUtilization() throws JSONException, Exception { public void testNodesResourceUtilization() throws JSONException, Exception {
WebResource r = resource(); WebResource r = resource();
MockNM nm1 = rm.registerNode("h1:1234", 5120); RMNode rmnode1 = getRunningRMNode("h1", 1234, 5120);
rm.sendNodeStarted(nm1); NodeId nodeId1 = rmnode1.getNodeID();
rm.waitForState(nm1.getNodeId(), NodeState.RUNNING);
RMNodeImpl node = (RMNodeImpl) rm.getRMContext().getRMNodes() RMNodeImpl node = (RMNodeImpl) rm.getRMContext().getRMNodes().get(nodeId1);
.get(nm1.getNodeId());
NodeHealthStatus nodeHealth = NodeHealthStatus.newInstance(true, NodeHealthStatus nodeHealth = NodeHealthStatus.newInstance(true,
"test health report", System.currentTimeMillis()); "test health report", System.currentTimeMillis());
ResourceUtilization nodeResource = ResourceUtilization.newInstance(4096, 0, ResourceUtilization nodeResource = ResourceUtilization.newInstance(4096, 0,
(float) 10.5); (float) 10.5);
ResourceUtilization containerResource = ResourceUtilization.newInstance( ResourceUtilization containerResource = ResourceUtilization.newInstance(
2048, 0, (float) 5.05); 2048, 0, (float) 5.05);
NodeStatus nodeStatus = NodeStatus.newInstance(nm1.getNodeId(), 0, NodeStatus nodeStatus =
NodeStatus.newInstance(nodeId1, 0,
new ArrayList<ContainerStatus>(), null, nodeHealth, containerResource, new ArrayList<ContainerStatus>(), null, nodeHealth, containerResource,
nodeResource, null); nodeResource, null);
node.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus, null)); node.handle(new RMNodeStatusEvent(nodeId1, nodeStatus, null));
rm.waitForState(nm1.getNodeId(), NodeState.RUNNING); rm.waitForState(nodeId1, NodeState.RUNNING);
ClientResponse response = r.path("ws").path("v1").path("cluster") ClientResponse response = r.path("ws").path("v1").path("cluster")
.path("nodes").accept(MediaType.APPLICATION_JSON) .path("nodes").accept(MediaType.APPLICATION_JSON)
@ -675,10 +668,11 @@ public void testNodesResourceUtilization() throws JSONException, Exception {
JSONObject info = nodeArray.getJSONObject(0); JSONObject info = nodeArray.getJSONObject(0);
// verify the resource utilization // verify the resource utilization
verifyNodeInfo(info, nm1); verifyNodeInfo(info, rmnode1);
} }
public void verifyNodesXML(NodeList nodes, MockNM nm) throws JSONException, public void verifyNodesXML(NodeList nodes, RMNode nm)
throws JSONException,
Exception { Exception {
for (int i = 0; i < nodes.getLength(); i++) { for (int i = 0; i < nodes.getLength(); i++) {
Element element = (Element) nodes.item(i); Element element = (Element) nodes.item(i);
@ -707,7 +701,7 @@ public void verifyNodesXML(NodeList nodes, MockNM nm) throws JSONException,
} }
} }
public void verifyNodeInfo(JSONObject nodeInfo, MockNM nm) public void verifyNodeInfo(JSONObject nodeInfo, RMNode nm)
throws JSONException, Exception { throws JSONException, Exception {
assertEquals("incorrect number of elements", 14, nodeInfo.length()); assertEquals("incorrect number of elements", 14, nodeInfo.length());
@ -729,7 +723,7 @@ public void verifyNodeInfo(JSONObject nodeInfo, MockNM nm)
resourceInfo.getDouble("containersCPUUsage")); resourceInfo.getDouble("containersCPUUsage"));
} }
public void verifyNodeInfoGeneric(MockNM nm, String state, String rack, public void verifyNodeInfoGeneric(RMNode node, String state, String rack,
String id, String nodeHostName, String id, String nodeHostName,
String nodeHTTPAddress, long lastHealthUpdate, String healthReport, String nodeHTTPAddress, long lastHealthUpdate, String healthReport,
int numContainers, long usedMemoryMB, long availMemoryMB, long usedVirtualCores, int numContainers, long usedMemoryMB, long availMemoryMB, long usedVirtualCores,
@ -739,20 +733,20 @@ public void verifyNodeInfoGeneric(MockNM nm, String state, String rack,
double containersCPUUsage) double containersCPUUsage)
throws JSONException, Exception { throws JSONException, Exception {
RMNode node = rm.getRMContext().getRMNodes().get(nm.getNodeId());
ResourceScheduler sched = rm.getResourceScheduler(); ResourceScheduler sched = rm.getResourceScheduler();
SchedulerNodeReport report = sched.getNodeReport(nm.getNodeId()); SchedulerNodeReport report = sched.getNodeReport(node.getNodeID());
WebServicesTestUtils.checkStringMatch("state", node.getState().toString(), WebServicesTestUtils.checkStringMatch("state", node.getState().toString(),
state); state);
WebServicesTestUtils.checkStringMatch("rack", node.getRackName(), rack); WebServicesTestUtils.checkStringMatch("rack", node.getRackName(), rack);
WebServicesTestUtils.checkStringMatch("id", nm.getNodeId().toString(), id); WebServicesTestUtils.checkStringMatch("id", node.getNodeID().toString(),
WebServicesTestUtils.checkStringMatch("nodeHostName", nm.getNodeId() id);
.getHost(), nodeHostName); WebServicesTestUtils.checkStringMatch("nodeHostName",
node.getNodeID().getHost(), nodeHostName);
WebServicesTestUtils.checkStringMatch("healthReport", WebServicesTestUtils.checkStringMatch("healthReport",
String.valueOf(node.getHealthReport()), healthReport); String.valueOf(node.getHealthReport()), healthReport);
String expectedHttpAddress = nm.getNodeId().getHost() + ":" String expectedHttpAddress =
+ nm.getHttpPort(); node.getNodeID().getHost() + ":" + node.getHttpPort();
WebServicesTestUtils.checkStringMatch("nodeHTTPAddress", WebServicesTestUtils.checkStringMatch("nodeHTTPAddress",
expectedHttpAddress, nodeHTTPAddress); expectedHttpAddress, nodeHTTPAddress);
WebServicesTestUtils.checkStringMatch("version", WebServicesTestUtils.checkStringMatch("version",