Переглянути джерело

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
     YARN-4132. Separate configs for nodemanager to resourcemanager connection
     timeout and retries (Chang Li via jlowe)
     timeout and retries (Chang Li via jlowe)
 
 
+    YARN-4292. ResourceUtilization should be a part of NodeInfo REST API.
+    (Sunil G via wangda)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not
     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 usedVirtualCores;
   protected long availableVirtualCores;
   protected long availableVirtualCores;
   protected ArrayList<String> nodeLabels = new ArrayList<String>();
   protected ArrayList<String> nodeLabels = new ArrayList<String>();
+  protected ResourceUtilizationInfo resourceUtilization;
 
 
   public NodeInfo() {
   public NodeInfo() {
   } // JAXB needs this
   } // JAXB needs this
@@ -82,6 +83,9 @@ public class NodeInfo {
       nodeLabels.addAll(labelSet);
       nodeLabels.addAll(labelSet);
       Collections.sort(nodeLabels);
       Collections.sort(nodeLabels);
     }
     }
+
+    // update node and containers resource utilization
+    this.resourceUtilization = new ResourceUtilizationInfo(ni);
   }
   }
 
 
   public String getRack() {
   public String getRack() {
@@ -139,4 +143,8 @@ public class NodeInfo {
   public ArrayList<String> getNodeLabels() {
   public ArrayList<String> getNodeLabels() {
     return this.nodeLabels;
     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.api.records.NodeState;
 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.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 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;
@@ -639,6 +640,44 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
     assertEquals("incorrect number of elements", 3, nodeArray.length());
     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,
   public void verifyNodesXML(NodeList nodes, MockNM nm) throws JSONException,
       Exception {
       Exception {
     for (int i = 0; i < nodes.getLength(); i++) {
     for (int i = 0; i < nodes.getLength(); i++) {
@@ -656,14 +695,23 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
           WebServicesTestUtils.getXmlLong(element, "availMemoryMB"),
           WebServicesTestUtils.getXmlLong(element, "availMemoryMB"),
           WebServicesTestUtils.getXmlLong(element, "usedVirtualCores"),
           WebServicesTestUtils.getXmlLong(element, "usedVirtualCores"),
           WebServicesTestUtils.getXmlLong(element,  "availableVirtualCores"),
           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)
   public void verifyNodeInfo(JSONObject nodeInfo, MockNM nm)
       throws JSONException, Exception {
       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"),
     verifyNodeInfoGeneric(nm, nodeInfo.getString("state"),
         nodeInfo.getString("rack"),
         nodeInfo.getString("rack"),
         nodeInfo.getString("id"), nodeInfo.getString("nodeHostName"),
         nodeInfo.getString("id"), nodeInfo.getString("nodeHostName"),
@@ -672,15 +720,23 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
         nodeInfo.getString("healthReport"), nodeInfo.getInt("numContainers"),
         nodeInfo.getString("healthReport"), nodeInfo.getInt("numContainers"),
         nodeInfo.getLong("usedMemoryMB"), nodeInfo.getLong("availMemoryMB"),
         nodeInfo.getLong("usedMemoryMB"), nodeInfo.getLong("availMemoryMB"),
         nodeInfo.getLong("usedVirtualCores"), nodeInfo.getLong("availableVirtualCores"),
         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,
   public void verifyNodeInfoGeneric(MockNM nm, 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, 
-      long availVirtualCores, String version)
+      long availVirtualCores, String version, int nodePhysicalMemoryMB,
+      int nodeVirtualMemoryMB, double nodeCPUUsage,
+      int containersPhysicalMemoryMB, int containersVirtualMemoryMB,
+      double containersCPUUsage)
       throws JSONException, Exception {
       throws JSONException, Exception {
 
 
     RMNode node = rm.getRMContext().getRMNodes().get(nm.getNodeId());
     RMNode node = rm.getRMContext().getRMNodes().get(nm.getNodeId());
@@ -701,6 +757,19 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
         expectedHttpAddress, nodeHTTPAddress);
         expectedHttpAddress, nodeHTTPAddress);
     WebServicesTestUtils.checkStringMatch("version",
     WebServicesTestUtils.checkStringMatch("version",
         node.getNodeManagerVersion(), 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();
     long expectedHealthUpdate = node.getLastHealthReportTime();
     assertEquals("lastHealthUpdate doesn't match, got: " + lastHealthUpdate
     assertEquals("lastHealthUpdate doesn't match, got: " + lastHealthUpdate