浏览代码

YARN-4292. ResourceUtilization should be a part of NodeInfo REST API. (Sunil G via wangda)

Wangda Tan 9 年之前
父节点
当前提交
a2c3bfc8c1

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -586,6 +586,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4132. Separate configs for nodemanager to resourcemanager connection
     timeout and retries (Chang Li via jlowe)
 
+    YARN-4292. ResourceUtilization should be a part of NodeInfo REST API.
+    (Sunil G via wangda)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeInfo.java

@@ -50,6 +50,7 @@ public class NodeInfo {
   protected long usedVirtualCores;
   protected long availableVirtualCores;
   protected ArrayList<String> nodeLabels = new ArrayList<String>();
+  protected ResourceUtilizationInfo resourceUtilization;
 
   public NodeInfo() {
   } // JAXB needs this
@@ -82,6 +83,9 @@ public class NodeInfo {
       nodeLabels.addAll(labelSet);
       Collections.sort(nodeLabels);
     }
+
+    // update node and containers resource utilization
+    this.resourceUtilization = new ResourceUtilizationInfo(ni);
   }
 
   public String getRack() {
@@ -139,4 +143,8 @@ public class NodeInfo {
   public ArrayList<String> getNodeLabels() {
     return this.nodeLabels;
   }
+
+  public ResourceUtilizationInfo getResourceUtilization() {
+    return this.resourceUtilization;
+  }
 }

+ 89 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceUtilizationInfo.java

@@ -0,0 +1,89 @@
+/**
+ * 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.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.yarn.server.api.records.ResourceUtilization;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+
+/**
+ * DAO object represents resource utilization of node and containers.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ResourceUtilizationInfo {
+
+  protected int nodePhysicalMemoryMB;
+  protected int nodeVirtualMemoryMB;
+  protected double nodeCPUUsage;
+  protected int aggregatedContainersPhysicalMemoryMB;
+  protected int aggregatedContainersVirtualMemoryMB;
+  protected double containersCPUUsage;
+
+  public ResourceUtilizationInfo() {
+  } // JAXB needs this
+
+  public ResourceUtilizationInfo(RMNode ni) {
+
+    // update node and containers resource utilization
+    ResourceUtilization nodeUtilization = ni.getNodeUtilization();
+    if (nodeUtilization != null) {
+      this.nodePhysicalMemoryMB = nodeUtilization.getPhysicalMemory();
+      this.nodeVirtualMemoryMB = nodeUtilization.getVirtualMemory();
+      this.nodeCPUUsage = nodeUtilization.getCPU();
+    }
+
+    ResourceUtilization containerAggrUtilization = ni
+        .getAggregatedContainersUtilization();
+    if (containerAggrUtilization != null) {
+      this.aggregatedContainersPhysicalMemoryMB = containerAggrUtilization
+          .getPhysicalMemory();
+      this.aggregatedContainersVirtualMemoryMB = containerAggrUtilization
+          .getVirtualMemory();
+      this.containersCPUUsage = containerAggrUtilization.getCPU();
+    }
+  }
+
+  public int getNodePhysicalMemoryMB() {
+    return nodePhysicalMemoryMB;
+  }
+
+  public int getNodeVirtualMemoryMB() {
+    return nodeVirtualMemoryMB;
+  }
+
+  public int getAggregatedContainersPhysicalMemoryMB() {
+    return aggregatedContainersPhysicalMemoryMB;
+  }
+
+  public int getAggregatedContainersVirtualMemoryMB() {
+    return aggregatedContainersVirtualMemoryMB;
+  }
+
+  public double getNodeCPUUsage() {
+    return nodeCPUUsage;
+  }
+
+  public double getContainersCPUUsage() {
+    return containersCPUUsage;
+  }
+}

+ 74 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java

@@ -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