|
@@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeState;
|
|
|
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.ResourceUtilization;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
|
@@ -639,6 +640,44 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
|
|
|
assertEquals("incorrect number of elements", 3, nodeArray.length());
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testNodesResourceUtilization() throws JSONException, Exception {
|
|
|
+ WebResource r = resource();
|
|
|
+ MockNM nm1 = rm.registerNode("h1:1234", 5120);
|
|
|
+ rm.sendNodeStarted(nm1);
|
|
|
+ rm.NMwaitForState(nm1.getNodeId(), NodeState.RUNNING);
|
|
|
+
|
|
|
+ RMNodeImpl node = (RMNodeImpl) rm.getRMContext().getRMNodes()
|
|
|
+ .get(nm1.getNodeId());
|
|
|
+ NodeHealthStatus nodeHealth = NodeHealthStatus.newInstance(true,
|
|
|
+ "test health report", System.currentTimeMillis());
|
|
|
+ ResourceUtilization nodeResource = ResourceUtilization.newInstance(4096, 0,
|
|
|
+ (float) 10.5);
|
|
|
+ ResourceUtilization containerResource = ResourceUtilization.newInstance(
|
|
|
+ 2048, 0, (float) 5.05);
|
|
|
+ NodeStatus nodeStatus = NodeStatus.newInstance(nm1.getNodeId(), 0,
|
|
|
+ new ArrayList<ContainerStatus>(), null, nodeHealth, containerResource,
|
|
|
+ nodeResource, null);
|
|
|
+ node.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus, null));
|
|
|
+ rm.NMwaitForState(nm1.getNodeId(), NodeState.RUNNING);
|
|
|
+
|
|
|
+ ClientResponse response = r.path("ws").path("v1").path("cluster")
|
|
|
+ .path("nodes").accept(MediaType.APPLICATION_JSON)
|
|
|
+ .get(ClientResponse.class);
|
|
|
+
|
|
|
+ assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
|
|
|
+ JSONObject json = response.getEntity(JSONObject.class);
|
|
|
+ assertEquals("incorrect number of elements", 1, json.length());
|
|
|
+ JSONObject nodes = json.getJSONObject("nodes");
|
|
|
+ assertEquals("incorrect number of elements", 1, nodes.length());
|
|
|
+ JSONArray nodeArray = nodes.getJSONArray("node");
|
|
|
+ assertEquals("incorrect number of elements", 1, nodeArray.length());
|
|
|
+ JSONObject info = nodeArray.getJSONObject(0);
|
|
|
+
|
|
|
+ // verify the resource utilization
|
|
|
+ verifyNodeInfo(info, nm1);
|
|
|
+ }
|
|
|
+
|
|
|
public void verifyNodesXML(NodeList nodes, MockNM nm) throws JSONException,
|
|
|
Exception {
|
|
|
for (int i = 0; i < nodes.getLength(); i++) {
|
|
@@ -656,14 +695,23 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
|
|
|
WebServicesTestUtils.getXmlLong(element, "availMemoryMB"),
|
|
|
WebServicesTestUtils.getXmlLong(element, "usedVirtualCores"),
|
|
|
WebServicesTestUtils.getXmlLong(element, "availableVirtualCores"),
|
|
|
- WebServicesTestUtils.getXmlString(element, "version"));
|
|
|
+ WebServicesTestUtils.getXmlString(element, "version"),
|
|
|
+ WebServicesTestUtils.getXmlInt(element, "nodePhysicalMemoryMB"),
|
|
|
+ WebServicesTestUtils.getXmlInt(element, "nodeVirtualMemoryMB"),
|
|
|
+ WebServicesTestUtils.getXmlFloat(element, "nodeCPUUsage"),
|
|
|
+ WebServicesTestUtils.getXmlInt(element,
|
|
|
+ "aggregatedContainersPhysicalMemoryMB"),
|
|
|
+ WebServicesTestUtils.getXmlInt(element,
|
|
|
+ "aggregatedContainersVirtualMemoryMB"),
|
|
|
+ WebServicesTestUtils.getXmlFloat(element, "containersCPUUsage"));
|
|
|
}
|
|
|
}
|
|
|
|
|
|
public void verifyNodeInfo(JSONObject nodeInfo, MockNM nm)
|
|
|
throws JSONException, Exception {
|
|
|
- assertEquals("incorrect number of elements", 13, nodeInfo.length());
|
|
|
+ assertEquals("incorrect number of elements", 14, nodeInfo.length());
|
|
|
|
|
|
+ JSONObject resourceInfo = nodeInfo.getJSONObject("resourceUtilization");
|
|
|
verifyNodeInfoGeneric(nm, nodeInfo.getString("state"),
|
|
|
nodeInfo.getString("rack"),
|
|
|
nodeInfo.getString("id"), nodeInfo.getString("nodeHostName"),
|
|
@@ -672,15 +720,23 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
|
|
|
nodeInfo.getString("healthReport"), nodeInfo.getInt("numContainers"),
|
|
|
nodeInfo.getLong("usedMemoryMB"), nodeInfo.getLong("availMemoryMB"),
|
|
|
nodeInfo.getLong("usedVirtualCores"), nodeInfo.getLong("availableVirtualCores"),
|
|
|
- nodeInfo.getString("version"));
|
|
|
-
|
|
|
+ nodeInfo.getString("version"),
|
|
|
+ resourceInfo.getInt("nodePhysicalMemoryMB"),
|
|
|
+ resourceInfo.getInt("nodeVirtualMemoryMB"),
|
|
|
+ resourceInfo.getDouble("nodeCPUUsage"),
|
|
|
+ resourceInfo.getInt("aggregatedContainersPhysicalMemoryMB"),
|
|
|
+ resourceInfo.getInt("aggregatedContainersVirtualMemoryMB"),
|
|
|
+ resourceInfo.getDouble("containersCPUUsage"));
|
|
|
}
|
|
|
|
|
|
public void verifyNodeInfoGeneric(MockNM nm, String state, String rack,
|
|
|
String id, String nodeHostName,
|
|
|
String nodeHTTPAddress, long lastHealthUpdate, String healthReport,
|
|
|
int numContainers, long usedMemoryMB, long availMemoryMB, long usedVirtualCores,
|
|
|
- long availVirtualCores, String version)
|
|
|
+ long availVirtualCores, String version, int nodePhysicalMemoryMB,
|
|
|
+ int nodeVirtualMemoryMB, double nodeCPUUsage,
|
|
|
+ int containersPhysicalMemoryMB, int containersVirtualMemoryMB,
|
|
|
+ double containersCPUUsage)
|
|
|
throws JSONException, Exception {
|
|
|
|
|
|
RMNode node = rm.getRMContext().getRMNodes().get(nm.getNodeId());
|
|
@@ -701,6 +757,19 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
|
|
|
expectedHttpAddress, nodeHTTPAddress);
|
|
|
WebServicesTestUtils.checkStringMatch("version",
|
|
|
node.getNodeManagerVersion(), version);
|
|
|
+ if (node.getNodeUtilization() != null) {
|
|
|
+ ResourceUtilization nodeResource = ResourceUtilization.newInstance(
|
|
|
+ nodePhysicalMemoryMB, nodeVirtualMemoryMB, (float) nodeCPUUsage);
|
|
|
+ assertEquals("nodeResourceUtilization doesn't match",
|
|
|
+ node.getNodeUtilization(), nodeResource);
|
|
|
+ }
|
|
|
+ if (node.getAggregatedContainersUtilization() != null) {
|
|
|
+ ResourceUtilization containerResource = ResourceUtilization.newInstance(
|
|
|
+ containersPhysicalMemoryMB, containersVirtualMemoryMB,
|
|
|
+ (float) containersCPUUsage);
|
|
|
+ assertEquals("containerResourceUtilization doesn't match",
|
|
|
+ node.getAggregatedContainersUtilization(), containerResource);
|
|
|
+ }
|
|
|
|
|
|
long expectedHealthUpdate = node.getLastHealthReportTime();
|
|
|
assertEquals("lastHealthUpdate doesn't match, got: " + lastHealthUpdate
|