Browse Source

YARN-10453. Add partition resource info to get-node-labels and label-mappings api responses. Contributed Akhil PB.

Sunil G 4 years ago
parent
commit
7435604a91

+ 16 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeIDsInfo.java

@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.PartitionInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo;
+
 import java.util.ArrayList;
 import java.util.List;
 
@@ -39,6 +43,9 @@ public class NodeIDsInfo {
   @XmlElement(name="nodes")
   protected ArrayList<String> nodeIDsList = new ArrayList<String>();
 
+  @XmlElement(name = "partitionInfo")
+  private PartitionInfo partitionInfo;
+
   public NodeIDsInfo() {
   } // JAXB needs this
 
@@ -46,7 +53,16 @@ public class NodeIDsInfo {
     this.nodeIDsList.addAll(nodeIdsList);
   }
 
+  public NodeIDsInfo(List<String> nodeIdsList, Resource resource) {
+    this(nodeIdsList);
+    this.partitionInfo = new PartitionInfo(new ResourceInfo(resource));
+  }
+
   public ArrayList<String> getNodeIDs() {
     return nodeIDsList;
   }
+
+  public PartitionInfo getPartitionInfo() {
+    return partitionInfo;
+  }
 }

+ 15 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java

@@ -184,6 +184,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntr
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.PartitionInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.RMQueueAclInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDefinitionInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo;
@@ -209,6 +210,7 @@ import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
 import org.apache.hadoop.yarn.util.AdHocLogDumper;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Times;
+import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
@@ -1296,8 +1298,10 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
       for (NodeId nodeId : entry.getValue()) {
         nodeIdStrList.add(nodeId.toString());
       }
+      Resource resource = rm.getRMContext().getNodeLabelManager()
+          .getResourceByLabel(entry.getKey().getName(), Resources.none());
       ltsMap.put(new NodeLabelInfo(entry.getKey()),
-          new NodeIDsInfo(nodeIdStrList));
+          new NodeIDsInfo(nodeIdStrList, resource));
     }
     return lts;
   }
@@ -1386,9 +1390,17 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
 
     List<NodeLabel> nodeLabels =
         rm.getRMContext().getNodeLabelManager().getClusterNodeLabels();
-    NodeLabelsInfo ret = new NodeLabelsInfo(nodeLabels);
 
-    return ret;
+    ArrayList<NodeLabelInfo> nodeLabelsInfo = new ArrayList<NodeLabelInfo>();
+    for (NodeLabel label: nodeLabels) {
+      Resource resource = rm.getRMContext().getNodeLabelManager()
+          .getResourceByLabel(label.getName(), Resources.none());
+      PartitionInfo partitionInfo =
+          new PartitionInfo(new ResourceInfo(resource));
+      nodeLabelsInfo.add(new NodeLabelInfo(label, partitionInfo));
+    }
+
+    return new NodeLabelsInfo(nodeLabelsInfo);
   }
 
   @POST

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

@@ -30,6 +30,7 @@ public class NodeLabelInfo {
 
   private String name;
   private boolean exclusivity;
+  private PartitionInfo partitionInfo;
 
   public NodeLabelInfo() {
     // JAXB needs this
@@ -50,6 +51,11 @@ public class NodeLabelInfo {
     this.exclusivity = label.isExclusive();
   }
 
+  public NodeLabelInfo(NodeLabel label, PartitionInfo partitionInfo) {
+    this(label);
+    this.partitionInfo = partitionInfo;
+  }
+
   public String getName() {
     return name;
   }
@@ -58,6 +64,10 @@ public class NodeLabelInfo {
     return exclusivity;
   }
 
+  public PartitionInfo getPartitionInfo() {
+    return partitionInfo;
+  }
+
   @Override
   public boolean equals(Object obj) {
     if (this == obj) {

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

@@ -0,0 +1,46 @@
+/**
+ * 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.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * XML element uses to represent partitionInfo.
+ */
+@XmlRootElement(name = "partitionInfo")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class PartitionInfo {
+
+  @XmlElement(name = "resourceAvailable")
+  private ResourceInfo resourceAvailable;
+
+  public PartitionInfo() {
+  }
+
+  public PartitionInfo(ResourceInfo resourceAvailable) {
+    this.resourceAvailable = resourceAvailable;
+  }
+
+  public ResourceInfo getResourceAvailable() {
+    return resourceAvailable;
+  }
+}

+ 65 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -130,7 +131,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("add-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(nlsifo, NodeLabelsInfo.class), MediaType.APPLICATION_JSON)
+            .entity(toJson(nlsifo, NodeLabelsInfo.class),
+                MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
 
     // Verify
@@ -154,7 +156,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("add-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(nlsifo, NodeLabelsInfo.class), MediaType.APPLICATION_JSON)
+            .entity(toJson(nlsifo, NodeLabelsInfo.class),
+                MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
 
     // Verify
@@ -690,6 +693,66 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
     validateJsonExceptionContent(response, expectedmessage);
   }
 
+  @Test
+  public void testNodeLabelPartitionInfo() throws Exception {
+    WebResource r = resource();
+
+    ClientResponse response;
+
+    // Add a node label
+    NodeLabelsInfo nlsifo = new NodeLabelsInfo();
+    nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("a"));
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("add-node-labels").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(nlsifo, NodeLabelsInfo.class), MediaType.APPLICATION_JSON)
+            .post(ClientResponse.class);
+
+    // Verify partition info in get-node-labels
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("get-node-labels").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertEquals(1, nlsifo.getNodeLabels().size());
+    for (NodeLabelInfo nl : nlsifo.getNodeLabelsInfo()) {
+      assertEquals("a", nl.getName());
+      assertTrue(nl.getExclusivity());
+      assertNotNull(nl.getPartitionInfo());
+      assertNotNull(nl.getPartitionInfo().getResourceAvailable());
+    }
+
+    // Add node label to a node
+    MultivaluedMapImpl params = new MultivaluedMapImpl();
+    params.add("labels", "a");
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("nodes").path("nodeId:0")
+            .path("replace-labels")
+            .queryParam("user.name", userName)
+            .queryParams(params)
+            .accept(MediaType.APPLICATION_JSON)
+            .post(ClientResponse.class);
+
+    // Verify partition info in label-mappings
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("label-mappings").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
+    LabelsToNodesInfo ltni = response.getEntity(LabelsToNodesInfo.class);
+    assertEquals(1, ltni.getLabelsToNodes().size());
+    NodeIDsInfo nodes = ltni.getLabelsToNodes().get(
+        new NodeLabelInfo("a"));
+    assertTrue(nodes.getNodeIDs().contains("nodeId:0"));
+    assertNotNull(nodes.getPartitionInfo());
+    assertNotNull(nodes.getPartitionInfo().getResourceAvailable());
+  }
+
   @SuppressWarnings("rawtypes")
   private String toJson(Object nsli, Class klass) throws Exception {
     StringWriter sw = new StringWriter();