You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by su...@apache.org on 2018/09/12 11:14:39 UTC

[01/29] hadoop git commit: YARN-7842. PB changes to carry node-attributes in NM heartbeat. Contributed by Weiwei Yang.

Repository: hadoop
Updated Branches:
  refs/heads/trunk 901156716 -> 5e64e62de


YARN-7842. PB changes to carry node-attributes in NM heartbeat. Contributed by Weiwei Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d9d93e39
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d9d93e39
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d9d93e39

Branch: refs/heads/trunk
Commit: d9d93e392508da2e8b88faefe002128e245fc8f3
Parents: 2475fb0
Author: Weiwei Yang <ww...@apache.org>
Authored: Wed Jan 31 20:28:41 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:00:59 2018 +0530

----------------------------------------------------------------------
 .../protocolrecords/NodeHeartbeatRequest.java   | 17 +++++++
 .../impl/pb/NodeHeartbeatRequestPBImpl.java     | 52 ++++++++++++++++++++
 .../yarn_server_common_service_protos.proto     |  5 ++
 .../protocolrecords/TestProtocolRecords.java    | 12 +++++
 4 files changed, 86 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9d93e39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java
index f238f79..4f99225 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 
 public abstract class NodeHeartbeatRequest {
   
@@ -61,6 +62,18 @@ public abstract class NodeHeartbeatRequest {
     return nodeHeartbeatRequest;
   }
 
+  public static NodeHeartbeatRequest newInstance(NodeStatus nodeStatus,
+      MasterKey lastKnownContainerTokenMasterKey,
+      MasterKey lastKnownNMTokenMasterKey, Set<NodeLabel> nodeLabels,
+      Set<NodeAttribute> nodeAttributes,
+      Map<ApplicationId, AppCollectorData> registeringCollectors) {
+    NodeHeartbeatRequest request = NodeHeartbeatRequest
+        .newInstance(nodeStatus, lastKnownContainerTokenMasterKey,
+            lastKnownNMTokenMasterKey, nodeLabels, registeringCollectors);
+    request.setNodeAttributes(nodeAttributes);
+    return request;
+  }
+
   public abstract NodeStatus getNodeStatus();
   public abstract void setNodeStatus(NodeStatus status);
 
@@ -85,4 +98,8 @@ public abstract class NodeHeartbeatRequest {
 
   public abstract void setRegisteringCollectors(Map<ApplicationId,
       AppCollectorData> appCollectorsMap);
+
+  public abstract Set<NodeAttribute> getNodeAttributes();
+
+  public abstract void setNodeAttributes(Set<NodeAttribute> nodeAttributes);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9d93e39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
index 1ffd223..c59127a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
@@ -27,6 +27,9 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
+import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos;
 import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
@@ -36,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.NodeLabelPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeLabelProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.AppCollectorDataProto;
@@ -60,6 +64,7 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
   private MasterKey lastKnownContainerTokenMasterKey = null;
   private MasterKey lastKnownNMTokenMasterKey = null;
   private Set<NodeLabel> labels = null;
+  private Set<NodeAttribute> attributes = null;
   private List<LogAggregationReport> logAggregationReportsForApps = null;
 
   private Map<ApplicationId, AppCollectorData> registeringCollectors = null;
@@ -115,6 +120,15 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
       }
       builder.setNodeLabels(newBuilder.build());
     }
+    if (this.attributes != null) {
+      builder.clearNodeAttributes();
+      YarnServerCommonServiceProtos.NodeAttributesProto.Builder attBuilder =
+          YarnServerCommonServiceProtos.NodeAttributesProto.newBuilder();
+      for (NodeAttribute attribute : attributes) {
+        attBuilder.addNodeAttributes(convertToProtoFormat(attribute));
+      }
+      builder.setNodeAttributes(attBuilder.build());
+    }
     if (this.logAggregationReportsForApps != null) {
       addLogAggregationStatusForAppsToProto();
     }
@@ -372,6 +386,44 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
   }
 
   @Override
+  public Set<NodeAttribute> getNodeAttributes() {
+    initNodeAttributes();
+    return this.attributes;
+  }
+
+  private void initNodeAttributes() {
+    if (this.attributes != null) {
+      return;
+    }
+    NodeHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasNodeAttributes()) {
+      return;
+    }
+    YarnServerCommonServiceProtos.NodeAttributesProto nodeAttributes =
+        p.getNodeAttributes();
+    attributes = new HashSet<>();
+    for (NodeAttributeProto attributeProto :
+        nodeAttributes.getNodeAttributesList()) {
+      attributes.add(convertFromProtoFormat(attributeProto));
+    }
+  }
+
+  @Override
+  public void setNodeAttributes(Set<NodeAttribute> nodeAttributes) {
+    maybeInitBuilder();
+    builder.clearNodeAttributes();
+    this.attributes = nodeAttributes;
+  }
+
+  private NodeAttributePBImpl convertFromProtoFormat(NodeAttributeProto p) {
+    return new NodeAttributePBImpl(p);
+  }
+
+  private NodeAttributeProto convertToProtoFormat(NodeAttribute attribute) {
+    return ((NodeAttributePBImpl) attribute).getProto();
+  }
+
+  @Override
   public List<LogAggregationReport> getLogAggregationReportsForApps() {
     if (this.logAggregationReportsForApps != null) {
       return this.logAggregationReportsForApps;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9d93e39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
index 387ddb4..0b8c4a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
@@ -58,6 +58,10 @@ message NodeLabelsProto {
   repeated NodeLabelProto nodeLabels = 1;
 }
 
+message NodeAttributesProto {
+  repeated NodeAttributeProto nodeAttributes = 1;
+}
+
 message RegisterNodeManagerRequestProto {
   optional NodeIdProto node_id = 1;
   optional int32 http_port = 3;
@@ -95,6 +99,7 @@ message NodeHeartbeatRequestProto {
   optional NodeLabelsProto nodeLabels = 4;
   repeated LogAggregationReportProto log_aggregation_reports_for_apps = 5;
   repeated AppCollectorDataProto registering_collectors = 6;
+  optional NodeAttributesProto nodeAttributes = 7;
 }
 
 message LogAggregationReportProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d9d93e39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestProtocolRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestProtocolRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestProtocolRecords.java
index 74f19e5..e6e79d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestProtocolRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestProtocolRecords.java
@@ -24,7 +24,9 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
+import com.google.common.collect.Sets;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
@@ -39,6 +41,8 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
@@ -173,6 +177,13 @@ public class TestProtocolRecords {
     nodeStatus.setOpportunisticContainersStatus(opportunisticContainersStatus);
     record.setNodeStatus(nodeStatus);
 
+    Set<NodeAttribute> attributeSet =
+        Sets.newHashSet(NodeAttribute.newInstance("attributeA",
+                NodeAttributeType.STRING, "valueA"),
+            NodeAttribute.newInstance("attributeB",
+                NodeAttributeType.STRING, "valueB"));
+    record.setNodeAttributes(attributeSet);
+
     NodeHeartbeatRequestPBImpl pb = new
         NodeHeartbeatRequestPBImpl(
         ((NodeHeartbeatRequestPBImpl) record).getProto());
@@ -183,6 +194,7 @@ public class TestProtocolRecords {
     Assert.assertEquals(321,
         pb.getNodeStatus().getOpportunisticContainersStatus()
             .getWaitQueueLength());
+    Assert.assertEquals(2, pb.getNodeAttributes().size());
   }
 
   @Test


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[20/29] hadoop git commit: YARN-7871. Node attributes reporting from NM to RM. Contributed by Weiwei Yang.

Posted by su...@apache.org.
YARN-7871. Node attributes reporting from NM to RM. Contributed by Weiwei Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3b3b6efe
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3b3b6efe
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3b3b6efe

Branch: refs/heads/trunk
Commit: 3b3b6efe2103244febfe6b4f61989e92bd7bb08a
Parents: 86d024e
Author: Naganarasimha <na...@apache.org>
Authored: Mon Mar 12 08:05:53 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  10 +-
 .../yarn/nodelabels/NodeAttributesManager.java  |  17 +-
 .../hadoop/yarn/nodelabels/NodeLabelUtil.java   |  19 ++
 .../src/main/resources/yarn-default.xml         |  24 +++
 .../yarn/server/nodemanager/NodeManager.java    |  70 +++++--
 .../server/nodemanager/NodeStatusUpdater.java   |  14 ++
 .../nodemanager/NodeStatusUpdaterImpl.java      |  70 ++++++-
 .../ConfigurationNodeAttributesProvider.java    |  90 +++++++++
 .../server/nodemanager/TestNodeManager.java     |   2 +-
 .../TestNodeStatusUpdaterForLabels.java         |  10 +-
 ...TestConfigurationNodeAttributesProvider.java | 185 +++++++++++++++++++
 .../resourcemanager/ResourceTrackerService.java |  30 +++
 .../nodelabels/NodeAttributesManagerImpl.java   |  52 ++++--
 .../TestResourceTrackerService.java             |  78 ++++++++
 .../nodelabels/TestNodeAttributesManager.java   |  99 ++++++++++
 15 files changed, 718 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 5bae5f4..5ca9998 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -3548,9 +3548,12 @@ public class YarnConfiguration extends Configuration {
   public static final String NM_NODE_LABELS_PROVIDER_CONFIG =
       NM_NODE_LABELS_PREFIX + "provider";
 
+  public static final String NM_NODE_ATTRIBUTES_PROVIDER_CONFIG =
+      NM_NODE_ATTRIBUTES_PREFIX + "provider";
+
   // whitelist names for the yarn.nodemanager.node-labels.provider
-  public static final String CONFIG_NODE_LABELS_PROVIDER = "config";
-  public static final String SCRIPT_NODE_LABELS_PROVIDER = "script";
+  public static final String CONFIG_NODE_DESCRIPTOR_PROVIDER = "config";
+  public static final String SCRIPT_NODE_DESCRIPTOR_PROVIDER = "script";
 
   private static final String NM_NODE_LABELS_PROVIDER_PREFIX =
       NM_NODE_LABELS_PREFIX + "provider.";
@@ -3582,6 +3585,9 @@ public class YarnConfiguration extends Configuration {
   public static final String NM_PROVIDER_CONFIGURED_NODE_PARTITION =
       NM_NODE_LABELS_PROVIDER_PREFIX + "configured-node-partition";
 
+  public static final String NM_PROVIDER_CONFIGURED_NODE_ATTRIBUTES =
+      NM_NODE_ATTRIBUTES_PROVIDER_PREFIX + "configured-node-attributes";
+
   private static final String RM_NODE_LABELS_PREFIX = RM_PREFIX
       + "node-labels.";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
index effda9b..ffa33cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
@@ -35,15 +35,18 @@ public abstract class NodeAttributesManager extends AbstractService {
 
   /**
    * To completely replace the mappings for a given node with the new Set of
-   * Attributes. If the mapping contains an attribute whose type does not match
-   * a previously existing Attribute under the same prefix (name space) then
-   * exception is thrown. Key would be name of the node and value would be set
-   * of Attributes to be mapped.
+   * Attributes which are under a given prefix. If the mapping contains an
+   * attribute whose type does not match a previously existing Attribute
+   * under the same prefix (name space) then exception is thrown.
+   * Key would be name of the node and value would be set of Attributes to
+   * be mapped. If the prefix is null, then all node attributes will be
+   * replaced regardless of what prefix they have.
    *
-   * @param nodeAttributeMapping
-   * @throws IOException
+   * @param prefix node attribute prefix
+   * @param nodeAttributeMapping host name to a set of node attributes mapping
+   * @throws IOException if failed to replace attributes
    */
-  public abstract void replaceNodeAttributes(
+  public abstract void replaceNodeAttributes(String prefix,
       Map<String, Set<NodeAttribute>> nodeAttributeMapping) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
index fdfd0ce..93a27a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import java.io.IOException;
 import java.util.Set;
 import java.util.regex.Pattern;
+import java.util.stream.Collectors;
 
 /**
  * Utility class for all NodeLabel and NodeAttribute operations.
@@ -125,4 +126,22 @@ public final class NodeLabelUtil {
       }
     }
   }
+
+  /**
+   * Filter a set of node attributes by a given prefix. Returns a filtered
+   * set of node attributes whose prefix equals the given prefix.
+   * If the prefix is null or empty, then the original set is returned.
+   * @param attributeSet node attribute set
+   * @param prefix node attribute prefix
+   * @return a filtered set of node attributes
+   */
+  public static Set<NodeAttribute> filterAttributesByPrefix(
+      Set<NodeAttribute> attributeSet, String prefix) {
+    if (Strings.isNullOrEmpty(prefix)) {
+      return attributeSet;
+    }
+    return attributeSet.stream().filter(
+        nodeAttribute -> prefix.equals(nodeAttribute.getAttributePrefix()))
+        .collect(Collectors.toSet());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 65b2a6d..cdc3c09 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -2904,6 +2904,20 @@
   <!-- Distributed Node Attributes Configuration -->
   <property>
     <description>
+      This property determines which provider will be plugged by the
+      node manager to collect node-attributes. Administrators can
+      configure "config", "script" or the class name of the provider.
+      Configured class needs to extend
+      org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeAttributesProvider.
+      If "config" is configured, then "ConfigurationNodeLabelsProvider" and if
+      "script" is configured, then "ScriptBasedNodeAttributesProvider"
+      will be used.
+    </description>
+    <name>yarn.nodemanager.node-attributes.provider</name>
+  </property>
+
+  <property>
+    <description>
       The node attribute script NM runs to collect node attributes.
       Script output Line starting with "NODE_ATTRIBUTE:" will be
       considered as a record of node attribute, attribute name, type
@@ -2941,6 +2955,16 @@
 
   <property>
     <description>
+      When "yarn.nodemanager.node-attributes.provider" is configured with
+      "config" then ConfigurationNodeAttributesProvider fetches node attributes
+      from this parameter.
+    </description>
+    <name>yarn.nodemanager.node-attributes.provider.configured-node-attributes</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
     Timeout in seconds for YARN node graceful decommission.
     This is the maximal time to wait for running containers and applications to complete
     before transition a DECOMMISSIONING node into DECOMMISSIONED.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index b54a6b7..6eda4a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -66,6 +66,9 @@ import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.nodelabels.ConfigurationNodeLabelsProvider;
 import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider;
 import org.apache.hadoop.yarn.server.nodemanager.nodelabels.ScriptBasedNodeLabelsProvider;
+import org.apache.hadoop.yarn.server.nodemanager.nodelabels.ScriptBasedNodeAttributesProvider;
+import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeAttributesProvider;
+import org.apache.hadoop.yarn.server.nodemanager.nodelabels.ConfigurationNodeAttributesProvider;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMLeveldbStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
@@ -123,6 +126,7 @@ public class NodeManager extends CompositeService
   private ApplicationACLsManager aclsManager;
   private NodeHealthCheckerService nodeHealthChecker;
   private NodeLabelsProvider nodeLabelsProvider;
+  private NodeAttributesProvider nodeAttributesProvider;
   private LocalDirsHandlerService dirsHandler;
   private Context context;
   private AsyncDispatcher dispatcher;
@@ -162,14 +166,45 @@ public class NodeManager extends CompositeService
   protected NodeStatusUpdater createNodeStatusUpdater(Context context,
       Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
     return new NodeStatusUpdaterImpl(context, dispatcher, healthChecker,
-        metrics, nodeLabelsProvider);
+        metrics);
   }
 
-  protected NodeStatusUpdater createNodeStatusUpdater(Context context,
-      Dispatcher dispatcher, NodeHealthCheckerService healthChecker,
-      NodeLabelsProvider nodeLabelsProvider) {
-    return new NodeStatusUpdaterImpl(context, dispatcher, healthChecker,
-        metrics, nodeLabelsProvider);
+  protected NodeAttributesProvider createNodeAttributesProvider(
+      Configuration conf) throws IOException {
+    NodeAttributesProvider attributesProvider = null;
+    String providerString =
+        conf.get(YarnConfiguration.NM_NODE_ATTRIBUTES_PROVIDER_CONFIG, null);
+    if (providerString == null || providerString.trim().length() == 0) {
+      return attributesProvider;
+    }
+    switch (providerString.trim().toLowerCase()) {
+    case YarnConfiguration.CONFIG_NODE_DESCRIPTOR_PROVIDER:
+      attributesProvider = new ConfigurationNodeAttributesProvider();
+      break;
+    case YarnConfiguration.SCRIPT_NODE_DESCRIPTOR_PROVIDER:
+      attributesProvider = new ScriptBasedNodeAttributesProvider();
+      break;
+    default:
+      try {
+        Class<? extends NodeAttributesProvider> labelsProviderClass =
+            conf.getClass(YarnConfiguration.NM_NODE_ATTRIBUTES_PROVIDER_CONFIG,
+                null, NodeAttributesProvider.class);
+        attributesProvider = labelsProviderClass.newInstance();
+      } catch (InstantiationException | IllegalAccessException
+          | RuntimeException e) {
+        LOG.error("Failed to create NodeAttributesProvider"
+                + " based on Configuration", e);
+        throw new IOException(
+            "Failed to create NodeAttributesProvider : "
+                + e.getMessage(), e);
+      }
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Distributed Node Attributes is enabled"
+          + " with provider class as : "
+          + attributesProvider.getClass().toString());
+    }
+    return attributesProvider;
   }
 
   protected NodeLabelsProvider createNodeLabelsProvider(Configuration conf)
@@ -182,10 +217,10 @@ public class NodeManager extends CompositeService
       return provider;
     }
     switch (providerString.trim().toLowerCase()) {
-    case YarnConfiguration.CONFIG_NODE_LABELS_PROVIDER:
+    case YarnConfiguration.CONFIG_NODE_DESCRIPTOR_PROVIDER:
       provider = new ConfigurationNodeLabelsProvider();
       break;
-    case YarnConfiguration.SCRIPT_NODE_LABELS_PROVIDER:
+    case YarnConfiguration.SCRIPT_NODE_DESCRIPTOR_PROVIDER:
       provider = new ScriptBasedNodeLabelsProvider();
       break;
     default:
@@ -407,16 +442,19 @@ public class NodeManager extends CompositeService
     ((NMContext)context).setContainerExecutor(exec);
     ((NMContext)context).setDeletionService(del);
 
-    nodeLabelsProvider = createNodeLabelsProvider(conf);
+    nodeStatusUpdater =
+        createNodeStatusUpdater(context, dispatcher, nodeHealthChecker);
 
-    if (null == nodeLabelsProvider) {
-      nodeStatusUpdater =
-          createNodeStatusUpdater(context, dispatcher, nodeHealthChecker);
-    } else {
+    nodeLabelsProvider = createNodeLabelsProvider(conf);
+    if (nodeLabelsProvider != null) {
       addIfService(nodeLabelsProvider);
-      nodeStatusUpdater =
-          createNodeStatusUpdater(context, dispatcher, nodeHealthChecker,
-              nodeLabelsProvider);
+      nodeStatusUpdater.setNodeLabelsProvider(nodeLabelsProvider);
+    }
+
+    nodeAttributesProvider = createNodeAttributesProvider(conf);
+    if (nodeAttributesProvider != null) {
+      addIfService(nodeAttributesProvider);
+      nodeStatusUpdater.setNodeAttributesProvider(nodeAttributesProvider);
     }
 
     nodeResourceMonitor = createNodeResourceMonitor();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdater.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdater.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdater.java
index 08892d2..142cbbc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdater.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdater.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.server.nodemanager;
 
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeAttributesProvider;
+import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider;
 
 public interface NodeStatusUpdater extends Service {
 
@@ -59,4 +61,16 @@ public interface NodeStatusUpdater extends Service {
    * @param ex exception that makes the node unhealthy
    */
   void reportException(Exception ex);
+
+  /**
+   * Sets a node attributes provider to node manager.
+   * @param provider
+   */
+  void setNodeAttributesProvider(NodeAttributesProvider provider);
+
+  /**
+   * Sets a node labels provider to the node manager.
+   * @param provider
+   */
+  void setNodeLabelsProvider(NodeLabelsProvider provider);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index 7be9ef7..df76ed7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceUtilization;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -85,6 +86,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.Contai
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePlugin;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePluginManager;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
+import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeAttributesProvider;
 import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider;
 import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
 import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
@@ -152,21 +154,16 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   Set<ContainerId> pendingContainersToRemove = new HashSet<ContainerId>();
 
   private NMNodeLabelsHandler nodeLabelsHandler;
-  private final NodeLabelsProvider nodeLabelsProvider;
+  private NMNodeAttributesHandler nodeAttributesHandler;
+  private NodeLabelsProvider nodeLabelsProvider;
+  private NodeAttributesProvider nodeAttributesProvider;
 
   public NodeStatusUpdaterImpl(Context context, Dispatcher dispatcher,
       NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
-    this(context, dispatcher, healthChecker, metrics, null);
-  }
-
-  public NodeStatusUpdaterImpl(Context context, Dispatcher dispatcher,
-      NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics,
-      NodeLabelsProvider nodeLabelsProvider) {
     super(NodeStatusUpdaterImpl.class.getName());
     this.healthChecker = healthChecker;
     this.context = context;
     this.dispatcher = dispatcher;
-    this.nodeLabelsProvider = nodeLabelsProvider;
     this.metrics = metrics;
     this.recentlyStoppedContainers = new LinkedHashMap<ContainerId, Long>();
     this.pendingCompletedContainers =
@@ -176,6 +173,16 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   }
 
   @Override
+  public void setNodeAttributesProvider(NodeAttributesProvider provider) {
+    this.nodeAttributesProvider = provider;
+  }
+
+  @Override
+  public void setNodeLabelsProvider(NodeLabelsProvider provider) {
+    this.nodeLabelsProvider = provider;
+  }
+
+  @Override
   protected void serviceInit(Configuration conf) throws Exception {
     this.totalResource = NodeManagerHardwareUtils.getNodeResources(conf);
     long memoryMb = totalResource.getMemorySize();
@@ -214,7 +221,11 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         YarnConfiguration.NM_RESOURCEMANAGER_MINIMUM_VERSION,
         YarnConfiguration.DEFAULT_NM_RESOURCEMANAGER_MINIMUM_VERSION);
 
-    nodeLabelsHandler = createNMNodeLabelsHandler(nodeLabelsProvider);
+    nodeLabelsHandler =
+        createNMNodeLabelsHandler(nodeLabelsProvider);
+    nodeAttributesHandler =
+        createNMNodeAttributesHandler(nodeAttributesProvider);
+
     // Default duration to track stopped containers on nodemanager is 10Min.
     // This should not be assigned very large value as it will remember all the
     // containers stopped during that time.
@@ -856,6 +867,43 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     }
   }
 
+  /**
+   * Returns a handler based on the configured node attributes provider.
+   * returns null if no provider is configured.
+   * @param provider
+   * @return attributes handler
+   */
+  private NMNodeAttributesHandler createNMNodeAttributesHandler(
+      NodeAttributesProvider provider) {
+    return provider == null ? null :
+        new NMDistributedNodeAttributesHandler(nodeAttributesProvider);
+  }
+
+  private interface NMNodeAttributesHandler {
+
+    /**
+     * @return the node attributes of this node manager.
+     */
+    Set<NodeAttribute> getNodeAttributesForHeartbeat();
+  }
+
+  private static class NMDistributedNodeAttributesHandler
+      implements NMNodeAttributesHandler {
+
+    private final NodeAttributesProvider attributesProvider;
+
+    protected NMDistributedNodeAttributesHandler(
+        NodeAttributesProvider provider) {
+      this.attributesProvider = provider;
+    }
+
+    @Override
+    public Set<NodeAttribute> getNodeAttributesForHeartbeat() {
+      return attributesProvider.getDescriptors();
+    }
+  }
+
+
   private static interface NMNodeLabelsHandler {
     /**
      * validates nodeLabels From Provider and returns it to the caller. Also
@@ -1071,6 +1119,9 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
           NodeHeartbeatResponse response = null;
           Set<NodeLabel> nodeLabelsForHeartbeat =
               nodeLabelsHandler.getNodeLabelsForHeartbeat();
+          Set<NodeAttribute> nodeAttributesForHeartbeat =
+              nodeAttributesHandler == null ? null :
+                  nodeAttributesHandler.getNodeAttributesForHeartbeat();
           NodeStatus nodeStatus = getNodeStatus(lastHeartbeatID);
           NodeHeartbeatRequest request =
               NodeHeartbeatRequest.newInstance(nodeStatus,
@@ -1079,6 +1130,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
                   NodeStatusUpdaterImpl.this.context
                       .getNMTokenSecretManager().getCurrentKey(),
                   nodeLabelsForHeartbeat,
+                  nodeAttributesForHeartbeat,
                   NodeStatusUpdaterImpl.this.context
                       .getRegisteringCollectors());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeAttributesProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeAttributesProvider.java
new file mode 100644
index 0000000..74341eb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeAttributesProvider.java
@@ -0,0 +1,90 @@
+/**
+ * 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.nodemanager.nodelabels;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.TimerTask;
+import java.util.Set;
+
+/**
+ * Configuration based node attributes provider.
+ */
+public class ConfigurationNodeAttributesProvider
+    extends NodeAttributesProvider {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ConfigurationNodeAttributesProvider.class);
+
+  public ConfigurationNodeAttributesProvider() {
+    super("Configuration Based Node Attributes Provider");
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    long taskInterval = conf.getLong(YarnConfiguration
+            .NM_NODE_ATTRIBUTES_PROVIDER_FETCH_INTERVAL_MS,
+        YarnConfiguration
+            .DEFAULT_NM_NODE_ATTRIBUTES_PROVIDER_FETCH_INTERVAL_MS);
+    this.setIntervalTime(taskInterval);
+    super.serviceInit(conf);
+  }
+
+  private void updateNodeAttributesFromConfig(Configuration conf)
+      throws IOException {
+    String configuredNodeAttributes = conf.get(
+        YarnConfiguration.NM_PROVIDER_CONFIGURED_NODE_ATTRIBUTES, null);
+    setDescriptors(parseAttributes(configuredNodeAttributes));
+  }
+
+  // TODO parse attributes from configuration
+  @VisibleForTesting
+  public Set<NodeAttribute> parseAttributes(String config)
+      throws IOException {
+    return new HashSet<>();
+  }
+
+  private class ConfigurationMonitorTimerTask extends TimerTask {
+    @Override
+    public void run() {
+      try {
+        updateNodeAttributesFromConfig(new YarnConfiguration());
+      } catch (Exception e) {
+        LOG.error("Failed to update node attributes from "
+            + YarnConfiguration.NM_PROVIDER_CONFIGURED_NODE_ATTRIBUTES, e);
+      }
+    }
+  }
+
+  @Override
+  protected void cleanUp() throws Exception {
+    // Nothing to cleanup
+  }
+
+  @Override
+  public TimerTask createTimerTask() {
+    return new ConfigurationMonitorTimerTask();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.java
index b31215b..b2c2f6e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.java
@@ -160,7 +160,7 @@ public class TestNodeManager {
 
       // With valid whitelisted configurations
       conf.set(YarnConfiguration.NM_NODE_LABELS_PROVIDER_CONFIG,
-          YarnConfiguration.CONFIG_NODE_LABELS_PROVIDER);
+          YarnConfiguration.CONFIG_NODE_DESCRIPTOR_PROVIDER);
       labelsProviderService = nodeManager.createNodeLabelsProvider(conf);
       Assert.assertNotNull("LabelsProviderService should be initialized When "
           + "node labels provider class is configured", labelsProviderService);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
index 7ef23cb..3e2d963 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
@@ -225,11 +225,10 @@ public class TestNodeStatusUpdaterForLabels extends NodeLabelTestBase {
 
       @Override
       protected NodeStatusUpdater createNodeStatusUpdater(Context context,
-          Dispatcher dispatcher, NodeHealthCheckerService healthChecker,
-          NodeLabelsProvider labelsProvider) {
+          Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
 
         return new NodeStatusUpdaterImpl(context, dispatcher, healthChecker,
-            metrics, labelsProvider) {
+            metrics) {
           @Override
           protected ResourceTracker getRMClient() {
             return resourceTracker;
@@ -325,11 +324,10 @@ public class TestNodeStatusUpdaterForLabels extends NodeLabelTestBase {
 
       @Override
       protected NodeStatusUpdater createNodeStatusUpdater(Context context,
-          Dispatcher dispatcher, NodeHealthCheckerService healthChecker,
-          NodeLabelsProvider labelsProvider) {
+          Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
 
         return new NodeStatusUpdaterImpl(context, dispatcher, healthChecker,
-            metrics, labelsProvider) {
+            metrics) {
           @Override
           protected ResourceTracker getRMClient() {
             return resourceTracker;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java
new file mode 100644
index 0000000..54cc8f0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java
@@ -0,0 +1,185 @@
+/**
+ * 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.nodemanager.nodelabels;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.BeforeClass;
+import org.junit.Before;
+import org.junit.AfterClass;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.Assert;
+
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Test class for node configuration node attributes provider.
+ */
+public class TestConfigurationNodeAttributesProvider {
+
+  private static File testRootDir = new File("target",
+      TestConfigurationNodeAttributesProvider.class.getName() + "-localDir")
+      .getAbsoluteFile();
+
+  private ConfigurationNodeAttributesProvider nodeAttributesProvider;
+
+  @BeforeClass
+  public static void create() {
+    testRootDir.mkdirs();
+  }
+
+  @Before
+  public void setup() {
+    nodeAttributesProvider = new ConfigurationNodeAttributesProvider();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (nodeAttributesProvider != null) {
+      nodeAttributesProvider.close();
+      nodeAttributesProvider.stop();
+    }
+  }
+
+  @AfterClass
+  public static void remove() throws Exception {
+    if (testRootDir.exists()) {
+      FileContext.getLocalFSFileContext()
+          .delete(new Path(testRootDir.getAbsolutePath()), true);
+    }
+  }
+
+  @Test(timeout=30000L)
+  public void testNodeAttributesFetchInterval()
+      throws IOException, InterruptedException {
+    Set<NodeAttribute> expectedAttributes1 = new HashSet<>();
+    expectedAttributes1.add(NodeAttribute
+        .newInstance("test.io", "host",
+            NodeAttributeType.STRING, "host1"));
+
+    Configuration conf = new Configuration();
+    // Set fetch interval to 1s for testing
+    conf.setLong(
+        YarnConfiguration.NM_NODE_ATTRIBUTES_PROVIDER_FETCH_INTERVAL_MS, 1000);
+    ConfigurationNodeAttributesProvider spyProvider =
+        Mockito.spy(nodeAttributesProvider);
+    Mockito.when(spyProvider.parseAttributes(Mockito.anyString()))
+        .thenReturn(expectedAttributes1);
+
+    spyProvider.init(conf);
+    spyProvider.start();
+
+    // Verify init value is honored.
+    Assert.assertEquals(expectedAttributes1, spyProvider.getDescriptors());
+
+    // Configuration provider provides a different set of attributes.
+    Set<NodeAttribute> expectedAttributes2 = new HashSet<>();
+    expectedAttributes2.add(NodeAttribute
+        .newInstance("test.io", "os",
+            NodeAttributeType.STRING, "windows"));
+    Mockito.when(spyProvider.parseAttributes(Mockito.anyString()))
+        .thenReturn(expectedAttributes2);
+
+    // Since we set fetch interval to 1s, it needs to wait for 1s until
+    // the updated attributes is updated to the provider. So we are expecting
+    // to see some old values for a short window.
+    ArrayList<String> keysMet = new ArrayList<>();
+    int numOfOldValue = 0;
+    int numOfNewValue = 0;
+    // Run 5 times in 500ms interval
+    int times=5;
+    while(times>0) {
+      Set<NodeAttribute> current = spyProvider.getDescriptors();
+      Assert.assertEquals(1, current.size());
+      String attributeName = current.iterator().next().getAttributeName();
+      if ("host".equals(attributeName)){
+        numOfOldValue++;
+      } else if ("os".equals(attributeName)) {
+        numOfNewValue++;
+      }
+      Thread.sleep(500);
+      times--;
+    }
+    // We should either see the old value or the new value.
+    Assert.assertEquals(5, numOfNewValue + numOfOldValue);
+    // Both values should be more than 0.
+    Assert.assertTrue(numOfOldValue > 0);
+    Assert.assertTrue(numOfNewValue > 0);
+  }
+
+  @Test
+  public void testDisableFetchNodeAttributes() throws IOException,
+      InterruptedException {
+    Set<NodeAttribute> expectedAttributes1 = new HashSet<>();
+    expectedAttributes1.add(NodeAttribute
+        .newInstance("test.io", "host",
+            NodeAttributeType.STRING, "host1"));
+
+    Configuration conf = new Configuration();
+    // Set fetch interval to -1 to disable refresh.
+    conf.setLong(
+        YarnConfiguration.NM_NODE_ATTRIBUTES_PROVIDER_FETCH_INTERVAL_MS, -1);
+    ConfigurationNodeAttributesProvider spyProvider =
+        Mockito.spy(nodeAttributesProvider);
+    Mockito.when(spyProvider.parseAttributes(Mockito.anyString()))
+        .thenReturn(expectedAttributes1);
+    spyProvider.init(conf);
+    spyProvider.start();
+
+    Assert.assertEquals(expectedAttributes1,
+        spyProvider.getDescriptors());
+
+    // The configuration added another attribute,
+    // as we disabled the fetch interval, this value cannot be
+    // updated to the provider.
+    Set<NodeAttribute> expectedAttributes2 = new HashSet<>();
+    expectedAttributes2.add(NodeAttribute
+        .newInstance("test.io", "os",
+            NodeAttributeType.STRING, "windows"));
+    Mockito.when(spyProvider.parseAttributes(Mockito.anyString()))
+        .thenReturn(expectedAttributes2);
+
+    // Wait a few seconds until we get the value update, expecting a failure.
+    try {
+      GenericTestUtils.waitFor(() -> {
+        Set<NodeAttribute> attributes = spyProvider.getDescriptors();
+        return "os".equalsIgnoreCase(attributes
+            .iterator().next().getAttributeName());
+      }, 500, 1000);
+    } catch (Exception e) {
+      // Make sure we get the timeout exception.
+      Assert.assertTrue(e instanceof TimeoutException);
+      return;
+    }
+
+    Assert.fail("Expecting a failure in previous check!");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index e997192..8a1a9a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -33,6 +33,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
 import org.apache.commons.collections.CollectionUtils;
+import com.google.common.collect.ImmutableMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -51,6 +52,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -646,6 +648,34 @@ public class ResourceTrackerService extends AbstractService implements
           this.rmContext.getNodeManagerQueueLimitCalculator()
               .createContainerQueuingLimit());
     }
+
+    // 8. Get node's attributes and update node-to-attributes mapping
+    // in RMNodeAttributeManager.
+    Set<NodeAttribute> nodeAttributes = request.getNodeAttributes();
+    if (nodeAttributes != null && !nodeAttributes.isEmpty()) {
+      nodeAttributes.forEach(nodeAttribute ->
+          LOG.debug(nodeId.toString() + " ATTRIBUTE : "
+              + nodeAttribute.toString()));
+
+      // Validate attributes
+      if (!nodeAttributes.stream().allMatch(
+          nodeAttribute -> NodeAttribute.PREFIX_DISTRIBUTED
+              .equals(nodeAttribute.getAttributePrefix()))) {
+        // All attributes must be in same prefix: nm.yarn.io.
+        // Since we have the checks in NM to make sure attributes reported
+        // in HB are with correct prefix, so it should not reach here.
+        LOG.warn("Reject invalid node attributes from host: "
+            + nodeId.toString() + ", attributes in HB must have prefix "
+            + NodeAttribute.PREFIX_DISTRIBUTED);
+      } else {
+        // Replace all distributed node attributes associated with this host
+        // with the new reported attributes in node attribute manager.
+        this.rmContext.getNodeAttributesManager()
+            .replaceNodeAttributes(NodeAttribute.PREFIX_DISTRIBUTED,
+                ImmutableMap.of(nodeId.getHost(), nodeAttributes));
+      }
+    }
+
     return nodeHeartBeatResponse;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
index a902ac6..04d74a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
@@ -33,6 +33,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
+import com.google.common.base.Strings;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -126,7 +127,8 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   private void internalUpdateAttributesOnNodes(
       Map<String, Map<NodeAttribute, AttributeValue>> nodeAttributeMapping,
       AttributeMappingOperationType op,
-      Map<NodeAttribute, RMNodeAttribute> newAttributesToBeAdded) {
+      Map<NodeAttribute, RMNodeAttribute> newAttributesToBeAdded,
+      String attributePrefix) {
     try {
       writeLock.lock();
 
@@ -156,8 +158,9 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
           break;
         case REPLACE:
           clusterAttributes.putAll(newAttributesToBeAdded);
-          replaceNodeToAttribute(nodeHost, node.getAttributes(), attributes);
-          node.replaceAttributes(attributes);
+          replaceNodeToAttribute(nodeHost, attributePrefix,
+              node.getAttributes(), attributes);
+          node.replaceAttributes(attributes, attributePrefix);
           break;
         default:
           break;
@@ -199,15 +202,23 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   private void addNodeToAttribute(String nodeHost,
       Map<NodeAttribute, AttributeValue> attributeMappings) {
     for (NodeAttribute attribute : attributeMappings.keySet()) {
-      clusterAttributes.get(attribute).addNode(nodeHost);
+      RMNodeAttribute rmNodeAttribute = clusterAttributes.get(attribute);
+      if (rmNodeAttribute != null) {
+        rmNodeAttribute.addNode(nodeHost);
+      } else {
+        clusterAttributes.put(attribute, new RMNodeAttribute(attribute));
+      }
     }
   }
 
-  private void replaceNodeToAttribute(String nodeHost,
+  private void replaceNodeToAttribute(String nodeHost, String prefix,
       Map<NodeAttribute, AttributeValue> oldAttributeMappings,
       Map<NodeAttribute, AttributeValue> newAttributeMappings) {
     if (oldAttributeMappings != null) {
-      removeNodeFromAttributes(nodeHost, oldAttributeMappings.keySet());
+      Set<NodeAttribute> toRemoveAttributes =
+          NodeLabelUtil.filterAttributesByPrefix(
+              oldAttributeMappings.keySet(), prefix);
+      removeNodeFromAttributes(nodeHost, toRemoveAttributes);
     }
     addNodeToAttribute(nodeHost, newAttributeMappings);
   }
@@ -432,8 +443,19 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
     }
 
     public void replaceAttributes(
-        Map<NodeAttribute, AttributeValue> attributesMapping) {
-      this.attributes.clear();
+        Map<NodeAttribute, AttributeValue> attributesMapping, String prefix) {
+      if (Strings.isNullOrEmpty(prefix)) {
+        this.attributes.clear();
+      } else {
+        Iterator<Entry<NodeAttribute, AttributeValue>> it =
+            this.attributes.entrySet().iterator();
+        while (it.hasNext()) {
+          Entry<NodeAttribute, AttributeValue> current = it.next();
+          if (prefix.equals(current.getKey().getAttributePrefix())) {
+            it.remove();
+          }
+        }
+      }
       this.attributes.putAll(attributesMapping);
     }
 
@@ -506,9 +528,10 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   }
 
   @Override
-  public void replaceNodeAttributes(
+  public void replaceNodeAttributes(String prefix,
       Map<String, Set<NodeAttribute>> nodeAttributeMapping) throws IOException {
-    processMapping(nodeAttributeMapping, AttributeMappingOperationType.REPLACE);
+    processMapping(nodeAttributeMapping,
+        AttributeMappingOperationType.REPLACE, prefix);
   }
 
   @Override
@@ -526,12 +549,19 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   private void processMapping(
       Map<String, Set<NodeAttribute>> nodeAttributeMapping,
       AttributeMappingOperationType mappingType) throws IOException {
+    processMapping(nodeAttributeMapping, mappingType, null);
+  }
+
+  private void processMapping(
+      Map<String, Set<NodeAttribute>> nodeAttributeMapping,
+      AttributeMappingOperationType mappingType, String attributePrefix)
+      throws IOException {
     Map<NodeAttribute, RMNodeAttribute> newAttributesToBeAdded =
         new HashMap<>();
     Map<String, Map<NodeAttribute, AttributeValue>> validMapping =
         validate(nodeAttributeMapping, newAttributesToBeAdded, false);
 
     internalUpdateAttributesOnNodes(validMapping, mappingType,
-        newAttributesToBeAdded);
+        newAttributesToBeAdded, attributePrefix);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
index fa0f5fd..a29e8a2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
@@ -37,6 +37,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.HashSet;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import javax.xml.parsers.DocumentBuilderFactory;
@@ -64,12 +65,16 @@ import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.nodelabels.AttributeValue;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.nodelabels.NodeLabelTestBase;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
@@ -818,6 +823,79 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
   }
 
   @Test
+  public void testNodeHeartbeatWithNodeAttributes() throws Exception {
+    writeToHostsFile("host2");
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH,
+        hostFile.getAbsolutePath());
+    rm = new MockRM(conf);
+    rm.start();
+
+    // Register to RM
+    ResourceTrackerService resourceTrackerService =
+        rm.getResourceTrackerService();
+    RegisterNodeManagerRequest registerReq =
+        Records.newRecord(RegisterNodeManagerRequest.class);
+    NodeId nodeId = NodeId.newInstance("host2", 1234);
+    Resource capability = BuilderUtils.newResource(1024, 1);
+    registerReq.setResource(capability);
+    registerReq.setNodeId(nodeId);
+    registerReq.setHttpPort(1234);
+    registerReq.setNMVersion(YarnVersionInfo.getVersion());
+    RegisterNodeManagerResponse registerResponse =
+        resourceTrackerService.registerNodeManager(registerReq);
+
+    Set<NodeAttribute> nodeAttributes = new HashSet<>();
+    nodeAttributes.add(NodeAttribute.newInstance(
+        NodeAttribute.PREFIX_DISTRIBUTED, "host",
+        NodeAttributeType.STRING, "host2"));
+
+    // Set node attributes in HB.
+    NodeHeartbeatRequest heartbeatReq =
+        Records.newRecord(NodeHeartbeatRequest.class);
+    NodeStatus nodeStatusObject = getNodeStatusObject(nodeId);
+    int responseId = nodeStatusObject.getResponseId();
+    heartbeatReq.setNodeStatus(nodeStatusObject);
+    heartbeatReq.setLastKnownNMTokenMasterKey(registerResponse
+        .getNMTokenMasterKey());
+    heartbeatReq.setLastKnownContainerTokenMasterKey(registerResponse
+        .getContainerTokenMasterKey());
+    heartbeatReq.setNodeAttributes(nodeAttributes);
+    resourceTrackerService.nodeHeartbeat(heartbeatReq);
+
+    // Ensure RM gets correct node attributes update.
+    NodeAttributesManager attributeManager =
+        rm.getRMContext().getNodeAttributesManager();
+    Map<NodeAttribute, AttributeValue> attrs = attributeManager
+        .getAttributesForNode(nodeId.getHost());
+    Assert.assertEquals(1, attrs.size());
+    NodeAttribute na = attrs.keySet().iterator().next();
+    Assert.assertEquals("host", na.getAttributeName());
+    Assert.assertEquals("host2", na.getAttributeValue());
+    Assert.assertEquals(NodeAttributeType.STRING, na.getAttributeType());
+
+
+    // Send another HB to RM with updated node atrribute
+    nodeAttributes.clear();
+    nodeAttributes.add(NodeAttribute.newInstance(
+        NodeAttribute.PREFIX_DISTRIBUTED, "host",
+        NodeAttributeType.STRING, "host3"));
+    nodeStatusObject = getNodeStatusObject(nodeId);
+    nodeStatusObject.setResponseId(++responseId);
+    heartbeatReq.setNodeStatus(nodeStatusObject);
+    heartbeatReq.setNodeAttributes(nodeAttributes);
+    resourceTrackerService.nodeHeartbeat(heartbeatReq);
+
+    // Make sure RM gets the updated attribute
+    attrs = attributeManager.getAttributesForNode(nodeId.getHost());
+    Assert.assertEquals(1, attrs.size());
+    na = attrs.keySet().iterator().next();
+    Assert.assertEquals("host", na.getAttributeName());
+    Assert.assertEquals("host3", na.getAttributeValue());
+    Assert.assertEquals(NodeAttributeType.STRING, na.getAttributeType());
+  }
+
+  @Test
   public void testNodeHeartBeatWithInvalidLabels() throws Exception {
     writeToHostsFile("host2");
     Configuration conf = new Configuration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b3b6efe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
index b639a74..07968d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
@@ -18,12 +18,14 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
 
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeAttributeType;
 import org.apache.hadoop.yarn.nodelabels.AttributeValue;
 import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
+import org.apache.hadoop.yarn.nodelabels.NodeLabelUtil;
 import org.junit.Test;
 import org.junit.Before;
 import org.junit.After;
@@ -255,4 +257,101 @@ public class TestNodeAttributesManager {
         .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[0]));
     Assert.assertEquals(2, allAttributesPerPrefix.size());
   }
+
+  @Test
+  public void testReplaceNodeAttributes() throws IOException {
+    Map<String, Set<NodeAttribute>> toAddAttributes = new HashMap<>();
+    Map<String, Set<NodeAttribute>> toReplaceMap = new HashMap<>();
+    Map<NodeAttribute, AttributeValue> nodeAttributes;
+    Set<NodeAttribute> filteredAttributes;
+    Set<NodeAttribute> clusterAttributes;
+
+    // Add 3 attributes to host1
+    //  yarn.test1.io/A1=host1_v1_1
+    //  yarn.test1.io/A2=host1_v1_2
+    //  yarn.test1.io/A3=host1_v1_3
+    toAddAttributes.put(HOSTNAMES[0],
+        createAttributesForTest(PREFIXES[0], 3, "A", "host1_v1"));
+
+    attributesManager.addNodeAttributes(toAddAttributes);
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]);
+    Assert.assertEquals(3, nodeAttributes.size());
+
+    // Add 10 distributed node attributes to host1
+    //  nn.yarn.io/dist-node-attribute1=dist_v1_1
+    //  nn.yarn.io/dist-node-attribute2=dist_v1_2
+    //  ...
+    //  nn.yarn.io/dist-node-attribute10=dist_v1_10
+    toAddAttributes.clear();
+    toAddAttributes.put(HOSTNAMES[0],
+        createAttributesForTest(NodeAttribute.PREFIX_DISTRIBUTED,
+            10, "dist-node-attribute", "dist_v1"));
+    attributesManager.addNodeAttributes(toAddAttributes);
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]);
+    Assert.assertEquals(13, nodeAttributes.size());
+    clusterAttributes = attributesManager.getClusterNodeAttributes(
+        Sets.newHashSet(NodeAttribute.PREFIX_DISTRIBUTED, PREFIXES[0]));
+    Assert.assertEquals(13, clusterAttributes.size());
+
+    // Replace by prefix
+    // Same distributed attributes names, but different values.
+    Set<NodeAttribute> toReplaceAttributes =
+        createAttributesForTest(NodeAttribute.PREFIX_DISTRIBUTED, 5,
+            "dist-node-attribute", "dist_v2");
+
+    attributesManager.replaceNodeAttributes(NodeAttribute.PREFIX_DISTRIBUTED,
+        ImmutableMap.of(HOSTNAMES[0], toReplaceAttributes));
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]);
+    Assert.assertEquals(8, nodeAttributes.size());
+    clusterAttributes = attributesManager.getClusterNodeAttributes(
+        Sets.newHashSet(NodeAttribute.PREFIX_DISTRIBUTED, PREFIXES[0]));
+    Assert.assertEquals(8, clusterAttributes.size());
+
+    // Now we have 5 distributed attributes
+    filteredAttributes = NodeLabelUtil.filterAttributesByPrefix(
+        nodeAttributes.keySet(), NodeAttribute.PREFIX_DISTRIBUTED);
+    Assert.assertEquals(5, filteredAttributes.size());
+    // Values are updated to have prefix dist_v2
+    Assert.assertTrue(filteredAttributes.stream().allMatch(
+        nodeAttribute ->
+            nodeAttribute.getAttributeValue().startsWith("dist_v2")));
+
+    // We still have 3 yarn.test1.io attributes
+    filteredAttributes = NodeLabelUtil.filterAttributesByPrefix(
+        nodeAttributes.keySet(), PREFIXES[0]);
+    Assert.assertEquals(3, filteredAttributes.size());
+
+    // Replace with prefix
+    // Different attribute names
+    toReplaceAttributes =
+        createAttributesForTest(NodeAttribute.PREFIX_DISTRIBUTED, 1,
+            "dist-node-attribute-v2", "dist_v3");
+    attributesManager.replaceNodeAttributes(NodeAttribute.PREFIX_DISTRIBUTED,
+        ImmutableMap.of(HOSTNAMES[0], toReplaceAttributes));
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]);
+    Assert.assertEquals(4, nodeAttributes.size());
+    clusterAttributes = attributesManager.getClusterNodeAttributes(
+        Sets.newHashSet(NodeAttribute.PREFIX_DISTRIBUTED));
+    Assert.assertEquals(1, clusterAttributes.size());
+    NodeAttribute att = clusterAttributes.iterator().next();
+    Assert.assertEquals("dist-node-attribute-v2_0", att.getAttributeName());
+    Assert.assertEquals(NodeAttribute.PREFIX_DISTRIBUTED,
+        att.getAttributePrefix());
+    Assert.assertEquals("dist_v3_0", att.getAttributeValue());
+
+    // Replace all attributes
+    toReplaceMap.put(HOSTNAMES[0],
+        createAttributesForTest(PREFIXES[1], 2, "B", "B_v1"));
+    attributesManager.replaceNodeAttributes(null, toReplaceMap);
+
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]);
+    Assert.assertEquals(2, nodeAttributes.size());
+    clusterAttributes = attributesManager
+        .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[1]));
+    Assert.assertEquals(2, clusterAttributes.size());
+    clusterAttributes = attributesManager
+        .getClusterNodeAttributes(Sets.newHashSet(
+            NodeAttribute.PREFIX_DISTRIBUTED));
+    Assert.assertEquals(0, clusterAttributes.size());
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[19/29] hadoop git commit: YARN-8094. Support configuration based Node Attribute provider. Contributed by Weiwei Yang.

Posted by su...@apache.org.
YARN-8094. Support configuration based Node Attribute provider. Contributed by Weiwei Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/440ff7f5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/440ff7f5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/440ff7f5

Branch: refs/heads/trunk
Commit: 440ff7f563df5e7db72dce020c3c3dc379f88c91
Parents: 6f4bc49
Author: Sunil G <su...@apache.org>
Authored: Sat Mar 31 19:53:06 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../ConfigurationNodeAttributesProvider.java    | 70 +++++++++++++++++-
 .../ScriptBasedNodeAttributesProvider.java      |  8 +++
 ...TestConfigurationNodeAttributesProvider.java | 74 ++++++++++++++++++++
 3 files changed, 150 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/440ff7f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeAttributesProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeAttributesProvider.java
index 74341eb..ab8a8b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeAttributesProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeAttributesProvider.java
@@ -18,13 +18,19 @@
 package org.apache.hadoop.yarn.server.nodemanager.nodelabels;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableSet;
+import org.apache.commons.lang3.EnumUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.nodelabels.NodeLabelUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.TimerTask;
 import java.util.Set;
@@ -38,6 +44,9 @@ public class ConfigurationNodeAttributesProvider
   private static final Logger LOG =
       LoggerFactory.getLogger(ConfigurationNodeAttributesProvider.class);
 
+  private static final String NODE_ATTRIBUTES_DELIMITER = ":";
+  private static final String NODE_ATTRIBUTE_DELIMITER = ",";
+
   public ConfigurationNodeAttributesProvider() {
     super("Configuration Based Node Attributes Provider");
   }
@@ -59,11 +68,68 @@ public class ConfigurationNodeAttributesProvider
     setDescriptors(parseAttributes(configuredNodeAttributes));
   }
 
-  // TODO parse attributes from configuration
   @VisibleForTesting
   public Set<NodeAttribute> parseAttributes(String config)
       throws IOException {
-    return new HashSet<>();
+    if (Strings.isNullOrEmpty(config)) {
+      return ImmutableSet.of();
+    }
+    Set<NodeAttribute> attributeSet = new HashSet<>();
+    // Configuration value should be in one line, format:
+    // "ATTRIBUTE_NAME,ATTRIBUTE_TYPE,ATTRIBUTE_VALUE",
+    // multiple node-attributes are delimited by ":".
+    // Each attribute str should not container any space.
+    String[] attributeStrs = config.split(NODE_ATTRIBUTES_DELIMITER);
+    for (String attributeStr : attributeStrs) {
+      String[] fields = attributeStr.split(NODE_ATTRIBUTE_DELIMITER);
+      if (fields.length != 3) {
+        throw new IOException("Invalid value for "
+            + YarnConfiguration.NM_PROVIDER_CONFIGURED_NODE_ATTRIBUTES
+            + "=" + config);
+      }
+
+      // We don't allow user config to overwrite our dist prefix,
+      // so disallow any prefix set in the configuration.
+      if (fields[0].contains("/")) {
+        throw new IOException("Node attribute set in "
+            + YarnConfiguration.NM_PROVIDER_CONFIGURED_NODE_ATTRIBUTES
+            + " should not contain any prefix.");
+      }
+
+      // Make sure attribute type is valid.
+      if (!EnumUtils.isValidEnum(NodeAttributeType.class, fields[1])) {
+        throw new IOException("Invalid node attribute type: "
+            + fields[1] + ", valid values are "
+            + Arrays.asList(NodeAttributeType.values()));
+      }
+
+      // Automatically setup prefix for collected attributes
+      NodeAttribute na = NodeAttribute.newInstance(
+          NodeAttribute.PREFIX_DISTRIBUTED,
+          fields[0],
+          NodeAttributeType.valueOf(fields[1]),
+          fields[2]);
+
+      // Since a NodeAttribute is identical with another one as long as
+      // their prefix and name are same, to avoid attributes getting
+      // overwritten by ambiguous attribute, make sure it fails in such
+      // case.
+      if (!attributeSet.add(na)) {
+        throw new IOException("Ambiguous node attribute is found: "
+            + na.toString() + ", a same attribute already exists");
+      }
+    }
+
+    // Before updating the attributes to the provider,
+    // verify if they are valid
+    try {
+      NodeLabelUtil.validateNodeAttributes(attributeSet);
+    } catch (IOException e) {
+      throw new IOException("Node attributes set by configuration property: "
+          + YarnConfiguration.NM_PROVIDER_CONFIGURED_NODE_ATTRIBUTES
+          + " is not valid. Detail message: " + e.getMessage());
+    }
+    return attributeSet;
   }
 
   private class ConfigurationMonitorTimerTask extends TimerTask {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/440ff7f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java
index 4621434..7e5aefc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java
@@ -117,6 +117,14 @@ public class ScriptBasedNodeAttributesProvider extends NodeAttributesProvider{
                 + NODE_ATTRIBUTE_DELIMITER + "ATTRIBUTE_VALUE; but get "
                 + nodeAttribute);
           }
+
+          // We don't allow script to overwrite our dist prefix,
+          // so disallow any prefix set in the script.
+          if (attributeStrs[0].contains("/")) {
+            throw new IOException("Node attributes reported by script"
+                + " should not contain any prefix.");
+          }
+
           // Automatically setup prefix for collected attributes
           NodeAttribute na = NodeAttribute
               .newInstance(NodeAttribute.PREFIX_DISTRIBUTED,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/440ff7f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java
index 54cc8f0..d4384b4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java
@@ -36,6 +36,7 @@ import org.mockito.Mockito;
 import java.io.File;
 import java.io.IOException;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.Set;
 import java.util.ArrayList;
 import java.util.concurrent.TimeoutException;
@@ -182,4 +183,77 @@ public class TestConfigurationNodeAttributesProvider {
 
     Assert.fail("Expecting a failure in previous check!");
   }
+
+  @Test
+  public void testFetchAttributesFromConfiguration() {
+    Configuration conf = new Configuration();
+    // Set fetch interval to -1 to disable refresh.
+    conf.setLong(
+        YarnConfiguration.NM_NODE_ATTRIBUTES_PROVIDER_FETCH_INTERVAL_MS, -1);
+    conf.setStrings(
+        YarnConfiguration.NM_PROVIDER_CONFIGURED_NODE_ATTRIBUTES, "");
+  }
+
+  @Test
+  public void testParseConfiguration() throws IOException {
+    // ATTRIBUTE_NAME,ATTRIBUTE_TYPE,ATTRIBUTE_VALUE
+    String attributesStr = "hostname,STRING,host1234:uptime,STRING,321543";
+    Set<NodeAttribute> attributes = nodeAttributesProvider
+        .parseAttributes(attributesStr);
+    Assert.assertEquals(2, attributes.size());
+    Iterator<NodeAttribute> ait = attributes.iterator();
+
+    while(ait.hasNext()) {
+      NodeAttribute at = ait.next();
+      if (at.getAttributeName().equals("hostname")) {
+        Assert.assertEquals("hostname", at.getAttributeName());
+        Assert.assertEquals(NodeAttribute.PREFIX_DISTRIBUTED,
+            at.getAttributePrefix());
+        Assert.assertEquals(NodeAttributeType.STRING,
+            at.getAttributeType());
+        Assert.assertEquals("host1234", at.getAttributeValue());
+      } else if (at.getAttributeName().equals("uptime")) {
+        Assert.assertEquals("uptime", at.getAttributeName());
+        Assert.assertEquals(NodeAttribute.PREFIX_DISTRIBUTED,
+            at.getAttributePrefix());
+        Assert.assertEquals(NodeAttributeType.STRING,
+            at.getAttributeType());
+        Assert.assertEquals("321543", at.getAttributeValue());
+      } else {
+        Assert.fail("Unexpected attribute");
+      }
+    }
+    // Missing type
+    attributesStr = "hostname,host1234";
+    try {
+      nodeAttributesProvider.parseAttributes(attributesStr);
+      Assert.fail("Expecting a parsing failure");
+    } catch (IOException e) {
+      Assert.assertNotNull(e);
+      Assert.assertTrue(e.getMessage().contains("Invalid value"));
+    }
+
+    // Extra prefix
+    attributesStr = "prefix/hostname,STRING,host1234";
+    try {
+      nodeAttributesProvider.parseAttributes(attributesStr);
+      Assert.fail("Expecting a parsing failure");
+    } catch (IOException e) {
+      Assert.assertNotNull(e);
+      Assert.assertTrue(e.getMessage()
+          .contains("should not contain any prefix."));
+    }
+
+    // Invalid type
+    attributesStr = "hostname,T,host1234";
+    try {
+      nodeAttributesProvider.parseAttributes(attributesStr);
+      Assert.fail("Expecting a parsing failure");
+    } catch (IOException e) {
+      e.printStackTrace();
+      Assert.assertNotNull(e);
+      Assert.assertTrue(e.getMessage()
+          .contains("Invalid node attribute type"));
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[28/29] hadoop git commit: YARN-8721. Relax NE node-attribute check when attribute doesn't exist on a node. Contributed by Sunil Govindan.

Posted by su...@apache.org.
YARN-8721. Relax NE node-attribute check when attribute doesn't exist on a node. Contributed by Sunil Govindan.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/52194351
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/52194351
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/52194351

Branch: refs/heads/trunk
Commit: 52194351e7df33b8438569c3a032f73d696c534d
Parents: 67ae81f
Author: Weiwei Yang <ww...@apache.org>
Authored: Tue Aug 28 17:25:19 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:01 2018 +0530

----------------------------------------------------------------------
 .../yarn/nodelabels/NodeAttributesManager.java  |   7 +
 .../nodelabels/NodeAttributesManagerImpl.java   |  23 +++
 .../scheduler/capacity/CapacityScheduler.java   |   6 +
 .../constraint/PlacementConstraintsUtil.java    |  90 +++++----
 .../scheduler/capacity/TestUtils.java           |  20 +-
 ...stSingleConstraintAppPlacementAllocator.java | 192 ++++++++++++++++++-
 6 files changed, 286 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/52194351/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
index 68c6ec6..20f72d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
 
 /**
@@ -126,4 +127,10 @@ public abstract class NodeAttributesManager extends AbstractService {
 
   // futuristic
   // public set<NodeId> getNodesMatchingExpression(String nodeLabelExp);
+
+  /**
+   * Refresh node attributes on a given node during RM recovery.
+   * @param nodeId Node Id
+   */
+  public abstract void refreshNodeAttributesToScheduler(NodeId nodeId);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52194351/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
index 9111d0f..906f41e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
@@ -725,4 +725,27 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   public void setRMContext(RMContext context) {
     this.rmContext  = context;
   }
+
+  /**
+   * Refresh node attributes on a given node during RM recovery.
+   * @param nodeId Node Id
+   */
+  public void refreshNodeAttributesToScheduler(NodeId nodeId) {
+    String hostName = nodeId.getHost();
+    Map<String, Set<NodeAttribute>> newNodeToAttributesMap =
+        new HashMap<>();
+    Host host = nodeCollections.get(hostName);
+    if (host == null || host.attributes == null) {
+      return;
+    }
+    newNodeToAttributesMap.put(hostName, host.attributes.keySet());
+
+    // Notify RM
+    if (rmContext != null && rmContext.getDispatcher() != null) {
+      LOG.info("Updated NodeAttribute event to RM:" + newNodeToAttributesMap
+          .values());
+      rmContext.getDispatcher().getEventHandler().handle(
+          new NodeAttributesUpdateSchedulerEvent(newNodeToAttributesMap));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52194351/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index a1d3f60..4b274df 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -1989,6 +1989,12 @@ public class CapacityScheduler extends
             schedulerNode.getTotalResource());
       }
 
+      // recover attributes from store if any.
+      if (rmContext.getNodeAttributesManager() != null) {
+        rmContext.getNodeAttributesManager()
+            .refreshNodeAttributesToScheduler(schedulerNode.getNodeID());
+      }
+
       Resource clusterResource = getClusterResource();
       getRootQueue().updateClusterResource(clusterResource,
           new ResourceLimits(clusterResource));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52194351/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java
index ccd334c..0227296 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java
@@ -113,7 +113,7 @@ public final class PlacementConstraintsUtil {
             || maxScopeCardinality <= desiredMaxCardinality);
   }
 
-  private static boolean canSatisfyNodeConstraintExpresssion(
+  private static boolean canSatisfyNodeConstraintExpression(
       SingleConstraint sc, TargetExpression targetExpression,
       SchedulerNode schedulerNode) {
     Set<String> values = targetExpression.getTargetValues();
@@ -138,45 +138,67 @@ public final class PlacementConstraintsUtil {
         return true;
       }
 
-      if (schedulerNode.getNodeAttributes() == null ||
-          !schedulerNode.getNodeAttributes().contains(requestAttribute)) {
-        if(LOG.isDebugEnabled()) {
+      return getNodeConstraintEvaluatedResult(schedulerNode, opCode,
+          requestAttribute);
+    }
+    return true;
+  }
+
+  private static boolean getNodeConstraintEvaluatedResult(
+      SchedulerNode schedulerNode,
+      NodeAttributeOpCode opCode, NodeAttribute requestAttribute) {
+    // In case, attributes in a node is empty or incoming attributes doesn't
+    // exist on given node, accept such nodes for scheduling if opCode is
+    // equals to NE. (for eg. java != 1.8 could be scheduled on a node
+    // where java is not configured.)
+    if (schedulerNode.getNodeAttributes() == null ||
+        !schedulerNode.getNodeAttributes().contains(requestAttribute)) {
+      if (opCode == NodeAttributeOpCode.NE) {
+        if (LOG.isDebugEnabled()) {
           LOG.debug("Incoming requestAttribute:" + requestAttribute
-              + "is not present in " + schedulerNode.getNodeID());
+              + "is not present in " + schedulerNode.getNodeID()
+              + ", however opcode is NE. Hence accept this node.");
         }
-        return false;
+        return true;
       }
-      boolean found = false;
-      for (Iterator<NodeAttribute> it = schedulerNode.getNodeAttributes()
-          .iterator(); it.hasNext();) {
-        NodeAttribute nodeAttribute = it.next();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Starting to compare Incoming requestAttribute :"
-              + requestAttribute
-              + " with requestAttribute value= " + requestAttribute
-              .getAttributeValue()
-              + ", stored nodeAttribute value=" + nodeAttribute
-              .getAttributeValue());
-        }
-        if (requestAttribute.equals(nodeAttribute)) {
-          if (isOpCodeMatches(requestAttribute, nodeAttribute, opCode)) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(
-                  "Incoming requestAttribute:" + requestAttribute
-                      + " matches with node:" + schedulerNode.getNodeID());
-            }
-            found = true;
-            return found;
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Incoming requestAttribute:" + requestAttribute
+            + "is not present in " + schedulerNode.getNodeID()
+            + ", skip such node.");
+      }
+      return false;
+    }
+
+    boolean found = false;
+    for (Iterator<NodeAttribute> it = schedulerNode.getNodeAttributes()
+        .iterator(); it.hasNext();) {
+      NodeAttribute nodeAttribute = it.next();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Starting to compare Incoming requestAttribute :"
+            + requestAttribute
+            + " with requestAttribute value= " + requestAttribute
+            .getAttributeValue()
+            + ", stored nodeAttribute value=" + nodeAttribute
+            .getAttributeValue());
+      }
+      if (requestAttribute.equals(nodeAttribute)) {
+        if (isOpCodeMatches(requestAttribute, nodeAttribute, opCode)) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                "Incoming requestAttribute:" + requestAttribute
+                    + " matches with node:" + schedulerNode.getNodeID());
           }
+          found = true;
+          return found;
         }
       }
-      if (!found) {
-        if(LOG.isDebugEnabled()) {
-          LOG.info("skip this node:" + schedulerNode.getNodeID()
-              + " for requestAttribute:" + requestAttribute);
-        }
-        return false;
+    }
+    if (!found) {
+      if (LOG.isDebugEnabled()) {
+        LOG.info("skip this node:" + schedulerNode.getNodeID()
+            + " for requestAttribute:" + requestAttribute);
       }
+      return false;
     }
     return true;
   }
@@ -217,7 +239,7 @@ public final class PlacementConstraintsUtil {
         }
       } else if (currentExp.getTargetType().equals(TargetType.NODE_ATTRIBUTE)) {
         // This is a node attribute expression, check it.
-        if (!canSatisfyNodeConstraintExpresssion(singleConstraint, currentExp,
+        if (!canSatisfyNodeConstraintExpression(singleConstraint, currentExp,
             schedulerNode)) {
           return false;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52194351/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
index b13790d..c692bae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
@@ -22,16 +22,7 @@ import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.NodeState;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceInformation;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.*;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -205,7 +196,14 @@ public class TestUtils {
     ApplicationId applicationId = BuilderUtils.newApplicationId(0l, appId);
     return ApplicationAttemptId.newInstance(applicationId, attemptId);
   }
-  
+
+  public static FiCaSchedulerNode getMockNodeWithAttributes(String host,
+      String rack, int port, int memory, Set<NodeAttribute> attributes) {
+    FiCaSchedulerNode node = getMockNode(host, rack, port, memory, 1);
+    when(node.getNodeAttributes()).thenReturn(attributes);
+    return node;
+  }
+
   public static FiCaSchedulerNode getMockNode(String host, String rack,
       int port, int memory) {
     return getMockNode(host, rack, port, memory, 1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52194351/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java
index 902c6d5..a665b8d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java
@@ -18,14 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
 
+import org.apache.hadoop.yarn.api.records.*;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTags;
-import org.apache.hadoop.yarn.api.records.ExecutionType;
-import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceSizing;
-import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
 import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -44,6 +38,8 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import java.util.HashSet;
+import java.util.Set;
 import java.util.function.LongBinaryOperator;
 
 import static org.mockito.Matchers.any;
@@ -326,4 +322,186 @@ public class TestSingleConstraintAppPlacementAllocator {
     Assert.assertFalse(allocator
         .precheckNode(node2, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY));
   }
+
+  @Test
+  public void testNodeAttributesFunctionality() {
+    // 1. Simple java=1.8 validation
+    SchedulingRequest schedulingRequest =
+        SchedulingRequest.newBuilder().executionType(
+            ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+            .allocationRequestId(10L).priority(Priority.newInstance(1))
+            .placementConstraintExpression(PlacementConstraints
+                .targetNodeAttribute(PlacementConstraints.NODE,
+                    NodeAttributeOpCode.EQ,
+                    PlacementConstraints.PlacementTargets
+                        .nodeAttribute("java", "1.8"),
+                    PlacementConstraints.PlacementTargets.nodePartition(""))
+                .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+            .build();
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
+    Set<NodeAttribute> attributes = new HashSet<>();
+    attributes.add(
+        NodeAttribute.newInstance("java", NodeAttributeType.STRING, "1.8"));
+    boolean result = allocator.canAllocate(NodeType.NODE_LOCAL,
+        TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
+            attributes));
+    Assert.assertTrue("Allocation should be success for java=1.8", result);
+
+    // 2. verify python!=3 validation
+    SchedulingRequest schedulingRequest2 =
+        SchedulingRequest.newBuilder().executionType(
+            ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+            .allocationRequestId(10L).priority(Priority.newInstance(1))
+            .placementConstraintExpression(PlacementConstraints
+                .targetNodeAttribute(PlacementConstraints.NODE,
+                    NodeAttributeOpCode.NE,
+                    PlacementConstraints.PlacementTargets
+                        .nodeAttribute("python", "3"),
+                    PlacementConstraints.PlacementTargets.nodePartition(""))
+                .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+            .build();
+    // Create allocator
+    allocator = new SingleConstraintAppPlacementAllocator();
+    allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest2, false);
+    attributes = new HashSet<>();
+    result = allocator.canAllocate(NodeType.NODE_LOCAL,
+        TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
+            attributes));
+    Assert.assertTrue("Allocation should be success as python doesn't exist",
+        result);
+
+    // 3. verify python!=3 validation when node has python=2
+    allocator = new SingleConstraintAppPlacementAllocator();
+    allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest2, false);
+    attributes = new HashSet<>();
+    attributes.add(
+        NodeAttribute.newInstance("python", NodeAttributeType.STRING, "2"));
+    result = allocator.canAllocate(NodeType.NODE_LOCAL,
+        TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
+            attributes));
+    Assert.assertTrue(
+        "Allocation should be success as python=3 doesn't exist in node",
+        result);
+
+    // 4. verify python!=3 validation when node has python=3
+    allocator = new SingleConstraintAppPlacementAllocator();
+    allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest2, false);
+    attributes = new HashSet<>();
+    attributes.add(
+        NodeAttribute.newInstance("python", NodeAttributeType.STRING, "3"));
+    result = allocator.canAllocate(NodeType.NODE_LOCAL,
+        TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
+            attributes));
+    Assert.assertFalse("Allocation should fail as python=3 exist in node",
+        result);
+  }
+
+  @Test
+  public void testConjunctionNodeAttributesFunctionality() {
+    // 1. verify and(python!=3:java=1.8) validation when node has python=3
+    SchedulingRequest schedulingRequest1 =
+        SchedulingRequest.newBuilder().executionType(
+            ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+            .allocationRequestId(10L).priority(Priority.newInstance(1))
+            .placementConstraintExpression(
+                PlacementConstraints.and(
+                    PlacementConstraints
+                        .targetNodeAttribute(PlacementConstraints.NODE,
+                            NodeAttributeOpCode.NE,
+                            PlacementConstraints.PlacementTargets
+                                .nodeAttribute("python", "3")),
+                    PlacementConstraints
+                        .targetNodeAttribute(PlacementConstraints.NODE,
+                            NodeAttributeOpCode.EQ,
+                            PlacementConstraints.PlacementTargets
+                                .nodeAttribute("java", "1.8")))
+                    .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+            .build();
+    allocator = new SingleConstraintAppPlacementAllocator();
+    allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest1, false);
+    Set<NodeAttribute> attributes = new HashSet<>();
+    attributes.add(
+        NodeAttribute.newInstance("python", NodeAttributeType.STRING, "3"));
+    attributes.add(
+        NodeAttribute.newInstance("java", NodeAttributeType.STRING, "1.8"));
+    boolean result = allocator.canAllocate(NodeType.NODE_LOCAL,
+        TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
+            attributes));
+    Assert.assertFalse("Allocation should fail as python=3 exists in node",
+        result);
+
+    // 2. verify and(python!=3:java=1.8) validation when node has python=2
+    // and java=1.8
+    allocator = new SingleConstraintAppPlacementAllocator();
+    allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest1, false);
+    attributes = new HashSet<>();
+    attributes.add(
+        NodeAttribute.newInstance("python", NodeAttributeType.STRING, "2"));
+    attributes.add(
+        NodeAttribute.newInstance("java", NodeAttributeType.STRING, "1.8"));
+    result = allocator.canAllocate(NodeType.NODE_LOCAL,
+        TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
+            attributes));
+    Assert.assertTrue("Allocation should be success as python=2 exists in node",
+        result);
+
+    // 3. verify or(python!=3:java=1.8) validation when node has python=3
+    SchedulingRequest schedulingRequest2 =
+        SchedulingRequest.newBuilder().executionType(
+            ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+            .allocationRequestId(10L).priority(Priority.newInstance(1))
+            .placementConstraintExpression(
+                PlacementConstraints.or(
+                    PlacementConstraints
+                        .targetNodeAttribute(PlacementConstraints.NODE,
+                            NodeAttributeOpCode.NE,
+                            PlacementConstraints.PlacementTargets
+                                .nodeAttribute("python", "3")),
+                    PlacementConstraints
+                        .targetNodeAttribute(PlacementConstraints.NODE,
+                            NodeAttributeOpCode.EQ,
+                            PlacementConstraints.PlacementTargets
+                                .nodeAttribute("java", "1.8")))
+                    .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+            .build();
+    allocator = new SingleConstraintAppPlacementAllocator();
+    allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest2, false);
+    attributes = new HashSet<>();
+    attributes.add(
+        NodeAttribute.newInstance("python", NodeAttributeType.STRING, "3"));
+    attributes.add(
+        NodeAttribute.newInstance("java", NodeAttributeType.STRING, "1.8"));
+    result = allocator.canAllocate(NodeType.NODE_LOCAL,
+        TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
+            attributes));
+    Assert.assertTrue("Allocation should be success as java=1.8 exists in node",
+        result);
+
+    // 4. verify or(python!=3:java=1.8) validation when node has python=3
+    // and java=1.7.
+    allocator = new SingleConstraintAppPlacementAllocator();
+    allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+    allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest2, false);
+    attributes = new HashSet<>();
+    attributes.add(
+        NodeAttribute.newInstance("python", NodeAttributeType.STRING, "3"));
+    attributes.add(
+        NodeAttribute.newInstance("java", NodeAttributeType.STRING, "1.7"));
+    result = allocator.canAllocate(NodeType.NODE_LOCAL,
+        TestUtils.getMockNodeWithAttributes("host1", "/rack1", 123, 1024,
+            attributes));
+    Assert
+        .assertFalse("Allocation should fail as java=1.8 doesnt exist in node",
+            result);
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/29] hadoop git commit: YARN-8092. Expose Node Attributes info via RM nodes REST API. Contributed by Weiwei Yang.

Posted by su...@apache.org.
YARN-8092. Expose Node Attributes info via RM nodes REST API. Contributed by Weiwei Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/89b3ebd1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/89b3ebd1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/89b3ebd1

Branch: refs/heads/trunk
Commit: 89b3ebd11ea93bf9d3521677f0fb9ab3c373a6ca
Parents: 440ff7f
Author: Sunil G <su...@apache.org>
Authored: Sun Apr 1 18:07:07 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/sls/nodemanager/NodeInfo.java   | 13 ++++
 .../yarn/sls/scheduler/RMNodeWrapper.java       | 12 ++++
 .../resourcemanager/ResourceTrackerService.java |  4 ++
 .../server/resourcemanager/rmnode/RMNode.java   | 13 ++++
 .../resourcemanager/rmnode/RMNodeImpl.java      | 15 +++++
 .../webapp/dao/NodeAttributeInfo.java           | 65 ++++++++++++++++++++
 .../webapp/dao/NodeAttributesInfo.java          | 49 +++++++++++++++
 .../resourcemanager/webapp/dao/NodeInfo.java    | 15 +++++
 .../yarn/server/resourcemanager/MockNodes.java  | 11 ++++
 9 files changed, 197 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/89b3ebd1/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
index 69946c8..65b8da0 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -219,6 +220,18 @@ public class NodeInfo {
       return null;
     }
 
+
+    @Override
+    public void setNodeAttributes(String prefix,
+        Set<NodeAttribute> nodeAttributes) {
+
+    }
+
+    @Override
+    public Map<String, Set<NodeAttribute>> getAllNodeAttributes() {
+      return null;
+    }
+
     @Override
     public RMContext getRMContext() {
       return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89b3ebd1/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
index a96b790..bf61f54 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.net.Node;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -206,6 +207,17 @@ public class RMNodeWrapper implements RMNode {
   public Map<String, Long> getAllocationTagsWithCount() {
     return node.getAllocationTagsWithCount();
   }
+  
+  @Override
+  public void setNodeAttributes(String prefix,
+      Set<NodeAttribute> nodeAttributes) {
+    node.setNodeAttributes(prefix, nodeAttributes);
+  }
+
+  @Override
+  public Map<String, Set<NodeAttribute>> getAllNodeAttributes() {
+    return node.getAllNodeAttributes();
+  }
 
   @Override
   public RMContext getRMContext() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89b3ebd1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index 8a1a9a7..4f4400f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -673,6 +673,10 @@ public class ResourceTrackerService extends AbstractService implements
         this.rmContext.getNodeAttributesManager()
             .replaceNodeAttributes(NodeAttribute.PREFIX_DISTRIBUTED,
                 ImmutableMap.of(nodeId.getHost(), nodeAttributes));
+
+        // Update node attributes to RMNode
+        rmNode.setNodeAttributes(NodeAttribute.PREFIX_DISTRIBUTED,
+            nodeAttributes);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89b3ebd1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
index 68a780e..33e5ef4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 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.NodeAttribute;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -195,4 +196,16 @@ public interface RMNode {
    * @return the RM context associated with this RM node.
    */
   RMContext getRMContext();
+
+ /**
+   * Sets node attributes per prefix.
+   * @param prefix node attribute prefix
+   * @param nodeAttributes node attributes
+   */
+  void setNodeAttributes(String prefix, Set<NodeAttribute> nodeAttributes);
+
+  /**
+   * @return all node attributes grouped by their prefix as a map.
+   */
+  Map<String, Set<NodeAttribute>> getAllNodeAttributes();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89b3ebd1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
index dfd93e2..e52b621 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -185,6 +186,9 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   private NodeHeartbeatResponse latestNodeHeartBeatResponse = recordFactory
       .newRecordInstance(NodeHeartbeatResponse.class);
 
+  // Node attributes, store by prefix
+  private Map<String, Set<NodeAttribute>> nodeAttributes = new HashMap<>();
+
   private static final StateMachineFactory<RMNodeImpl,
                                            NodeState,
                                            RMNodeEventType,
@@ -1546,4 +1550,15 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   public RMContext getRMContext() {
     return this.context;
   }
+
+  @Override
+  public void setNodeAttributes(String prefix,
+      Set<NodeAttribute> nodeAttributeSet) {
+    this.nodeAttributes.put(prefix, nodeAttributeSet);
+  }
+
+  @Override
+  public Map<String, Set<NodeAttribute>> getAllNodeAttributes() {
+    return this.nodeAttributes;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89b3ebd1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeAttributeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeAttributeInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeAttributeInfo.java
new file mode 100644
index 0000000..bbc2ec3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeAttributeInfo.java
@@ -0,0 +1,65 @@
+/**
+ * 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 org.apache.hadoop.yarn.api.records.NodeAttribute;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * DAO for node an attribute record.
+ */
+@XmlRootElement(name = "nodeAttributeInfo")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class NodeAttributeInfo {
+
+  private String prefix;
+  private String name;
+  private String type;
+  private String value;
+
+  public NodeAttributeInfo() {
+    // JAXB needs this
+  }
+
+  public NodeAttributeInfo(NodeAttribute nodeAttribute) {
+    this.prefix = nodeAttribute.getAttributePrefix();
+    this.name = nodeAttribute.getAttributeName();
+    this.type = nodeAttribute.getAttributeType().toString();
+    this.value = nodeAttribute.getAttributeValue();
+  }
+
+  public String getPrefix() {
+    return prefix;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public String getValue() {
+    return value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89b3ebd1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeAttributesInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeAttributesInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeAttributesInfo.java
new file mode 100644
index 0000000..1f4c25f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeAttributesInfo.java
@@ -0,0 +1,49 @@
+/**
+ * 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;
+import java.util.ArrayList;
+
+/**
+ * DAO for a list of node attributes info.
+ */
+@XmlRootElement(name = "nodeAttributesInfo")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class NodeAttributesInfo {
+
+  @XmlElement(name = "nodeAttributeInfo")
+  private ArrayList<NodeAttributeInfo> nodeAttributesInfo =
+      new ArrayList<>();
+
+  public NodeAttributesInfo() {
+    // JAXB needs this
+  }
+
+  public void addNodeAttributeInfo(NodeAttributeInfo attributeInfo) {
+    this.nodeAttributesInfo.add(attributeInfo);
+  }
+
+  public ArrayList<NodeAttributeInfo> getNodeAttributesInfo() {
+    return nodeAttributesInfo;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89b3ebd1/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 46a6e60..ecf66b9 100644
--- a/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
+++ b/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
@@ -27,6 +27,7 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
@@ -62,6 +63,7 @@ public class NodeInfo {
   protected ResourceUtilizationInfo resourceUtilization;
   protected ResourceInfo usedResource;
   protected ResourceInfo availableResource;
+  protected NodeAttributesInfo nodeAttributesInfo;
 
   public NodeInfo() {
   } // JAXB needs this
@@ -113,6 +115,19 @@ public class NodeInfo {
       Collections.sort(nodeLabels);
     }
 
+    // add attributes
+    Map<String, Set<NodeAttribute>> nodeAttributes =
+        ni.getAllNodeAttributes();
+    nodeAttributesInfo = new NodeAttributesInfo();
+    if (nodeAttributes != null) {
+      for (Set<NodeAttribute> attrs : nodeAttributes.values()) {
+        for (NodeAttribute attribute : attrs) {
+          NodeAttributeInfo info = new NodeAttributeInfo(attribute);
+          this.nodeAttributesInfo.addNodeAttributeInfo(info);
+        }
+      }
+    }
+
     // add allocation tags
     allocationTags = new AllocationTagsInfo();
     Map<String, Long> allocationTagsInfo = ni.getAllocationTagsWithCount();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89b3ebd1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
index c444b6e..b6c0cc1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.net.Node;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -284,6 +285,16 @@ public class MockNodes {
     public Map<String, Long> getAllocationTagsWithCount() {
       return null;
     }
+    
+    public void setNodeAttributes(String prefix,
+        Set<NodeAttribute> nodeAttributes) {
+
+    }
+
+    @Override
+    public Map<String, Set<NodeAttribute>> getAllNodeAttributes() {
+      return null;
+    }
 
     @Override
     public RMContext getRMContext() {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[11/29] hadoop git commit: YARN-7875. Node Attribute store for storing and recovering attributes. Contributed by Bibin A Chundatt.

Posted by su...@apache.org.
YARN-7875. Node Attribute store for storing and recovering attributes. Contributed by Bibin A Chundatt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b9890d1f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b9890d1f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b9890d1f

Branch: refs/heads/trunk
Commit: b9890d1f6604d572cd9f1cb719e2a4da08b31eff
Parents: a6590c1
Author: Sunil G <su...@apache.org>
Authored: Fri Apr 6 07:09:27 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  16 ++
 .../yarn/nodelabels/NodeAttributeStore.java     |  77 ++++++
 .../yarn/nodelabels/NodeAttributesManager.java  |  11 +
 .../hadoop/yarn/nodelabels/RMNodeAttribute.java |   6 -
 .../nodelabels/store/AbstractFSNodeStore.java   |   2 +-
 .../yarn/nodelabels/store/FSStoreOpHandler.java |  21 +-
 .../store/op/AddNodeToAttributeLogOp.java       |  71 +++++
 .../nodelabels/store/op/FSNodeStoreLogOp.java   |  17 ++
 .../store/op/NodeAttributeMirrorOp.java         |  64 +++++
 .../store/op/RemoveNodeToAttributeLogOp.java    |  71 +++++
 .../store/op/ReplaceNodeToAttributeLogOp.java   |  73 ++++++
 .../yarn/nodelabels/store/op/package-info.java  |  21 ++
 .../src/main/resources/yarn-default.xml         |  16 ++
 .../FileSystemNodeAttributeStore.java           | 102 ++++++++
 .../nodelabels/NodeAttributesManagerImpl.java   | 100 ++++++-
 .../TestResourceTrackerService.java             |  10 +
 .../TestFileSystemNodeAttributeStore.java       | 260 +++++++++++++++++++
 .../nodelabels/TestNodeAttributesManager.java   |  13 +-
 18 files changed, 935 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 5ca9998..edad4d2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -3481,6 +3481,22 @@ public class YarnConfiguration extends Configuration {
       + "fs-store.root-dir";
 
   /**
+   * Node-attribute configurations.
+   */
+  public static final String NODE_ATTRIBUTE_PREFIX =
+      YARN_PREFIX + "node-attribute.";
+  /**
+   * Node attribute store implementation class.
+   */
+  public static final String FS_NODE_ATTRIBUTE_STORE_IMPL_CLASS =
+      NODE_ATTRIBUTE_PREFIX + "fs-store.impl.class";
+  /**
+   * File system not attribute store directory.
+   */
+  public static final String FS_NODE_ATTRIBUTE_STORE_ROOT_DIR =
+      NODE_ATTRIBUTE_PREFIX + "fs-store.root-dir";
+
+  /**
    * Flag to indicate if the node labels feature enabled, by default it's
    * disabled
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributeStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributeStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributeStore.java
new file mode 100644
index 0000000..8e9f9ff
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributeStore.java
@@ -0,0 +1,77 @@
+/**
+ * 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.nodelabels;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Interface class for Node label store.
+ */
+public interface NodeAttributeStore extends Closeable {
+
+  /**
+   * Replace labels on node.
+   *
+   * @param nodeToAttribute node to attribute list.
+   * @throws IOException
+   */
+  void replaceNodeAttributes(List<NodeToAttributes> nodeToAttribute)
+      throws IOException;
+
+  /**
+   * Add attribute to node.
+   *
+   * @param nodeToAttribute node to attribute list.
+   * @throws IOException
+   */
+  void addNodeAttributes(List<NodeToAttributes> nodeToAttribute)
+      throws IOException;
+
+  /**
+   * Remove attribute from node.
+   *
+   * @param nodeToAttribute node to attribute list.
+   * @throws IOException
+   */
+  void removeNodeAttributes(List<NodeToAttributes> nodeToAttribute)
+      throws IOException;
+
+  /**
+   * Initialize based on configuration and NodeAttributesManager.
+   *
+   * @param configuration configuration instance.
+   * @param mgr nodeattributemanager instance.
+   * @throws Exception
+   */
+  void init(Configuration configuration, NodeAttributesManager mgr)
+      throws Exception;
+
+  /**
+   * Recover store on resourcemanager startup.
+   * @throws IOException
+   * @throws YarnException
+   */
+  void recover() throws IOException, YarnException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
index ffa33cf..ec7d30d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
@@ -19,11 +19,13 @@
 package org.apache.hadoop.yarn.nodelabels;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
 
 /**
  * This class captures all interactions for Attributes with RM.
@@ -101,6 +103,15 @@ public abstract class NodeAttributesManager extends AbstractService {
   public abstract Map<NodeAttribute, AttributeValue> getAttributesForNode(
       String hostName);
 
+  /**
+   * Get All node to Attributes list based on filter.
+   *
+   * @return List<NodeToAttributes> nodeToAttributes matching filter.If empty
+   * or null is passed as argument will return all.
+   */
+  public abstract List<NodeToAttributes> getNodeToAttributes(
+      Set<String> prefix);
+
   // futuristic
   // public set<NodeId> getNodesMatchingExpression(String nodeLabelExp);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java
index 5a709c6..3b2bd16 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java
@@ -53,12 +53,6 @@ public class RMNodeAttribute extends AbstractLabel {
     this.attribute = attribute;
   }
 
-  public RMNodeAttribute(String attributeName) {
-    super(attributeName);
-    attribute = NodeAttribute.newInstance(attributeName,
-        NodeAttributeType.STRING, CommonNodeLabelsManager.NO_LABEL);
-  }
-
   public NodeAttributeType getAttributeType() {
     return attribute.getAttributeType();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java
index a47cacf..216fc79 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java
@@ -64,7 +64,7 @@ public abstract class AbstractFSNodeStore<M> {
     initFileSystem(conf);
     // mkdir of root dir path
     fs.mkdirs(fsWorkingPath);
-
+    LOG.info("Created store directory :" + fsWorkingPath);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java
index 0f7f53d..a626537 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java
@@ -17,13 +17,17 @@
  */
 package org.apache.hadoop.yarn.nodelabels.store;
 
-import static org.apache.hadoop.yarn.nodelabels.store.FSStoreOpHandler
-    .StoreType.NODE_LABEL_STORE;
+import static org.apache.hadoop.yarn.nodelabels.store.FSStoreOpHandler.StoreType.NODE_ATTRIBUTE;
+import static org.apache.hadoop.yarn.nodelabels.store.FSStoreOpHandler.StoreType.NODE_LABEL_STORE;
 import org.apache.hadoop.yarn.nodelabels.store.op.AddClusterLabelOp;
+import org.apache.hadoop.yarn.nodelabels.store.op.AddNodeToAttributeLogOp;
 import org.apache.hadoop.yarn.nodelabels.store.op.FSNodeStoreLogOp;
+import org.apache.hadoop.yarn.nodelabels.store.op.NodeAttributeMirrorOp;
 import org.apache.hadoop.yarn.nodelabels.store.op.NodeLabelMirrorOp;
 import org.apache.hadoop.yarn.nodelabels.store.op.NodeToLabelOp;
 import org.apache.hadoop.yarn.nodelabels.store.op.RemoveClusterLabelOp;
+import org.apache.hadoop.yarn.nodelabels.store.op.RemoveNodeToAttributeLogOp;
+import org.apache.hadoop.yarn.nodelabels.store.op.ReplaceNodeToAttributeLogOp;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -39,7 +43,7 @@ public class FSStoreOpHandler {
 
   public enum StoreType {
     NODE_LABEL_STORE,
-    NODE_LABEL_ATTRIBUTE;
+    NODE_ATTRIBUTE
   }
 
   static {
@@ -47,13 +51,24 @@ public class FSStoreOpHandler {
     mirrorOp = new HashMap<>();
 
     // registerLog edit log operation
+
+    //Node Label Operations
     registerLog(NODE_LABEL_STORE, AddClusterLabelOp.OPCODE, AddClusterLabelOp.class);
     registerLog(NODE_LABEL_STORE, NodeToLabelOp.OPCODE, NodeToLabelOp.class);
     registerLog(NODE_LABEL_STORE, RemoveClusterLabelOp.OPCODE, RemoveClusterLabelOp.class);
 
+    //NodeAttibute operation
+    registerLog(NODE_ATTRIBUTE, AddNodeToAttributeLogOp.OPCODE, AddNodeToAttributeLogOp.class);
+    registerLog(NODE_ATTRIBUTE, RemoveNodeToAttributeLogOp.OPCODE, RemoveNodeToAttributeLogOp.class);
+    registerLog(NODE_ATTRIBUTE, ReplaceNodeToAttributeLogOp.OPCODE, ReplaceNodeToAttributeLogOp.class);
+
     // registerLog Mirror op
 
+    // Node label mirror operation
     registerMirror(NODE_LABEL_STORE, NodeLabelMirrorOp.class);
+    //Node attribute mirror operation
+    registerMirror(NODE_ATTRIBUTE, NodeAttributeMirrorOp.class);
+
   }
 
   private static void registerMirror(StoreType type,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/AddNodeToAttributeLogOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/AddNodeToAttributeLogOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/AddNodeToAttributeLogOp.java
new file mode 100644
index 0000000..4b92bcf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/AddNodeToAttributeLogOp.java
@@ -0,0 +1,71 @@
+/**
+ * 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.nodelabels.store.op;
+
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingRequestPBImpl;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+
+/**
+ * File system Add Node to attribute mapping.
+ */
+public class AddNodeToAttributeLogOp
+    extends FSNodeStoreLogOp<NodeAttributesManager> {
+
+  private List<NodeToAttributes> attributes;
+
+  public static final int OPCODE = 0;
+
+  @Override
+  public void write(OutputStream os, NodeAttributesManager mgr)
+      throws IOException {
+    ((NodesToAttributesMappingRequestPBImpl) NodesToAttributesMappingRequest
+        .newInstance(AttributeMappingOperationType.ADD, attributes, false))
+        .getProto().writeDelimitedTo(os);
+  }
+
+  @Override
+  public void recover(InputStream is, NodeAttributesManager mgr)
+      throws IOException {
+    NodesToAttributesMappingRequest request =
+        new NodesToAttributesMappingRequestPBImpl(
+            YarnServerResourceManagerServiceProtos
+                .NodesToAttributesMappingRequestProto
+                .parseDelimitedFrom(is));
+    mgr.addNodeAttributes(getNodeToAttributesMap(request));
+  }
+
+  public AddNodeToAttributeLogOp setAttributes(
+      List<NodeToAttributes> attributesList) {
+    this.attributes = attributesList;
+    return this;
+  }
+
+  @Override
+  public int getOpCode() {
+    return OPCODE;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/FSNodeStoreLogOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/FSNodeStoreLogOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/FSNodeStoreLogOp.java
index cd739c0..bf4d1b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/FSNodeStoreLogOp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/FSNodeStoreLogOp.java
@@ -17,10 +17,18 @@
  */
 package org.apache.hadoop.yarn.nodelabels.store.op;
 
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.nodelabels.store.StoreOp;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
 
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * Defines all FileSystem editlog operation. All node label and attribute
@@ -32,4 +40,13 @@ public abstract class FSNodeStoreLogOp<M>
     implements StoreOp<OutputStream, InputStream, M> {
 
   public abstract int getOpCode();
+
+  protected Map<String, Set<NodeAttribute>> getNodeToAttributesMap(
+      NodesToAttributesMappingRequest request) {
+    List<NodeToAttributes> attributes = request.getNodesToAttributes();
+    Map<String, Set<NodeAttribute>> nodeToAttrMap = new HashMap<>();
+    attributes.forEach((v) -> nodeToAttrMap
+        .put(v.getNode(), new HashSet<>(v.getNodeAttributes())));
+    return nodeToAttrMap;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeAttributeMirrorOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeAttributeMirrorOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeAttributeMirrorOp.java
new file mode 100644
index 0000000..dca0555
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeAttributeMirrorOp.java
@@ -0,0 +1,64 @@
+/**
+ * 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.nodelabels.store.op;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingRequestPBImpl;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * File System Node Attribute Mirror read and write operation.
+ */
+public class NodeAttributeMirrorOp
+    extends FSNodeStoreLogOp<NodeAttributesManager> {
+
+  @Override
+  public void write(OutputStream os, NodeAttributesManager mgr)
+      throws IOException {
+    ((NodesToAttributesMappingRequestPBImpl) NodesToAttributesMappingRequest
+        .newInstance(AttributeMappingOperationType.REPLACE,
+            mgr.getNodeToAttributes(
+                ImmutableSet.of(NodeAttribute.PREFIX_CENTRALIZED)), false))
+        .getProto().writeDelimitedTo(os);
+  }
+
+  @Override
+  public void recover(InputStream is, NodeAttributesManager mgr)
+      throws IOException {
+    NodesToAttributesMappingRequest request =
+        new NodesToAttributesMappingRequestPBImpl(
+            YarnServerResourceManagerServiceProtos
+                .NodesToAttributesMappingRequestProto
+                .parseDelimitedFrom(is));
+    mgr.replaceNodeAttributes(NodeAttribute.PREFIX_CENTRALIZED,
+        getNodeToAttributesMap(request));
+  }
+
+  @Override
+  public int getOpCode() {
+    return -1;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/RemoveNodeToAttributeLogOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/RemoveNodeToAttributeLogOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/RemoveNodeToAttributeLogOp.java
new file mode 100644
index 0000000..1d13077
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/RemoveNodeToAttributeLogOp.java
@@ -0,0 +1,71 @@
+/**
+ * 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.nodelabels.store.op;
+
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingRequestPBImpl;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+
+/**
+ * File system remove node attribute from node operation.
+ */
+public class RemoveNodeToAttributeLogOp
+    extends FSNodeStoreLogOp<NodeAttributesManager> {
+
+  private List<NodeToAttributes> attributes;
+
+  public static final int OPCODE = 1;
+
+  @Override
+  public void write(OutputStream os, NodeAttributesManager mgr)
+      throws IOException {
+    ((NodesToAttributesMappingRequestPBImpl) NodesToAttributesMappingRequest
+        .newInstance(AttributeMappingOperationType.REMOVE, attributes, false))
+        .getProto().writeDelimitedTo(os);
+  }
+
+  @Override
+  public void recover(InputStream is, NodeAttributesManager mgr)
+      throws IOException {
+    NodesToAttributesMappingRequest request =
+        new NodesToAttributesMappingRequestPBImpl(
+            YarnServerResourceManagerServiceProtos
+                .NodesToAttributesMappingRequestProto
+                .parseDelimitedFrom(is));
+    mgr.removeNodeAttributes(getNodeToAttributesMap(request));
+  }
+
+  public RemoveNodeToAttributeLogOp setAttributes(
+      List<NodeToAttributes> attrs) {
+    this.attributes = attrs;
+    return this;
+  }
+
+  @Override
+  public int getOpCode() {
+    return OPCODE;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/ReplaceNodeToAttributeLogOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/ReplaceNodeToAttributeLogOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/ReplaceNodeToAttributeLogOp.java
new file mode 100644
index 0000000..54d7651
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/ReplaceNodeToAttributeLogOp.java
@@ -0,0 +1,73 @@
+/**
+ * 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.nodelabels.store.op;
+
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingRequestPBImpl;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+
+/**
+ * File system replace node attribute from node operation.
+ */
+public class ReplaceNodeToAttributeLogOp
+    extends FSNodeStoreLogOp<NodeAttributesManager> {
+
+  private List<NodeToAttributes> attributes;
+  public static final int OPCODE = 2;
+
+  @Override
+  public void write(OutputStream os, NodeAttributesManager mgr)
+      throws IOException {
+    ((NodesToAttributesMappingRequestPBImpl) NodesToAttributesMappingRequest
+        .newInstance(AttributeMappingOperationType.REPLACE, attributes, false))
+        .getProto().writeDelimitedTo(os);
+  }
+
+  @Override
+  public void recover(InputStream is, NodeAttributesManager mgr)
+      throws IOException {
+    NodesToAttributesMappingRequest request =
+        new NodesToAttributesMappingRequestPBImpl(
+            YarnServerResourceManagerServiceProtos
+                .NodesToAttributesMappingRequestProto
+                .parseDelimitedFrom(is));
+    //Only CENTRALIZED is stored to FS system
+    mgr.replaceNodeAttributes(NodeAttribute.PREFIX_CENTRALIZED,
+        getNodeToAttributesMap(request));
+  }
+
+  public ReplaceNodeToAttributeLogOp setAttributes(
+      List<NodeToAttributes> attrs) {
+    this.attributes = attrs;
+    return this;
+  }
+
+  @Override
+  public int getOpCode() {
+    return OPCODE;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/package-info.java
new file mode 100644
index 0000000..f6fb3d3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.nodelabels.store.op;
+import org.apache.hadoop.classification.InterfaceAudience;
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index cdc3c09..0700902 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -3965,4 +3965,20 @@
     <name>yarn.nodemanager.elastic-memory-control.timeout-sec</name>
     <value>5</value>
   </property>
+  <property>
+    <description>
+      URI for NodeAttributeManager. The default value is
+      /tmp/hadoop-yarn-${user}/node-attribute/ in the local filesystem.
+    </description>
+    <name>yarn.node-attribute.fs-store.root-dir</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+      Choose different implementation of node attribute's storage
+    </description>
+    <name>yarn.node-attribute.fs-store.impl.class</name>
+    <value>org.apache.hadoop.yarn.server.resourcemanager.nodelabels.FileSystemNodeAttributeStore</value>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/FileSystemNodeAttributeStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/FileSystemNodeAttributeStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/FileSystemNodeAttributeStore.java
new file mode 100644
index 0000000..01df250
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/FileSystemNodeAttributeStore.java
@@ -0,0 +1,102 @@
+/**
+ * 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.nodelabels;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributeStore;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
+import org.apache.hadoop.yarn.nodelabels.store.AbstractFSNodeStore;
+import org.apache.hadoop.yarn.nodelabels.store.FSStoreOpHandler;
+import org.apache.hadoop.yarn.nodelabels.store.op.AddNodeToAttributeLogOp;
+import org.apache.hadoop.yarn.nodelabels.store.op.RemoveNodeToAttributeLogOp;
+import org.apache.hadoop.yarn.nodelabels.store.op.ReplaceNodeToAttributeLogOp;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * File system node attribute implementation.
+ */
+public class FileSystemNodeAttributeStore
+    extends AbstractFSNodeStore<NodeAttributesManager>
+    implements NodeAttributeStore {
+
+  protected static final Log LOG =
+      LogFactory.getLog(FileSystemNodeAttributeStore.class);
+
+  protected static final String DEFAULT_DIR_NAME = "node-attribute";
+  protected static final String MIRROR_FILENAME = "nodeattribute.mirror";
+  protected static final String EDITLOG_FILENAME = "nodeattribute.editlog";
+
+  public FileSystemNodeAttributeStore() {
+    super(FSStoreOpHandler.StoreType.NODE_ATTRIBUTE);
+  }
+
+  private String getDefaultFSNodeAttributeRootDir() throws IOException {
+    // default is in local: /tmp/hadoop-yarn-${user}/node-attribute/
+    return "file:///tmp/hadoop-yarn-" + UserGroupInformation.getCurrentUser()
+        .getShortUserName() + "/" + DEFAULT_DIR_NAME;
+  }
+
+  @Override
+  public void init(Configuration conf, NodeAttributesManager mgr)
+      throws Exception {
+    StoreSchema schema = new StoreSchema(EDITLOG_FILENAME, MIRROR_FILENAME);
+    initStore(conf, new Path(
+        conf.get(YarnConfiguration.FS_NODE_ATTRIBUTE_STORE_ROOT_DIR,
+            getDefaultFSNodeAttributeRootDir())), schema, mgr);
+  }
+
+  @Override
+  public void replaceNodeAttributes(List<NodeToAttributes> nodeToAttribute)
+      throws IOException {
+    ReplaceNodeToAttributeLogOp op = new ReplaceNodeToAttributeLogOp();
+    writeToLog(op.setAttributes(nodeToAttribute));
+  }
+
+  @Override
+  public void addNodeAttributes(List<NodeToAttributes> nodeAttributeMapping)
+      throws IOException {
+    AddNodeToAttributeLogOp op = new AddNodeToAttributeLogOp();
+    writeToLog(op.setAttributes(nodeAttributeMapping));
+  }
+
+  @Override
+  public void removeNodeAttributes(List<NodeToAttributes> nodeAttributeMapping)
+      throws IOException {
+    RemoveNodeToAttributeLogOp op = new RemoveNodeToAttributeLogOp();
+    writeToLog(op.setAttributes(nodeAttributeMapping));
+  }
+
+  @Override
+  public void recover() throws IOException, YarnException {
+    super.recoverFromStore();
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.closeFSStore();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
index 04d74a8..b4686e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
@@ -32,24 +32,31 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
+import java.util.ArrayList;
+import java.util.List;
 
 import com.google.common.base.Strings;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.nodelabels.AttributeValue;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributeStore;
 import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.nodelabels.NodeLabelUtil;
 import org.apache.hadoop.yarn.nodelabels.RMNodeAttribute;
 import org.apache.hadoop.yarn.nodelabels.StringAttributeValue;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
 
 /**
  * Manager holding the attributes to Labels.
@@ -63,7 +70,8 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
    */
   public static final String EMPTY_ATTRIBUTE_VALUE = "";
 
-  private Dispatcher dispatcher;
+  Dispatcher dispatcher;
+  NodeAttributeStore store;
 
   // TODO may be we can have a better collection here.
   // this will be updated to get the attributeName to NM mapping
@@ -121,7 +129,21 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   }
 
   protected void initNodeAttributeStore(Configuration conf) throws Exception {
-    // TODO to generalize and make use of the FileSystemNodeLabelsStore
+    this.store =getAttributeStoreClass(conf);
+    this.store.init(conf, this);
+    this.store.recover();
+  }
+
+  private NodeAttributeStore getAttributeStoreClass(Configuration conf) {
+    try {
+      return ReflectionUtils.newInstance(
+          conf.getClass(YarnConfiguration.FS_NODE_ATTRIBUTE_STORE_IMPL_CLASS,
+              FileSystemNodeAttributeStore.class, NodeAttributeStore.class),
+          conf);
+    } catch (Exception e) {
+      throw new YarnRuntimeException(
+          "Could not instantiate Node Attribute Store ", e);
+    }
   }
 
   private void internalUpdateAttributesOnNodes(
@@ -174,7 +196,8 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
 
       LOG.info(logMsg);
 
-      if (null != dispatcher) {
+      if (null != dispatcher && NodeAttribute.PREFIX_CENTRALIZED
+          .equals(attributePrefix)) {
         dispatcher.getEventHandler()
             .handle(new NodeAttributesStoreEvent(nodeAttributeMapping, op));
       }
@@ -382,6 +405,32 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
     }
   }
 
+  @Override
+  public List<NodeToAttributes> getNodeToAttributes(Set<String> prefix) {
+    try {
+      readLock.lock();
+      List<NodeToAttributes> nodeToAttributes = new ArrayList<>();
+      nodeCollections.forEach((k, v) -> {
+        List<NodeAttribute> attrs;
+        if (prefix == null || prefix.isEmpty()) {
+          attrs = new ArrayList<>(v.getAttributes().keySet());
+        } else {
+          attrs = new ArrayList<>();
+          for (Entry<NodeAttribute, AttributeValue> nodeAttr : v.attributes
+              .entrySet()) {
+            if (prefix.contains(nodeAttr.getKey().getAttributePrefix())) {
+              attrs.add(nodeAttr.getKey());
+            }
+          }
+        }
+        nodeToAttributes.add(NodeToAttributes.newInstance(k, attrs));
+      });
+      return nodeToAttributes;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
   public void activateNode(NodeId nodeId, Resource resource) {
     try {
       writeLock.lock();
@@ -524,7 +573,29 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
 
   // Dispatcher related code
   protected void handleStoreEvent(NodeAttributesStoreEvent event) {
-    // TODO Need to extend the File
+    List<NodeToAttributes> mappingList = new ArrayList<>();
+    Map<String, Map<NodeAttribute, AttributeValue>> nodeToAttr =
+        event.getNodeAttributeMappingList();
+    nodeToAttr.forEach((k, v) -> mappingList
+        .add(NodeToAttributes.newInstance(k, new ArrayList<>(v.keySet()))));
+    try {
+      switch (event.getOperation()) {
+      case REPLACE:
+        store.replaceNodeAttributes(mappingList);
+        break;
+      case ADD:
+        store.addNodeAttributes(mappingList);
+        break;
+      case REMOVE:
+        store.removeNodeAttributes(mappingList);
+        break;
+      default:
+        LOG.warn("Unsupported operation");
+      }
+    } catch (IOException e) {
+      LOG.error("Failed to store attribute modification to storage");
+      throw new YarnRuntimeException(e);
+    }
   }
 
   @Override
@@ -549,7 +620,8 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   private void processMapping(
       Map<String, Set<NodeAttribute>> nodeAttributeMapping,
       AttributeMappingOperationType mappingType) throws IOException {
-    processMapping(nodeAttributeMapping, mappingType, null);
+    processMapping(nodeAttributeMapping, mappingType,
+        NodeAttribute.PREFIX_CENTRALIZED);
   }
 
   private void processMapping(
@@ -564,4 +636,22 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
     internalUpdateAttributesOnNodes(validMapping, mappingType,
         newAttributesToBeAdded, attributePrefix);
   }
+
+  protected void stopDispatcher() {
+    AsyncDispatcher asyncDispatcher = (AsyncDispatcher) dispatcher;
+    if (null != asyncDispatcher) {
+      asyncDispatcher.stop();
+    }
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    // finalize store
+    stopDispatcher();
+
+    // only close store when we enabled store persistent
+    if (null != store) {
+      store.close();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
index a29e8a2..adb7fe0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import org.apache.hadoop.yarn.nodelabels.NodeAttributeStore;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.FileSystemNodeAttributeStore;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
@@ -828,6 +830,14 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
     Configuration conf = new Configuration();
     conf.set(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH,
         hostFile.getAbsolutePath());
+    conf.setClass(YarnConfiguration.FS_NODE_ATTRIBUTE_STORE_IMPL_CLASS,
+        FileSystemNodeAttributeStore.class, NodeAttributeStore.class);
+    File tempDir = File.createTempFile("nattr", ".tmp");
+    tempDir.delete();
+    tempDir.mkdirs();
+    tempDir.deleteOnExit();
+    conf.set(YarnConfiguration.FS_NODE_ATTRIBUTE_STORE_ROOT_DIR,
+        tempDir.getAbsolutePath());
     rm = new MockRM(conf);
     rm.start();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
new file mode 100644
index 0000000..e2ee8b4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
@@ -0,0 +1,260 @@
+/**
+ * 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.nodelabels;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.InlineDispatcher;
+import org.apache.hadoop.yarn.nodelabels.AttributeValue;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributeStore;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+public class TestFileSystemNodeAttributeStore {
+
+  private MockNodeAttrbuteManager mgr = null;
+  private Configuration conf = null;
+
+  private static class MockNodeAttrbuteManager
+      extends NodeAttributesManagerImpl {
+    @Override
+    protected void initDispatcher(Configuration conf) {
+      super.dispatcher = new InlineDispatcher();
+    }
+
+    @Override
+    protected void startDispatcher() {
+      //Do nothing
+    }
+
+    @Override
+    protected void stopDispatcher() {
+      //Do nothing
+    }
+  }
+
+  @Before
+  public void before() throws IOException {
+    mgr = new MockNodeAttrbuteManager();
+    conf = new Configuration();
+    conf.setClass(YarnConfiguration.FS_NODE_ATTRIBUTE_STORE_IMPL_CLASS,
+        FileSystemNodeAttributeStore.class, NodeAttributeStore.class);
+    File tempDir = File.createTempFile("nattr", ".tmp");
+    tempDir.delete();
+    tempDir.mkdirs();
+    tempDir.deleteOnExit();
+    conf.set(YarnConfiguration.FS_NODE_ATTRIBUTE_STORE_ROOT_DIR,
+        tempDir.getAbsolutePath());
+    mgr.init(conf);
+    mgr.start();
+  }
+
+  @After
+  public void after() throws IOException {
+    FileSystemNodeAttributeStore fsStore =
+        ((FileSystemNodeAttributeStore) mgr.store);
+    fsStore.getFs().delete(fsStore.getFsWorkingPath(), true);
+    mgr.stop();
+  }
+
+  @Test(timeout = 10000)
+  public void testRecoverWithMirror() throws Exception {
+
+    //------host0----
+    // add       -GPU & FPGA
+    // remove    -GPU
+    // replace   -Docker
+    //------host1----
+    // add--GPU
+    NodeAttribute docker = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "DOCKER",
+            NodeAttributeType.STRING, "docker-0");
+    NodeAttribute gpu = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "GPU",
+            NodeAttributeType.STRING, "nvidia");
+    NodeAttribute fpga = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "FPGA",
+            NodeAttributeType.STRING, "asus");
+
+    Map<String, Set<NodeAttribute>> toAddAttributes = new HashMap<>();
+    toAddAttributes.put("host0", ImmutableSet.of(gpu, fpga));
+    toAddAttributes.put("host1", ImmutableSet.of(gpu));
+    // Add node attribute
+    mgr.addNodeAttributes(toAddAttributes);
+
+    Assert.assertEquals("host0 size", 2,
+        mgr.getAttributesForNode("host0").size());
+    // Add test to remove
+    toAddAttributes.clear();
+    toAddAttributes.put("host0", ImmutableSet.of(gpu));
+    mgr.removeNodeAttributes(toAddAttributes);
+
+    // replace nodeattribute
+    toAddAttributes.clear();
+    toAddAttributes.put("host0", ImmutableSet.of(docker));
+    mgr.replaceNodeAttributes(NodeAttribute.PREFIX_CENTRALIZED,
+        toAddAttributes);
+    Map<NodeAttribute, AttributeValue> attrs =
+        mgr.getAttributesForNode("host0");
+    Assert.assertEquals(attrs.size(), 1);
+    Assert.assertEquals(attrs.keySet().toArray()[0], docker);
+    mgr.stop();
+
+    // Start new attribute manager with same path
+    mgr = new MockNodeAttrbuteManager();
+    mgr.init(conf);
+    mgr.start();
+
+    mgr.getAttributesForNode("host0");
+    Assert.assertEquals("host0 size", 1,
+        mgr.getAttributesForNode("host0").size());
+    Assert.assertEquals("host1 size", 1,
+        mgr.getAttributesForNode("host1").size());
+    attrs = mgr.getAttributesForNode("host0");
+    Assert.assertEquals(attrs.size(), 1);
+    Assert.assertEquals(attrs.keySet().toArray()[0], docker);
+    //------host0----
+    // current       - docker
+    // replace       - gpu
+    //----- host1----
+    // current       - gpu
+    // add           - docker
+    toAddAttributes.clear();
+    toAddAttributes.put("host0", ImmutableSet.of(gpu));
+    mgr.replaceNodeAttributes(NodeAttribute.PREFIX_CENTRALIZED,
+        toAddAttributes);
+
+    toAddAttributes.clear();
+    toAddAttributes.put("host1", ImmutableSet.of(docker));
+    mgr.addNodeAttributes(toAddAttributes);
+    // Recover from mirror and edit log
+    mgr.stop();
+
+    mgr = new MockNodeAttrbuteManager();
+    mgr.init(conf);
+    mgr.start();
+    Assert.assertEquals("host0 size", 1,
+        mgr.getAttributesForNode("host0").size());
+    Assert.assertEquals("host1 size", 2,
+        mgr.getAttributesForNode("host1").size());
+    attrs = mgr.getAttributesForNode("host0");
+    Assert.assertEquals(attrs.size(), 1);
+    Assert.assertEquals(attrs.keySet().toArray()[0], gpu);
+    attrs = mgr.getAttributesForNode("host1");
+    Assert.assertTrue(attrs.keySet().contains(docker));
+    Assert.assertTrue(attrs.keySet().contains(gpu));
+  }
+
+  @Test(timeout = 10000)
+  public void testRecoverFromEditLog() throws Exception {
+    NodeAttribute docker = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "DOCKER",
+            NodeAttributeType.STRING, "docker-0");
+    NodeAttribute gpu = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "GPU",
+            NodeAttributeType.STRING, "nvidia");
+    NodeAttribute fpga = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "FPGA",
+            NodeAttributeType.STRING, "asus");
+
+    Map<String, Set<NodeAttribute>> toAddAttributes = new HashMap<>();
+    toAddAttributes.put("host0", ImmutableSet.of(gpu, fpga));
+    toAddAttributes.put("host1", ImmutableSet.of(docker));
+
+    // Add node attribute
+    mgr.addNodeAttributes(toAddAttributes);
+
+    Assert.assertEquals("host0 size", 2,
+        mgr.getAttributesForNode("host0").size());
+
+    //  Increase editlog operation
+    for (int i = 0; i < 5; i++) {
+      // Add gpu host1
+      toAddAttributes.clear();
+      toAddAttributes.put("host0", ImmutableSet.of(gpu));
+      mgr.removeNodeAttributes(toAddAttributes);
+
+      // Add gpu host1
+      toAddAttributes.clear();
+      toAddAttributes.put("host1", ImmutableSet.of(docker));
+      mgr.addNodeAttributes(toAddAttributes);
+
+      // Remove GPU replace
+      toAddAttributes.clear();
+      toAddAttributes.put("host0", ImmutableSet.of(gpu));
+      mgr.replaceNodeAttributes(NodeAttribute.PREFIX_CENTRALIZED,
+          toAddAttributes);
+
+      // Add fgpa host1
+      toAddAttributes.clear();
+      toAddAttributes.put("host1", ImmutableSet.of(gpu));
+      mgr.addNodeAttributes(toAddAttributes);
+    }
+    mgr.stop();
+
+    // Start new attribute manager with same path
+    mgr = new MockNodeAttrbuteManager();
+    mgr.init(conf);
+    mgr.start();
+
+    Assert.assertEquals("host0 size", 1,
+        mgr.getAttributesForNode("host0").size());
+    Assert.assertEquals("host1 size", 2,
+        mgr.getAttributesForNode("host1").size());
+
+    toAddAttributes.clear();
+    NodeAttribute replaced =
+        NodeAttribute.newInstance("GPU2", NodeAttributeType.STRING, "nvidia2");
+    toAddAttributes.put("host0", ImmutableSet.of(replaced));
+    mgr.replaceNodeAttributes(NodeAttribute.PREFIX_CENTRALIZED,
+        toAddAttributes);
+    mgr.stop();
+
+    mgr = new MockNodeAttrbuteManager();
+    mgr.init(conf);
+    mgr.start();
+    Map<NodeAttribute, AttributeValue> valueMap =
+        mgr.getAttributesForNode("host0");
+    Map.Entry<NodeAttribute, AttributeValue> entry =
+        valueMap.entrySet().iterator().next();
+    NodeAttribute attribute = entry.getKey();
+    Assert.assertEquals("host0 size", 1,
+        mgr.getAttributesForNode("host0").size());
+    Assert.assertEquals("host1 size", 2,
+        mgr.getAttributesForNode("host1").size());
+    checkNodeAttributeEqual(replaced, attribute);
+  }
+
+  public void checkNodeAttributeEqual(NodeAttribute atr1, NodeAttribute atr2) {
+    Assert.assertEquals(atr1.getAttributeType(), atr2.getAttributeType());
+    Assert.assertEquals(atr1.getAttributeName(), atr2.getAttributeName());
+    Assert.assertEquals(atr1.getAttributePrefix(), atr2.getAttributePrefix());
+    Assert.assertEquals(atr1.getAttributeValue(), atr2.getAttributeValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9890d1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
index 07968d4..b8c5bc9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
@@ -23,7 +23,9 @@ import com.google.common.collect.Sets;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.nodelabels.AttributeValue;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributeStore;
 import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.nodelabels.NodeLabelUtil;
 import org.junit.Test;
@@ -31,6 +33,7 @@ import org.junit.Before;
 import org.junit.After;
 import org.junit.Assert;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -49,9 +52,17 @@ public class TestNodeAttributesManager {
       new String[] {"host1", "host2", "host3"};
 
   @Before
-  public void init() {
+  public void init() throws IOException {
     Configuration conf = new Configuration();
     attributesManager = new NodeAttributesManagerImpl();
+    conf.setClass(YarnConfiguration.FS_NODE_ATTRIBUTE_STORE_IMPL_CLASS,
+        FileSystemNodeAttributeStore.class, NodeAttributeStore.class);
+    File tempDir = File.createTempFile("nattr", ".tmp");
+    tempDir.delete();
+    tempDir.mkdirs();
+    tempDir.deleteOnExit();
+    conf.set(YarnConfiguration.FS_NODE_ATTRIBUTE_STORE_ROOT_DIR,
+        tempDir.getAbsolutePath());
     attributesManager.init(conf);
     attributesManager.start();
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/29] hadoop git commit: YARN-6855. [YARN-3409] CLI Proto Modifications to support Node Attributes. Contributed by Naganarasimha G R.

Posted by su...@apache.org.
YARN-6855. [YARN-3409] CLI Proto Modifications to support Node Attributes. Contributed by Naganarasimha G R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1f42ce90
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1f42ce90
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1f42ce90

Branch: refs/heads/trunk
Commit: 1f42ce907a4987ea095ae0c237b89274c689ed99
Parents: 9011567
Author: Naganarasimha <na...@apache.org>
Authored: Sun Jan 21 00:53:02 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:00:59 2018 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/api/records/NodeAttribute.java  |  78 ++++++++
 .../yarn/api/records/NodeAttributeType.java     |  35 ++++
 .../ResourceManagerAdministrationProtocol.java  |  13 +-
 .../AttributeMappingOperationType.java          |  42 ++++
 .../api/protocolrecords/NodeToAttributes.java   |  59 ++++++
 .../NodesToAttributesMappingRequest.java        |  69 +++++++
 .../NodesToAttributesMappingResponse.java       |  27 +++
 ...esourcemanager_administration_protocol.proto |   1 +
 ..._server_resourcemanager_service_protos.proto |  21 ++
 .../src/main/proto/yarn_protos.proto            |  11 ++
 .../records/impl/pb/NodeAttributePBImpl.java    | 155 +++++++++++++++
 ...nagerAdministrationProtocolPBClientImpl.java |  26 ++-
 ...agerAdministrationProtocolPBServiceImpl.java |  31 ++-
 .../impl/pb/NodeToAttributesPBImpl.java         | 161 +++++++++++++++
 .../NodesToAttributesMappingRequestPBImpl.java  | 194 +++++++++++++++++++
 .../NodesToAttributesMappingResponsePBImpl.java |  47 +++++
 .../hadoop/yarn/api/TestPBImplRecords.java      |  34 +++-
 .../yarn/server/MockResourceManagerFacade.java  |  17 +-
 .../server/resourcemanager/AdminService.java    |  10 +
 .../DefaultRMAdminRequestInterceptor.java       |   9 +
 .../router/rmadmin/RouterRMAdminService.java    |  10 +
 .../PassThroughRMAdminRequestInterceptor.java   |   9 +
 22 files changed, 1044 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
new file mode 100644
index 0000000..13081f3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
@@ -0,0 +1,78 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * Node Attribute is a kind of a label which represents one of the
+ * attribute/feature of a Node. Its different from node partition label as
+ * resource guarantees across the queues will not be maintained for these type
+ * of labels.
+ * </p>
+ * <p>
+ * A given Node can be mapped with any kind of attribute, few examples are
+ * HAS_SSD=true, JAVA_VERSION=JDK1.8, OS_TYPE=WINDOWS.
+ * </p>
+ * <p>
+ * Its not compulsory for all the attributes to have value, empty string is the
+ * default value of the <code>NodeAttributeType.STRING</code>
+ * </p>
+ *
+ */
+@Public
+@Unstable
+public abstract class NodeAttribute {
+
+  public static NodeAttribute newInstance(String attributeName,
+      NodeAttributeType attributeType, String attributeValue) {
+    NodeAttribute nodeAttribute = Records.newRecord(NodeAttribute.class);
+    nodeAttribute.setAttributeName(attributeName);
+    nodeAttribute.setAttributeType(attributeType);
+    nodeAttribute.setAttributeValue(attributeValue);
+    return nodeAttribute;
+  }
+
+  @Public
+  @Unstable
+  public abstract String getAttributeName();
+
+  @Public
+  @Unstable
+  public abstract void setAttributeName(String attributeName);
+
+  @Public
+  @Unstable
+  public abstract String getAttributeValue();
+
+  @Public
+  @Unstable
+  public abstract void setAttributeValue(String attributeValue);
+
+  @Public
+  @Unstable
+  public abstract NodeAttributeType getAttributeType();
+
+  @Public
+  @Unstable
+  public abstract void setAttributeType(NodeAttributeType attributeType);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeType.java
new file mode 100644
index 0000000..3f281c8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeType.java
@@ -0,0 +1,35 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * <p>
+ * Type of a <code>node Attribute</code>.
+ * </p>
+ * Based on this attribute expressions and values will be evaluated.
+ */
+@Public
+@Unstable
+public enum NodeAttributeType {
+  /** string type node attribute. */
+  STRING
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
index 8523342..58bb270 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
@@ -37,6 +39,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRespons
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesResponse;
@@ -52,8 +56,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequ
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesResponse;
 
 @Private
 public interface ResourceManagerAdministrationProtocol extends GetUserMappingsProtocol {
@@ -144,4 +146,11 @@ public interface ResourceManagerAdministrationProtocol extends GetUserMappingsPr
   public RefreshClusterMaxPriorityResponse refreshClusterMaxPriority(
       RefreshClusterMaxPriorityRequest request) throws YarnException,
       IOException;
+
+
+  @Private
+  @Idempotent
+  public NodesToAttributesMappingResponse mapAttributesToNodes(
+      NodesToAttributesMappingRequest request) throws YarnException,
+      IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/AttributeMappingOperationType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/AttributeMappingOperationType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/AttributeMappingOperationType.java
new file mode 100644
index 0000000..5de1504
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/AttributeMappingOperationType.java
@@ -0,0 +1,42 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * <p>
+ * Type of node to attribute mapping operation.
+ * </p>
+ *
+ */
+@Public
+@Unstable
+public enum AttributeMappingOperationType {
+  /** Replaces the existing node to attribute mapping with new mapping.*/
+  REPLACE,
+
+  /** Add attribute(s) to a node and if it already exists will update the
+   *  value.*/
+  ADD,
+
+  /** Removes attribute(s) mapped to a node. */
+  REMOVE
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeToAttributes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeToAttributes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeToAttributes.java
new file mode 100644
index 0000000..b2e38b4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeToAttributes.java
@@ -0,0 +1,59 @@
+/**
+ * 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.api.protocolrecords;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Represents a mapping of Node id to list of attributes.
+ */
+@Public
+@Unstable
+public abstract class NodeToAttributes {
+
+  public static NodeToAttributes newInstance(String node,
+      List<NodeAttribute> attributes) {
+    NodeToAttributes nodeIdToAttributes =
+        Records.newRecord(NodeToAttributes.class);
+    nodeIdToAttributes.setNode(node);
+    nodeIdToAttributes.setNodeAttributes(attributes);
+    return nodeIdToAttributes;
+  }
+
+  @Public
+  @Unstable
+  public abstract String getNode();
+
+  @Public
+  @Unstable
+  public abstract void setNode(String node);
+
+  @Public
+  @Unstable
+  public abstract List<NodeAttribute> getNodeAttributes();
+
+  @Public
+  @Unstable
+  public abstract void setNodeAttributes(List<NodeAttribute> attributes);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingRequest.java
new file mode 100644
index 0000000..71421ed
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingRequest.java
@@ -0,0 +1,69 @@
+/**
+ * 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.api.protocolrecords;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * list of node-attribute mapping request info.
+ */
+@Public
+@Unstable
+public abstract class NodesToAttributesMappingRequest {
+
+  public static NodesToAttributesMappingRequest newInstance(
+      AttributeMappingOperationType operation,
+      List<NodeToAttributes> nodesToAttributes, boolean failOnUnknownNodes) {
+    NodesToAttributesMappingRequest request =
+        Records.newRecord(NodesToAttributesMappingRequest.class);
+    request.setNodesToAttributes(nodesToAttributes);
+    request.setFailOnUnknownNodes(failOnUnknownNodes);
+    request.setOperation(operation);
+    return request;
+  }
+
+  @Public
+  @Unstable
+  public abstract void setNodesToAttributes(
+      List<NodeToAttributes> nodesToAttributes);
+
+  @Public
+  @Unstable
+  public abstract List<NodeToAttributes> getNodesToAttributes();
+
+  @Public
+  @Unstable
+  public abstract void setFailOnUnknownNodes(boolean failOnUnknownNodes);
+
+  @Public
+  @Unstable
+  public abstract boolean getFailOnUnknownNodes();
+
+  @Public
+  @Unstable
+  public abstract void setOperation(AttributeMappingOperationType operation);
+
+  @Public
+  @Unstable
+  public abstract AttributeMappingOperationType getOperation();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingResponse.java
new file mode 100644
index 0000000..8e44adf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingResponse.java
@@ -0,0 +1,27 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.yarn.util.Records;
+
+public class NodesToAttributesMappingResponse {
+  public static NodesToAttributesMappingResponse newInstance() {
+    return Records.newRecord(NodesToAttributesMappingResponse.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto
index 1134623..032aa8e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto
@@ -45,4 +45,5 @@ service ResourceManagerAdministrationProtocolService {
   rpc replaceLabelsOnNodes(ReplaceLabelsOnNodeRequestProto) returns (ReplaceLabelsOnNodeResponseProto);
   rpc checkForDecommissioningNodes(CheckForDecommissioningNodesRequestProto) returns (CheckForDecommissioningNodesResponseProto);
   rpc refreshClusterMaxPriority(RefreshClusterMaxPriorityRequestProto) returns (RefreshClusterMaxPriorityResponseProto);
+  rpc mapAttributesToNodes(NodesToAttributesMappingRequestProto) returns (NodesToAttributesMappingResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
index e8c92d9..5b93aec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
@@ -130,6 +130,27 @@ enum DecommissionTypeProto {
   GRACEFUL = 2;
   FORCEFUL = 3;
 }
+
+
+enum AttributeMappingOperationTypeProto {
+  REPLACE = 1;
+  ADD = 2;
+  REMOVE = 3;
+}
+
+message NodesToAttributesMappingRequestProto {
+  optional AttributeMappingOperationTypeProto operation = 1 [default = REPLACE];
+  repeated NodeToAttributesProto nodeToAttributes = 2;
+  optional bool failOnUnknownNodes = 3;
+}
+
+message NodeToAttributesProto {
+  optional string node = 1;
+  repeated NodeAttributeProto nodeAttributes = 2;
+}
+
+message NodesToAttributesMappingResponseProto {
+}
 //////////////////////////////////////////////////////////////////
 ///////////// RM Failover related records ////////////////////////
 //////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index d6138e8..41f5ccb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -372,6 +372,17 @@ message NodeLabelProto {
   optional bool isExclusive = 2 [default = true]; 
 }
 
+enum NodeAttributeTypeProto {
+  STRING = 1;
+}
+
+message NodeAttributeProto {
+  optional string attributeName = 1;
+  optional NodeAttributeTypeProto attributeType = 2;
+  optional string attributeValue = 3;
+}
+
+
 enum ContainerTypeProto {
   APPLICATION_MASTER = 1;
   TASK = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
new file mode 100644
index 0000000..11c9c48
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
@@ -0,0 +1,155 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeTypeProto;
+
+public class NodeAttributePBImpl extends NodeAttribute {
+  private NodeAttributeProto proto = NodeAttributeProto.getDefaultInstance();
+  private NodeAttributeProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public NodeAttributePBImpl() {
+    builder = NodeAttributeProto.newBuilder();
+  }
+
+  public NodeAttributePBImpl(NodeAttributeProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public NodeAttributeProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = NodeAttributeProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public String getAttributeName() {
+    NodeAttributeProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasAttributeName()) {
+      return null;
+    }
+    return p.getAttributeName();
+  }
+
+  @Override
+  public void setAttributeName(String attributeName) {
+    maybeInitBuilder();
+    builder.setAttributeName(attributeName);
+  }
+
+  @Override
+  public String getAttributeValue() {
+    NodeAttributeProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasAttributeValue()) {
+      return null;
+    }
+    return p.getAttributeValue();
+  }
+
+  @Override
+  public void setAttributeValue(String attributeValue) {
+    maybeInitBuilder();
+    builder.setAttributeValue(attributeValue);
+  }
+
+  @Override
+  public NodeAttributeType getAttributeType() {
+    NodeAttributeProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasAttributeType()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getAttributeType());
+  }
+
+  @Override
+  public void setAttributeType(NodeAttributeType attributeType) {
+    maybeInitBuilder();
+    if (attributeType == null) {
+      builder.clearAttributeType();
+      return;
+    }
+    builder.setAttributeType(convertToProtoFormat(attributeType));
+  }
+
+  private NodeAttributeTypeProto convertToProtoFormat(
+      NodeAttributeType attributeType) {
+    return NodeAttributeTypeProto.valueOf(attributeType.name());
+  }
+
+  private NodeAttributeType convertFromProtoFormat(
+      NodeAttributeTypeProto containerState) {
+    return NodeAttributeType.valueOf(containerState.name());
+  }
+
+  @Override
+  public String toString() {
+    return " name-" + getAttributeName() + ":value-" + getAttributeValue()
+        + ":type-" + getAttributeType();
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (obj instanceof NodeAttribute) {
+      NodeAttribute other = (NodeAttribute) obj;
+      if (!compare(getAttributeName(), other.getAttributeName())) {
+        return false;
+      }
+      if (!compare(getAttributeValue(), other.getAttributeValue())) {
+        return false;
+      }
+      if (!compare(getAttributeType(), other.getAttributeType())) {
+        return false;
+      }
+      return true;
+    }
+    return false;
+  }
+
+  private static boolean compare(Object left, Object right) {
+    if (left == null) {
+      return right == null;
+    } else {
+      return left.equals(right);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java
index 077edf3..156ed05 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java
@@ -33,9 +33,11 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddTo
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.CheckForDecommissioningNodesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshClusterMaxPriorityRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResourcesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshQueuesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshSuperUserGroupsConfigurationRequestProto;
@@ -43,13 +45,14 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.Refre
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RemoveFromClusterNodeLabelsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ReplaceLabelsOnNodeRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeResourceRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResourcesRequestProto;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocolPB;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
@@ -76,11 +79,15 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNod
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshClusterMaxPriorityRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshClusterMaxPriorityResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshQueuesRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshQueuesResponsePBImpl;
@@ -96,8 +103,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOn
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOnNodeResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceResponsePBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesRequestPBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesResponsePBImpl;
 
 import com.google.protobuf.ServiceException;
 
@@ -323,4 +328,19 @@ public class ResourceManagerAdministrationProtocolPBClientImpl implements Resour
       return null;
     }
   }
+
+  @Override
+  public NodesToAttributesMappingResponse mapAttributesToNodes(
+      NodesToAttributesMappingRequest request)
+      throws YarnException, IOException {
+    NodesToAttributesMappingRequestProto requestProto =
+        ((NodesToAttributesMappingRequestPBImpl) request).getProto();
+    try {
+      return new NodesToAttributesMappingResponsePBImpl(
+          proxy.mapAttributesToNodes(null, requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java
index aafce08..0036339 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java
@@ -28,11 +28,15 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.Check
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.CheckForDecommissioningNodesResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshClusterMaxPriorityRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshClusterMaxPriorityResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResourcesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResourcesResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshQueuesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshQueuesResponseProto;
@@ -48,13 +52,13 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.Repla
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ReplaceLabelsOnNodeResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeResourceRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeResourceResponseProto;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResourcesRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResourcesResponseProto;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocolPB;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityResponse;
@@ -71,11 +75,15 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNod
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshClusterMaxPriorityRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshClusterMaxPriorityResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshQueuesRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshQueuesResponsePBImpl;
@@ -91,8 +99,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOn
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOnNodeResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceResponsePBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesRequestPBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesResponsePBImpl;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -336,4 +342,21 @@ public class ResourceManagerAdministrationProtocolPBServiceImpl implements Resou
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public NodesToAttributesMappingResponseProto mapAttributesToNodes(
+      RpcController controller, NodesToAttributesMappingRequestProto proto)
+      throws ServiceException {
+    NodesToAttributesMappingRequest request =
+        new NodesToAttributesMappingRequestPBImpl(proto);
+    try {
+      NodesToAttributesMappingResponse response =
+          real.mapAttributesToNodes(request);
+      return ((NodesToAttributesMappingResponsePBImpl) response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
new file mode 100644
index 0000000..7b52d03
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
@@ -0,0 +1,161 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodeToAttributesProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodeToAttributesProtoOrBuilder;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+
+public class NodeToAttributesPBImpl extends NodeToAttributes {
+  private NodeToAttributesProto proto =
+      NodeToAttributesProto.getDefaultInstance();
+  private NodeToAttributesProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private List<NodeAttribute> nodeAttributes = null;
+
+  public NodeToAttributesPBImpl() {
+    builder = NodeToAttributesProto.newBuilder();
+  }
+
+  public NodeToAttributesPBImpl(NodeToAttributesProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    if (this.nodeAttributes != null) {
+      for (NodeAttribute nodeAttribute : nodeAttributes) {
+        builder.addNodeAttributes(
+            ((NodeAttributePBImpl) nodeAttribute).getProto());
+      }
+    }
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  public NodeToAttributesProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = NodeToAttributesProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public String getNode() {
+    NodeToAttributesProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasNode()) {
+      return null;
+    }
+    return p.getNode();
+  }
+
+  @Override
+  public void setNode(String node) {
+    maybeInitBuilder();
+    builder.setNode(node);
+  }
+
+  private void initNodeAttributes() {
+    if (this.nodeAttributes != null) {
+      return;
+    }
+
+    NodeToAttributesProtoOrBuilder p = viaProto ? proto : builder;
+    List<NodeAttributeProto> nodeAttributesProtoList =
+        p.getNodeAttributesList();
+    List<NodeAttribute> attributes = new ArrayList<>();
+    if (nodeAttributesProtoList == null
+        || nodeAttributesProtoList.size() == 0) {
+      this.nodeAttributes = attributes;
+      return;
+    }
+    for (NodeAttributeProto nodeAttributeProto : nodeAttributesProtoList) {
+      attributes.add(new NodeAttributePBImpl(nodeAttributeProto));
+    }
+    this.nodeAttributes = attributes;
+  }
+
+  @Override
+  public List<NodeAttribute> getNodeAttributes() {
+    initNodeAttributes();
+    return this.nodeAttributes;
+  }
+
+  @Override
+  public void setNodeAttributes(List<NodeAttribute> attributes) {
+    if (nodeAttributes == null) {
+      nodeAttributes = new ArrayList<>();
+    }
+    nodeAttributes.clear();
+    nodeAttributes.addAll(attributes);
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (obj instanceof NodeToAttributes) {
+      NodeToAttributes other = (NodeToAttributes) obj;
+      if (getNodeAttributes() == null) {
+        if (other.getNodeAttributes() != null) {
+          return false;
+        }
+      } else if (!getNodeAttributes().containsAll(other.getNodeAttributes())) {
+        return false;
+      }
+
+      if (getNode() == null) {
+        if (other.getNode() != null) {
+          return false;
+        }
+      } else if (!getNode().equals(other.getNode())) {
+        return false;
+      }
+
+      return true;
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
new file mode 100644
index 0000000..b319b26
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
@@ -0,0 +1,194 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AttributeMappingOperationTypeProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodeToAttributesProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+
+public class NodesToAttributesMappingRequestPBImpl
+    extends NodesToAttributesMappingRequest {
+  private NodesToAttributesMappingRequestProto proto =
+      NodesToAttributesMappingRequestProto.getDefaultInstance();
+  private NodesToAttributesMappingRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private List<NodeToAttributes> nodeAttributesMapping = null;
+
+  public NodesToAttributesMappingRequestPBImpl() {
+    builder = NodesToAttributesMappingRequestProto.newBuilder();
+  }
+
+  public NodesToAttributesMappingRequestPBImpl(
+      NodesToAttributesMappingRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    if (this.nodeAttributesMapping != null) {
+      for (NodeToAttributes nodeAttributes : nodeAttributesMapping) {
+        builder.addNodeToAttributes(
+            ((NodeToAttributesPBImpl) nodeAttributes).getProto());
+      }
+    }
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  public NodesToAttributesMappingRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = NodesToAttributesMappingRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public void setNodesToAttributes(List<NodeToAttributes> nodesToAttributes) {
+    if (nodeAttributesMapping == null) {
+      nodeAttributesMapping = new ArrayList<>();
+    }
+    if(nodesToAttributes == null) {
+      throw new IllegalArgumentException("nodesToAttributes cannot be null");
+    }
+    nodeAttributesMapping.clear();
+    nodeAttributesMapping.addAll(nodesToAttributes);
+  }
+
+  private void initNodeAttributesMapping() {
+    if (this.nodeAttributesMapping != null) {
+      return;
+    }
+
+    NodesToAttributesMappingRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<NodeToAttributesProto> nodeAttributesProtoList =
+        p.getNodeToAttributesList();
+    List<NodeToAttributes> attributes = new ArrayList<>();
+    if (nodeAttributesProtoList == null
+        || nodeAttributesProtoList.size() == 0) {
+      this.nodeAttributesMapping = attributes;
+      return;
+    }
+    for (NodeToAttributesProto nodeAttributeProto : nodeAttributesProtoList) {
+      attributes.add(new NodeToAttributesPBImpl(nodeAttributeProto));
+    }
+    this.nodeAttributesMapping = attributes;
+  }
+
+  @Override
+  public List<NodeToAttributes> getNodesToAttributes() {
+    initNodeAttributesMapping();
+    return this.nodeAttributesMapping;
+  }
+
+  @Override
+  public void setFailOnUnknownNodes(boolean failOnUnknownNodes) {
+    maybeInitBuilder();
+    builder.setFailOnUnknownNodes(failOnUnknownNodes);
+  }
+
+  @Override
+  public boolean getFailOnUnknownNodes() {
+    NodesToAttributesMappingRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    return p.getFailOnUnknownNodes();
+  }
+
+  @Override
+  public void setOperation(AttributeMappingOperationType operation) {
+    maybeInitBuilder();
+    builder.setOperation(convertToProtoFormat(operation));
+  }
+
+  private AttributeMappingOperationTypeProto convertToProtoFormat(
+      AttributeMappingOperationType operation) {
+    return AttributeMappingOperationTypeProto.valueOf(operation.name());
+  }
+
+  private AttributeMappingOperationType convertFromProtoFormat(
+      AttributeMappingOperationTypeProto operationTypeProto) {
+    return AttributeMappingOperationType.valueOf(operationTypeProto.name());
+  }
+
+  @Override
+  public AttributeMappingOperationType getOperation() {
+    NodesToAttributesMappingRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (!p.hasOperation()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getOperation());
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (obj instanceof NodesToAttributesMappingRequest) {
+      NodesToAttributesMappingRequest other =
+          (NodesToAttributesMappingRequest) obj;
+      if (getNodesToAttributes() == null) {
+        if (other.getNodesToAttributes() != null) {
+          return false;
+        }
+      } else if (!getNodesToAttributes()
+          .containsAll(other.getNodesToAttributes())) {
+        return false;
+      }
+
+      if (getOperation() == null) {
+        if (other.getOperation() != null) {
+          return false;
+        }
+      } else if (!getOperation().equals(other.getOperation())) {
+        return false;
+      }
+
+      return getFailOnUnknownNodes() == other.getFailOnUnknownNodes();
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingResponsePBImpl.java
new file mode 100644
index 0000000..0554ef8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingResponsePBImpl.java
@@ -0,0 +1,47 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingResponseProto;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
+
+public class NodesToAttributesMappingResponsePBImpl
+    extends NodesToAttributesMappingResponse {
+
+  private NodesToAttributesMappingResponseProto proto =
+      NodesToAttributesMappingResponseProto.getDefaultInstance();
+  private NodesToAttributesMappingResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public NodesToAttributesMappingResponsePBImpl() {
+    builder = NodesToAttributesMappingResponseProto.newBuilder();
+  }
+
+  public NodesToAttributesMappingResponsePBImpl(
+      NodesToAttributesMappingResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public NodesToAttributesMappingResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 4c660c0..80bbac4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -124,6 +124,7 @@ import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -176,6 +177,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ExecutionTypeRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeLabelPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeReportPBImpl;
@@ -214,6 +216,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ContainerRetryContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeLabelProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
@@ -238,6 +241,8 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddTo
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddToClusterNodeLabelsResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.CheckForDecommissioningNodesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.CheckForDecommissioningNodesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodeToAttributesProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesRequestProto;
@@ -320,10 +325,14 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationResponseP
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllResourceProfilesResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileResponseProto;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodeToAttributesPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesRequestPBImpl;
@@ -434,7 +443,12 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
     generateByNewInstance(ResourceSizing.class);
     generateByNewInstance(SchedulingRequest.class);
     generateByNewInstance(RejectedSchedulingRequest.class);
-  }
+    //for Node attribute support
+    generateByNewInstance(NodeAttribute.class);
+    generateByNewInstance(NodeToAttributes.class);
+    generateByNewInstance(NodesToAttributesMappingRequest.class);
+ 
+ }
 
   @Test
   public void testAllocateRequestPBImpl() throws Exception {
@@ -1228,4 +1242,22 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
     validatePBImplRecord(GetAllResourceTypeInfoResponsePBImpl.class,
         YarnServiceProtos.GetAllResourceTypeInfoResponseProto.class);
   }
+
+  @Test
+  public void testNodeAttributePBImpl() throws Exception {
+    validatePBImplRecord(NodeAttributePBImpl.class,
+	NodeAttributeProto.class);
+  }
+
+  @Test
+  public void testNodeToAttributesPBImpl() throws Exception {
+    validatePBImplRecord(NodeToAttributesPBImpl.class,
+	NodeToAttributesProto.class);
+  }
+
+  @Test
+  public void testNodesToAttributesMappingRequestPBImpl() throws Exception {
+    validatePBImplRecord(NodesToAttributesMappingRequestPBImpl.class,
+	NodesToAttributesMappingRequestProto.class);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
index 00278a0..b9cebfe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
@@ -164,6 +164,8 @@ import org.apache.hadoop.yarn.util.Records;
 import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 
 import com.google.common.base.Strings;
 
@@ -181,15 +183,15 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
 
   private HashSet<ApplicationId> applicationMap = new HashSet<>();
   private HashSet<ApplicationId> keepContainerOnUams = new HashSet<>();
-  private HashMap<ApplicationAttemptId, List<ContainerId>>
-      applicationContainerIdMap = new HashMap<>();
+  private HashMap<ApplicationAttemptId, List<ContainerId>> applicationContainerIdMap =
+      new HashMap<ApplicationAttemptId, List<ContainerId>>();
   private AtomicInteger containerIndex = new AtomicInteger(0);
   private Configuration conf;
   private int subClusterId;
   final private AtomicInteger applicationCounter = new AtomicInteger(0);
 
   // True if the Mock RM is running, false otherwise.
-  // This property allows us to write tests for specific scenario as YARN RM
+  // This property allows us to write tests for specific scenario as Yarn RM
   // down e.g. network issue, failover.
   private boolean isRunning;
 
@@ -487,7 +489,7 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     if (request.getApplicationSubmissionContext().getUnmanagedAM()
         || request.getApplicationSubmissionContext()
             .getKeepContainersAcrossApplicationAttempts()) {
-      keepContainerOnUams.add(appId);
+	keepContainerOnUams.add(appId);
     }
     return SubmitApplicationResponse.newInstance();
   }
@@ -505,7 +507,6 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
         throw new ApplicationNotFoundException(
             "Trying to kill an absent application: " + appId);
       }
-      keepContainerOnUams.remove(appId);
     }
     LOG.info("Force killing application: " + appId);
     return KillApplicationResponse.newInstance(true);
@@ -893,4 +894,10 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
       GetAllResourceTypeInfoRequest request) throws YarnException, IOException {
     return null;
   }
+
+  @Override
+  public NodesToAttributesMappingResponse mapAttributesToNodes(NodesToAttributesMappingRequest request)
+    throws YarnException, IOException {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index 3c117bc..bff3f2c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -66,6 +66,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsR
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
@@ -961,4 +963,12 @@ public class AdminService extends CompositeService implements
 
     rm.getRMContext().getScheduler().setClusterMaxPriority(conf);
   }
+
+  @Override
+  public NodesToAttributesMappingResponse mapAttributesToNodes(
+      NodesToAttributesMappingRequest request)
+      throws YarnException, IOException {
+    return recordFactory
+        .newRecordInstance(NodesToAttributesMappingResponse.class);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java
index 7e6a1ff..b4140c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java
@@ -32,6 +32,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsR
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
@@ -212,4 +214,11 @@ public class DefaultRMAdminRequestInterceptor
   public String[] getGroupsForUser(String userName) throws IOException {
     return rmAdminProxy.getGroupsForUser(userName);
   }
+
+  @Override
+  public NodesToAttributesMappingResponse mapAttributesToNodes(
+      NodesToAttributesMappingRequest request)
+      throws YarnException, IOException {
+    return rmAdminProxy.mapAttributesToNodes(request);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java
index ef30613..a806e9d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java
@@ -43,6 +43,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsR
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
@@ -419,4 +421,12 @@ public class RouterRMAdminService extends AbstractService
     RequestInterceptorChainWrapper pipeline = getInterceptorChain();
     return pipeline.getRootInterceptor().refreshClusterMaxPriority(request);
   }
+
+  @Override
+  public NodesToAttributesMappingResponse mapAttributesToNodes(
+      NodesToAttributesMappingRequest request)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().mapAttributesToNodes(request);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f42ce90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java
index 38dcc3d..4d6a3ea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java
@@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsR
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
@@ -145,4 +147,11 @@ public class PassThroughRMAdminRequestInterceptor
     return getNextInterceptor().getGroupsForUser(user);
   }
 
+  @Override
+  public NodesToAttributesMappingResponse mapAttributesToNodes(
+      NodesToAttributesMappingRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().mapAttributesToNodes(request);
+  }
+
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/29] hadoop git commit: YARN-6858. Attribute Manager to store and provide node attributes in RM. Contributed by Naganarasimha G R.

Posted by su...@apache.org.
YARN-6858. Attribute Manager to store and provide node attributes in RM. Contributed by Naganarasimha G R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2f7712be
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2f7712be
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2f7712be

Branch: refs/heads/trunk
Commit: 2f7712be09d5b68978547f95b1be2352cdd46266
Parents: d312b5c
Author: Sunil G <su...@apache.org>
Authored: Fri Feb 23 08:01:58 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../records/impl/pb/NodeAttributePBImpl.java    |  15 +-
 .../hadoop/yarn/nodelabels/AbstractLabel.java   |  71 +++
 .../AttributeExpressionOperation.java           |  26 +
 .../hadoop/yarn/nodelabels/AttributeValue.java  |  53 ++
 .../nodelabels/CommonNodeLabelsManager.java     |  24 +-
 .../yarn/nodelabels/NodeAttributesManager.java  |  99 ++++
 .../hadoop/yarn/nodelabels/NodeLabelUtil.java   |  97 ++++
 .../hadoop/yarn/nodelabels/RMNodeAttribute.java | 104 ++++
 .../hadoop/yarn/nodelabels/RMNodeLabel.java     | 109 ++--
 .../yarn/nodelabels/StringAttributeValue.java   |  61 +++
 .../nodemanager/NodeStatusUpdaterImpl.java      |  16 +-
 .../resourcemanager/RMActiveServiceContext.java |  14 +
 .../yarn/server/resourcemanager/RMContext.java  |   5 +
 .../server/resourcemanager/RMContextImpl.java   |  11 +
 .../server/resourcemanager/ResourceManager.java |  13 +-
 .../nodelabels/NodeAttributesManagerImpl.java   | 527 +++++++++++++++++++
 .../nodelabels/NodeAttributesStoreEvent.java    |  51 ++
 .../NodeAttributesStoreEventType.java           |  26 +
 18 files changed, 1220 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
index 7810939..12a0ecc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
@@ -120,7 +120,13 @@ public class NodeAttributePBImpl extends NodeAttribute {
 
   @Override
   public int hashCode() {
-    return getProto().hashCode();
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((getAttributePrefix() == null) ? 0
+        : getAttributePrefix().hashCode());
+    result = prime * result
+        + ((getAttributeName() == null) ? 0 : getAttributeName().hashCode());
+    return result;
   }
 
   @Override
@@ -133,13 +139,10 @@ public class NodeAttributePBImpl extends NodeAttribute {
     }
     if (obj instanceof NodeAttribute) {
       NodeAttribute other = (NodeAttribute) obj;
-      if (!compare(getAttributeName(), other.getAttributeName())) {
-        return false;
-      }
-      if (!compare(getAttributeValue(), other.getAttributeValue())) {
+      if (!compare(getAttributePrefix(), other.getAttributePrefix())) {
         return false;
       }
-      if (!compare(getAttributeType(), other.getAttributeType())) {
+      if (!compare(getAttributeName(), other.getAttributeName())) {
         return false;
       }
       return true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/AbstractLabel.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/AbstractLabel.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/AbstractLabel.java
new file mode 100644
index 0000000..6a44574
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/AbstractLabel.java
@@ -0,0 +1,71 @@
+/**
+ * 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.nodelabels;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+/**
+ * Generic class capturing the information required commonly across Partitions
+ * and Attributes.
+ */
+public abstract class AbstractLabel {
+
+  private Resource resource;
+  private int numActiveNMs;
+  private String labelName;
+
+  public AbstractLabel() {
+    super();
+  }
+
+  public AbstractLabel(String labelName) {
+    this(labelName, Resource.newInstance(0, 0), 0);
+  }
+
+  public AbstractLabel(String labelName, Resource resource, int numActiveNMs) {
+    super();
+    this.resource = resource;
+    this.numActiveNMs = numActiveNMs;
+    this.labelName = labelName;
+  }
+
+  public void addNode(Resource nodeRes) {
+    Resources.addTo(resource, nodeRes);
+    numActiveNMs++;
+  }
+
+  public void removeNode(Resource nodeRes) {
+    Resources.subtractFrom(resource, nodeRes);
+    numActiveNMs--;
+  }
+
+  public Resource getResource() {
+    return Resource.newInstance(this.resource);
+  }
+
+  public int getNumActiveNMs() {
+    return numActiveNMs;
+  }
+
+  public String getLabelName() {
+    return labelName;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/AttributeExpressionOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/AttributeExpressionOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/AttributeExpressionOperation.java
new file mode 100644
index 0000000..8754314
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/AttributeExpressionOperation.java
@@ -0,0 +1,26 @@
+/**
+ * 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.nodelabels;
+
+/**
+ * Operations which are allowed in Node Attributes Expression.
+ */
+public enum AttributeExpressionOperation {
+  LT, GT, IN, NOTIN
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/AttributeValue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/AttributeValue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/AttributeValue.java
new file mode 100644
index 0000000..d1d75cf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/AttributeValue.java
@@ -0,0 +1,53 @@
+/**
+ * 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.nodelabels;
+
+import java.io.IOException;
+
+/**
+ * Interface to capture operations on AttributeValue.
+ */
+public interface AttributeValue {
+
+  /**
+   * @return original value which was set.
+   */
+  String getValue();
+
+  /**
+   * validate the value based on the type and initialize for further compare
+   * operations.
+   *
+   * @param value
+   * @throws IOException
+   */
+  void validateAndInitializeValue(String value) throws IOException;
+
+  /**
+   * compare the value against the other based on the
+   * AttributeExpressionOperation.
+   *
+   * @param other
+   * @param op
+   * @return true if value <code>other</code> matches the current value for the
+   *         operation <code>op</code>.
+   */
+  boolean compareForOperation(AttributeValue other,
+      AttributeExpressionOperation op);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
index 612b701..b5f4757 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
@@ -35,7 +35,6 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
-import java.util.regex.Pattern;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.logging.Log;
@@ -65,15 +64,12 @@ import com.google.common.collect.ImmutableSet;
 @Private
 public class CommonNodeLabelsManager extends AbstractService {
   protected static final Log LOG = LogFactory.getLog(CommonNodeLabelsManager.class);
-  private static final int MAX_LABEL_LENGTH = 255;
   public static final Set<String> EMPTY_STRING_SET = Collections
       .unmodifiableSet(new HashSet<String>(0));
   public static final Set<NodeLabel> EMPTY_NODELABEL_SET = Collections
       .unmodifiableSet(new HashSet<NodeLabel>(0));
   public static final String ANY = "*";
   public static final Set<String> ACCESS_ANY_LABEL_SET = ImmutableSet.of(ANY);
-  private static final Pattern LABEL_PATTERN = Pattern
-      .compile("^[0-9a-zA-Z][0-9a-zA-Z-_]*");
   public static final int WILDCARD_PORT = 0;
   // Flag to identify startup for removelabel
   private boolean initNodeLabelStoreInProgress = false;
@@ -112,7 +108,7 @@ public class CommonNodeLabelsManager extends AbstractService {
   /**
    * A <code>Host</code> can have multiple <code>Node</code>s 
    */
-  protected static class Host {
+  public static class Host {
     public Set<String> labels;
     public Map<NodeId, Node> nms;
     
@@ -317,7 +313,7 @@ public class CommonNodeLabelsManager extends AbstractService {
     // do a check before actual adding them, will throw exception if any of them
     // doesn't meet label name requirement
     for (NodeLabel label : labels) {
-      checkAndThrowLabelName(label.getName());
+      NodeLabelUtil.checkAndThrowLabelName(label.getName());
     }
 
     for (NodeLabel label : labels) {
@@ -969,22 +965,6 @@ public class CommonNodeLabelsManager extends AbstractService {
     }
   }
 
-  public static void checkAndThrowLabelName(String label) throws IOException {
-    if (label == null || label.isEmpty() || label.length() > MAX_LABEL_LENGTH) {
-      throw new IOException("label added is empty or exceeds "
-          + MAX_LABEL_LENGTH + " character(s)");
-    }
-    label = label.trim();
-
-    boolean match = LABEL_PATTERN.matcher(label).matches();
-
-    if (!match) {
-      throw new IOException("label name should only contains "
-          + "{0-9, a-z, A-Z, -, _} and should not started with {-,_}"
-          + ", now it is=" + label);
-    }
-  }
-
   private void checkExclusivityMatch(Collection<NodeLabel> labels)
       throws IOException {
     ArrayList<NodeLabel> mismatchlabels = new ArrayList<NodeLabel>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
new file mode 100644
index 0000000..63f3dcf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
@@ -0,0 +1,99 @@
+/**
+ * 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.nodelabels;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+
+/**
+ * This class captures all interactions for Attributes with RM.
+ */
+public abstract class NodeAttributesManager extends AbstractService {
+  public NodeAttributesManager(String name) {
+    super(name);
+  }
+
+  /**
+   * To completely replace the mappings for a given node with the new Set of
+   * Attributes. If the mapping contains an attribute whose type does not match
+   * a previously existing Attribute under the same prefix (name space) then
+   * exception is thrown. Key would be name of the node and value would be set
+   * of Attributes to be mapped.
+   *
+   * @param nodeAttributeMapping
+   * @throws IOException
+   */
+  public abstract void replaceNodeAttributes(
+      Map<String, Set<NodeAttribute>> nodeAttributeMapping) throws IOException;
+
+  /**
+   * It adds or updates the attribute mapping for a given node with out
+   * impacting other existing attribute mapping. Key would be name of the node
+   * and value would be set of Attributes to be mapped.
+   *
+   * @param nodeAttributeMapping
+   * @throws IOException
+   */
+  public abstract void addNodeAttributes(
+      Map<String, Set<NodeAttribute>> nodeAttributeMapping) throws IOException;
+
+  /**
+   * It removes the specified attribute mapping for a given node with out
+   * impacting other existing attribute mapping. Key would be name of the node
+   * and value would be set of Attributes to be removed.
+   *
+   * @param nodeAttributeMapping
+   * @throws IOException
+   */
+  public abstract void removeNodeAttributes(
+      Map<String, Set<NodeAttribute>> nodeAttributeMapping) throws IOException;
+
+  /**
+   * @param prefix set of prefix string's for which the attributes needs to
+   *          returned
+   * @return set of node Attributes objects for the specified set of prefixes,
+   *         else return all
+   */
+  public abstract Set<NodeAttribute> getClusterNodeAttributes(
+      Set<String> prefix);
+
+  /**
+   * Given a attribute set, return what all Nodes have attribute mapped to it.
+   *
+   * @return a Map, of attribute to set of hostnames
+   */
+  //TODO need to handle as part of REST patch.
+/*  public abstract Map<NodeAttribute, Set<String>> getAttributesToNodes(
+      Set<NodeAttribute> attributes);*/
+
+  /**
+   * NodeAttribute to AttributeValue Map.
+   *
+   * @return Map<NodeAttribute, AttributeValue> mapping of Attribute to Value.
+   */
+  public abstract Map<NodeAttribute, AttributeValue> getAttributesForNode(
+      String hostName);
+
+  // futuristic
+  // public set<NodeId> getNodesMatchingExpression(String nodeLabelExp);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
new file mode 100644
index 0000000..d918712
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
@@ -0,0 +1,97 @@
+/**
+ * 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.nodelabels;
+
+import java.io.IOException;
+import java.util.regex.Pattern;
+
+/**
+ * Utility class for all NodeLabel and NodeAttribute operations.
+ */
+public final class NodeLabelUtil {
+  private NodeLabelUtil() {
+  }
+
+  private static final int MAX_LABEL_LENGTH = 255;
+  private static final Pattern LABEL_OR_VALUE_PATTERN =
+      Pattern.compile("^[0-9a-zA-Z][0-9a-zA-Z-_]*");
+  private static final Pattern PREFIX_PATTERN =
+      Pattern.compile("^[0-9a-zA-Z][0-9a-zA-Z-_\\.]*");
+
+  public static void checkAndThrowLabelName(String label) throws IOException {
+    if (label == null || label.isEmpty() || label.length() > MAX_LABEL_LENGTH) {
+      throw new IOException("label added is empty or exceeds "
+          + MAX_LABEL_LENGTH + " character(s)");
+    }
+    label = label.trim();
+
+    boolean match = LABEL_OR_VALUE_PATTERN.matcher(label).matches();
+
+    if (!match) {
+      throw new IOException("label name should only contains "
+          + "{0-9, a-z, A-Z, -, _} and should not started with {-,_}"
+          + ", now it is= " + label);
+    }
+  }
+
+  public static void checkAndThrowAttributeValue(String value)
+      throws IOException {
+    if (value == null) {
+      return;
+    } else if (value.trim().length() > MAX_LABEL_LENGTH) {
+      throw new IOException("Attribute value added exceeds " + MAX_LABEL_LENGTH
+          + " character(s)");
+
+    }
+    value = value.trim();
+    if(value.isEmpty()) {
+      return;
+    }
+
+    boolean match = LABEL_OR_VALUE_PATTERN.matcher(value).matches();
+
+    if (!match) {
+      throw new IOException("attribute value should only contains "
+          + "{0-9, a-z, A-Z, -, _} and should not started with {-,_}"
+          + ", now it is= " + value);
+    }
+  }
+
+  public static void checkAndThrowAttributePrefix(String prefix)
+      throws IOException {
+    if (prefix == null) {
+      throw new IOException("Attribute prefix cannot be null.");
+    }
+    if (prefix.trim().length() > MAX_LABEL_LENGTH) {
+      throw new IOException("Attribute value added exceeds " + MAX_LABEL_LENGTH
+          + " character(s)");
+    }
+    prefix = prefix.trim();
+    if(prefix.isEmpty()) {
+      return;
+    }
+
+    boolean match = PREFIX_PATTERN.matcher(prefix).matches();
+
+    if (!match) {
+      throw new IOException("attribute value should only contains "
+          + "{0-9, a-z, A-Z, -, _,.} and should not started with {-,_}"
+          + ", now it is= " + prefix);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java
new file mode 100644
index 0000000..5a709c6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java
@@ -0,0 +1,104 @@
+/**
+ * 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.nodelabels;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+/**
+ * Reference of NodeAttribute in RM.
+ */
+public class RMNodeAttribute extends AbstractLabel {
+
+  private NodeAttribute attribute;
+  // TODO need to revisit whether we need to make this concurrent implementation
+  private Set<String> nodes = new HashSet<>();
+
+  public RMNodeAttribute(NodeAttribute attribute) {
+    this(attribute.getAttributeName(), Resource.newInstance(0, 0), 0,
+        attribute);
+  }
+
+  public RMNodeAttribute(String labelName, Resource res, int activeNMs,
+      NodeAttribute attribute) {
+    super(labelName, res, activeNMs);
+    this.attribute = attribute;
+  }
+
+  public NodeAttribute getAttribute() {
+    return attribute;
+  }
+
+  public void setAttribute(NodeAttribute attribute) {
+    this.attribute = attribute;
+  }
+
+  public RMNodeAttribute(String attributeName) {
+    super(attributeName);
+    attribute = NodeAttribute.newInstance(attributeName,
+        NodeAttributeType.STRING, CommonNodeLabelsManager.NO_LABEL);
+  }
+
+  public NodeAttributeType getAttributeType() {
+    return attribute.getAttributeType();
+  }
+
+  public void addNode(String node) {
+    nodes.add(node);
+  }
+
+  public void removeNode(String node) {
+    nodes.remove(node);
+  }
+
+  public Set<String> getAssociatedNodeIds() {
+    return new HashSet<String>(nodes);
+  }
+
+  @Override
+  public int hashCode() {
+    return attribute.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    RMNodeAttribute other = (RMNodeAttribute) obj;
+    if (attribute == null) {
+      if (other.attribute != null) {
+        return false;
+      }
+    } else if (!attribute.equals(other.attribute)) {
+      return false;
+    }
+    return true;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
index 693a58a..a8d151c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
@@ -27,13 +27,13 @@ import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-public class RMNodeLabel implements Comparable<RMNodeLabel> {
-  private Resource resource;
-  private int numActiveNMs;
-  private String labelName;
-  private Set<NodeId> nodeIds;
+/**
+ * Partition representation in RM.
+ */
+public class RMNodeLabel extends AbstractLabel implements Comparable<RMNodeLabel> {
   private boolean exclusive;
   private NodeLabel nodeLabel;
+  private Set<NodeId> nodeIds;
 
   public RMNodeLabel(NodeLabel nodeLabel) {
     this(nodeLabel.getName(), Resource.newInstance(0, 0), 0,
@@ -47,48 +47,12 @@ public class RMNodeLabel implements Comparable<RMNodeLabel> {
   
   protected RMNodeLabel(String labelName, Resource res, int activeNMs,
       boolean exclusive) {
-    this.labelName = labelName;
-    this.resource = res;
-    this.numActiveNMs = activeNMs;
-    this.nodeIds = new HashSet<NodeId>();
+    super(labelName, res, activeNMs);
     this.exclusive = exclusive;
     this.nodeLabel = NodeLabel.newInstance(labelName, exclusive);
+    nodeIds = new HashSet<NodeId>();
   }
 
-  public void addNodeId(NodeId node) {
-    nodeIds.add(node);
-  }
-
-  public void removeNodeId(NodeId node) {
-    nodeIds.remove(node);
-  }
-  
-  public Set<NodeId> getAssociatedNodeIds() {
-    return new HashSet<NodeId>(nodeIds);
-  }
-
-  public void addNode(Resource nodeRes) {
-    Resources.addTo(resource, nodeRes);
-    numActiveNMs++;
-  }
-  
-  public void removeNode(Resource nodeRes) {
-    Resources.subtractFrom(resource, nodeRes);
-    numActiveNMs--;
-  }
-
-  public Resource getResource() {
-    return this.resource;
-  }
-
-  public int getNumActiveNMs() {
-    return numActiveNMs;
-  }
-  
-  public String getLabelName() {
-    return labelName;
-  }
-  
   public void setIsExclusive(boolean exclusive) {
     this.exclusive = exclusive;
   }
@@ -97,42 +61,57 @@ public class RMNodeLabel implements Comparable<RMNodeLabel> {
     return this.exclusive;
   }
   
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof RMNodeLabel) {
+      RMNodeLabel other = (RMNodeLabel) obj;
+      return Resources.equals(getResource(), other.getResource())
+          && StringUtils.equals(getLabelName(), other.getLabelName())
+          && (other.getNumActiveNMs() == getNumActiveNMs());
+    }
+    return false;
+  }
+
+
   public RMNodeLabel getCopy() {
-    return new RMNodeLabel(labelName, resource, numActiveNMs, exclusive);
+    return new RMNodeLabel(getLabelName(), getResource(), getNumActiveNMs(),
+        exclusive);
   }
   
-  public NodeLabel getNodeLabel() {
-    return this.nodeLabel;
+  @Override
+  public int hashCode() {
+    final int prime = 502357;
+    return (int) ((((long) getLabelName().hashCode() << 8)
+        + (getResource().hashCode() << 4) + getNumActiveNMs()) % prime);
   }
 
+
   @Override
   public int compareTo(RMNodeLabel o) {
     // We should always put empty label entry first after sorting
-    if (labelName.isEmpty() != o.getLabelName().isEmpty()) {
-      if (labelName.isEmpty()) {
+    if (getLabelName().isEmpty() != o.getLabelName().isEmpty()) {
+      if (getLabelName().isEmpty()) {
         return -1;
       }
       return 1;
     }
     
-    return labelName.compareTo(o.getLabelName());
+    return getLabelName().compareTo(o.getLabelName());
   }
-  
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof RMNodeLabel) {
-      RMNodeLabel other = (RMNodeLabel) obj;
-      return Resources.equals(resource, other.getResource())
-          && StringUtils.equals(labelName, other.getLabelName())
-          && (other.getNumActiveNMs() == numActiveNMs); 
-    }
-    return false;
+
+  public NodeLabel getNodeLabel() {
+    return this.nodeLabel;
   }
-  
-  @Override
-  public int hashCode() {
-    final int prime = 502357;
-    return (int) ((((long) labelName.hashCode() << 8)
-        + (resource.hashCode() << 4) + numActiveNMs) % prime);
+
+  public void addNodeId(NodeId node) {
+    nodeIds.add(node);
+  }
+
+  public void removeNodeId(NodeId node) {
+    nodeIds.remove(node);
+  }
+
+  public Set<NodeId> getAssociatedNodeIds() {
+    return new HashSet<NodeId>(nodeIds);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/StringAttributeValue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/StringAttributeValue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/StringAttributeValue.java
new file mode 100644
index 0000000..12343a7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/StringAttributeValue.java
@@ -0,0 +1,61 @@
+/**
+ * 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.nodelabels;
+
+import java.io.IOException;
+
+/**
+ * Attribute value for String NodeAttributeType.
+ */
+public class StringAttributeValue implements AttributeValue {
+  private String value = "";
+
+  @Override
+  public boolean compareForOperation(AttributeValue other,
+      AttributeExpressionOperation op) {
+    if (other instanceof StringAttributeValue) {
+      StringAttributeValue otherString = (StringAttributeValue) other;
+      switch (op) {
+      case IN:
+        return value.equals(otherString.value);
+      case NOTIN:
+        return !value.equals(otherString.value);
+      default:
+        return false;
+      }
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public void validateAndInitializeValue(String valueStr) throws IOException {
+    NodeLabelUtil.checkAndThrowAttributeValue(valueStr);
+    this.value = valueStr;
+  }
+
+  @Override
+  public String getValue() {
+    return value;
+  }
+
+  public String toString() {
+    return getValue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index 33096bd..7be9ef7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -34,11 +34,6 @@ import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
 
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePlugin;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePluginManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataInputByteBuffer;
@@ -64,6 +59,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.impl.pb.RecordFactoryPBImpl;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.nodelabels.NodeLabelUtil;
 import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
 import org.apache.hadoop.yarn.server.api.ServerRMProxy;
@@ -76,23 +72,27 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResp
 import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
 import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
-import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 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.OpportunisticContainersStatus;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePlugin;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePluginManager;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider;
 import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
 import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
-import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -1012,7 +1012,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       StringBuilder errorMsg = new StringBuilder("");
       while (iterator.hasNext()) {
         try {
-          CommonNodeLabelsManager
+          NodeLabelUtil
               .checkAndThrowLabelName(iterator.next().getName());
         } catch (IOException e) {
           errorMsg.append(e.getMessage());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
index 8fb0de6..1596598 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
@@ -101,6 +102,7 @@ public class RMActiveServiceContext {
   private ApplicationMasterService applicationMasterService;
 
   private RMNodeLabelsManager nodeLabelManager;
+  private NodeAttributesManager nodeAttributesManager;
   private RMDelegatedNodeLabelsUpdater rmDelegatedNodeLabelsUpdater;
   private long epoch;
   private Clock systemClock = SystemClock.getInstance();
@@ -407,6 +409,18 @@ public class RMActiveServiceContext {
 
   @Private
   @Unstable
+  public NodeAttributesManager getNodeAttributesManager() {
+    return nodeAttributesManager;
+  }
+
+  @Private
+  @Unstable
+  public void setNodeAttributesManager(NodeAttributesManager mgr) {
+    nodeAttributesManager = mgr;
+  }
+
+  @Private
+  @Unstable
   public AllocationTagsManager getAllocationTagsManager() {
     return allocationTagsManager;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index a30ff76..d3daa05 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.ConfigurationProvider;
 import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
@@ -133,6 +134,10 @@ public interface RMContext extends ApplicationMasterServiceContext {
   
   public void setNodeLabelManager(RMNodeLabelsManager mgr);
 
+  NodeAttributesManager getNodeAttributesManager();
+
+  void setNodeAttributesManager(NodeAttributesManager mgr);
+
   RMDelegatedNodeLabelsUpdater getRMDelegatedNodeLabelsUpdater();
 
   void setRMDelegatedNodeLabelsUpdater(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index cb1d56f..5b295f6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.ConfigurationProvider;
 import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
@@ -505,6 +506,11 @@ public class RMContextImpl implements RMContext {
   }
 
   @Override
+  public void setNodeAttributesManager(NodeAttributesManager mgr) {
+    activeServiceContext.setNodeAttributesManager(mgr);
+  }
+
+  @Override
   public AllocationTagsManager getAllocationTagsManager() {
     return activeServiceContext.getAllocationTagsManager();
   }
@@ -632,4 +638,9 @@ public class RMContextImpl implements RMContext {
     this.activeServiceContext.setResourceProfilesManager(mgr);
   }
   // Note: Read java doc before adding any services over here.
+
+  @Override
+  public NodeAttributesManager getNodeAttributesManager() {
+    return activeServiceContext.getNodeAttributesManager();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index d591f94..81ef337 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -62,15 +62,17 @@ import org.apache.hadoop.yarn.event.EventDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
 import org.apache.hadoop.yarn.server.resourcemanager.federation.FederationStateStoreService;
+import org.apache.hadoop.yarn.server.resourcemanager.metrics.CombinedSystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.NoOpSystemMetricPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV1Publisher;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV2Publisher;
-import org.apache.hadoop.yarn.server.resourcemanager.metrics.CombinedSystemMetricsPublisher;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeAttributesManagerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
@@ -517,6 +519,11 @@ public class ResourceManager extends CompositeService
     return new RMNodeLabelsManager();
   }
 
+  protected NodeAttributesManager createNodeAttributesManager()
+      throws InstantiationException, IllegalAccessException {
+    return new NodeAttributesManagerImpl();
+  }
+
   protected AllocationTagsManager createAllocationTagsManager() {
     return new AllocationTagsManager(this.rmContext);
   }
@@ -656,6 +663,10 @@ public class ResourceManager extends CompositeService
       addService(nlm);
       rmContext.setNodeLabelManager(nlm);
 
+      NodeAttributesManager nam = createNodeAttributesManager();
+      addService(nam);
+      rmContext.setNodeAttributesManager(nam);
+
       AllocationTagsManager allocationTagsManager =
           createAllocationTagsManager();
       rmContext.setAllocationTagsManager(allocationTagsManager);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
new file mode 100644
index 0000000..2e63a7c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
@@ -0,0 +1,527 @@
+/**
+ * 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.nodelabels;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.nodelabels.AttributeValue;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
+import org.apache.hadoop.yarn.nodelabels.NodeLabelUtil;
+import org.apache.hadoop.yarn.nodelabels.RMNodeAttribute;
+import org.apache.hadoop.yarn.nodelabels.StringAttributeValue;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
+
+/**
+ * Manager holding the attributes to Labels.
+ */
+public class NodeAttributesManagerImpl extends NodeAttributesManager {
+  protected static final Log LOG =
+      LogFactory.getLog(NodeAttributesManagerImpl.class);
+  /**
+   * If a user doesn't specify value for a label, then empty string is
+   * considered as default.
+   */
+  public static final String EMPTY_ATTRIBUTE_VALUE = "";
+
+  private Dispatcher dispatcher;
+
+  // TODO may be we can have a better collection here.
+  // this will be updated to get the attributeName to NM mapping
+  private ConcurrentHashMap<NodeAttribute, RMNodeAttribute> clusterAttributes =
+      new ConcurrentHashMap<>();
+
+  // hostname -> (Map (attributeName -> NodeAttribute))
+  // Instead of NodeAttribute, plan to have it in future as AttributeValue
+  // AttributeValue
+  // / \
+  // StringNodeAttributeValue LongAttributeValue
+  // and convert the configured value to the specific type so that the
+  // expression evaluations are faster
+  private ConcurrentMap<String, Host> nodeCollections =
+      new ConcurrentHashMap<>();
+
+  private final ReadLock readLock;
+  private final WriteLock writeLock;
+
+  public NodeAttributesManagerImpl() {
+    super("NodeAttributesManagerImpl");
+    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    readLock = lock.readLock();
+    writeLock = lock.writeLock();
+  }
+
+  protected void initDispatcher(Configuration conf) {
+    // create async handler
+    dispatcher = new AsyncDispatcher("AttributeNodeLabelsManager dispatcher");
+    AsyncDispatcher asyncDispatcher = (AsyncDispatcher) dispatcher;
+    asyncDispatcher.init(conf);
+    asyncDispatcher.setDrainEventsOnStop();
+  }
+
+  protected void startDispatcher() {
+    // start dispatcher
+    AsyncDispatcher asyncDispatcher = (AsyncDispatcher) dispatcher;
+    asyncDispatcher.start();
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    initNodeAttributeStore(getConfig());
+    // init dispatcher only when service start, because recover will happen in
+    // service init, we don't want to trigger any event handling at that time.
+    initDispatcher(getConfig());
+
+    if (null != dispatcher) {
+      dispatcher.register(NodeAttributesStoreEventType.class,
+          new ForwardingEventHandler());
+    }
+
+    startDispatcher();
+    super.serviceStart();
+  }
+
+  protected void initNodeAttributeStore(Configuration conf) throws Exception {
+    // TODO to generalize and make use of the FileSystemNodeLabelsStore
+  }
+
+  private void internalUpdateAttributesOnNodes(
+      Map<String, Map<NodeAttribute, AttributeValue>> nodeAttributeMapping,
+      AttributeMappingOperationType op,
+      Map<NodeAttribute, RMNodeAttribute> newAttributesToBeAdded) {
+    try {
+      writeLock.lock();
+
+      // shows node->attributes Mapped as part of this operation.
+      StringBuilder logMsg = new StringBuilder(op.name());
+      logMsg.append(" attributes on nodes:");
+      // do update labels from nodes
+      for (Entry<String, Map<NodeAttribute, AttributeValue>> entry : nodeAttributeMapping
+          .entrySet()) {
+        String nodeHost = entry.getKey();
+        Map<NodeAttribute, AttributeValue> attributes = entry.getValue();
+
+        Host node = nodeCollections.get(nodeHost);
+        if (node == null) {
+          node = new Host(nodeHost);
+        }
+        switch (op) {
+        case REMOVE:
+          removeNodeFromAttributes(nodeHost, attributes.keySet());
+          node.removeAttributes(attributes);
+          break;
+        case ADD:
+          clusterAttributes.putAll(newAttributesToBeAdded);
+          addNodeToAttribute(nodeHost, attributes);
+          node.addAttributes(attributes);
+          break;
+        case REPLACE:
+          clusterAttributes.putAll(newAttributesToBeAdded);
+          replaceNodeToAttribute(nodeHost, node.getAttributes(), attributes);
+          node.replaceAttributes(attributes);
+          break;
+        default:
+          break;
+        }
+        logMsg.append(" NM = ");
+        logMsg.append(entry.getKey());
+        logMsg.append(", attributes=[ ");
+        logMsg.append(StringUtils.join(entry.getValue().entrySet(), ","));
+        logMsg.append("] ,");
+      }
+
+      LOG.info(logMsg);
+
+      if (null != dispatcher) {
+        dispatcher.getEventHandler()
+            .handle(new NodeAttributesStoreEvent(nodeAttributeMapping, op));
+      }
+
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private void removeNodeFromAttributes(String nodeHost,
+      Set<NodeAttribute> attributeMappings) {
+    for (NodeAttribute attribute : attributeMappings) {
+      clusterAttributes.get(attribute).removeNode(nodeHost);
+    }
+  }
+
+  private void addNodeToAttribute(String nodeHost,
+      Map<NodeAttribute, AttributeValue> attributeMappings) {
+    for (NodeAttribute attribute : attributeMappings.keySet()) {
+      clusterAttributes.get(attribute).addNode(nodeHost);
+    }
+  }
+
+  private void replaceNodeToAttribute(String nodeHost,
+      Map<NodeAttribute, AttributeValue> oldAttributeMappings,
+      Map<NodeAttribute, AttributeValue> newAttributeMappings) {
+    if (oldAttributeMappings != null) {
+      removeNodeFromAttributes(nodeHost, oldAttributeMappings.keySet());
+    }
+    addNodeToAttribute(nodeHost, newAttributeMappings);
+  }
+
+  /**
+   * @param nodeAttributeMapping
+   * @param newAttributesToBeAdded
+   * @param isRemoveOperation : to indicate whether its a remove operation.
+   * @return Map<String, Map<NodeAttribute, AttributeValue>>, node -> Map(
+   *         NodeAttribute -> AttributeValue)
+   * @throws IOException : on invalid mapping in the current request or against
+   *           already existing NodeAttributes.
+   */
+  protected Map<String, Map<NodeAttribute, AttributeValue>> validate(
+      Map<String, Set<NodeAttribute>> nodeAttributeMapping,
+      Map<NodeAttribute, RMNodeAttribute> newAttributesToBeAdded,
+      boolean isRemoveOperation) throws IOException {
+    Map<String, Map<NodeAttribute, AttributeValue>> nodeToAttributesMap =
+        new TreeMap<>();
+    Map<NodeAttribute, AttributeValue> attributesValues;
+    Set<Entry<String, Set<NodeAttribute>>> entrySet =
+        nodeAttributeMapping.entrySet();
+    for (Entry<String, Set<NodeAttribute>> nodeToAttrMappingEntry : entrySet) {
+      attributesValues = new HashMap<>();
+      String node = nodeToAttrMappingEntry.getKey().trim();
+      if (nodeToAttrMappingEntry.getValue().isEmpty()) {
+        // no attributes to map mostly remove operation
+        continue;
+      }
+
+      // validate for attributes
+      for (NodeAttribute attribute : nodeToAttrMappingEntry.getValue()) {
+        String attributeName = attribute.getAttributeName().trim();
+        NodeLabelUtil.checkAndThrowLabelName(attributeName);
+        NodeLabelUtil
+            .checkAndThrowAttributePrefix(attribute.getAttributePrefix());
+
+        // ensure trimmed values are set back
+        attribute.setAttributeName(attributeName);
+        attribute.setAttributePrefix(attribute.getAttributePrefix().trim());
+
+        // verify for type against prefix/attributeName
+        if (validateForAttributeTypeMismatch(isRemoveOperation, attribute,
+            newAttributesToBeAdded)) {
+          newAttributesToBeAdded.put(attribute,
+              new RMNodeAttribute(attribute));
+        }
+        // TODO type based value setting needs to be done using a factory
+        StringAttributeValue value = new StringAttributeValue();
+        value.validateAndInitializeValue(
+            normalizeAttributeValue(attribute.getAttributeValue()));
+        attributesValues.put(attribute, value);
+      }
+      nodeToAttributesMap.put(node, attributesValues);
+    }
+    return nodeToAttributesMap;
+  }
+
+  /**
+   *
+   * @param isRemoveOperation
+   * @param attribute
+   * @param newAttributes
+   * @return Whether its a new Attribute added
+   * @throws IOException
+   */
+  private boolean validateForAttributeTypeMismatch(boolean isRemoveOperation,
+      NodeAttribute attribute,
+      Map<NodeAttribute, RMNodeAttribute> newAttributes)
+      throws IOException {
+    if (isRemoveOperation && !clusterAttributes.containsKey(attribute)) {
+      // no need to validate anything as its remove operation and attribute
+      // doesn't exist.
+      return false; // no need to add as its remove operation
+    } else {
+      // already existing or attribute is mapped to another Node in the
+      // current command, then check whether the attribute type is matching
+      NodeAttribute existingAttribute =
+          (clusterAttributes.containsKey((attribute))
+              ? clusterAttributes.get(attribute).getAttribute()
+              : (newAttributes.containsKey(attribute)
+                  ? newAttributes.get(attribute).getAttribute()
+                  : null));
+      if (existingAttribute == null) {
+        return true;
+      } else if (existingAttribute.getAttributeType() != attribute
+          .getAttributeType()) {
+        throw new IOException("Attribute name - type is not matching with "
+            + "already configured mapping for the attribute "
+            + attribute.getAttributeName() + " existing : "
+            + existingAttribute.getAttributeType() + ", new :"
+            + attribute.getAttributeType());
+      }
+      return false;
+    }
+  }
+
+  protected String normalizeAttributeValue(String value) {
+    if (value != null) {
+      return value.trim();
+    }
+    return EMPTY_ATTRIBUTE_VALUE;
+  }
+
+  @Override
+  public Set<NodeAttribute> getClusterNodeAttributes(Set<String> prefix) {
+    Set<NodeAttribute> attributes = new HashSet<>();
+    try {
+      readLock.lock();
+      attributes.addAll(clusterAttributes.keySet());
+    } finally {
+      readLock.unlock();
+    }
+    if (prefix != null && prefix.isEmpty()) {
+      Iterator<NodeAttribute> iterator = attributes.iterator();
+      while (iterator.hasNext()) {
+        NodeAttribute attribute = iterator.next();
+        if (!prefix.contains(attribute.getAttributePrefix())) {
+          iterator.remove();
+        }
+      }
+    }
+    return attributes;
+  }
+
+  // TODO need to handle as part of REST patch.
+  /*
+   * @Override public Map<NodeAttribute, Set<String>> getAttributesToNodes(
+   * Set<NodeAttribute> attributes) { try { readLock.lock(); boolean
+   * fetchAllAttributes = (attributes == null || attributes.isEmpty());
+   * Map<NodeAttribute, Set<String>> attributesToNodes = new HashMap<>(); for
+   * (Entry<NodeAttribute, RMAttributeNodeLabel> attributeEntry :
+   * attributeCollections .entrySet()) { if (fetchAllAttributes ||
+   * attributes.contains(attributeEntry.getKey())) {
+   * attributesToNodes.put(attributeEntry.getKey(),
+   * attributeEntry.getValue().getAssociatedNodeIds()); } } return
+   * attributesToNodes; } finally { readLock.unlock(); } }
+   */
+
+  public Resource getResourceByAttribute(NodeAttribute attribute) {
+    try {
+      readLock.lock();
+      return clusterAttributes.containsKey(attribute)
+          ? clusterAttributes.get(attribute).getResource()
+          : Resource.newInstance(0, 0);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public Map<NodeAttribute, AttributeValue> getAttributesForNode(
+      String hostName) {
+    try {
+      readLock.lock();
+      return nodeCollections.containsKey(hostName)
+          ? nodeCollections.get(hostName).getAttributes()
+          : new HashMap<>();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public void activateNode(NodeId nodeId, Resource resource) {
+    try {
+      writeLock.lock();
+      String hostName = nodeId.getHost();
+      Host host = nodeCollections.get(hostName);
+      if (host == null) {
+        host = new Host(hostName);
+        nodeCollections.put(hostName, host);
+      }
+      host.activateNode(resource);
+      for (NodeAttribute attribute : host.getAttributes().keySet()) {
+        clusterAttributes.get(attribute).removeNode(resource);
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void deactivateNode(NodeId nodeId) {
+    try {
+      writeLock.lock();
+      Host host = nodeCollections.get(nodeId.getHost());
+      for (NodeAttribute attribute : host.getAttributes().keySet()) {
+        clusterAttributes.get(attribute).removeNode(host.getResource());
+      }
+      host.deactivateNode();
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void updateNodeResource(NodeId node, Resource newResource) {
+    deactivateNode(node);
+    activateNode(node, newResource);
+  }
+
+  /**
+   * A <code>Host</code> can have multiple <code>Node</code>s.
+   */
+  public static class Host {
+    private String hostName;
+    private Map<NodeAttribute, AttributeValue> attributes;
+    private Resource resource;
+    private boolean isActive;
+
+    private Map<NodeAttribute, AttributeValue> getAttributes() {
+      return attributes;
+    }
+
+    public void setAttributes(Map<NodeAttribute, AttributeValue> attributes) {
+      this.attributes = attributes;
+    }
+
+    public void removeAttributes(
+        Map<NodeAttribute, AttributeValue> attributesMapping) {
+      for (NodeAttribute attribute : attributesMapping.keySet()) {
+        this.attributes.remove(attribute);
+      }
+    }
+
+    public void replaceAttributes(
+        Map<NodeAttribute, AttributeValue> attributesMapping) {
+      this.attributes.clear();
+      this.attributes.putAll(attributesMapping);
+    }
+
+    public void addAttributes(
+        Map<NodeAttribute, AttributeValue> attributesMapping) {
+      this.attributes.putAll(attributesMapping);
+    }
+
+    public Resource getResource() {
+      return resource;
+    }
+
+    public void setResource(Resource resourceParam) {
+      this.resource = resourceParam;
+    }
+
+    public boolean isActive() {
+      return isActive;
+    }
+
+    public void deactivateNode() {
+      this.isActive = false;
+      this.resource = Resource.newInstance(0, 0);
+    }
+
+    public void activateNode(Resource r) {
+      this.isActive = true;
+      this.resource = r;
+    }
+
+    public String getHostName() {
+      return hostName;
+    }
+
+    public void setHostName(String hostName) {
+      this.hostName = hostName;
+    }
+
+    public Host(String hostName) {
+      this(hostName, new HashMap<NodeAttribute, AttributeValue>());
+    }
+
+    public Host(String hostName,
+        Map<NodeAttribute, AttributeValue> attributes) {
+      this(hostName, attributes, Resource.newInstance(0, 0), false);
+    }
+
+    public Host(String hostName, Map<NodeAttribute, AttributeValue> attributes,
+        Resource resource, boolean isActive) {
+      super();
+      this.attributes = attributes;
+      this.resource = resource;
+      this.isActive = isActive;
+      this.hostName = hostName;
+    }
+  }
+
+  private final class ForwardingEventHandler
+      implements EventHandler<NodeAttributesStoreEvent> {
+
+    @Override
+    public void handle(NodeAttributesStoreEvent event) {
+      handleStoreEvent(event);
+    }
+  }
+
+  // Dispatcher related code
+  protected void handleStoreEvent(NodeAttributesStoreEvent event) {
+    // TODO Need to extend the File
+  }
+
+  @Override
+  public void replaceNodeAttributes(
+      Map<String, Set<NodeAttribute>> nodeAttributeMapping) throws IOException {
+    processMapping(nodeAttributeMapping, AttributeMappingOperationType.REPLACE);
+  }
+
+  @Override
+  public void addNodeAttributes(
+      Map<String, Set<NodeAttribute>> nodeAttributeMapping) throws IOException {
+    processMapping(nodeAttributeMapping, AttributeMappingOperationType.ADD);
+  }
+
+  @Override
+  public void removeNodeAttributes(
+      Map<String, Set<NodeAttribute>> nodeAttributeMapping) throws IOException {
+    processMapping(nodeAttributeMapping, AttributeMappingOperationType.REMOVE);
+  }
+
+  private void processMapping(
+      Map<String, Set<NodeAttribute>> nodeAttributeMapping,
+      AttributeMappingOperationType mappingType) throws IOException {
+    Map<NodeAttribute, RMNodeAttribute> newAttributesToBeAdded =
+        new HashMap<>();
+    Map<String, Map<NodeAttribute, AttributeValue>> validMapping =
+        validate(nodeAttributeMapping, newAttributesToBeAdded, false);
+
+    internalUpdateAttributesOnNodes(validMapping, mappingType,
+        newAttributesToBeAdded);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesStoreEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesStoreEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesStoreEvent.java
new file mode 100644
index 0000000..d04e8cf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesStoreEvent.java
@@ -0,0 +1,51 @@
+/**
+ * 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.nodelabels;
+
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.hadoop.yarn.nodelabels.AttributeValue;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
+
+/**
+ * Event capturing details to store the Node Attributes in the backend store.
+ */
+public class NodeAttributesStoreEvent
+    extends AbstractEvent<NodeAttributesStoreEventType> {
+  private Map<String, Map<NodeAttribute, AttributeValue>> nodeAttributeMapping;
+  private AttributeMappingOperationType operation;
+
+  public NodeAttributesStoreEvent(
+      Map<String, Map<NodeAttribute, AttributeValue>> nodeAttributeMappingList,
+      AttributeMappingOperationType operation) {
+    super(NodeAttributesStoreEventType.STORE_ATTRIBUTES);
+    this.nodeAttributeMapping = nodeAttributeMappingList;
+    this.operation = operation;
+  }
+
+  public Map<String, Map<NodeAttribute, AttributeValue>> getNodeAttributeMappingList() {
+    return nodeAttributeMapping;
+  }
+
+  public AttributeMappingOperationType getOperation() {
+    return operation;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f7712be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesStoreEventType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesStoreEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesStoreEventType.java
new file mode 100644
index 0000000..c433b19
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesStoreEventType.java
@@ -0,0 +1,26 @@
+/**
+ * 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.nodelabels;
+
+/**
+ * Event type to store the NodeAttributes.
+ */
+public enum NodeAttributesStoreEventType {
+ STORE_ATTRIBUTES
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[14/29] hadoop git commit: YARN-7892. Revisit NodeAttribute class structure. Contributed by Naganarasimha G R.

Posted by su...@apache.org.
YARN-7892. Revisit NodeAttribute class structure. Contributed by  Naganarasimha G R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8cf6a9a2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8cf6a9a2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8cf6a9a2

Branch: refs/heads/trunk
Commit: 8cf6a9a2bde9d4ccad95a514885634792889ec7f
Parents: 5dc7d6e
Author: bibinchundatt <bi...@apache.org>
Authored: Sun May 13 09:35:11 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../hadoop/mapred/ResourceMgrDelegate.java      |   9 +-
 .../GetAttributesToNodesRequest.java            |  34 +++--
 .../GetAttributesToNodesResponse.java           |  33 +++--
 .../GetClusterNodeAttributesResponse.java       |  23 +--
 .../hadoop/yarn/api/records/NodeAttribute.java  |  17 +--
 .../yarn/api/records/NodeAttributeInfo.java     |  62 ++++++++
 .../yarn/api/records/NodeAttributeKey.java      |  66 +++++++++
 .../yarn/api/records/NodeToAttributeValue.java  |  57 ++++++++
 .../src/main/proto/yarn_protos.proto            |  27 +++-
 .../src/main/proto/yarn_service_protos.proto    |   4 +-
 .../hadoop/yarn/client/api/YarnClient.java      |  22 +--
 .../yarn/client/api/impl/YarnClientImpl.java    |   9 +-
 .../pb/GetAttributesToNodesRequestPBImpl.java   |  45 +++---
 .../pb/GetAttributesToNodesResponsePBImpl.java  |  83 +++++++----
 .../GetClusterNodeAttributesResponsePBImpl.java |  68 ++++-----
 .../impl/pb/NodeAttributeInfoPBImpl.java        | 143 +++++++++++++++++++
 .../records/impl/pb/NodeAttributeKeyPBImpl.java | 140 ++++++++++++++++++
 .../records/impl/pb/NodeAttributePBImpl.java    |  79 ++++------
 .../impl/pb/NodeToAttributeValuePBImpl.java     | 137 ++++++++++++++++++
 .../yarn/nodelabels/NodeAttributesManager.java  |  16 ++-
 .../hadoop/yarn/nodelabels/NodeLabelUtil.java   |  14 +-
 .../hadoop/yarn/nodelabels/RMNodeAttribute.java |  18 +--
 .../hadoop/yarn/api/TestPBImplRecords.java      |  45 ++++--
 ...TestConfigurationNodeAttributesProvider.java |  17 ++-
 .../TestScriptBasedNodeAttributesProvider.java  |   5 +-
 .../server/resourcemanager/AdminService.java    |   2 +-
 .../server/resourcemanager/ClientRMService.java |  31 +++-
 .../resourcemanager/ResourceTrackerService.java |   2 +-
 .../nodelabels/NodeAttributesManagerImpl.java   | 114 ++++++++-------
 .../nodelabels/NodeLabelsUtils.java             |  19 +++
 .../webapp/dao/NodeAttributeInfo.java           |   4 +-
 .../resourcemanager/TestClientRMService.java    | 110 ++++++++------
 .../TestResourceTrackerService.java             |   4 +-
 .../TestFileSystemNodeAttributeStore.java       |   6 +-
 .../nodelabels/TestNodeAttributesManager.java   |   9 +-
 .../clientrm/FederationClientInterceptor.java   |   2 +-
 36 files changed, 1119 insertions(+), 357 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
index 1a7f308..2cb3716 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
@@ -59,10 +59,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.api.records.NodeAttributeInfo;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.NodeToAttributeValue;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -541,14 +544,14 @@ public class ResourceMgrDelegate extends YarnClient {
   }
 
   @Override
-  public Set<NodeAttribute> getClusterAttributes()
+  public Set<NodeAttributeInfo> getClusterAttributes()
       throws YarnException, IOException {
     return client.getClusterAttributes();
   }
 
   @Override
-  public Map<NodeAttribute, Set<String>> getAttributesToNodes(
-      Set<NodeAttribute> attributes) throws YarnException, IOException {
+  public Map<NodeAttributeKey, List<NodeToAttributeValue>> getAttributesToNodes(
+      Set<NodeAttributeKey> attributes) throws YarnException, IOException {
     return client.getAttributesToNodes(attributes);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesRequest.java
index d9531b0..94814e9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesRequest.java
@@ -17,22 +17,24 @@
  */
 package org.apache.hadoop.yarn.api.protocolrecords;
 
-import static org.apache.hadoop.classification.InterfaceAudience.*;
-import static org.apache.hadoop.classification.InterfaceStability.*;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
-import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
 import org.apache.hadoop.yarn.util.Records;
 
-import java.util.Set;
-
 /**
  * <p>
- * The request from clients to get attribtues to nodes mapping
- * in the cluster from the <code>ResourceManager</code>.
+ * The request from clients to get node to attribute value mapping for all or
+ * given set of Node AttributeKey's in the cluster from the
+ * <code>ResourceManager</code>.
  * </p>
  *
  * @see ApplicationClientProtocol#getAttributesToNodes
- * (GetAttributesToNodesRequest)
+ *      (GetAttributesToNodesRequest)
  */
 @Public
 @Evolving
@@ -43,7 +45,7 @@ public abstract class GetAttributesToNodesRequest {
   }
 
   public static GetAttributesToNodesRequest newInstance(
-      Set<NodeAttribute> attributes) {
+      Set<NodeAttributeKey> attributes) {
     GetAttributesToNodesRequest request =
         Records.newRecord(GetAttributesToNodesRequest.class);
     request.setNodeAttributes(attributes);
@@ -51,20 +53,22 @@ public abstract class GetAttributesToNodesRequest {
   }
 
   /**
-   * Set node attributes for which the mapping is required.
+   * Set node attributeKeys for which the mapping of hostname to attribute value
+   * is required.
    *
-   * @param attributes Set<NodeAttribute> provided.
+   * @param attributes Set<NodeAttributeKey> provided.
    */
   @Public
   @Unstable
-  public abstract void setNodeAttributes(Set<NodeAttribute> attributes);
+  public abstract void setNodeAttributes(Set<NodeAttributeKey> attributes);
 
   /**
-   * Get node attributes for which mapping mapping is required.
+   * Get node attributeKeys for which mapping of hostname to attribute value is
+   * required.
    *
-   * @return Set<NodeAttribute>
+   * @return Set<NodeAttributeKey>
    */
   @Public
   @Unstable
-  public abstract Set<NodeAttribute> getNodeAttributes();
+  public abstract Set<NodeAttributeKey> getNodeAttributes();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java
index 4fdb1f7..9bd529f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java
@@ -17,29 +17,30 @@
  */
 package org.apache.hadoop.yarn.api.protocolrecords;
 
-import static org.apache.hadoop.classification.InterfaceAudience.Public;
-import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
-import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.api.records.NodeToAttributeValue;
 import org.apache.hadoop.yarn.util.Records;
 
-import java.util.Map;
-import java.util.Set;
-
 /**
  * <p>
  * The response sent by the <code>ResourceManager</code> to a client requesting
- * attributes to hostname mapping.
+ * node to attribute value mapping for all or given set of Node AttributeKey's.
  * </p>
  *
  * @see ApplicationClientProtocol#getAttributesToNodes
- * (GetAttributesToNodesRequest)
+ *      (GetAttributesToNodesRequest)
  */
 @Public
 @Evolving
 public abstract class GetAttributesToNodesResponse {
   public static GetAttributesToNodesResponse newInstance(
-      Map<NodeAttribute, Set<String>> map) {
+      Map<NodeAttributeKey, List<NodeToAttributeValue>> map) {
     GetAttributesToNodesResponse response =
         Records.newRecord(GetAttributesToNodesResponse.class);
     response.setAttributeToNodes(map);
@@ -48,15 +49,17 @@ public abstract class GetAttributesToNodesResponse {
 
   @Public
   @Evolving
-  public abstract void setAttributeToNodes(Map<NodeAttribute, Set<String>> map);
+  public abstract void setAttributeToNodes(
+      Map<NodeAttributeKey, List<NodeToAttributeValue>> map);
 
-  /*
-   * Get attributes to node hostname mapping.
+  /**
+   * Get mapping of NodeAttributeKey to its associated mapping of list of
+   * NodeToAttributeValuenode to attribute value.
    *
-   * @return Map<NodeAttribute, Set<String>> node attributes to hostname
-   * mapping.
+   * @return Map<NodeAttributeKey, List<NodeToAttributeValue>> node attributes
+   *         to list of NodeToAttributeValuenode.
    */
   @Public
   @Evolving
-  public abstract Map<NodeAttribute, Set<String>> getAttributesToNodes();
+  public abstract Map<NodeAttributeKey, List<NodeToAttributeValue>> getAttributesToNodes();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeAttributesResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeAttributesResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeAttributesResponse.java
index cc3cae4..b0ccd90 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeAttributesResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeAttributesResponse.java
@@ -17,14 +17,15 @@
  */
 package org.apache.hadoop.yarn.api.protocolrecords;
 
-import static org.apache.hadoop.classification.InterfaceAudience.*;
-import static org.apache.hadoop.classification.InterfaceStability.*;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
-import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeInfo;
 import org.apache.hadoop.yarn.util.Records;
 
-import java.util.Set;
-
 /**
  * <p>
  * The response sent by the <code>ResourceManager</code> to a client requesting
@@ -45,7 +46,7 @@ public abstract class GetClusterNodeAttributesResponse {
    * @return GetClusterNodeAttributesResponse.
    */
   public static GetClusterNodeAttributesResponse newInstance(
-      Set<NodeAttribute> attributes) {
+      Set<NodeAttributeInfo> attributes) {
     GetClusterNodeAttributesResponse response =
         Records.newRecord(GetClusterNodeAttributesResponse.class);
     response.setNodeAttributes(attributes);
@@ -55,18 +56,18 @@ public abstract class GetClusterNodeAttributesResponse {
   /**
    * Set node attributes to the response.
    *
-   * @param attributes Node attributes
+   * @param attributes Map of Node attributeKey to Type.
    */
   @Public
   @Unstable
-  public abstract void setNodeAttributes(Set<NodeAttribute> attributes);
+  public abstract void setNodeAttributes(Set<NodeAttributeInfo> attributes);
 
   /**
-   * Get node attributes of the response.
+   * Get node attributes from the response.
    *
-   * @return Node attributes
+   * @return Node attributes.
    */
   @Public
   @Unstable
-  public abstract Set<NodeAttribute> getNodeAttributes();
+  public abstract Set<NodeAttributeInfo> getNodeAttributes();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
index 25ac9ab..7064939 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
@@ -58,8 +58,9 @@ public abstract class NodeAttribute {
       String attributeName, NodeAttributeType attributeType,
       String attributeValue) {
     NodeAttribute nodeAttribute = Records.newRecord(NodeAttribute.class);
-    nodeAttribute.setAttributePrefix(attributePrefix);
-    nodeAttribute.setAttributeName(attributeName);
+    NodeAttributeKey nodeAttributeKey =
+        NodeAttributeKey.newInstance(attributePrefix, attributeName);
+    nodeAttribute.setAttributeKey(nodeAttributeKey);
     nodeAttribute.setAttributeType(attributeType);
     nodeAttribute.setAttributeValue(attributeValue);
     return nodeAttribute;
@@ -67,19 +68,11 @@ public abstract class NodeAttribute {
 
   @Public
   @Unstable
-  public abstract String getAttributePrefix();
+  public abstract NodeAttributeKey getAttributeKey();
 
   @Public
   @Unstable
-  public abstract void setAttributePrefix(String attributePrefix);
-
-  @Public
-  @Unstable
-  public abstract String getAttributeName();
-
-  @Public
-  @Unstable
-  public abstract void setAttributeName(String attributeName);
+  public abstract void setAttributeKey(NodeAttributeKey attributeKey);
 
   @Public
   @Unstable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeInfo.java
new file mode 100644
index 0000000..d294333
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeInfo.java
@@ -0,0 +1,62 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * Node Attribute Info describes a NodeAttribute.
+ * </p>
+ */
+@Public
+@Unstable
+public abstract class NodeAttributeInfo {
+
+  public static NodeAttributeInfo newInstance(NodeAttribute nodeAttribute) {
+    return newInstance(nodeAttribute.getAttributeKey(),
+        nodeAttribute.getAttributeType());
+  }
+
+  public static NodeAttributeInfo newInstance(NodeAttributeKey nodeAttributeKey,
+      NodeAttributeType attributeType) {
+    NodeAttributeInfo nodeAttribute =
+        Records.newRecord(NodeAttributeInfo.class);
+    nodeAttribute.setAttributeKey(nodeAttributeKey);
+    nodeAttribute.setAttributeType(attributeType);
+    return nodeAttribute;
+  }
+
+  @Public
+  @Unstable
+  public abstract NodeAttributeKey getAttributeKey();
+
+  @Public
+  @Unstable
+  public abstract void setAttributeKey(NodeAttributeKey attributeKey);
+
+  @Public
+  @Unstable
+  public abstract NodeAttributeType getAttributeType();
+
+  @Public
+  @Unstable
+  public abstract void setAttributeType(NodeAttributeType attributeType);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeKey.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeKey.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeKey.java
new file mode 100644
index 0000000..35ff26f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeKey.java
@@ -0,0 +1,66 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * Node AttributeKey uniquely identifies a given Node Attribute. Node Attribute
+ * is identified based on attribute prefix and name.
+ * </p>
+ * <p>
+ * Node Attribute Prefix is used as namespace to segregate the attributes.
+ * </p>
+ */
+@Public
+@Unstable
+public abstract class NodeAttributeKey {
+
+  public static NodeAttributeKey newInstance(String attributeName) {
+    return newInstance(NodeAttribute.PREFIX_CENTRALIZED, attributeName);
+  }
+
+  public static NodeAttributeKey newInstance(String attributePrefix,
+      String attributeName) {
+    NodeAttributeKey nodeAttributeKey =
+        Records.newRecord(NodeAttributeKey.class);
+    nodeAttributeKey.setAttributePrefix(attributePrefix);
+    nodeAttributeKey.setAttributeName(attributeName);
+    return nodeAttributeKey;
+  }
+
+  @Public
+  @Unstable
+  public abstract String getAttributePrefix();
+
+  @Public
+  @Unstable
+  public abstract void setAttributePrefix(String attributePrefix);
+
+  @Public
+  @Unstable
+  public abstract String getAttributeName();
+
+  @Public
+  @Unstable
+  public abstract void setAttributeName(String attributeName);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeToAttributeValue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeToAttributeValue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeToAttributeValue.java
new file mode 100644
index 0000000..0bcb8b6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeToAttributeValue.java
@@ -0,0 +1,57 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * Mapping of Attribute Value to a Node.
+ * </p>
+ */
+@Public
+@Unstable
+public abstract class NodeToAttributeValue {
+  public static NodeToAttributeValue newInstance(String hostname,
+      String attributeValue) {
+    NodeToAttributeValue nodeToAttributeValue =
+        Records.newRecord(NodeToAttributeValue.class);
+    nodeToAttributeValue.setAttributeValue(attributeValue);
+    nodeToAttributeValue.setHostname(hostname);
+    return nodeToAttributeValue;
+  }
+
+  @Public
+  @Unstable
+  public abstract String getAttributeValue();
+
+  @Public
+  @Unstable
+  public abstract void setAttributeValue(String attributeValue);
+
+  @Public
+  @Unstable
+  public abstract String getHostname();
+
+  @Public
+  @Unstable
+  public abstract void setHostname(String hostname);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 5576ee6..aca9471 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -376,16 +376,31 @@ enum NodeAttributeTypeProto {
   STRING = 1;
 }
 
-message NodeAttributeProto {
-  optional string attributePrefix = 1;
+message NodeAttributeKeyProto {
+  optional string attributePrefix = 1 [default="rm.yarn.io"];
   required string attributeName = 2;
-  optional NodeAttributeTypeProto attributeType = 3 [default = STRING];
-  optional string attributeValue = 4 [default=""];
+}
+
+message NodeAttributeProto {
+  required NodeAttributeKeyProto attributeKey = 1;
+  optional NodeAttributeTypeProto attributeType = 2 [default = STRING];
+  optional string attributeValue = 3 [default=""];
+}
+
+
+message NodeAttributeInfoProto {
+  required NodeAttributeKeyProto attributeKey = 1;
+  required NodeAttributeTypeProto attributeType = 2;
+}
+
+message NodeToAttributeValueProto {
+  required string hostname = 1;
+  required string attributeValue = 2;
 }
 
 message AttributeToNodesProto {
-  required NodeAttributeProto nodeAttribute = 1;
-  repeated string hostnames = 2;
+  required NodeAttributeKeyProto nodeAttribute = 1;
+  repeated NodeToAttributeValueProto nodeValueMap = 2;
 }
 
 message NodeToAttributesProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index 439780b..248f775 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -264,11 +264,11 @@ message GetClusterNodeAttributesRequestProto {
 }
 
 message GetClusterNodeAttributesResponseProto {
-  repeated NodeAttributeProto nodeAttributes = 1;
+  repeated NodeAttributeInfoProto nodeAttributes = 1;
 }
 
 message GetAttributesToNodesRequestProto {
-  repeated NodeAttributeProto nodeAttributes = 1;
+  repeated NodeAttributeKeyProto nodeAttributes = 1;
 }
 
 message GetAttributesToNodesResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
index 0099845..f51b2f9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
@@ -52,10 +52,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.api.records.NodeAttributeInfo;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.NodeToAttributeValue;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
@@ -914,26 +917,27 @@ public abstract class YarnClient extends AbstractService {
    */
   @Public
   @Unstable
-  public abstract Set<NodeAttribute> getClusterAttributes()
+  public abstract Set<NodeAttributeInfo> getClusterAttributes()
       throws YarnException, IOException;
 
   /**
    * <p>
-   * The interface used by client to get Attributes to nodes mapping
-   * for specified node attributes in existing cluster.
+   * The interface used by client to get mapping of AttributeKey to associated
+   * NodeToAttributeValue list for specified node attributeKeys in the cluster.
    * </p>
    *
-   * @param attributes Attributes for which Attributes to nodes mapping has to
-   *                   be retrieved.If empty or null is set then will return
-   *                   all attributes to node mapping in cluster.
-   * @return Attributes to nodes mappings for specific Attributes.
+   * @param attributes AttributeKeys for which associated NodeToAttributeValue
+   *          mapping value has to be retrieved. If empty or null is set then
+   *          will return mapping for all attributeKeys in the cluster
+   * @return mapping of AttributeKey to List of associated
+   *         NodeToAttributeValue's.
    * @throws YarnException
    * @throws IOException
    */
   @Public
   @Unstable
-  public abstract Map<NodeAttribute, Set<String>> getAttributesToNodes(
-      Set<NodeAttribute> attributes) throws YarnException, IOException;
+  public abstract Map<NodeAttributeKey, List<NodeToAttributeValue>> getAttributesToNodes(
+      Set<NodeAttributeKey> attributes) throws YarnException, IOException;
 
   /**
    * <p>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
index a08d35d..acfc3ff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
@@ -99,10 +99,13 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.api.records.NodeAttributeInfo;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.NodeToAttributeValue;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
@@ -981,7 +984,7 @@ public class YarnClientImpl extends YarnClient {
   }
 
   @Override
-  public Set<NodeAttribute> getClusterAttributes()
+  public Set<NodeAttributeInfo> getClusterAttributes()
       throws YarnException, IOException {
     GetClusterNodeAttributesRequest request =
         GetClusterNodeAttributesRequest.newInstance();
@@ -989,8 +992,8 @@ public class YarnClientImpl extends YarnClient {
   }
 
   @Override
-  public Map<NodeAttribute, Set<String>> getAttributesToNodes(
-      Set<NodeAttribute> attributes) throws YarnException, IOException {
+  public Map<NodeAttributeKey, List<NodeToAttributeValue>> getAttributesToNodes(
+      Set<NodeAttributeKey> attributes) throws YarnException, IOException {
     GetAttributesToNodesRequest request =
         GetAttributesToNodesRequest.newInstance(attributes);
     return rmClient.getAttributesToNodes(request).getAttributesToNodes();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesRequestPBImpl.java
index a84fb44..15a360c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesRequestPBImpl.java
@@ -17,21 +17,21 @@
  */
 package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 
-import com.google.protobuf.TextFormat;
-import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
-import org.apache.hadoop.yarn.api.records.NodeAttribute;
-import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAttributesToNodesRequestProto;
-
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 
-import static org.apache.hadoop.classification.InterfaceAudience.*;
-import static org.apache.hadoop.classification.InterfaceStability.*;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributeKeyPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeKeyProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAttributesToNodesRequestProto;
+
+import com.google.protobuf.TextFormat;
 
 /**
  * Attributes to nodes mapping request.
@@ -41,7 +41,7 @@ import static org.apache.hadoop.classification.InterfaceStability.*;
 public class GetAttributesToNodesRequestPBImpl
     extends GetAttributesToNodesRequest {
 
-  private Set<NodeAttribute> nodeAttributes = null;
+  private Set<NodeAttributeKey> nodeAttributes = null;
 
   private GetAttributesToNodesRequestProto proto =
       GetAttributesToNodesRequestProto.getDefaultInstance();
@@ -86,9 +86,9 @@ public class GetAttributesToNodesRequestPBImpl
     if (nodeAttributes == null) {
       return;
     }
-    Iterable<NodeAttributeProto> iterable =
-        () -> new Iterator<NodeAttributeProto>() {
-          private Iterator<NodeAttribute> iter = nodeAttributes.iterator();
+    Iterable<NodeAttributeKeyProto> iterable =
+        () -> new Iterator<NodeAttributeKeyProto>() {
+          private Iterator<NodeAttributeKey> iter = nodeAttributes.iterator();
 
           @Override
           public boolean hasNext() {
@@ -96,7 +96,7 @@ public class GetAttributesToNodesRequestPBImpl
           }
 
           @Override
-          public NodeAttributeProto next() {
+          public NodeAttributeKeyProto next() {
             return convertToProtoFormat(iter.next());
           }
 
@@ -110,12 +110,13 @@ public class GetAttributesToNodesRequestPBImpl
     builder.addAllNodeAttributes(iterable);
   }
 
-  private NodeAttributePBImpl convertFromProtoFormat(NodeAttributeProto p) {
-    return new NodeAttributePBImpl(p);
+  private NodeAttributeKeyPBImpl convertFromProtoFormat(
+      NodeAttributeKeyProto p) {
+    return new NodeAttributeKeyPBImpl(p);
   }
 
-  private NodeAttributeProto convertToProtoFormat(NodeAttribute t) {
-    return ((NodeAttributePBImpl) t).getProto();
+  private NodeAttributeKeyProto convertToProtoFormat(NodeAttributeKey t) {
+    return ((NodeAttributeKeyPBImpl) t).getProto();
   }
 
   private void maybeInitBuilder() {
@@ -131,7 +132,7 @@ public class GetAttributesToNodesRequestPBImpl
     }
     YarnServiceProtos.GetAttributesToNodesRequestProtoOrBuilder p =
         viaProto ? proto : builder;
-    List<NodeAttributeProto> nodeAttributesList = p.getNodeAttributesList();
+    List<NodeAttributeKeyProto> nodeAttributesList = p.getNodeAttributesList();
     this.nodeAttributes = new HashSet<>();
     nodeAttributesList
         .forEach((v) -> nodeAttributes.add(convertFromProtoFormat(v)));
@@ -159,7 +160,7 @@ public class GetAttributesToNodesRequestPBImpl
   }
 
   @Override
-  public void setNodeAttributes(Set<NodeAttribute> attributes) {
+  public void setNodeAttributes(Set<NodeAttributeKey> attributes) {
     maybeInitBuilder();
     if (nodeAttributes == null) {
       builder.clearNodeAttributes();
@@ -168,7 +169,7 @@ public class GetAttributesToNodesRequestPBImpl
   }
 
   @Override
-  public Set<NodeAttribute> getNodeAttributes() {
+  public Set<NodeAttributeKey> getNodeAttributes() {
     initNodeAttributes();
     return this.nodeAttributes;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
index 175c10e..11999bdc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
@@ -17,23 +17,24 @@
  */
 package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 
-import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
-import org.apache.hadoop.yarn.api.records.NodeAttribute;
-import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.AttributeToNodesProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAttributesToNodesResponseProto;
-
+import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
-import static org.apache.hadoop.classification.InterfaceAudience.*;
-import static org.apache.hadoop.classification.InterfaceStability.*;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.api.records.NodeToAttributeValue;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributeKeyPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeToAttributeValuePBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.AttributeToNodesProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeKeyProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeToAttributeValueProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAttributesToNodesResponseProto;
 
 /**
  * Attributes to nodes response.
@@ -48,7 +49,7 @@ public class GetAttributesToNodesResponsePBImpl
   private GetAttributesToNodesResponseProto.Builder builder = null;
   private boolean viaProto = false;
 
-  private Map<NodeAttribute, Set<String>> attributesToNodes;
+  private Map<NodeAttributeKey, List<NodeToAttributeValue>> attributesToNodes;
 
   public GetAttributesToNodesResponsePBImpl() {
     this.builder = GetAttributesToNodesResponseProto.newBuilder();
@@ -70,10 +71,15 @@ public class GetAttributesToNodesResponsePBImpl
     this.attributesToNodes = new HashMap<>();
 
     for (AttributeToNodesProto c : list) {
-      Set<String> setNodes = new HashSet<>(c.getHostnamesList());
-      if (!setNodes.isEmpty()) {
-        this.attributesToNodes
-            .put(convertFromProtoFormat(c.getNodeAttribute()), setNodes);
+      List<NodeToAttributeValueProto> nodeValueMapList =
+          c.getNodeValueMapList();
+      List<NodeToAttributeValue> nodeToAttributeValue = new ArrayList<>();
+      for (NodeToAttributeValueProto valueProto : nodeValueMapList) {
+        nodeToAttributeValue.add(convertFromProtoFormat(valueProto));
+      }
+      if (!nodeToAttributeValue.isEmpty()) {
+        this.attributesToNodes.put(convertFromProtoFormat(c.getNodeAttribute()),
+            nodeToAttributeValue);
       }
     }
   }
@@ -94,7 +100,7 @@ public class GetAttributesToNodesResponsePBImpl
     Iterable<AttributeToNodesProto> iterable =
         () -> new Iterator<AttributeToNodesProto>() {
 
-          private Iterator<Map.Entry<NodeAttribute, Set<String>>> iter =
+          private Iterator<Map.Entry<NodeAttributeKey, List<NodeToAttributeValue>>> iter =
               attributesToNodes.entrySet().iterator();
 
           @Override
@@ -104,14 +110,18 @@ public class GetAttributesToNodesResponsePBImpl
 
           @Override
           public AttributeToNodesProto next() {
-            Map.Entry<NodeAttribute, Set<String>> now = iter.next();
-            Set<String> hostNames = new HashSet<>();
-            for (String host : now.getValue()) {
-              hostNames.add(host);
+            Map.Entry<NodeAttributeKey, List<NodeToAttributeValue>> attrToNodes
+                      = iter.next();
+
+            AttributeToNodesProto.Builder attrToNodesBuilder =
+                AttributeToNodesProto.newBuilder().setNodeAttribute(
+                    convertToProtoFormat(attrToNodes.getKey()));
+            for (NodeToAttributeValue hostToAttrVal : attrToNodes.getValue()) {
+              attrToNodesBuilder
+                  .addNodeValueMap(convertToProtoFormat(hostToAttrVal));
             }
-            return AttributeToNodesProto.newBuilder()
-                .setNodeAttribute(convertToProtoFormat(now.getKey()))
-                .addAllHostnames(hostNames).build();
+
+            return attrToNodesBuilder.build();
           }
 
           @Override
@@ -122,12 +132,22 @@ public class GetAttributesToNodesResponsePBImpl
     builder.addAllAttributesToNodes(iterable);
   }
 
-  private NodeAttributePBImpl convertFromProtoFormat(NodeAttributeProto p) {
-    return new NodeAttributePBImpl(p);
+  private NodeAttributeKey convertFromProtoFormat(NodeAttributeKeyProto p) {
+    return new NodeAttributeKeyPBImpl(p);
+  }
+
+  private NodeAttributeKeyProto convertToProtoFormat(NodeAttributeKey t) {
+    return ((NodeAttributeKeyPBImpl) t).getProto();
+  }
+
+  private NodeToAttributeValue convertFromProtoFormat(
+      NodeToAttributeValueProto p) {
+    return new NodeToAttributeValuePBImpl(p);
   }
 
-  private NodeAttributeProto convertToProtoFormat(NodeAttribute t) {
-    return ((NodeAttributePBImpl) t).getProto();
+  private NodeToAttributeValueProto convertToProtoFormat(
+      NodeToAttributeValue t) {
+    return ((NodeToAttributeValuePBImpl) t).getProto();
   }
 
   private void mergeLocalToBuilder() {
@@ -170,14 +190,15 @@ public class GetAttributesToNodesResponsePBImpl
   }
 
   @Override
-  public void setAttributeToNodes(Map<NodeAttribute, Set<String>> map) {
+  public void setAttributeToNodes(
+      Map<NodeAttributeKey, List<NodeToAttributeValue>> map) {
     initAttributesToNodes();
     attributesToNodes.clear();
     attributesToNodes.putAll(map);
   }
 
   @Override
-  public Map<NodeAttribute, Set<String>> getAttributesToNodes() {
+  public Map<NodeAttributeKey, List<NodeToAttributeValue>> getAttributesToNodes() {
     initAttributesToNodes();
     return this.attributesToNodes;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeAttributesResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeAttributesResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeAttributesResponsePBImpl.java
index 385155f..41cd808 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeAttributesResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeAttributesResponsePBImpl.java
@@ -17,19 +17,19 @@
  */
 package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 
-import static org.apache.hadoop.classification.InterfaceAudience.*;
-import static org.apache.hadoop.classification.InterfaceStability.*;
-import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
-import org.apache.hadoop.yarn.api.records.NodeAttribute;
-import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterNodeAttributesResponseProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
-
-import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
+import org.apache.hadoop.yarn.api.records.NodeAttributeInfo;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributeInfoPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeInfoProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterNodeAttributesResponseProto;
 
 /**
  * Cluster node attributes response.
@@ -42,7 +42,7 @@ public class GetClusterNodeAttributesResponsePBImpl
   private GetClusterNodeAttributesResponseProto proto =
       GetClusterNodeAttributesResponseProto.getDefaultInstance();
   private GetClusterNodeAttributesResponseProto.Builder builder = null;
-  private Set<NodeAttribute> updatedNodeAttributes;
+  private Set<NodeAttributeInfo> clusterNodeAttributes;
   private boolean viaProto = false;
 
   public GetClusterNodeAttributesResponsePBImpl() {
@@ -72,7 +72,7 @@ public class GetClusterNodeAttributesResponsePBImpl
   }
 
   private void mergeLocalToBuilder() {
-    if (this.updatedNodeAttributes != null) {
+    if (this.clusterNodeAttributes != null) {
       addNodeAttributesToProto();
     }
   }
@@ -80,11 +80,12 @@ public class GetClusterNodeAttributesResponsePBImpl
   private void addNodeAttributesToProto() {
     maybeInitBuilder();
     builder.clearNodeAttributes();
-    List<NodeAttributeProto> protoList = new ArrayList<>();
-    for (NodeAttribute r : this.updatedNodeAttributes) {
-      protoList.add(convertToProtoFormat(r));
+    if (clusterNodeAttributes == null || clusterNodeAttributes.isEmpty()) {
+      return;
     }
-    builder.addAllNodeAttributes(protoList);
+
+    builder.addAllNodeAttributes(clusterNodeAttributes.stream()
+        .map(s -> convertToProtoFormat(s)).collect(Collectors.toSet()));
   }
 
   @Override
@@ -112,41 +113,44 @@ public class GetClusterNodeAttributesResponsePBImpl
   }
 
   @Override
-  public synchronized void setNodeAttributes(Set<NodeAttribute> attributes) {
+  public synchronized void setNodeAttributes(
+      Set<NodeAttributeInfo> attributes) {
     maybeInitBuilder();
-    this.updatedNodeAttributes = new HashSet<>();
+    this.clusterNodeAttributes = new HashSet<>();
     if (attributes == null) {
       builder.clearNodeAttributes();
       return;
     }
-    this.updatedNodeAttributes.addAll(attributes);
+    this.clusterNodeAttributes.addAll(attributes);
   }
 
   @Override
-  public synchronized Set<NodeAttribute> getNodeAttributes() {
-    if (this.updatedNodeAttributes != null) {
-      return this.updatedNodeAttributes;
+  public synchronized Set<NodeAttributeInfo> getNodeAttributes() {
+    if (this.clusterNodeAttributes != null) {
+      return this.clusterNodeAttributes;
     }
     initLocalNodeAttributes();
-    return this.updatedNodeAttributes;
+    return this.clusterNodeAttributes;
   }
 
   private void initLocalNodeAttributes() {
     YarnServiceProtos.GetClusterNodeAttributesResponseProtoOrBuilder p =
         viaProto ? proto : builder;
-    List<NodeAttributeProto> attributesProtoList = p.getNodeAttributesList();
-    this.updatedNodeAttributes = new HashSet<>();
-    for (NodeAttributeProto r : attributesProtoList) {
-      this.updatedNodeAttributes.add(convertFromProtoFormat(r));
-    }
+    List<NodeAttributeInfoProto> attributesProtoList =
+        p.getNodeAttributesList();
+    this.clusterNodeAttributes = new HashSet<>();
+    clusterNodeAttributes.addAll(attributesProtoList.stream()
+        .map(attr -> convertFromProtoFormat(attr)).collect(Collectors.toSet()));
   }
 
-  private NodeAttribute convertFromProtoFormat(NodeAttributeProto p) {
-    return new NodeAttributePBImpl(p);
+  private NodeAttributeInfoProto convertToProtoFormat(
+      NodeAttributeInfo attributeInfo) {
+    return ((NodeAttributeInfoPBImpl)attributeInfo).getProto();
   }
 
-  private NodeAttributeProto convertToProtoFormat(NodeAttribute t) {
-    return ((NodeAttributePBImpl) t).getProto();
+  private NodeAttributeInfo convertFromProtoFormat(
+      NodeAttributeInfoProto nodeAttributeInfoProto) {
+    return new NodeAttributeInfoPBImpl(nodeAttributeInfoProto);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributeInfoPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributeInfoPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributeInfoPBImpl.java
new file mode 100644
index 0000000..bff6335
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributeInfoPBImpl.java
@@ -0,0 +1,143 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.api.records.NodeAttributeInfo;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeKeyProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeInfoProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeInfoProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeTypeProto;
+
+/**
+ * Implementation for NodeAttributeInfo.
+ *
+ */
+public class NodeAttributeInfoPBImpl extends NodeAttributeInfo {
+  private NodeAttributeInfoProto proto =
+      NodeAttributeInfoProto.getDefaultInstance();
+  private NodeAttributeInfoProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public NodeAttributeInfoPBImpl() {
+    builder = NodeAttributeInfoProto.newBuilder();
+  }
+
+  public NodeAttributeInfoPBImpl(NodeAttributeInfoProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public NodeAttributeInfoProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = NodeAttributeInfoProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public NodeAttributeKey getAttributeKey() {
+    NodeAttributeInfoProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasAttributeKey()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getAttributeKey());
+  }
+
+  @Override
+  public void setAttributeKey(NodeAttributeKey attributeKey) {
+    maybeInitBuilder();
+    if (attributeKey == null) {
+      builder.clearAttributeKey();
+      return;
+    }
+    builder.setAttributeKey(convertToProtoFormat(attributeKey));
+  }
+
+  @Override
+  public NodeAttributeType getAttributeType() {
+    NodeAttributeInfoProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasAttributeType()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getAttributeType());
+  }
+
+  @Override
+  public void setAttributeType(NodeAttributeType attributeType) {
+    maybeInitBuilder();
+    if (attributeType == null) {
+      builder.clearAttributeType();
+      return;
+    }
+    builder.setAttributeType(convertToProtoFormat(attributeType));
+  }
+
+  private NodeAttributeTypeProto convertToProtoFormat(
+      NodeAttributeType attributeType) {
+    return NodeAttributeTypeProto.valueOf(attributeType.name());
+  }
+
+  private NodeAttributeType convertFromProtoFormat(
+      NodeAttributeTypeProto containerState) {
+    return NodeAttributeType.valueOf(containerState.name());
+  }
+
+  private NodeAttributeKeyPBImpl convertFromProtoFormat(
+      NodeAttributeKeyProto attributeKeyProto) {
+    return new NodeAttributeKeyPBImpl(attributeKeyProto);
+  }
+
+  private NodeAttributeKeyProto convertToProtoFormat(
+      NodeAttributeKey attributeKey) {
+    return ((NodeAttributeKeyPBImpl) attributeKey).getProto();
+  }
+
+  @Override
+  public int hashCode() {
+    return getAttributeKey().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (obj instanceof NodeAttributeInfo) {
+      NodeAttributeInfo other = (NodeAttributeInfo) obj;
+      getAttributeKey().equals(other.getAttributeKey());
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return getAttributeKey().toString() + ":Type-" + getAttributeType();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributeKeyPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributeKeyPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributeKeyPBImpl.java
new file mode 100644
index 0000000..921e767
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributeKeyPBImpl.java
@@ -0,0 +1,140 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeKeyProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeKeyProtoOrBuilder;
+
+/**
+ * Implementation for NodeAttributeKey.
+ *
+ */
+@Private
+@Unstable
+public class NodeAttributeKeyPBImpl extends NodeAttributeKey {
+  private NodeAttributeKeyProto proto =
+      NodeAttributeKeyProto.getDefaultInstance();
+  private NodeAttributeKeyProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public NodeAttributeKeyPBImpl() {
+    builder = NodeAttributeKeyProto.newBuilder();
+  }
+
+  public NodeAttributeKeyPBImpl(NodeAttributeKeyProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public NodeAttributeKeyProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = NodeAttributeKeyProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public String getAttributePrefix() {
+    NodeAttributeKeyProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getAttributePrefix();
+  }
+
+  @Override
+  public void setAttributePrefix(String attributePrefix) {
+    maybeInitBuilder();
+    if (attributePrefix == null) {
+      builder.clearAttributePrefix();
+      return;
+    }
+    builder.setAttributePrefix(attributePrefix);
+  }
+
+  @Override
+  public String getAttributeName() {
+    NodeAttributeKeyProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasAttributeName()) {
+      return null;
+    }
+    return p.getAttributeName();
+  }
+
+  @Override
+  public void setAttributeName(String attributeName) {
+    maybeInitBuilder();
+    if (attributeName == null) {
+      builder.clearAttributeName();
+      return;
+    }
+    builder.setAttributeName(attributeName);
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((getAttributePrefix() == null) ? 0
+        : getAttributePrefix().hashCode());
+    result = prime * result
+        + ((getAttributeName() == null) ? 0 : getAttributeName().hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (obj instanceof NodeAttributeKey) {
+      NodeAttributeKey other = (NodeAttributeKey) obj;
+      if (!compare(getAttributePrefix(), other.getAttributePrefix())) {
+        return false;
+      }
+      if (!compare(getAttributeName(), other.getAttributeName())) {
+        return false;
+      }
+      return true;
+    }
+    return false;
+  }
+
+  private static boolean compare(Object left, Object right) {
+    if (left == null) {
+      return right == null;
+    } else {
+      return left.equals(right);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "Prefix-" + getAttributePrefix() + " :Name-" + getAttributeName();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
index 12a0ecc..0002567 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
@@ -18,12 +18,21 @@
 
 package org.apache.hadoop.yarn.api.records.impl.pb;
 
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
 import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeKeyProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeTypeProto;
 
+/**
+ * Implementation for NodeAttribute.
+ */
+@Private
+@Unstable
 public class NodeAttributePBImpl extends NodeAttribute {
   private NodeAttributeProto proto = NodeAttributeProto.getDefaultInstance();
   private NodeAttributeProto.Builder builder = null;
@@ -52,22 +61,22 @@ public class NodeAttributePBImpl extends NodeAttribute {
   }
 
   @Override
-  public String getAttributeName() {
+  public NodeAttributeKey getAttributeKey() {
     NodeAttributeProtoOrBuilder p = viaProto ? proto : builder;
-    if (!p.hasAttributeName()) {
+    if (!p.hasAttributeKey()) {
       return null;
     }
-    return p.getAttributeName();
+    return convertFromProtoFormat(p.getAttributeKey());
   }
 
   @Override
-  public void setAttributeName(String attributeName) {
+  public void setAttributeKey(NodeAttributeKey attributeKey) {
     maybeInitBuilder();
-    if(attributeName == null) {
-      builder.clearAttributeName();
+    if(attributeKey == null) {
+      builder.clearAttributeKey();
       return;
     }
-    builder.setAttributeName(attributeName);
+    builder.setAttributeKey(convertToProtoFormat(attributeKey));
   }
 
   @Override
@@ -118,15 +127,19 @@ public class NodeAttributePBImpl extends NodeAttribute {
     return NodeAttributeType.valueOf(containerState.name());
   }
 
+  private NodeAttributeKeyPBImpl convertFromProtoFormat(
+      NodeAttributeKeyProto attributeKeyProto) {
+    return new NodeAttributeKeyPBImpl(attributeKeyProto);
+  }
+
+  private NodeAttributeKeyProto convertToProtoFormat(
+      NodeAttributeKey attributeKey) {
+    return ((NodeAttributeKeyPBImpl)attributeKey).getProto();
+  }
+
   @Override
   public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((getAttributePrefix() == null) ? 0
-        : getAttributePrefix().hashCode());
-    result = prime * result
-        + ((getAttributeName() == null) ? 0 : getAttributeName().hashCode());
-    return result;
+    return getAttributeKey().hashCode();
   }
 
   @Override
@@ -139,47 +152,15 @@ public class NodeAttributePBImpl extends NodeAttribute {
     }
     if (obj instanceof NodeAttribute) {
       NodeAttribute other = (NodeAttribute) obj;
-      if (!compare(getAttributePrefix(), other.getAttributePrefix())) {
-        return false;
-      }
-      if (!compare(getAttributeName(), other.getAttributeName())) {
-        return false;
-      }
+      getAttributeKey().equals(other.getAttributeKey());
       return true;
     }
     return false;
   }
 
-  private static boolean compare(Object left, Object right) {
-    if (left == null) {
-      return right == null;
-    } else {
-      return left.equals(right);
-    }
-  }
-
-  @Override
-  public String getAttributePrefix() {
-    NodeAttributeProtoOrBuilder p = viaProto ? proto : builder;
-    if (!p.hasAttributePrefix()) {
-      return null;
-    }
-    return p.getAttributePrefix();
-  }
-
-  @Override
-  public void setAttributePrefix(String attributePrefix) {
-    maybeInitBuilder();
-    if(attributePrefix == null) {
-      builder.clearAttributePrefix();
-      return;
-    }
-    builder.setAttributePrefix(attributePrefix);
-  }
-
   @Override
   public String toString() {
-    return "Prefix-" + getAttributePrefix() + " :Name-" + getAttributeName()
-        + ":Value-" + getAttributeValue() + ":Type-" + getAttributeType();
+    return getAttributeKey().toString() + ":Value-" + getAttributeValue()
+        + ":Type-" + getAttributeType();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeToAttributeValuePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeToAttributeValuePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeToAttributeValuePBImpl.java
new file mode 100644
index 0000000..7a79876
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeToAttributeValuePBImpl.java
@@ -0,0 +1,137 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.NodeToAttributeValue;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeToAttributeValueProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeToAttributeValueProtoOrBuilder;
+
+/**
+ * PB Implementation for NodeToAttributeValue.
+ *
+ */
+public class NodeToAttributeValuePBImpl extends NodeToAttributeValue {
+  private NodeToAttributeValueProto proto =
+      NodeToAttributeValueProto.getDefaultInstance();
+  private NodeToAttributeValueProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public NodeToAttributeValuePBImpl() {
+    builder = NodeToAttributeValueProto.newBuilder();
+  }
+
+  public NodeToAttributeValuePBImpl(NodeToAttributeValueProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public NodeToAttributeValueProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = NodeToAttributeValueProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public String getAttributeValue() {
+    NodeToAttributeValueProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getAttributeValue();
+  }
+
+  @Override
+  public void setAttributeValue(String attributeValue) {
+    maybeInitBuilder();
+    if (attributeValue == null) {
+      builder.clearAttributeValue();
+      return;
+    }
+    builder.setAttributeValue(attributeValue);
+  }
+
+  @Override
+  public String getHostname() {
+    NodeToAttributeValueProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasHostname()) {
+      return null;
+    }
+    return p.getHostname();
+  }
+
+  @Override
+  public void setHostname(String hostname) {
+    maybeInitBuilder();
+    if (hostname == null) {
+      builder.clearHostname();
+      return;
+    }
+    builder.setHostname(hostname);
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result
+        + ((getAttributeValue() == null) ? 0 : getAttributeValue().hashCode());
+    result = prime * result
+        + ((getHostname() == null) ? 0 : getHostname().hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (obj instanceof NodeToAttributeValue) {
+      NodeToAttributeValue other = (NodeToAttributeValue) obj;
+      if (!compare(getAttributeValue(), other.getAttributeValue())) {
+        return false;
+      }
+      if (!compare(getHostname(), other.getHostname())) {
+        return false;
+      }
+      return true;
+    }
+    return false;
+  }
+
+  private static boolean compare(Object left, Object right) {
+    if (left == null) {
+      return right == null;
+    } else {
+      return left.equals(right);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "Name-" + getHostname() + " : Attribute Value-"
+        + getAttributeValue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
index 79c53e2..68c6ec6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
@@ -25,6 +25,7 @@ import java.util.Set;
 
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
 
 /**
@@ -81,20 +82,21 @@ public abstract class NodeAttributesManager extends AbstractService {
    *
    * @param prefix set of prefix string's for which the attributes needs to
    *          returned
-   * @return set of node Attributes
+   * @return Set of node Attributes
    */
   public abstract Set<NodeAttribute> getClusterNodeAttributes(
       Set<String> prefix);
 
   /**
-   * Given a attribute set, return what all Nodes have attribute mapped to it.
-   * If the attributes set is null or empty, all attributes mapping are
-   * returned.
+   * Return a map of Nodes to attribute value for the given NodeAttributeKeys.
+   * If the attributeKeys set is null or empty, then mapping for all attributes
+   * are returned.
    *
-   * @return a Map of attributes to set of hostnames.
+   * @return a Map of attributeKeys to a map of hostnames to its attribute
+   *         values.
    */
-  public abstract Map<NodeAttribute, Set<String>> getAttributesToNodes(
-      Set<NodeAttribute> attributes);
+  public abstract Map<NodeAttributeKey, Map<String, AttributeValue>> getAttributesToNodes(
+      Set<NodeAttributeKey> attributes);
 
   /**
    * NodeAttribute to AttributeValue Map.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
index 93a27a9..0dd0755 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.nodelabels;
 
 import com.google.common.base.Strings;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
 
 import java.io.IOException;
 import java.util.Set;
@@ -115,14 +116,18 @@ public final class NodeLabelUtil {
       throws IOException {
     if (attributeSet != null && !attributeSet.isEmpty()) {
       for (NodeAttribute nodeAttribute : attributeSet) {
-        String prefix = nodeAttribute.getAttributePrefix();
+        NodeAttributeKey attributeKey = nodeAttribute.getAttributeKey();
+        if (attributeKey == null) {
+          throw new IOException("AttributeKey  must be set");
+        }
+        String prefix = attributeKey.getAttributePrefix();
         if (Strings.isNullOrEmpty(prefix)) {
           throw new IOException("Attribute prefix must be set");
         }
         // Verify attribute prefix format.
         checkAndThrowAttributePrefix(prefix);
         // Verify attribute name format.
-        checkAndThrowLabelName(nodeAttribute.getAttributeName());
+        checkAndThrowLabelName(attributeKey.getAttributeName());
       }
     }
   }
@@ -140,8 +145,9 @@ public final class NodeLabelUtil {
     if (Strings.isNullOrEmpty(prefix)) {
       return attributeSet;
     }
-    return attributeSet.stream().filter(
-        nodeAttribute -> prefix.equals(nodeAttribute.getAttributePrefix()))
+    return attributeSet.stream()
+        .filter(nodeAttribute -> prefix
+            .equals(nodeAttribute.getAttributeKey().getAttributePrefix()))
         .collect(Collectors.toSet());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java
index 3b2bd16..ffe36c8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeAttribute.java
@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.yarn.nodelabels;
 
-import java.util.HashSet;
-import java.util.Set;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeAttributeType;
@@ -32,11 +32,11 @@ public class RMNodeAttribute extends AbstractLabel {
 
   private NodeAttribute attribute;
   // TODO need to revisit whether we need to make this concurrent implementation
-  private Set<String> nodes = new HashSet<>();
+  private Map<String, AttributeValue> nodes = new HashMap<>();
 
   public RMNodeAttribute(NodeAttribute attribute) {
-    this(attribute.getAttributeName(), Resource.newInstance(0, 0), 0,
-        attribute);
+    this(attribute.getAttributeKey().getAttributeName(),
+        Resource.newInstance(0, 0), 0, attribute);
   }
 
   public RMNodeAttribute(String labelName, Resource res, int activeNMs,
@@ -57,16 +57,16 @@ public class RMNodeAttribute extends AbstractLabel {
     return attribute.getAttributeType();
   }
 
-  public void addNode(String node) {
-    nodes.add(node);
+  public void addNode(String node, AttributeValue attributeValue) {
+    nodes.put(node, attributeValue);
   }
 
   public void removeNode(String node) {
     nodes.remove(node);
   }
 
-  public Set<String> getAssociatedNodeIds() {
-    return new HashSet<String>(nodes);
+  public Map<String, AttributeValue> getAssociatedNodeIds() {
+    return new HashMap<String,  AttributeValue>(nodes);
   }
 
   @Override


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[17/29] hadoop git commit: YARN-8100. Support API interface to query cluster attributes and attribute to nodes. Contributed by Bibin A Chundatt.

Posted by su...@apache.org.
YARN-8100. Support API interface to query cluster attributes and attribute to nodes. Contributed by Bibin A Chundatt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0a01b135
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0a01b135
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0a01b135

Branch: refs/heads/trunk
Commit: 0a01b1350d69dacaf101d52e54f67c91e28ab1ee
Parents: b9890d1
Author: Naganarasimha <na...@apache.org>
Authored: Tue Apr 10 07:28:53 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../hadoop/mapred/ResourceMgrDelegate.java      |  13 ++
 .../hadoop/mapred/TestClientRedirect.java       |  17 ++
 .../yarn/api/ApplicationClientProtocol.java     |  37 ++++
 .../GetAttributesToNodesRequest.java            |  70 +++++++
 .../GetAttributesToNodesResponse.java           |  62 +++++++
 .../GetClusterNodeAttributesRequest.java        |  47 +++++
 .../GetClusterNodeAttributesResponse.java       |  72 ++++++++
 .../hadoop/yarn/conf/YarnConfiguration.java     |   2 +-
 .../main/proto/applicationclient_protocol.proto |   2 +
 .../src/main/proto/yarn_protos.proto            |   4 +
 .../src/main/proto/yarn_service_protos.proto    |  15 ++
 .../hadoop/yarn/client/api/YarnClient.java      |  36 +++-
 .../yarn/client/api/impl/YarnClientImpl.java    |  21 ++-
 .../ApplicationClientProtocolPBClientImpl.java  |  37 ++++
 .../ApplicationClientProtocolPBServiceImpl.java |  44 +++++
 .../pb/GetAttributesToNodesRequestPBImpl.java   | 175 ++++++++++++++++++
 .../pb/GetAttributesToNodesResponsePBImpl.java  | 184 +++++++++++++++++++
 .../GetClusterNodeAttributesRequestPBImpl.java  |  75 ++++++++
 .../GetClusterNodeAttributesResponsePBImpl.java | 156 ++++++++++++++++
 .../yarn/nodelabels/NodeAttributesManager.java  |   9 +-
 .../hadoop/yarn/api/TestPBImplRecords.java      |  28 +++
 .../yarn/server/MockResourceManagerFacade.java  |  17 ++
 .../server/resourcemanager/ClientRMService.java |  30 +++
 .../nodelabels/NodeAttributesManagerImpl.java   |  35 ++--
 .../resourcemanager/TestClientRMService.java    | 126 +++++++++++++
 .../DefaultClientRequestInterceptor.java        |  17 ++
 .../clientrm/FederationClientInterceptor.java   |  17 ++
 .../router/clientrm/RouterClientRMService.java  |  19 ++
 .../PassThroughClientRequestInterceptor.java    |  17 ++
 29 files changed, 1362 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
index ac4b73b..4f96a6b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -538,4 +539,16 @@ public class ResourceMgrDelegate extends YarnClient {
       throws YarnException, IOException {
     return client.getResourceTypeInfo();
   }
+
+  @Override
+  public Set<NodeAttribute> getClusterAttributes()
+      throws YarnException, IOException {
+    return client.getClusterAttributes();
+  }
+
+  @Override
+  public Map<NodeAttribute, Set<String>> getAttributesToNodes(
+      Set<NodeAttribute> attributes) throws YarnException, IOException {
+    return client.getAttributesToNodes(attributes);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
index f97d0a4..23a1a85 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
@@ -82,8 +82,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
@@ -521,6 +525,19 @@ public class TestClientRedirect {
         throws YarnException, IOException {
       return null;
     }
+
+    @Override
+    public GetAttributesToNodesResponse getAttributesToNodes(
+        GetAttributesToNodesRequest request) throws YarnException, IOException {
+      return null;
+    }
+
+    @Override
+    public GetClusterNodeAttributesResponse getClusterNodeAttributes(
+        GetClusterNodeAttributesRequest request)
+        throws YarnException, IOException {
+      return null;
+    }
   }
 
   class HistoryService extends AMService implements HSClientProtocol {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
index 3c4e4d0..8661a78 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
@@ -27,8 +27,12 @@ import org.apache.hadoop.io.retry.Idempotent;
 import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
@@ -642,4 +646,37 @@ public interface ApplicationClientProtocol extends ApplicationBaseProtocol {
   @Unstable
   GetAllResourceTypeInfoResponse getResourceTypeInfo(
       GetAllResourceTypeInfoRequest request) throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by client to get attributes to nodes mappings
+   * available in ResourceManager.
+   * </p>
+   *
+   * @param request request to get details of attributes to nodes mapping.
+   * @return Response containing the details of attributes to nodes mappings.
+   * @throws YarnException if any error happens inside YARN
+   * @throws IOException   incase of other errors
+   */
+  @Public
+  @Unstable
+  GetAttributesToNodesResponse getAttributesToNodes(
+      GetAttributesToNodesRequest request) throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by client to get node attributes available in
+   * ResourceManager.
+   * </p>
+   *
+   * @param request request to get node attributes collection of this cluster.
+   * @return Response containing node attributes collection.
+   * @throws YarnException if any error happens inside YARN.
+   * @throws IOException   incase of other errors.
+   */
+  @Public
+  @Unstable
+  GetClusterNodeAttributesResponse getClusterNodeAttributes(
+      GetClusterNodeAttributesRequest request)
+      throws YarnException, IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesRequest.java
new file mode 100644
index 0000000..d9531b0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesRequest.java
@@ -0,0 +1,70 @@
+/**
+ * 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.api.protocolrecords;
+
+import static org.apache.hadoop.classification.InterfaceAudience.*;
+import static org.apache.hadoop.classification.InterfaceStability.*;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.Set;
+
+/**
+ * <p>
+ * The request from clients to get attribtues to nodes mapping
+ * in the cluster from the <code>ResourceManager</code>.
+ * </p>
+ *
+ * @see ApplicationClientProtocol#getAttributesToNodes
+ * (GetAttributesToNodesRequest)
+ */
+@Public
+@Evolving
+public abstract class GetAttributesToNodesRequest {
+
+  public static GetAttributesToNodesRequest newInstance() {
+    return Records.newRecord(GetAttributesToNodesRequest.class);
+  }
+
+  public static GetAttributesToNodesRequest newInstance(
+      Set<NodeAttribute> attributes) {
+    GetAttributesToNodesRequest request =
+        Records.newRecord(GetAttributesToNodesRequest.class);
+    request.setNodeAttributes(attributes);
+    return request;
+  }
+
+  /**
+   * Set node attributes for which the mapping is required.
+   *
+   * @param attributes Set<NodeAttribute> provided.
+   */
+  @Public
+  @Unstable
+  public abstract void setNodeAttributes(Set<NodeAttribute> attributes);
+
+  /**
+   * Get node attributes for which mapping mapping is required.
+   *
+   * @return Set<NodeAttribute>
+   */
+  @Public
+  @Unstable
+  public abstract Set<NodeAttribute> getNodeAttributes();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java
new file mode 100644
index 0000000..4fdb1f7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java
@@ -0,0 +1,62 @@
+/**
+ * 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.api.protocolrecords;
+
+import static org.apache.hadoop.classification.InterfaceAudience.Public;
+import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * <p>
+ * The response sent by the <code>ResourceManager</code> to a client requesting
+ * attributes to hostname mapping.
+ * </p>
+ *
+ * @see ApplicationClientProtocol#getAttributesToNodes
+ * (GetAttributesToNodesRequest)
+ */
+@Public
+@Evolving
+public abstract class GetAttributesToNodesResponse {
+  public static GetAttributesToNodesResponse newInstance(
+      Map<NodeAttribute, Set<String>> map) {
+    GetAttributesToNodesResponse response =
+        Records.newRecord(GetAttributesToNodesResponse.class);
+    response.setAttributeToNodes(map);
+    return response;
+  }
+
+  @Public
+  @Evolving
+  public abstract void setAttributeToNodes(Map<NodeAttribute, Set<String>> map);
+
+  /*
+   * Get attributes to node hostname mapping.
+   *
+   * @return Map<NodeAttribute, Set<String>> node attributes to hostname
+   * mapping.
+   */
+  @Public
+  @Evolving
+  public abstract Map<NodeAttribute, Set<String>> getAttributesToNodes();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeAttributesRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeAttributesRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeAttributesRequest.java
new file mode 100644
index 0000000..ca81f9a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeAttributesRequest.java
@@ -0,0 +1,47 @@
+/**
+ * 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.api.protocolrecords;
+
+import static org.apache.hadoop.classification.InterfaceAudience.*;
+import static org.apache.hadoop.classification.InterfaceStability.*;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * The request from clients to get node attributes in the cluster from the
+ * <code>ResourceManager</code>.
+ * </p>
+ *
+ * @see ApplicationClientProtocol#getClusterNodeAttributes
+ * (GetClusterNodeAttributesRequest)
+ */
+@Public
+@Evolving
+public abstract class GetClusterNodeAttributesRequest {
+
+  /**
+   * Create new instance of GetClusterNodeAttributesRequest.
+   *
+   * @return GetClusterNodeAttributesRequest is returned.
+   */
+  public static GetClusterNodeAttributesRequest newInstance() {
+    return Records.newRecord(GetClusterNodeAttributesRequest.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeAttributesResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeAttributesResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeAttributesResponse.java
new file mode 100644
index 0000000..cc3cae4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetClusterNodeAttributesResponse.java
@@ -0,0 +1,72 @@
+/**
+ * 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.api.protocolrecords;
+
+import static org.apache.hadoop.classification.InterfaceAudience.*;
+import static org.apache.hadoop.classification.InterfaceStability.*;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.Set;
+
+/**
+ * <p>
+ * The response sent by the <code>ResourceManager</code> to a client requesting
+ * a node attributes in cluster.
+ * </p>
+ *
+ * @see ApplicationClientProtocol#getClusterNodeAttributes
+ * (GetClusterNodeAttributesRequest)
+ */
+@Public
+@Evolving
+public abstract class GetClusterNodeAttributesResponse {
+
+  /**
+   * Create instance of GetClusterNodeAttributesResponse.
+   *
+   * @param attributes
+   * @return GetClusterNodeAttributesResponse.
+   */
+  public static GetClusterNodeAttributesResponse newInstance(
+      Set<NodeAttribute> attributes) {
+    GetClusterNodeAttributesResponse response =
+        Records.newRecord(GetClusterNodeAttributesResponse.class);
+    response.setNodeAttributes(attributes);
+    return response;
+  }
+
+  /**
+   * Set node attributes to the response.
+   *
+   * @param attributes Node attributes
+   */
+  @Public
+  @Unstable
+  public abstract void setNodeAttributes(Set<NodeAttribute> attributes);
+
+  /**
+   * Get node attributes of the response.
+   *
+   * @return Node attributes
+   */
+  @Public
+  @Unstable
+  public abstract Set<NodeAttribute> getNodeAttributes();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index edad4d2..a82801d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -3491,7 +3491,7 @@ public class YarnConfiguration extends Configuration {
   public static final String FS_NODE_ATTRIBUTE_STORE_IMPL_CLASS =
       NODE_ATTRIBUTE_PREFIX + "fs-store.impl.class";
   /**
-   * File system not attribute store directory.
+   * File system node attribute store directory.
    */
   public static final String FS_NODE_ATTRIBUTE_STORE_ROOT_DIR =
       NODE_ATTRIBUTE_PREFIX + "fs-store.root-dir";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
index 81adef1..eeb884c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
@@ -64,4 +64,6 @@ service ApplicationClientProtocolService {
   rpc getResourceProfiles(GetAllResourceProfilesRequestProto) returns (GetAllResourceProfilesResponseProto);
   rpc getResourceProfile(GetResourceProfileRequestProto) returns (GetResourceProfileResponseProto);
   rpc getResourceTypeInfo(GetAllResourceTypeInfoRequestProto) returns (GetAllResourceTypeInfoResponseProto);
+  rpc getClusterNodeAttributes (GetClusterNodeAttributesRequestProto) returns (GetClusterNodeAttributesResponseProto);
+  rpc getAttributesToNodes (GetAttributesToNodesRequestProto) returns (GetAttributesToNodesResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 815e989..2b796ff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -383,6 +383,10 @@ message NodeAttributeProto {
   optional string attributeValue = 4 [default=""];
 }
 
+message AttributeToNodesProto {
+  required NodeAttributeProto nodeAttribute = 1;
+  repeated string hostnames = 2;
+}
 
 enum ContainerTypeProto {
   APPLICATION_MASTER = 1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index acd452d..084457b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -260,6 +260,21 @@ message GetClusterNodeLabelsResponseProto {
   repeated NodeLabelProto nodeLabels = 2;
 }
 
+message GetClusterNodeAttributesRequestProto {
+}
+
+message GetClusterNodeAttributesResponseProto {
+  repeated NodeAttributeProto nodeAttributes = 1;
+}
+
+message GetAttributesToNodesRequestProto {
+  repeated NodeAttributeProto nodeAttributes = 1;
+}
+
+message GetAttributesToNodesResponseProto {
+  repeated AttributeToNodesProto attributeToNodes = 1;
+}
+
 message UpdateApplicationPriorityRequestProto {
   required ApplicationIdProto applicationId = 1;
   required PriorityProto applicationPriority = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
index 26c99e3..ca0b7b7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
@@ -52,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -900,4 +900,38 @@ public abstract class YarnClient extends AbstractService {
   @Unstable
   public abstract List<ResourceTypeInfo> getResourceTypeInfo()
       throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by client to get node attributes in the cluster.
+   * </p>
+   *
+   * @return cluster node attributes collection
+   * @throws YarnException when there is a failure in
+   *                       {@link ApplicationClientProtocol}
+   * @throws IOException   when there is a failure in
+   *                       {@link ApplicationClientProtocol}
+   */
+  @Public
+  @Unstable
+  public abstract Set<NodeAttribute> getClusterAttributes()
+      throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by client to get Attributes to nodes mapping
+   * for specified node attributes in existing cluster.
+   * </p>
+   *
+   * @param attributes Attributes for which Attributes to nodes mapping has to
+   *                   be retrieved.If empty or null is set then will return
+   *                   all attributes to node mapping in cluster.
+   * @return Attributes to nodes mappings for specific Attributes.
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Public
+  @Unstable
+  public abstract Map<NodeAttribute, Set<String>> getAttributesToNodes(
+      Set<NodeAttribute> attributes) throws YarnException, IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
index 1ceb462..2c7496e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -52,8 +51,10 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
@@ -96,6 +97,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -104,7 +106,6 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.api.records.ResourceTypeInfo;
 import org.apache.hadoop.yarn.api.records.SignalContainerCommand;
 import org.apache.hadoop.yarn.api.records.Token;
@@ -977,4 +978,20 @@ public class YarnClientImpl extends YarnClient {
         GetAllResourceTypeInfoRequest.newInstance();
     return rmClient.getResourceTypeInfo(request).getResourceTypeInfo();
   }
+
+  @Override
+  public Set<NodeAttribute> getClusterAttributes()
+      throws YarnException, IOException {
+    GetClusterNodeAttributesRequest request =
+        GetClusterNodeAttributesRequest.newInstance();
+    return rmClient.getClusterNodeAttributes(request).getNodeAttributes();
+  }
+
+  @Override
+  public Map<NodeAttribute, Set<String>> getAttributesToNodes(
+      Set<NodeAttribute> attributes) throws YarnException, IOException {
+    GetAttributesToNodesRequest request =
+        GetAttributesToNodesRequest.newInstance(attributes);
+    return rmClient.getAttributesToNodes(request).getAttributesToNodes();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
index fd5096a..4cf0548 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
@@ -43,8 +43,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
@@ -107,8 +111,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportRe
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetAttributesToNodesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetAttributesToNodesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterMetricsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterMetricsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeAttributesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeAttributesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeLabelsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodesRequestPBImpl;
@@ -673,4 +681,33 @@ public class ApplicationClientProtocolPBClientImpl implements ApplicationClientP
       return null;
     }
   }
+
+  @Override
+  public GetAttributesToNodesResponse getAttributesToNodes(
+      GetAttributesToNodesRequest request) throws YarnException, IOException {
+    YarnServiceProtos.GetAttributesToNodesRequestProto requestProto =
+        ((GetAttributesToNodesRequestPBImpl) request).getProto();
+    try {
+      return new GetAttributesToNodesResponsePBImpl(
+          proxy.getAttributesToNodes(null, requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
+
+  @Override
+  public GetClusterNodeAttributesResponse getClusterNodeAttributes(
+      GetClusterNodeAttributesRequest request)
+      throws YarnException, IOException {
+    YarnServiceProtos.GetClusterNodeAttributesRequestProto requestProto =
+        ((GetClusterNodeAttributesRequestPBImpl) request).getProto();
+    try {
+      return new GetClusterNodeAttributesResponsePBImpl(
+          proxy.getClusterNodeAttributes(null, requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
index 423287e..8e53f08 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
@@ -35,7 +35,10 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRes
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse;
@@ -73,8 +76,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportRe
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetAttributesToNodesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetAttributesToNodesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterMetricsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterMetricsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeAttributesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeAttributesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeLabelsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodesRequestPBImpl;
@@ -184,6 +191,8 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllResourceTypeInfoResp
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllResourceProfilesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterNodeAttributesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAttributesToNodesResponseProto;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -694,4 +703,39 @@ public class ApplicationClientProtocolPBServiceImpl implements ApplicationClient
       throw new ServiceException(ie);
     }
   }
+
+  @Override
+  public GetClusterNodeAttributesResponseProto getClusterNodeAttributes(
+      RpcController controller,
+      YarnServiceProtos.GetClusterNodeAttributesRequestProto proto)
+      throws ServiceException {
+    GetClusterNodeAttributesRequest req =
+        new GetClusterNodeAttributesRequestPBImpl(proto);
+    try {
+      GetClusterNodeAttributesResponse resp =
+          real.getClusterNodeAttributes(req);
+      return ((GetClusterNodeAttributesResponsePBImpl) resp).getProto();
+    } catch (YarnException ye) {
+      throw new ServiceException(ye);
+    } catch (IOException ie) {
+      throw new ServiceException(ie);
+    }
+  }
+
+  @Override
+  public GetAttributesToNodesResponseProto getAttributesToNodes(
+      RpcController controller,
+      YarnServiceProtos.GetAttributesToNodesRequestProto proto)
+      throws ServiceException {
+    GetAttributesToNodesRequestPBImpl req =
+        new GetAttributesToNodesRequestPBImpl(proto);
+    try {
+      GetAttributesToNodesResponse resp = real.getAttributesToNodes(req);
+      return ((GetAttributesToNodesResponsePBImpl) resp).getProto();
+    } catch (YarnException ye) {
+      throw new ServiceException(ye);
+    } catch (IOException ie) {
+      throw new ServiceException(ie);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesRequestPBImpl.java
new file mode 100644
index 0000000..a84fb44
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesRequestPBImpl.java
@@ -0,0 +1,175 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import com.google.protobuf.TextFormat;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAttributesToNodesRequestProto;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.classification.InterfaceAudience.*;
+import static org.apache.hadoop.classification.InterfaceStability.*;
+
+/**
+ * Attributes to nodes mapping request.
+ */
+@Private
+@Unstable
+public class GetAttributesToNodesRequestPBImpl
+    extends GetAttributesToNodesRequest {
+
+  private Set<NodeAttribute> nodeAttributes = null;
+
+  private GetAttributesToNodesRequestProto proto =
+      GetAttributesToNodesRequestProto.getDefaultInstance();
+  private GetAttributesToNodesRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public GetAttributesToNodesRequestPBImpl() {
+    builder = GetAttributesToNodesRequestProto.newBuilder();
+  }
+
+  public GetAttributesToNodesRequestPBImpl(
+      GetAttributesToNodesRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetAttributesToNodesRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.nodeAttributes != null) {
+      addLocalAttributesToProto();
+    }
+  }
+
+  private void addLocalAttributesToProto() {
+    maybeInitBuilder();
+    builder.clearNodeAttributes();
+    if (nodeAttributes == null) {
+      return;
+    }
+    Iterable<NodeAttributeProto> iterable =
+        () -> new Iterator<NodeAttributeProto>() {
+          private Iterator<NodeAttribute> iter = nodeAttributes.iterator();
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public NodeAttributeProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+
+          }
+        };
+
+    builder.addAllNodeAttributes(iterable);
+  }
+
+  private NodeAttributePBImpl convertFromProtoFormat(NodeAttributeProto p) {
+    return new NodeAttributePBImpl(p);
+  }
+
+  private NodeAttributeProto convertToProtoFormat(NodeAttribute t) {
+    return ((NodeAttributePBImpl) t).getProto();
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetAttributesToNodesRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void initNodeAttributes() {
+    if (this.nodeAttributes != null) {
+      return;
+    }
+    YarnServiceProtos.GetAttributesToNodesRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<NodeAttributeProto> nodeAttributesList = p.getNodeAttributesList();
+    this.nodeAttributes = new HashSet<>();
+    nodeAttributesList
+        .forEach((v) -> nodeAttributes.add(convertFromProtoFormat(v)));
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public void setNodeAttributes(Set<NodeAttribute> attributes) {
+    maybeInitBuilder();
+    if (nodeAttributes == null) {
+      builder.clearNodeAttributes();
+    }
+    this.nodeAttributes = attributes;
+  }
+
+  @Override
+  public Set<NodeAttribute> getNodeAttributes() {
+    initNodeAttributes();
+    return this.nodeAttributes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
new file mode 100644
index 0000000..ab6204e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
@@ -0,0 +1,184 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.AttributeToNodesProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAttributesToNodesResponseProto;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.hadoop.classification.InterfaceAudience.*;
+import static org.apache.hadoop.classification.InterfaceStability.*;
+
+/**
+ * Attributes to nodes response.
+ */
+@Private
+@Unstable
+public class GetAttributesToNodesResponsePBImpl
+    extends GetAttributesToNodesResponse {
+
+  private GetAttributesToNodesResponseProto proto =
+      GetAttributesToNodesResponseProto.getDefaultInstance();
+  private GetAttributesToNodesResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private Map<NodeAttribute, Set<String>> attributesToNodes;
+
+  public GetAttributesToNodesResponsePBImpl() {
+    this.builder = GetAttributesToNodesResponseProto.newBuilder();
+  }
+
+  public GetAttributesToNodesResponsePBImpl(
+      GetAttributesToNodesResponseProto proto) {
+    this.proto = proto;
+    this.viaProto = true;
+  }
+
+  private void initAttributesToNodes() {
+    if (this.attributesToNodes != null) {
+      return;
+    }
+    YarnServiceProtos.GetAttributesToNodesResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<AttributeToNodesProto> list = p.getAttributeToNodesList();
+    this.attributesToNodes = new HashMap<>();
+
+    for (AttributeToNodesProto c : list) {
+      Set<String> setNodes = new HashSet<>(c.getHostnamesList());
+      if (!setNodes.isEmpty()) {
+        this.attributesToNodes
+            .put(convertFromProtoFormat(c.getNodeAttribute()), setNodes);
+      }
+    }
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetAttributesToNodesResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void addAttributesToNodesToProto() {
+    maybeInitBuilder();
+    builder.clearAttributeToNodes();
+    if (attributesToNodes == null) {
+      return;
+    }
+    Iterable<AttributeToNodesProto> iterable =
+        () -> new Iterator<AttributeToNodesProto>() {
+
+          private Iterator<Map.Entry<NodeAttribute, Set<String>>> iter =
+              attributesToNodes.entrySet().iterator();
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+
+          @Override
+          public AttributeToNodesProto next() {
+            Map.Entry<NodeAttribute, Set<String>> now = iter.next();
+            Set<String> hostNames = new HashSet<>();
+            for (String host : now.getValue()) {
+              hostNames.add(host);
+            }
+            return AttributeToNodesProto.newBuilder()
+                .setNodeAttribute(convertToProtoFormat(now.getKey()))
+                .addAllHostnames(hostNames).build();
+          }
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+        };
+    builder.addAllAttributeToNodes(iterable);
+  }
+
+  private NodeAttributePBImpl convertFromProtoFormat(NodeAttributeProto p) {
+    return new NodeAttributePBImpl(p);
+  }
+
+  private NodeAttributeProto convertToProtoFormat(NodeAttribute t) {
+    return ((NodeAttributePBImpl) t).getProto();
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.attributesToNodes != null) {
+      addAttributesToNodesToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  public GetAttributesToNodesResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    assert false : "hashCode not designed";
+    return 0;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public void setAttributeToNodes(Map<NodeAttribute, Set<String>> map) {
+    initAttributesToNodes();
+    attributesToNodes.clear();
+    attributesToNodes.putAll(map);
+  }
+
+  @Override
+  public Map<NodeAttribute, Set<String>> getAttributesToNodes() {
+    initAttributesToNodes();
+    return this.attributesToNodes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeAttributesRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeAttributesRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeAttributesRequestPBImpl.java
new file mode 100644
index 0000000..bf5ab40
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeAttributesRequestPBImpl.java
@@ -0,0 +1,75 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import com.google.protobuf.TextFormat;
+import static org.apache.hadoop.classification.InterfaceAudience.*;
+import static org.apache.hadoop.classification.InterfaceStability.*;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterNodeAttributesRequestProto;
+
+/**
+ * Request to get cluster node attributes.
+ */
+@Private
+@Unstable
+public class GetClusterNodeAttributesRequestPBImpl
+    extends GetClusterNodeAttributesRequest {
+
+  private GetClusterNodeAttributesRequestProto proto =
+      GetClusterNodeAttributesRequestProto.getDefaultInstance();
+  private GetClusterNodeAttributesRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public GetClusterNodeAttributesRequestPBImpl() {
+    builder = GetClusterNodeAttributesRequestProto.newBuilder();
+  }
+
+  public GetClusterNodeAttributesRequestPBImpl(
+      GetClusterNodeAttributesRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetClusterNodeAttributesRequestProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeAttributesResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeAttributesResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeAttributesResponsePBImpl.java
new file mode 100644
index 0000000..385155f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetClusterNodeAttributesResponsePBImpl.java
@@ -0,0 +1,156 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import static org.apache.hadoop.classification.InterfaceAudience.*;
+import static org.apache.hadoop.classification.InterfaceStability.*;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterNodeAttributesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Cluster node attributes response.
+ */
+@Private
+@Unstable
+public class GetClusterNodeAttributesResponsePBImpl
+    extends GetClusterNodeAttributesResponse {
+
+  private GetClusterNodeAttributesResponseProto proto =
+      GetClusterNodeAttributesResponseProto.getDefaultInstance();
+  private GetClusterNodeAttributesResponseProto.Builder builder = null;
+  private Set<NodeAttribute> updatedNodeAttributes;
+  private boolean viaProto = false;
+
+  public GetClusterNodeAttributesResponsePBImpl() {
+    builder = GetClusterNodeAttributesResponseProto.newBuilder();
+  }
+
+  public GetClusterNodeAttributesResponsePBImpl(
+      GetClusterNodeAttributesResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public synchronized GetClusterNodeAttributesResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.updatedNodeAttributes != null) {
+      addNodeAttributesToProto();
+    }
+  }
+
+  private void addNodeAttributesToProto() {
+    maybeInitBuilder();
+    builder.clearNodeAttributes();
+    List<NodeAttributeProto> protoList = new ArrayList<>();
+    for (NodeAttribute r : this.updatedNodeAttributes) {
+      protoList.add(convertToProtoFormat(r));
+    }
+    builder.addAllNodeAttributes(protoList);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    assert false : "hashCode not designed";
+    return 0;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetClusterNodeAttributesResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public synchronized void setNodeAttributes(Set<NodeAttribute> attributes) {
+    maybeInitBuilder();
+    this.updatedNodeAttributes = new HashSet<>();
+    if (attributes == null) {
+      builder.clearNodeAttributes();
+      return;
+    }
+    this.updatedNodeAttributes.addAll(attributes);
+  }
+
+  @Override
+  public synchronized Set<NodeAttribute> getNodeAttributes() {
+    if (this.updatedNodeAttributes != null) {
+      return this.updatedNodeAttributes;
+    }
+    initLocalNodeAttributes();
+    return this.updatedNodeAttributes;
+  }
+
+  private void initLocalNodeAttributes() {
+    YarnServiceProtos.GetClusterNodeAttributesResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<NodeAttributeProto> attributesProtoList = p.getNodeAttributesList();
+    this.updatedNodeAttributes = new HashSet<>();
+    for (NodeAttributeProto r : attributesProtoList) {
+      this.updatedNodeAttributes.add(convertFromProtoFormat(r));
+    }
+  }
+
+  private NodeAttribute convertFromProtoFormat(NodeAttributeProto p) {
+    return new NodeAttributePBImpl(p);
+  }
+
+  private NodeAttributeProto convertToProtoFormat(NodeAttribute t) {
+    return ((NodeAttributePBImpl) t).getProto();
+  }
+
+  @Override
+  public String toString() {
+    return getProto().toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
index ec7d30d..3816051 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
@@ -88,12 +88,13 @@ public abstract class NodeAttributesManager extends AbstractService {
 
   /**
    * Given a attribute set, return what all Nodes have attribute mapped to it.
+   * If the attributes set is null or empty, all attributes mapping are
+   * returned.
    *
-   * @return a Map, of attribute to set of hostnames
+   * @return a Map of attributes to set of hostnames.
    */
-  //TODO need to handle as part of REST patch.
-/*  public abstract Map<NodeAttribute, Set<String>> getAttributesToNodes(
-      Set<NodeAttribute> attributes);*/
+  public abstract Map<NodeAttribute, Set<String>> getAttributesToNodes(
+      Set<NodeAttribute> attributes);
 
   /**
    * NodeAttribute to AttributeValue Map.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 1af4191..9397dd8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -53,8 +53,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportRe
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetAttributesToNodesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetAttributesToNodesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterMetricsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterMetricsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeAttributesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeAttributesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeLabelsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodesRequestPBImpl;
@@ -1259,4 +1263,28 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
     validatePBImplRecord(NodesToAttributesMappingRequestPBImpl.class,
         NodesToAttributesMappingRequestProto.class);
   }
+
+  @Test
+  public void testGetAttributesToNodesRequestPBImpl() throws Exception {
+    validatePBImplRecord(GetAttributesToNodesRequestPBImpl.class,
+        YarnServiceProtos.GetAttributesToNodesRequestProto.class);
+  }
+
+  @Test
+  public void testGetAttributesToNodesResponsePBImpl() throws Exception {
+    validatePBImplRecord(GetAttributesToNodesResponsePBImpl.class,
+        YarnServiceProtos.GetAttributesToNodesResponseProto.class);
+  }
+
+  @Test
+  public void testGetClusterNodeAttributesRequestPBImpl() throws Exception {
+    validatePBImplRecord(GetClusterNodeAttributesRequestPBImpl.class,
+        YarnServiceProtos.GetClusterNodeAttributesRequestProto.class);
+  }
+
+  @Test
+  public void testGetClusterNodeAttributesResponsePBImpl() throws Exception {
+    validatePBImplRecord(GetClusterNodeAttributesResponsePBImpl.class,
+        YarnServiceProtos.GetClusterNodeAttributesResponseProto.class);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
index b9cebfe..47b51f8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
@@ -53,8 +53,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
@@ -896,6 +900,19 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   }
 
   @Override
+  public GetAttributesToNodesResponse getAttributesToNodes(
+      GetAttributesToNodesRequest request) throws YarnException, IOException {
+    return null;
+  }
+
+  @Override
+  public GetClusterNodeAttributesResponse getClusterNodeAttributes(
+      GetClusterNodeAttributesRequest request)
+      throws YarnException, IOException {
+    return null;
+  }
+
+  @Override
   public NodesToAttributesMappingResponse mapAttributesToNodes(NodesToAttributesMappingRequest request)
     throws YarnException, IOException {
     return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index be99753..05c11cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -66,8 +66,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
@@ -127,6 +131,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -148,6 +153,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.Keys;
@@ -1838,6 +1844,30 @@ public class ClientRMService extends AbstractService implements
     return response;
   }
 
+  @Override
+  public GetAttributesToNodesResponse getAttributesToNodes(
+      GetAttributesToNodesRequest request) throws YarnException, IOException {
+    NodeAttributesManager attributesManager =
+        rmContext.getNodeAttributesManager();
+    GetAttributesToNodesResponse response = GetAttributesToNodesResponse
+        .newInstance(attributesManager
+            .getAttributesToNodes(request.getNodeAttributes()));
+    return response;
+  }
+
+  @Override
+  public GetClusterNodeAttributesResponse getClusterNodeAttributes(
+      GetClusterNodeAttributesRequest request)
+      throws YarnException, IOException {
+    NodeAttributesManager attributesManager =
+        rmContext.getNodeAttributesManager();
+    Set<NodeAttribute> attributes =
+        attributesManager.getClusterNodeAttributes(null);
+    GetClusterNodeAttributesResponse response =
+        GetClusterNodeAttributesResponse.newInstance(attributes);
+    return response;
+  }
+
   @VisibleForTesting
   public void setDisplayPerUserApps(boolean displayPerUserApps) {
     this.filterAppsByUser = displayPerUserApps;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
index b4686e6..09671f1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
@@ -36,7 +36,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import com.google.common.base.Strings;
-import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -367,19 +367,26 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
     return attributes;
   }
 
-  // TODO need to handle as part of REST patch.
-  /*
-   * @Override public Map<NodeAttribute, Set<String>> getAttributesToNodes(
-   * Set<NodeAttribute> attributes) { try { readLock.lock(); boolean
-   * fetchAllAttributes = (attributes == null || attributes.isEmpty());
-   * Map<NodeAttribute, Set<String>> attributesToNodes = new HashMap<>(); for
-   * (Entry<NodeAttribute, RMAttributeNodeLabel> attributeEntry :
-   * attributeCollections .entrySet()) { if (fetchAllAttributes ||
-   * attributes.contains(attributeEntry.getKey())) {
-   * attributesToNodes.put(attributeEntry.getKey(),
-   * attributeEntry.getValue().getAssociatedNodeIds()); } } return
-   * attributesToNodes; } finally { readLock.unlock(); } }
-   */
+  @Override
+  public Map<NodeAttribute, Set<String>> getAttributesToNodes(
+      Set<NodeAttribute> attributes) {
+    try {
+      readLock.lock();
+      boolean fetchAllAttributes = (attributes == null || attributes.isEmpty());
+      Map<NodeAttribute, Set<String>> attributesToNodes = new HashMap<>();
+      for (Entry<NodeAttribute, RMNodeAttribute> attributeEntry :
+          clusterAttributes.entrySet()) {
+        if (fetchAllAttributes || attributes
+            .contains(attributeEntry.getKey())) {
+          attributesToNodes.put(attributeEntry.getKey(),
+              attributeEntry.getValue().getAssociatedNodeIds());
+        }
+      }
+      return attributesToNodes;
+    } finally {
+      readLock.unlock();
+    }
+  }
 
   public Resource getResourceByAttribute(NodeAttribute attribute) {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index 6644e44..4a6b366 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -18,6 +18,13 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
@@ -2000,6 +2007,125 @@ public class TestClientRMService {
   }
 
   @Test(timeout = 120000)
+  public void testGetClusterNodeAttributes() throws IOException, YarnException {
+    MockRM rm = new MockRM() {
+      protected ClientRMService createClientRMService() {
+        return new ClientRMService(this.rmContext, scheduler, this.rmAppManager,
+            this.applicationACLsManager, this.queueACLsManager,
+            this.getRMContext().getRMDelegationTokenSecretManager());
+      }
+    };
+    rm.start();
+
+    NodeAttributesManager mgr = rm.getRMContext().getNodeAttributesManager();
+    NodeId host1 = NodeId.newInstance("host1", 0);
+    NodeId host2 = NodeId.newInstance("host2", 0);
+    NodeAttribute gpu = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "GPU",
+            NodeAttributeType.STRING, "nvida");
+    NodeAttribute os = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "OS",
+            NodeAttributeType.STRING, "windows64");
+    NodeAttribute docker = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_DISTRIBUTED, "DOCKER",
+            NodeAttributeType.STRING, "docker0");
+    Map<String, Set<NodeAttribute>> nodes = new HashMap<>();
+    nodes.put(host1.getHost(), ImmutableSet.of(gpu, os));
+    nodes.put(host2.getHost(), ImmutableSet.of(docker));
+    mgr.addNodeAttributes(nodes);
+    // Create a client.
+    Configuration conf = new Configuration();
+    YarnRPC rpc = YarnRPC.create(conf);
+    InetSocketAddress rmAddress = rm.getClientRMService().getBindAddress();
+    LOG.info("Connecting to ResourceManager at " + rmAddress);
+    ApplicationClientProtocol client = (ApplicationClientProtocol) rpc
+        .getProxy(ApplicationClientProtocol.class, rmAddress, conf);
+
+    GetClusterNodeAttributesRequest request =
+        GetClusterNodeAttributesRequest.newInstance();
+    GetClusterNodeAttributesResponse response =
+        client.getClusterNodeAttributes(request);
+    Set<NodeAttribute> attributes = response.getNodeAttributes();
+    Assert.assertEquals("Size not correct", 3, attributes.size());
+    Assert.assertTrue(attributes.contains(gpu));
+    Assert.assertTrue(attributes.contains(os));
+    Assert.assertTrue(attributes.contains(docker));
+    rpc.stopProxy(client, conf);
+    rm.close();
+  }
+
+  @Test(timeout = 120000)
+  public void testGetAttributesToNodes() throws IOException, YarnException {
+    MockRM rm = new MockRM() {
+      protected ClientRMService createClientRMService() {
+        return new ClientRMService(this.rmContext, scheduler, this.rmAppManager,
+            this.applicationACLsManager, this.queueACLsManager,
+            this.getRMContext().getRMDelegationTokenSecretManager());
+      }
+    };
+    rm.start();
+
+    NodeAttributesManager mgr = rm.getRMContext().getNodeAttributesManager();
+    String node1 = "host1";
+    String node2 = "host2";
+    NodeAttribute gpu = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "GPU",
+            NodeAttributeType.STRING, "nvida");
+    NodeAttribute os = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "OS",
+            NodeAttributeType.STRING, "windows64");
+    NodeAttribute docker = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_DISTRIBUTED, "DOCKER",
+            NodeAttributeType.STRING, "docker0");
+    NodeAttribute dist = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_DISTRIBUTED, "VERSION",
+            NodeAttributeType.STRING, "3_0_2");
+    Map<String, Set<NodeAttribute>> nodes = new HashMap<>();
+    nodes.put(node1, ImmutableSet.of(gpu, os, dist));
+    nodes.put(node2, ImmutableSet.of(docker, dist));
+    mgr.addNodeAttributes(nodes);
+    // Create a client.
+    Configuration conf = new Configuration();
+    YarnRPC rpc = YarnRPC.create(conf);
+    InetSocketAddress rmAddress = rm.getClientRMService().getBindAddress();
+    LOG.info("Connecting to ResourceManager at " + rmAddress);
+    ApplicationClientProtocol client = (ApplicationClientProtocol) rpc
+        .getProxy(ApplicationClientProtocol.class, rmAddress, conf);
+
+    GetAttributesToNodesRequest request =
+        GetAttributesToNodesRequest.newInstance();
+    GetAttributesToNodesResponse response =
+        client.getAttributesToNodes(request);
+    Map<NodeAttribute, Set<String>> attrs = response.getAttributesToNodes();
+    Assert.assertEquals(response.getAttributesToNodes().size(), 4);
+    Assert.assertEquals(attrs.get(dist).size(), 2);
+    Assert.assertEquals(attrs.get(os).size(), 1);
+    Assert.assertEquals(attrs.get(gpu).size(), 1);
+    Assert.assertTrue(attrs.get(dist).contains(node1));
+    Assert.assertTrue(attrs.get(dist).contains(node2));
+    Assert.assertTrue(attrs.get(docker).contains(node2));
+
+    GetAttributesToNodesRequest request2 =
+        GetAttributesToNodesRequest.newInstance(ImmutableSet.of(docker));
+    GetAttributesToNodesResponse response2 =
+        client.getAttributesToNodes(request2);
+    Map<NodeAttribute, Set<String>> attrs2 = response2.getAttributesToNodes();
+    Assert.assertEquals(response2.getAttributesToNodes().size(), 1);
+    Assert.assertTrue(attrs.get(docker).contains(node2));
+
+    GetAttributesToNodesRequest request3 =
+        GetAttributesToNodesRequest.newInstance(ImmutableSet.of(docker, os));
+    GetAttributesToNodesResponse response3 =
+        client.getAttributesToNodes(request3);
+    Map<NodeAttribute, Set<String>> attrs3 = response3.getAttributesToNodes();
+    Assert.assertEquals(response3.getAttributesToNodes().size(), 2);
+    Assert.assertTrue(attrs.get(os).contains(node1));
+    Assert.assertTrue(attrs.get(docker).contains(node2));
+    rpc.stopProxy(client, conf);
+    rm.close();
+  }
+
+  @Test(timeout = 120000)
   public void testUpdatePriorityAndKillAppWithZeroClusterResource()
       throws Exception {
     int maxPriority = 10;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/29] hadoop git commit: YARN-7840. Update PB for prefix support of node attributes. Contributed by Naganarasimha G R.

Posted by su...@apache.org.
YARN-7840. Update PB for prefix support of node attributes. Contributed by Naganarasimha G R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4458b277
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4458b277
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4458b277

Branch: refs/heads/trunk
Commit: 4458b2772f481259453ab5472a476bdebb8c835b
Parents: d9d93e3
Author: bibinchundatt <bi...@apache.org>
Authored: Fri Feb 2 10:31:00 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/api/records/NodeAttribute.java  | 22 ++++++++++-
 .../src/main/proto/yarn_protos.proto            |  7 ++--
 .../records/impl/pb/NodeAttributePBImpl.java    | 39 +++++++++++++++++---
 .../hadoop/yarn/api/TestPBImplRecords.java      |  7 ++--
 4 files changed, 61 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4458b277/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
index 13081f3..01c70b2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
@@ -37,15 +37,27 @@ import org.apache.hadoop.yarn.util.Records;
  * Its not compulsory for all the attributes to have value, empty string is the
  * default value of the <code>NodeAttributeType.STRING</code>
  * </p>
- *
+ * <p>
+ * Node Attribute Prefix is used as namespace to segregate the attributes.
+ * </p>
  */
 @Public
 @Unstable
 public abstract class NodeAttribute {
 
+  public static final String DEFAULT_PREFIX = "";
+
   public static NodeAttribute newInstance(String attributeName,
       NodeAttributeType attributeType, String attributeValue) {
+    return newInstance(DEFAULT_PREFIX, attributeName, attributeType,
+        attributeValue);
+  }
+
+  public static NodeAttribute newInstance(String attributePrefix,
+      String attributeName, NodeAttributeType attributeType,
+      String attributeValue) {
     NodeAttribute nodeAttribute = Records.newRecord(NodeAttribute.class);
+    nodeAttribute.setAttributePrefix(attributePrefix);
     nodeAttribute.setAttributeName(attributeName);
     nodeAttribute.setAttributeType(attributeType);
     nodeAttribute.setAttributeValue(attributeValue);
@@ -54,6 +66,14 @@ public abstract class NodeAttribute {
 
   @Public
   @Unstable
+  public abstract String getAttributePrefix();
+
+  @Public
+  @Unstable
+  public abstract void setAttributePrefix(String attributePrefix);
+
+  @Public
+  @Unstable
   public abstract String getAttributeName();
 
   @Public

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4458b277/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 41f5ccb..815e989 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -377,9 +377,10 @@ enum NodeAttributeTypeProto {
 }
 
 message NodeAttributeProto {
-  optional string attributeName = 1;
-  optional NodeAttributeTypeProto attributeType = 2;
-  optional string attributeValue = 3;
+  optional string attributePrefix = 1;
+  required string attributeName = 2;
+  optional NodeAttributeTypeProto attributeType = 3 [default = STRING];
+  optional string attributeValue = 4 [default=""];
 }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4458b277/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
index 11c9c48..7810939 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
@@ -63,6 +63,10 @@ public class NodeAttributePBImpl extends NodeAttribute {
   @Override
   public void setAttributeName(String attributeName) {
     maybeInitBuilder();
+    if(attributeName == null) {
+      builder.clearAttributeName();
+      return;
+    }
     builder.setAttributeName(attributeName);
   }
 
@@ -78,6 +82,10 @@ public class NodeAttributePBImpl extends NodeAttribute {
   @Override
   public void setAttributeValue(String attributeValue) {
     maybeInitBuilder();
+    if(attributeValue == null) {
+      builder.clearAttributeValue();
+      return;
+    }
     builder.setAttributeValue(attributeValue);
   }
 
@@ -111,12 +119,6 @@ public class NodeAttributePBImpl extends NodeAttribute {
   }
 
   @Override
-  public String toString() {
-    return " name-" + getAttributeName() + ":value-" + getAttributeValue()
-        + ":type-" + getAttributeType();
-  }
-
-  @Override
   public int hashCode() {
     return getProto().hashCode();
   }
@@ -152,4 +154,29 @@ public class NodeAttributePBImpl extends NodeAttribute {
       return left.equals(right);
     }
   }
+
+  @Override
+  public String getAttributePrefix() {
+    NodeAttributeProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasAttributePrefix()) {
+      return null;
+    }
+    return p.getAttributePrefix();
+  }
+
+  @Override
+  public void setAttributePrefix(String attributePrefix) {
+    maybeInitBuilder();
+    if(attributePrefix == null) {
+      builder.clearAttributePrefix();
+      return;
+    }
+    builder.setAttributePrefix(attributePrefix);
+  }
+
+  @Override
+  public String toString() {
+    return "Prefix-" + getAttributePrefix() + " :Name-" + getAttributeName()
+        + ":Value-" + getAttributeValue() + ":Type-" + getAttributeType();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4458b277/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 80bbac4..1af4191 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -1245,19 +1245,18 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
 
   @Test
   public void testNodeAttributePBImpl() throws Exception {
-    validatePBImplRecord(NodeAttributePBImpl.class,
-	NodeAttributeProto.class);
+    validatePBImplRecord(NodeAttributePBImpl.class, NodeAttributeProto.class);
   }
 
   @Test
   public void testNodeToAttributesPBImpl() throws Exception {
     validatePBImplRecord(NodeToAttributesPBImpl.class,
-	NodeToAttributesProto.class);
+        NodeToAttributesProto.class);
   }
 
   @Test
   public void testNodesToAttributesMappingRequestPBImpl() throws Exception {
     validatePBImplRecord(NodesToAttributesMappingRequestPBImpl.class,
-	NodesToAttributesMappingRequestProto.class);
+        NodesToAttributesMappingRequestProto.class);
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[22/29] hadoop git commit: YARN-8574. Allow dot in attribute values. Contributed by Bibin A Chundatt.

Posted by su...@apache.org.
YARN-8574. Allow dot in attribute values. Contributed by Bibin A Chundatt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8c947398
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8c947398
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8c947398

Branch: refs/heads/trunk
Commit: 8c947398ab2de642cb49a59e5e9508d196d4bc30
Parents: eb08543
Author: Naganarasimha <na...@apache.org>
Authored: Sun Jul 29 13:52:52 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:01 2018 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/nodelabels/NodeLabelUtil.java   |  4 +-
 .../yarn/nodelabels/TestNodeLabelUtil.java      | 47 ++++++++++++++++++++
 2 files changed, 50 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c947398/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
index 0dd0755..395ff81 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
@@ -38,6 +38,8 @@ public final class NodeLabelUtil {
       Pattern.compile("^[0-9a-zA-Z][0-9a-zA-Z-_]*");
   private static final Pattern PREFIX_PATTERN =
       Pattern.compile("^[0-9a-zA-Z][0-9a-zA-Z-_\\.]*");
+  private static final Pattern ATTRIBUTE_VALUE_PATTERN =
+      Pattern.compile("^[0-9a-zA-Z][0-9a-zA-Z-_.]*");
 
   public static void checkAndThrowLabelName(String label) throws IOException {
     if (label == null || label.isEmpty() || label.length() > MAX_LABEL_LENGTH) {
@@ -69,7 +71,7 @@ public final class NodeLabelUtil {
       return;
     }
 
-    boolean match = LABEL_OR_VALUE_PATTERN.matcher(value).matches();
+    boolean match = ATTRIBUTE_VALUE_PATTERN.matcher(value).matches();
 
     if (!match) {
       throw new IOException("attribute value should only contains "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c947398/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestNodeLabelUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestNodeLabelUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestNodeLabelUtil.java
new file mode 100644
index 0000000..d43da4f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestNodeLabelUtil.java
@@ -0,0 +1,47 @@
+/**
+ * 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.nodelabels;
+
+import static org.junit.Assert.fail;
+import org.junit.Test;
+
+public class TestNodeLabelUtil {
+
+  @Test
+  public void testAttributeValueAddition() {
+    String[] values =
+        new String[] {"1_8", "1.8", "ABZ", "ABZ", "az", "a-z","a_z", "123456789"};
+    for (String val : values) {
+      try {
+        NodeLabelUtil.checkAndThrowAttributeValue(val);
+      } catch (Exception e) {
+        fail("Valid values for NodeAttributeValue :" + val);
+      }
+    }
+
+    String[] invalidVals = new String[] {"_18","1,8","1/5",".15","1\\5"};
+    for (String val : invalidVals) {
+      try {
+        NodeLabelUtil.checkAndThrowAttributeValue(val);
+        fail("Valid values for NodeAttributeValue :" + val);
+      } catch (Exception e) {
+        // IGNORE
+      }
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[16/29] hadoop git commit: YARN-8100. Support API interface to query cluster attributes and attribute to nodes. Contributed by Bibin A Chundatt.

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java
index 5ce4803..f6adb43 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java
@@ -39,8 +39,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
@@ -327,6 +331,19 @@ public class DefaultClientRequestInterceptor
     return clientRMProxy.getResourceTypeInfo(request);
   }
 
+  @Override
+  public GetAttributesToNodesResponse getAttributesToNodes(
+      GetAttributesToNodesRequest request) throws YarnException, IOException {
+    return clientRMProxy.getAttributesToNodes(request);
+  }
+
+  @Override
+  public GetClusterNodeAttributesResponse getClusterNodeAttributes(
+      GetClusterNodeAttributesRequest request)
+      throws YarnException, IOException {
+    return clientRMProxy.getClusterNodeAttributes(request);
+  }
+
   @VisibleForTesting
   public void setRMClient(ApplicationClientProtocol clientRM) {
     this.clientRMProxy = clientRM;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
index 8b48c8c..bf006a4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
@@ -55,8 +55,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
@@ -826,4 +830,17 @@ public class FederationClientInterceptor
     executorService.shutdown();
     super.shutdown();
   }
+
+  @Override
+  public GetAttributesToNodesResponse getAttributesToNodes(
+      GetAttributesToNodesRequest request) throws YarnException, IOException {
+    throw new NotImplementedException("Code is not implemented");
+  }
+
+  @Override
+  public GetClusterNodeAttributesResponse getClusterNodeAttributes(
+      GetClusterNodeAttributesRequest request)
+      throws YarnException, IOException {
+    throw new NotImplementedException("Code is not implemented");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java
index bbb8047..3237dd4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java
@@ -50,8 +50,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
@@ -430,6 +434,21 @@ public class RouterClientRMService extends AbstractService
     return pipeline.getRootInterceptor().getResourceTypeInfo(request);
   }
 
+  @Override
+  public GetAttributesToNodesResponse getAttributesToNodes(
+      GetAttributesToNodesRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAttributesToNodes(request);
+  }
+
+  @Override
+  public GetClusterNodeAttributesResponse getClusterNodeAttributes(
+      GetClusterNodeAttributesRequest request)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getClusterNodeAttributes(request);
+  }
+
   @VisibleForTesting
   protected RequestInterceptorChainWrapper getInterceptorChain()
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a01b135/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java
index cb1b529..96da4c4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java
@@ -36,8 +36,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
@@ -288,4 +292,17 @@ public class PassThroughClientRequestInterceptor
       GetAllResourceTypeInfoRequest request) throws YarnException, IOException {
     return getNextInterceptor().getResourceTypeInfo(request);
   }
+
+  @Override
+  public GetAttributesToNodesResponse getAttributesToNodes(
+      GetAttributesToNodesRequest request) throws YarnException, IOException {
+    return getNextInterceptor().getAttributesToNodes(request);
+  }
+
+  @Override
+  public GetClusterNodeAttributesResponse getClusterNodeAttributes(
+      GetClusterNodeAttributesRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().getClusterNodeAttributes(request);
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[12/29] hadoop git commit: YARN-7965. NodeAttributeManager add/get API is not working properly. Contributed by Weiwei Yang.

Posted by su...@apache.org.
YARN-7965. NodeAttributeManager add/get API is not working properly. Contributed by Weiwei Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/86d024ef
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/86d024ef
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/86d024ef

Branch: refs/heads/trunk
Commit: 86d024ef2a5844ac723c7b597689afa4b598ee27
Parents: ffcabd2
Author: Naganarasimha <na...@apache.org>
Authored: Tue Feb 27 18:46:16 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../yarn/nodelabels/NodeAttributesManager.java  |   8 +-
 .../nodelabels/NodeAttributesManagerImpl.java   |  38 ++-
 .../nodelabels/TestNodeAttributesManager.java   | 258 +++++++++++++++++++
 3 files changed, 288 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/86d024ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
index 63f3dcf..effda9b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
@@ -69,10 +69,14 @@ public abstract class NodeAttributesManager extends AbstractService {
       Map<String, Set<NodeAttribute>> nodeAttributeMapping) throws IOException;
 
   /**
+   * Returns a set of node attributes whose prefix is one of the given
+   * prefixes; if the prefix set is null or empty, all attributes are returned;
+   * if prefix set is given but no mapping could be found, an empty set
+   * is returned.
+   *
    * @param prefix set of prefix string's for which the attributes needs to
    *          returned
-   * @return set of node Attributes objects for the specified set of prefixes,
-   *         else return all
+   * @return set of node Attributes
    */
   public abstract Set<NodeAttribute> getClusterNodeAttributes(
       Set<String> prefix);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86d024ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
index 2e63a7c..a902ac6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
@@ -27,6 +27,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentHashMap.KeySetView;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
@@ -141,6 +142,7 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
         Host node = nodeCollections.get(nodeHost);
         if (node == null) {
           node = new Host(nodeHost);
+          nodeCollections.put(nodeHost, node);
         }
         switch (op) {
         case REMOVE:
@@ -181,8 +183,16 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
 
   private void removeNodeFromAttributes(String nodeHost,
       Set<NodeAttribute> attributeMappings) {
-    for (NodeAttribute attribute : attributeMappings) {
-      clusterAttributes.get(attribute).removeNode(nodeHost);
+    for (NodeAttribute rmAttribute : attributeMappings) {
+      RMNodeAttribute host = clusterAttributes.get(rmAttribute);
+      if (host != null) {
+        host.removeNode(nodeHost);
+        // If there is no other host has such attribute,
+        // remove it from the global mapping.
+        if (host.getAssociatedNodeIds().isEmpty()) {
+          clusterAttributes.remove(rmAttribute);
+        }
+      }
     }
   }
 
@@ -305,19 +315,19 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   @Override
   public Set<NodeAttribute> getClusterNodeAttributes(Set<String> prefix) {
     Set<NodeAttribute> attributes = new HashSet<>();
-    try {
-      readLock.lock();
-      attributes.addAll(clusterAttributes.keySet());
-    } finally {
-      readLock.unlock();
+    KeySetView<NodeAttribute, RMNodeAttribute> allAttributes =
+        clusterAttributes.keySet();
+    // Return all if prefix is not given.
+    if (prefix == null || prefix.isEmpty()) {
+      attributes.addAll(allAttributes);
+      return attributes;
     }
-    if (prefix != null && prefix.isEmpty()) {
-      Iterator<NodeAttribute> iterator = attributes.iterator();
-      while (iterator.hasNext()) {
-        NodeAttribute attribute = iterator.next();
-        if (!prefix.contains(attribute.getAttributePrefix())) {
-          iterator.remove();
-        }
+    // Try search attributes by prefix and return valid ones.
+    Iterator<NodeAttribute> iterator = allAttributes.iterator();
+    while (iterator.hasNext()) {
+      NodeAttribute current = iterator.next();
+      if (prefix.contains(current.getAttributePrefix())) {
+        attributes.add(current);
       }
     }
     return attributes;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86d024ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
new file mode 100644
index 0000000..b639a74
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
@@ -0,0 +1,258 @@
+/**
+ * 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.nodelabels;
+
+import com.google.common.collect.Sets;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.nodelabels.AttributeValue;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
+import org.junit.Test;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Assert;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Unit tests for node attribute manager.
+ */
+public class TestNodeAttributesManager {
+
+  private NodeAttributesManager attributesManager;
+  private final static String[] PREFIXES =
+      new String[] {"yarn.test1.io", "yarn.test2.io", "yarn.test3.io"};
+  private final static String[] HOSTNAMES =
+      new String[] {"host1", "host2", "host3"};
+
+  @Before
+  public void init() {
+    Configuration conf = new Configuration();
+    attributesManager = new NodeAttributesManagerImpl();
+    attributesManager.init(conf);
+    attributesManager.start();
+  }
+
+  @After
+  public void cleanUp() {
+    if (attributesManager != null) {
+      attributesManager.stop();
+    }
+  }
+
+  private Set<NodeAttribute> createAttributesForTest(String attributePrefix,
+      int numOfAttributes, String attributeNamePrefix,
+      String attributeValuePrefix) {
+    Set<NodeAttribute> attributes = new HashSet<>();
+    for (int i = 0; i< numOfAttributes; i++) {
+      NodeAttribute attribute = NodeAttribute.newInstance(
+          attributePrefix, attributeNamePrefix + "_" + i,
+          NodeAttributeType.STRING, attributeValuePrefix + "_" + i);
+      attributes.add(attribute);
+    }
+    return attributes;
+  }
+
+  private boolean sameAttributeSet(Set<NodeAttribute> set1,
+      Set<NodeAttribute> set2) {
+    return Sets.difference(set1, set2).isEmpty();
+  }
+
+  @Test
+  public void testAddNodeAttributes() throws IOException {
+    Map<String, Set<NodeAttribute>> toAddAttributes = new HashMap<>();
+    Map<NodeAttribute, AttributeValue> nodeAttributes;
+
+    // Add 3 attributes to host1
+    //  yarn.test1.io/A1=host1_v1_1
+    //  yarn.test1.io/A2=host1_v1_2
+    //  yarn.test1.io/A3=host1_v1_3
+    toAddAttributes.put(HOSTNAMES[0],
+        createAttributesForTest(PREFIXES[0], 3, "A", "host1_v1"));
+
+    attributesManager.addNodeAttributes(toAddAttributes);
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]);
+
+    Assert.assertEquals(3, nodeAttributes.size());
+    Assert.assertTrue(sameAttributeSet(toAddAttributes.get(HOSTNAMES[0]),
+        nodeAttributes.keySet()));
+
+    // Add 2 attributes to host2
+    //  yarn.test1.io/A1=host2_v1_1
+    //  yarn.test1.io/A2=host2_v1_2
+    toAddAttributes.clear();
+    toAddAttributes.put(HOSTNAMES[1],
+        createAttributesForTest(PREFIXES[0], 2, "A", "host2_v1"));
+    attributesManager.addNodeAttributes(toAddAttributes);
+
+    // Verify host1 attributes are still valid.
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]);
+    Assert.assertEquals(3, nodeAttributes.size());
+
+    // Verify new added host2 attributes are correctly updated.
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[1]);
+    Assert.assertEquals(2, nodeAttributes.size());
+    Assert.assertTrue(sameAttributeSet(toAddAttributes.get(HOSTNAMES[1]),
+        nodeAttributes.keySet()));
+
+    // Cluster wide, it only has 3 attributes.
+    //  yarn.test1.io/A1
+    //  yarn.test1.io/A2
+    //  yarn.test1.io/A3
+    Set<NodeAttribute> clusterAttributes = attributesManager
+        .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[0]));
+    Assert.assertEquals(3, clusterAttributes.size());
+
+    // Query for attributes under a non-exist prefix,
+    // ensure it returns an empty set.
+    clusterAttributes = attributesManager
+        .getClusterNodeAttributes(Sets.newHashSet("non_exist_prefix"));
+    Assert.assertEquals(0, clusterAttributes.size());
+
+    // Not provide any prefix, ensure it returns all attributes.
+    clusterAttributes = attributesManager.getClusterNodeAttributes(null);
+    Assert.assertEquals(3, clusterAttributes.size());
+
+    // Add some other attributes with different prefixes on host1 and host2.
+    toAddAttributes.clear();
+
+    // Host1
+    //  yarn.test2.io/A_1=host1_v2_1
+    //  ...
+    //  yarn.test2.io/A_10=host1_v2_10
+    toAddAttributes.put(HOSTNAMES[0],
+        createAttributesForTest(PREFIXES[1], 10, "C", "host1_v2"));
+    // Host2
+    //  yarn.test2.io/C_1=host1_v2_1
+    //  ...
+    //  yarn.test2.io/C_20=host1_v2_20
+    toAddAttributes.put(HOSTNAMES[1],
+        createAttributesForTest(PREFIXES[1], 20, "C", "host1_v2"));
+    attributesManager.addNodeAttributes(toAddAttributes);
+
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]);
+    Assert.assertEquals(13, nodeAttributes.size());
+
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[1]);
+    Assert.assertEquals(22, nodeAttributes.size());
+  }
+
+  @Test
+  public void testRemoveNodeAttributes() throws IOException {
+    Map<String, Set<NodeAttribute>> toAddAttributes = new HashMap<>();
+    Map<String, Set<NodeAttribute>> toRemoveAttributes = new HashMap<>();
+    Set<NodeAttribute> allAttributesPerPrefix = new HashSet<>();
+    Map<NodeAttribute, AttributeValue> nodeAttributes;
+
+    // Host1 -----------------------
+    //  yarn.test1.io
+    //    A1=host1_v1_1
+    //    A2=host1_v1_2
+    //    A3=host1_v1_3
+    //  yarn.test2.io
+    //    B1=host1_v2_1
+    //    ...
+    //    B5=host5_v2_5
+    // Host2 -----------------------
+    //  yarn.test1.io
+    //    A1=host2_v1_1
+    //    A2=host2_v1_2
+    //  yarn.test3.io
+    //    C1=host2_v3_1
+    //    c2=host2_v3_2
+    Set<NodeAttribute> host1set = new HashSet<>();
+    Set<NodeAttribute> host1set1 =
+        createAttributesForTest(PREFIXES[0], 3, "A", "host1_v1");
+    Set<NodeAttribute> host1set2 =
+        createAttributesForTest(PREFIXES[1], 5, "B", "host1_v1");
+    host1set.addAll(host1set1);
+    host1set.addAll(host1set2);
+
+    Set<NodeAttribute> host2set = new HashSet<>();
+    Set<NodeAttribute> host2set1 =
+        createAttributesForTest(PREFIXES[0], 2, "A", "host2_v1");
+    Set<NodeAttribute> host2set2 =
+        createAttributesForTest(PREFIXES[2], 2, "C", "host2_v3");
+    host2set.addAll(host2set1);
+    host2set.addAll(host2set2);
+
+    toAddAttributes.put(HOSTNAMES[0], host1set);
+    toAddAttributes.put(HOSTNAMES[1], host2set);
+    attributesManager.addNodeAttributes(toAddAttributes);
+
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]);
+    Assert.assertEquals(8, nodeAttributes.size());
+
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[1]);
+    Assert.assertEquals(4, nodeAttributes.size());
+
+    allAttributesPerPrefix = attributesManager
+        .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[0]));
+    Assert.assertEquals(3, allAttributesPerPrefix.size());
+    allAttributesPerPrefix = attributesManager
+        .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[1]));
+    Assert.assertEquals(5, allAttributesPerPrefix.size());
+    allAttributesPerPrefix = attributesManager
+        .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[2]));
+    Assert.assertEquals(2, allAttributesPerPrefix.size());
+
+    // Remove "yarn.test1.io/A_2" from host1
+    Set<NodeAttribute> attributes2rm1 = new HashSet<>();
+    attributes2rm1.add(NodeAttribute.newInstance(PREFIXES[0], "A_2",
+        NodeAttributeType.STRING, "anyValue"));
+    toRemoveAttributes.put(HOSTNAMES[0], attributes2rm1);
+    attributesManager.removeNodeAttributes(toRemoveAttributes);
+
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]);
+    Assert.assertEquals(7, nodeAttributes.size());
+
+    // Remove again, but give a non-exist attribute name
+    attributes2rm1.clear();
+    toRemoveAttributes.clear();
+    attributes2rm1.add(NodeAttribute.newInstance(PREFIXES[0], "non_exist_name",
+        NodeAttributeType.STRING, "anyValue"));
+    toRemoveAttributes.put(HOSTNAMES[0], attributes2rm1);
+    attributesManager.removeNodeAttributes(toRemoveAttributes);
+
+    nodeAttributes = attributesManager.getAttributesForNode(HOSTNAMES[0]);
+    Assert.assertEquals(7, nodeAttributes.size());
+
+    // Remove "yarn.test1.io/A_2" from host2 too,
+    // by then there will be no such attribute exist in the cluster.
+    Set<NodeAttribute> attributes2rm2 = new HashSet<>();
+    attributes2rm2.add(NodeAttribute.newInstance(PREFIXES[0], "A_2",
+        NodeAttributeType.STRING, "anyValue"));
+    toRemoveAttributes.clear();
+    toRemoveAttributes.put(HOSTNAMES[1], attributes2rm2);
+    attributesManager.removeNodeAttributes(toRemoveAttributes);
+
+    // Make sure cluster wide attributes are still consistent.
+    // Since both host1 and host2 doesn't have "yarn.test1.io/A_2",
+    // get all attributes under prefix "yarn.test1.io" should only return
+    // us A_1 and A_3.
+    allAttributesPerPrefix = attributesManager
+        .getClusterNodeAttributes(Sets.newHashSet(PREFIXES[0]));
+    Assert.assertEquals(2, allAttributesPerPrefix.size());
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[27/29] hadoop git commit: YARN-8103. Add CLI interface to query node attributes. Contributed by Bibin A Chundatt.

Posted by su...@apache.org.
YARN-8103. Add CLI interface to query node attributes. Contributed by Bibin A Chundatt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/eb08543c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/eb08543c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/eb08543c

Branch: refs/heads/trunk
Commit: eb08543c7a5d1fd97a1915dbc5a11a2ba2066ba1
Parents: 7618342
Author: Naganarasimha <na...@apache.org>
Authored: Thu Jun 28 08:13:09 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:01 2018 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/sls/nodemanager/NodeInfo.java   |  12 +-
 .../yarn/sls/scheduler/RMNodeWrapper.java       |   8 +-
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |   4 +-
 .../hadoop/yarn/api/records/NodeReport.java     |  13 +
 .../src/main/proto/yarn_protos.proto            |   1 +
 .../hadoop/yarn/client/cli/ClusterCLI.java      |  17 +
 .../yarn/client/cli/NodeAttributesCLI.java      | 893 +++++++++++++------
 .../apache/hadoop/yarn/client/cli/NodeCLI.java  |  13 +-
 .../hadoop/yarn/client/cli/TestClusterCLI.java  |  32 +-
 .../yarn/client/cli/TestNodeAttributesCLI.java  | 331 +++++--
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |  31 +-
 .../impl/pb/NodeAttributeInfoPBImpl.java        |  10 +-
 .../records/impl/pb/NodeAttributePBImpl.java    |  12 +-
 .../api/records/impl/pb/NodeReportPBImpl.java   |  44 +-
 .../hadoop/yarn/server/utils/BuilderUtils.java  |   6 +-
 .../server/resourcemanager/AdminService.java    |   5 +-
 .../server/resourcemanager/ClientRMService.java |   5 +-
 .../resourcemanager/ResourceTrackerService.java |   4 -
 .../server/resourcemanager/rmnode/RMNode.java   |  11 +-
 .../resourcemanager/rmnode/RMNodeImpl.java      |  18 +-
 .../resourcemanager/webapp/dao/NodeInfo.java    |  13 +-
 .../yarn/server/resourcemanager/MockNodes.java  |   4 +-
 .../resourcemanager/TestRMAdminService.java     |  14 +-
 23 files changed, 1049 insertions(+), 452 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
index 65b8da0..2eee351 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.sls.nodemanager;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -220,16 +221,9 @@ public class NodeInfo {
       return null;
     }
 
-
-    @Override
-    public void setNodeAttributes(String prefix,
-        Set<NodeAttribute> nodeAttributes) {
-
-    }
-
     @Override
-    public Map<String, Set<NodeAttribute>> getAllNodeAttributes() {
-      return null;
+    public Set<NodeAttribute> getAllNodeAttributes() {
+      return Collections.emptySet();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
index bf61f54..248b634 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
@@ -209,13 +209,7 @@ public class RMNodeWrapper implements RMNode {
   }
   
   @Override
-  public void setNodeAttributes(String prefix,
-      Set<NodeAttribute> nodeAttributes) {
-    node.setNodeAttributes(prefix, nodeAttributes);
-  }
-
-  @Override
-  public Map<String, Set<NodeAttribute>> getAllNodeAttributes() {
+  public Set<NodeAttribute> getAllNodeAttributes() {
     return node.getAllNodeAttributes();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/bin/yarn
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn
index 7cd838f..8290fcd 100755
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn
@@ -55,7 +55,7 @@ function hadoop_usage
   hadoop_add_subcommand "timelinereader" client "run the timeline reader server"
   hadoop_add_subcommand "timelineserver" daemon "run the timeline server"
   hadoop_add_subcommand "top" client "view cluster information"
-  hadoop_add_subcommand "node-attributes" "map node to attibutes"
+  hadoop_add_subcommand "nodeattributes" client "node attributes cli client"
   hadoop_add_subcommand "version" client "print the version"
   hadoop_generate_usage "${HADOOP_SHELL_EXECNAME}" true
 }
@@ -187,7 +187,7 @@ ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}"
       hadoop_add_classpath "$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/lib/*"
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderServer'
     ;;
-	node-attributes)
+	nodeattributes)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="false"
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.client.cli.NodeAttributesCLI'
 	;;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
index 3a80641..625ad23 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java
@@ -258,4 +258,17 @@ public abstract class NodeReport {
    * Set the node update type (null indicates absent node update type).
    * */
   public void setNodeUpdateType(NodeUpdateType nodeUpdateType) {}
+
+  /**
+   * Set the node attributes of node.
+   *
+   * @param nodeAttributes set of node attributes.
+   */
+  public abstract void setNodeAttributes(Set<NodeAttribute> nodeAttributes);
+
+  /**
+   * Get node attributes of node.
+   * @return the set of node attributes.
+   */
+  public abstract Set<NodeAttribute> getNodeAttributes();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index aca9471..10b36c7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -355,6 +355,7 @@ message NodeReportProto {
   optional ResourceUtilizationProto node_utilization = 12;
   optional uint32 decommissioning_timeout = 13;
   optional NodeUpdateTypeProto node_update_type = 14;
+  repeated NodeAttributeProto node_attributes = 15;
 }
 
 message NodeIdToLabelsProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ClusterCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ClusterCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ClusterCLI.java
index a29b0db..4d93949 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ClusterCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ClusterCLI.java
@@ -36,6 +36,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.api.records.NodeAttributeInfo;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -52,6 +53,7 @@ public class ClusterCLI extends YarnCLI {
   public static final String LIST_LABELS_CMD = "list-node-labels";
   public static final String DIRECTLY_ACCESS_NODE_LABEL_STORE =
       "directly-access-node-label-store";
+  public static final String LIST_CLUSTER_ATTRIBUTES="list-node-attributes";
   public static final String CMD = "cluster";
   private boolean accessLocal = false;
   static CommonNodeLabelsManager localNodeLabelsManager = null;
@@ -71,6 +73,8 @@ public class ClusterCLI extends YarnCLI {
 
     opts.addOption("lnl", LIST_LABELS_CMD, false,
         "List cluster node-label collection");
+    opts.addOption("lna", LIST_CLUSTER_ATTRIBUTES, false,
+        "List cluster node-attribute collection");
     opts.addOption("h", HELP_CMD, false, "Displays help for all commands.");
     opts.addOption("dnl", DIRECTLY_ACCESS_NODE_LABEL_STORE, false,
         "This is DEPRECATED, will be removed in future releases. Directly access node label store, "
@@ -102,6 +106,8 @@ public class ClusterCLI extends YarnCLI {
 
     if (parsedCli.hasOption(LIST_LABELS_CMD)) {
       printClusterNodeLabels();
+    } else if(parsedCli.hasOption(LIST_CLUSTER_ATTRIBUTES)){
+      printClusterNodeAttributes();
     } else if (parsedCli.hasOption(HELP_CMD)) {
       printUsage(opts);
       return 0;
@@ -112,6 +118,17 @@ public class ClusterCLI extends YarnCLI {
     return 0;
   }
 
+  private void printClusterNodeAttributes() throws IOException, YarnException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter pw = new PrintWriter(
+        new OutputStreamWriter(baos, Charset.forName("UTF-8")));
+    for (NodeAttributeInfo attribute : client.getClusterAttributes()) {
+      pw.println(attribute.toString());
+    }
+    pw.close();
+    sysout.println(baos.toString("UTF-8"));
+  }
+
   void printClusterNodeLabels() throws YarnException, IOException {
     List<NodeLabel> nodeLabels = null;
     if (accessLocal) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java
index df5a57d..13d5e24 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java
@@ -18,29 +18,30 @@
 
 package org.apache.hadoop.yarn.client.cli;
 
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.MissingArgumentException;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.UnrecognizedOptionException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.ha.HAAdmin.UsageInfo;
-import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeInfo;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
 import org.apache.hadoop.yarn.api.records.NodeAttributeType;
 import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -50,13 +51,24 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperati
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * CLI to map attributes to Nodes.
- *
  */
 public class NodeAttributesCLI extends Configured implements Tool {
 
@@ -64,351 +76,640 @@ public class NodeAttributesCLI extends Configured implements Tool {
       "Invalid Node to attribute mapping : ";
 
   protected static final String USAGE_YARN_NODE_ATTRIBUTES =
-      "Usage: yarn node-attributes ";
+      "Usage: yarn nodeattributes ";
+
+  protected static final String MISSING_ARGUMENT =
+      "Missing argument for command";
 
   protected static final String NO_MAPPING_ERR_MSG =
       "No node-to-attributes mappings are specified";
 
-  protected final static Map<String, UsageInfo> NODE_ATTRIB_USAGE =
-      ImmutableMap.<String, UsageInfo>builder()
-          .put("-replace",
-              new UsageInfo(
-                  "<\"node1:attribute[(type)][=value],attribute1[=value],"
-                      + "attribute2  node2:attribute2[=value],attribute3\">",
-                  " Replace the node to attributes mapping information at the"
-                      + " ResourceManager with the new mapping. Currently"
-                      + " supported attribute type. And string is the default"
-                      + " type too. Attribute value if not specified for string"
-                      + " type value will be considered as empty string."
-                      + " Replaced node-attributes should not violate the"
-                      + " existing attribute to attribute type mapping."))
-          .put("-add",
-              new UsageInfo(
-                  "<\"node1:attribute[(type)][=value],attribute1[=value],"
-                      + "attribute2  node2:attribute2[=value],attribute3\">",
-                  " Adds or updates the node to attributes mapping information"
-                      + " at the ResourceManager. Currently supported attribute"
-                      + " type is string. And string is the default type too."
-                      + " Attribute value if not specified for string type"
-                      + " value will be considered as empty string. Added or"
-                      + " updated node-attributes should not violate the"
-                      + " existing attribute to attribute type mapping."))
-          .put("-remove",
-              new UsageInfo("<\"node1:attribute,attribute1 node2:attribute2\">",
-                  " Removes the specified node to attributes mapping"
-                      + " information at the ResourceManager"))
-          .put("-failOnUnknownNodes",
-              new UsageInfo("",
-                  "Can be used optionally along with other options. When its"
-                      + " set, it will fail if specified nodes are unknown."))
-          .build();
-
-  /** Output stream for errors, for use in tests. */
+  private static final String DEFAULT_SEPARATOR = System.lineSeparator();
+  public static final String INVALID_COMMAND_USAGE = "Invalid Command Usage : ";
+  /**
+   * Output stream for errors, for use in tests.
+   */
   private PrintStream errOut = System.err;
 
   public NodeAttributesCLI() {
     super();
   }
 
-  public NodeAttributesCLI(Configuration conf) {
-    super(conf);
-  }
-
   protected void setErrOut(PrintStream errOut) {
     this.errOut = errOut;
   }
 
-  private void printHelpMsg(String cmd) {
-    StringBuilder builder = new StringBuilder();
-    UsageInfo usageInfo = null;
-    if (cmd != null && !(cmd.trim().isEmpty())) {
-      usageInfo = NODE_ATTRIB_USAGE.get(cmd);
-    }
-    if (usageInfo != null) {
-      if (usageInfo.args == null) {
-        builder.append("   " + cmd + ":\n" + usageInfo.help);
-      } else {
-        String space = (usageInfo.args == "") ? "" : " ";
-        builder.append(
-            "   " + cmd + space + usageInfo.args + " :\n" + usageInfo.help);
-      }
-    } else {
-      // help for all commands
-      builder.append("Usage: yarn node-attributes\n");
-      for (Map.Entry<String, UsageInfo> cmdEntry : NODE_ATTRIB_USAGE
-          .entrySet()) {
-        usageInfo = cmdEntry.getValue();
-        builder.append("   " + cmdEntry.getKey() + " " + usageInfo.args
-            + " :\n " + usageInfo.help + "\n");
-      }
-      builder.append("   -help" + " [cmd]\n");
-    }
-    errOut.println(builder);
+  protected AdminCommandHandler getAdminCommandHandler() {
+    return new AdminCommandHandler();
   }
 
-  private static void buildIndividualUsageMsg(String cmd,
-      StringBuilder builder) {
-    UsageInfo usageInfo = NODE_ATTRIB_USAGE.get(cmd);
-    if (usageInfo == null) {
-      return;
-    }
-    if (usageInfo.args == null) {
-      builder.append(USAGE_YARN_NODE_ATTRIBUTES + cmd + "\n");
-    } else {
-      String space = (usageInfo.args == "") ? "" : " ";
-      builder.append(
-          USAGE_YARN_NODE_ATTRIBUTES + cmd + space + usageInfo.args + "\n");
-    }
+  protected ClientCommandHandler getClientCommandHandler() {
+    return new ClientCommandHandler();
   }
 
-  private static void buildUsageMsgForAllCmds(StringBuilder builder) {
-    builder.append("Usage: yarn node-attributes\n");
-    for (Map.Entry<String, UsageInfo> cmdEntry : NODE_ATTRIB_USAGE.entrySet()) {
-      UsageInfo usageInfo = cmdEntry.getValue();
-      builder.append("   " + cmdEntry.getKey() + " " + usageInfo.args + "\n");
+  void printUsage(String cmd, boolean desc, CommandHandler... handlers)
+      throws UnsupportedEncodingException {
+    StringBuilder usageBuilder = new StringBuilder();
+    usageBuilder.append(USAGE_YARN_NODE_ATTRIBUTES);
+    boolean satisfied = false;
+    for (CommandHandler cmdHandlers : handlers) {
+      satisfied |= cmdHandlers.getHelp(cmd, usageBuilder, desc);
+    }
+    if (!satisfied) {
+      printUsage(desc, handlers);
+    } else {
+      print(usageBuilder);
     }
-    builder.append("   -help" + " [cmd]\n");
   }
 
-  /**
-   * Displays format of commands.
-   *
-   * @param cmd The command that is being executed.
-   */
-  private void printUsage(String cmd) {
+  private void printUsage(boolean desc, CommandHandler... handlers)
+      throws UnsupportedEncodingException {
     StringBuilder usageBuilder = new StringBuilder();
-    if (NODE_ATTRIB_USAGE.containsKey(cmd)) {
-      buildIndividualUsageMsg(cmd, usageBuilder);
-    } else {
-      buildUsageMsgForAllCmds(usageBuilder);
+    usageBuilder.append(USAGE_YARN_NODE_ATTRIBUTES);
+    for (CommandHandler cmdHandlers : handlers) {
+      cmdHandlers.getHelp(usageBuilder, desc);
     }
-    errOut.println(usageBuilder);
-  }
 
-  private void printUsage() {
-    printUsage("");
+    // append help with usage
+    usageBuilder.append(DEFAULT_SEPARATOR)
+        .append(" -help [cmd] List help of commands");
+    print(usageBuilder);
   }
 
-  protected ResourceManagerAdministrationProtocol createAdminProtocol()
-      throws IOException {
-    // Get the current configuration
-    final YarnConfiguration conf = new YarnConfiguration(getConf());
-    return ClientRMProxy.createRMProxy(conf,
-        ResourceManagerAdministrationProtocol.class);
+  private void print(StringBuilder usageBuilder)
+      throws UnsupportedEncodingException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter pw =
+        new PrintWriter(new OutputStreamWriter(baos, Charset.forName("UTF-8")));
+    pw.write(usageBuilder.toString());
+    pw.close();
+    errOut.println(baos.toString("UTF-8"));
   }
 
-  @Override
-  public void setConf(Configuration conf) {
-    if (conf != null) {
-      conf = addSecurityConfiguration(conf);
+  private Options buildOptions(CommandHandler... handlers) {
+    Options opts = new Options();
+    for (CommandHandler handler : handlers) {
+      Options handlerOpts = handler.getOptions();
+      handlerOpts.getOptions().iterator()
+          .forEachRemaining(option -> opts.addOption((Option) option));
     }
-    super.setConf(conf);
-  }
-
-  /**
-   * Add the requisite security principal settings to the given Configuration,
-   * returning a copy.
-   *
-   * @param conf the original config
-   * @return a copy with the security settings added
-   */
-  private static Configuration addSecurityConfiguration(Configuration conf) {
-    // Make a copy so we don't mutate it. Also use an YarnConfiguration to
-    // force loading of yarn-site.xml.
-    conf = new YarnConfiguration(conf);
-    conf.set(CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY,
-        conf.get(YarnConfiguration.RM_PRINCIPAL, ""));
-    return conf;
+    return opts;
   }
 
-  @Override
   public int run(String[] args) throws Exception {
+
+    int exitCode = -1;
+
+    AdminCommandHandler adminCmdHandler = getAdminCommandHandler();
+    ClientCommandHandler clientCmdHandler = getClientCommandHandler();
+
+    // Build options
+    Options opts = buildOptions(adminCmdHandler, clientCmdHandler);
+
     if (args.length < 1) {
-      printUsage();
+      printUsage(false, adminCmdHandler, clientCmdHandler);
       return -1;
     }
 
-    int exitCode = -1;
-    int i = 0;
-    String cmd = args[i++];
+    // Handle command separate
+    if (handleHelpCommand(args, adminCmdHandler, clientCmdHandler)) {
+      return 0;
+    }
 
-    if ("-help".equals(cmd)) {
-      exitCode = 0;
-      if (args.length >= 2) {
-        printHelpMsg(args[i]);
+    CommandLine cliParser;
+    CommandHandler handler = null;
+    try {
+      cliParser = new GnuParser().parse(opts, args);
+      handler = adminCmdHandler.canHandleCommand(cliParser) ?
+          adminCmdHandler :
+          clientCmdHandler.canHandleCommand(cliParser) ?
+              clientCmdHandler :
+              null;
+      if (handler == null) {
+        errOut.println(INVALID_COMMAND_USAGE);
+        printUsage(false, adminCmdHandler, clientCmdHandler);
+        return exitCode;
       } else {
-        printHelpMsg("");
+        return handler.handleCommand(cliParser);
       }
+    } catch (UnrecognizedOptionException e) {
+      errOut.println(INVALID_COMMAND_USAGE);
+      printUsage(false, adminCmdHandler, clientCmdHandler);
+      return exitCode;
+    } catch (MissingArgumentException ex) {
+      errOut.println(MISSING_ARGUMENT);
+      printUsage(true, adminCmdHandler, clientCmdHandler);
+      return exitCode;
+    } catch (IllegalArgumentException arge) {
+      errOut.println(arge.getLocalizedMessage());
+      // print admin command detail
+      printUsage(true, handler);
+      return exitCode;
+    } catch (Exception e) {
+      errOut.println(e.toString());
+      printUsage(true, handler);
       return exitCode;
     }
+  }
 
-    try {
-      if ("-replace".equals(cmd)) {
-        exitCode = handleNodeAttributeMapping(args,
-            AttributeMappingOperationType.REPLACE);
-      } else if ("-add".equals(cmd)) {
-        exitCode =
-            handleNodeAttributeMapping(args, AttributeMappingOperationType.ADD);
-      } else if ("-remove".equals(cmd)) {
-        exitCode = handleNodeAttributeMapping(args,
-            AttributeMappingOperationType.REMOVE);
+  private boolean handleHelpCommand(String[] args, CommandHandler... handlers)
+      throws UnsupportedEncodingException {
+    if (args[0].equals("-help")) {
+      if (args.length == 2) {
+        printUsage(args[1], true, handlers);
       } else {
-        exitCode = -1;
-        errOut.println(cmd.substring(1) + ": Unknown command");
-        printUsage();
+        printUsage(true, handlers);
       }
-    } catch (IllegalArgumentException arge) {
-      exitCode = -1;
-      errOut.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
-      printUsage(cmd);
-    } catch (RemoteException e) {
-      //
-      // This is a error returned by hadoop server. Print
-      // out the first line of the error message, ignore the stack trace.
-      exitCode = -1;
-      try {
-        String[] content;
-        content = e.getLocalizedMessage().split("\n");
-        errOut.println(cmd.substring(1) + ": " + content[0]);
-      } catch (Exception ex) {
-        errOut.println(cmd.substring(1) + ": " + ex.getLocalizedMessage());
+      return true;
+    }
+    return false;
+  }
+
+  public static void main(String[] args) throws Exception {
+    int result = ToolRunner.run(new NodeAttributesCLI(), args);
+    System.exit(result);
+  }
+
+  /**
+   * Abstract class for command handler.
+   */
+  public static abstract class CommandHandler extends Configured {
+
+    private Options options;
+
+    private LinkedList<String> order = new LinkedList<>();
+    private String header;
+
+    protected CommandHandler(String header) {
+      this(new YarnConfiguration());
+      this.header = header;
+    }
+
+    protected CommandHandler(Configuration conf) {
+      super(conf);
+      options = buildOptions();
+    }
+
+    public boolean canHandleCommand(CommandLine parse) {
+      ArrayList<Option> arrayList = new ArrayList<Option>(options.getOptions());
+      return arrayList.stream().anyMatch(opt -> parse.hasOption(opt.getOpt()));
+    }
+
+    public abstract int handleCommand(CommandLine parse)
+        throws IOException, YarnException;
+
+    public abstract Options buildOptions();
+
+    public Options getOptions() {
+      return options;
+    }
+
+    public boolean getHelp(String cmd, StringBuilder strcnd, boolean addDesc) {
+      Option opt = options.getOption(cmd);
+      if (opt != null) {
+        strcnd.append(DEFAULT_SEPARATOR).append(" -").append(opt.getOpt());
+        if (opt.hasArg()) {
+          strcnd.append(" <").append(opt.getArgName()).append(">");
+        }
+        if (addDesc) {
+          strcnd.append(DEFAULT_SEPARATOR).append("\t")
+              .append(opt.getDescription());
+        }
       }
-    } catch (Exception e) {
-      exitCode = -1;
-      errOut.println(cmd.substring(1) + ": " + e.getLocalizedMessage());
+      return opt == null;
+    }
+
+    public void getHelp(StringBuilder builder, boolean description) {
+      builder.append(DEFAULT_SEPARATOR).append(DEFAULT_SEPARATOR)
+          .append(header);
+      for (String option : order) {
+        getHelp(option, builder, description);
+      }
+    }
+
+    protected void addOrder(String key){
+      order.add(key);
     }
-    return exitCode;
   }
 
-  private int handleNodeAttributeMapping(String args[],
-      AttributeMappingOperationType operation)
-      throws IOException, YarnException, ParseException {
-    Options opts = new Options();
-    opts.addOption(operation.name().toLowerCase(), true,
-        operation.name().toLowerCase());
-    opts.addOption("failOnUnknownNodes", false, "Fail on unknown nodes.");
-    int exitCode = -1;
-    CommandLine cliParser = null;
-    try {
-      cliParser = new GnuParser().parse(opts, args);
-    } catch (MissingArgumentException ex) {
-      errOut.println(NO_MAPPING_ERR_MSG);
-      printUsage(args[0]);
-      return exitCode;
+  /**
+   * Client commands handler.
+   */
+  public static class ClientCommandHandler extends CommandHandler {
+
+    private static final String LIST_ALL_ATTRS = "list";
+
+    private static final String NODESTOATTR = "nodestoattributes";
+    private static final String NODES = "nodes";
+
+    private static final String ATTRTONODES = "attributestonodes";
+    private static final String ATTRIBUTES = "attributes";
+
+    public static final String SPLITPATTERN = "/";
+
+    private static final String NODEATTRIBUTE =
+        "%40s\t%10s\t%20s" + DEFAULT_SEPARATOR;
+    private static final String NODEATTRIBUTEINFO =
+        "%40s\t%15s" + DEFAULT_SEPARATOR;
+    private static final String HOSTNAMEVAL = "%40s\t%15s" + DEFAULT_SEPARATOR;
+
+    private PrintStream sysOut = System.out;
+
+    public ClientCommandHandler() {
+      super("Client Commands:");
+
+    }
+
+    public void setSysOut(PrintStream out) {
+      this.sysOut = out;
+    }
+
+    @Override
+    public int handleCommand(CommandLine parse)
+        throws IOException, YarnException {
+      if (parse.hasOption(LIST_ALL_ATTRS)) {
+        return printClusterAttributes();
+      } else if (parse.hasOption(NODESTOATTR)) {
+        String[] nodes = new String[0];
+        if (parse.hasOption(NODES)) {
+          nodes = parse.getOptionValues(NODES);
+        }
+        return printAttributesByNode(nodes);
+      } else if (parse.hasOption(ATTRTONODES)) {
+        String[] attrKeys = {};
+        if (parse.hasOption(ATTRIBUTES)) {
+          attrKeys = parse.getOptionValues(ATTRIBUTES);
+        }
+        return printNodesByAttributes(attrKeys);
+      }
+      return 0;
+    }
+
+    protected ApplicationClientProtocol createApplicationProtocol()
+        throws IOException {
+      // Get the current configuration
+      final YarnConfiguration conf = new YarnConfiguration(getConf());
+      return ClientRMProxy.createRMProxy(conf, ApplicationClientProtocol.class);
+    }
+
+    public int printNodesByAttributes(String[] attrs)
+        throws YarnException, IOException {
+      ApplicationClientProtocol protocol = createApplicationProtocol();
+      HashSet<NodeAttributeKey> set = new HashSet<>();
+
+      for (String attr : attrs) {
+        String[] attrFields = attr.split(SPLITPATTERN);
+        if (attrFields.length == 1) {
+          set.add(NodeAttributeKey.newInstance(attrFields[0]));
+        } else if (attrFields.length == 2) {
+          set.add(NodeAttributeKey.newInstance(attrFields[0], attrFields[1]));
+        } else {
+          throw new IllegalArgumentException(
+              " Attribute format not correct. Should be <[prefix]/[name]> :"
+                  + attr);
+        }
+      }
+
+      GetAttributesToNodesRequest request =
+          GetAttributesToNodesRequest.newInstance(set);
+      GetAttributesToNodesResponse response =
+          protocol.getAttributesToNodes(request);
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      PrintWriter writer = new PrintWriter(
+          new OutputStreamWriter(baos, Charset.forName("UTF-8")));
+      writer.format(HOSTNAMEVAL, "Hostname", "Attribute-value");
+      response.getAttributesToNodes().forEach((attributeKey, v) -> {
+        writer.println(getKeyString(attributeKey) + " :");
+        v.iterator().forEachRemaining(attrVal -> writer
+            .format(HOSTNAMEVAL, attrVal.getHostname(),
+                attrVal.getAttributeValue()));
+      });
+      writer.close();
+      sysOut.println(baos.toString("UTF-8"));
+      return 0;
+    }
+
+    private int printAttributesByNode(String[] nodeArray)
+        throws YarnException, IOException {
+      ApplicationClientProtocol protocol = createApplicationProtocol();
+      HashSet<String> nodes = new HashSet<>(Arrays.asList(nodeArray));
+      GetNodesToAttributesRequest request =
+          GetNodesToAttributesRequest.newInstance(nodes);
+      GetNodesToAttributesResponse response =
+          protocol.getNodesToAttributes(request);
+      Map<String, Set<NodeAttribute>> nodeToAttrs =
+          response.getNodeToAttributes();
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      PrintWriter writer = new PrintWriter(
+          new OutputStreamWriter(baos, Charset.forName("UTF-8")));
+      writer.printf(NODEATTRIBUTE, "Attribute", "Type", "Value");
+      nodeToAttrs.forEach((node, v) -> {
+        // print node header
+        writer.println(node + ":");
+        v.iterator().forEachRemaining(attr -> writer
+            .format(NODEATTRIBUTE, getKeyString(attr.getAttributeKey()),
+                attr.getAttributeType().name(), attr.getAttributeValue()));
+      });
+      writer.close();
+      sysOut.println(baos.toString("UTF-8"));
+      return 0;
+    }
+
+    private int printClusterAttributes() throws IOException, YarnException {
+      ApplicationClientProtocol protocol = createApplicationProtocol();
+      GetClusterNodeAttributesRequest request =
+          GetClusterNodeAttributesRequest.newInstance();
+      GetClusterNodeAttributesResponse response =
+          protocol.getClusterNodeAttributes(request);
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      PrintWriter writer = new PrintWriter(
+          new OutputStreamWriter(baos, Charset.forName("UTF-8")));
+      writer.format(NODEATTRIBUTEINFO, "Attribute", "Type");
+      for (NodeAttributeInfo attr : response.getNodeAttributes()) {
+        writer.format(NODEATTRIBUTEINFO, getKeyString(attr.getAttributeKey()),
+            attr.getAttributeType().name());
+      }
+      writer.close();
+      sysOut.println(baos.toString("UTF-8"));
+      return 0;
+    }
+
+    private String getKeyString(NodeAttributeKey key) {
+      StringBuilder sb = new StringBuilder();
+      sb.append(key.getAttributePrefix()).append("/")
+          .append(key.getAttributeName());
+      return sb.toString();
+    }
+
+    @Override
+    public Options buildOptions() {
+      Options clientOptions = new Options();
+      clientOptions.addOption(
+          new Option(LIST_ALL_ATTRS, false, "List all attributes in cluster"));
+
+      // group by command
+      OptionGroup nodeToAttr = new OptionGroup();
+      Option attrtonodes = new Option(NODESTOATTR, false,
+          "Lists all mapping to nodes to attributes");
+      Option nodes = new Option(NODES,
+          "Works with [" + LIST_ALL_ATTRS + "] to specify node hostnames "
+              + "whose mappings are required to be displayed.");
+      nodes.setValueSeparator(',');
+      nodes.setArgName("Host Names");
+      nodes.setArgs(Option.UNLIMITED_VALUES);
+      nodeToAttr.addOption(attrtonodes);
+      nodeToAttr.addOption(nodes);
+      clientOptions.addOptionGroup(nodeToAttr);
+
+      // Defines as groups to add extendability for later
+      OptionGroup attrToNodes = new OptionGroup();
+      attrToNodes.addOption(new Option(ATTRTONODES, false,
+          "Displays mapping of "
+              + "attributes to nodes and attribute values grouped by "
+              + "attributes"));
+      Option attrs = new Option(ATTRIBUTES, "Works with [" + ATTRTONODES
+          + "] to specify attributes whose mapping "
+          + "are required to be displayed.");
+      attrs.setValueSeparator(',');
+      attrs.setArgName("Attributes");
+      attrs.setArgs(Option.UNLIMITED_VALUES);
+      attrToNodes.addOption(attrs);
+      clientOptions.addOptionGroup(attrToNodes);
+
+      // DEFINE ORDER
+      addOrder(LIST_ALL_ATTRS);
+      addOrder(NODESTOATTR);
+      addOrder(NODES);
+      addOrder(ATTRTONODES);
+      addOrder(ATTRIBUTES);
+      return clientOptions;
     }
-    List<NodeToAttributes> buildNodeLabelsMapFromStr =
-        buildNodeLabelsMapFromStr(
-            cliParser.getOptionValue(operation.name().toLowerCase()),
-            operation != AttributeMappingOperationType.REPLACE, operation);
-    NodesToAttributesMappingRequest request = NodesToAttributesMappingRequest
-        .newInstance(operation, buildNodeLabelsMapFromStr,
-            cliParser.hasOption("failOnUnknownNodes"));
-    ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol();
-    adminProtocol.mapAttributesToNodes(request);
-    return 0;
   }
 
   /**
-   * args are expected to be of the format
-   * node1:java(string)=8,ssd(boolean)=false node2:ssd(boolean)=true
+   * Admin commands handler.
    */
-  private List<NodeToAttributes> buildNodeLabelsMapFromStr(String args,
-      boolean validateForAttributes, AttributeMappingOperationType operation) {
-    Map<String,NodeToAttributes> nodeToAttributesMap = new HashMap<>();
-    for (String nodeToAttributesStr : args.split("[ \n]")) {
-      // for each node to attribute mapping
-      nodeToAttributesStr = nodeToAttributesStr.trim();
-      if (nodeToAttributesStr.isEmpty()
-          || nodeToAttributesStr.startsWith("#")) {
-        continue;
+  public static class AdminCommandHandler extends CommandHandler {
+
+    private static final String ADD = "add";
+    private static final String REMOVE = "remove";
+    private static final String REPLACE = "replace";
+    private static final String FAILUNKNOWNNODES = "failOnUnknownNodes";
+
+    AdminCommandHandler() {
+      super("Admin Commands:");
+    }
+
+    @Override
+    public Options buildOptions() {
+      Options adminOptions = new Options();
+      Option replace = new Option(REPLACE, true,
+          "Replace the node to attributes mapping information at the"
+              + " ResourceManager with the new mapping. Currently"
+              + " supported attribute type. And string is the default"
+              + " type too. Attribute value if not specified for string"
+              + " type value will be considered as empty string."
+              + " Replaced node-attributes should not violate the"
+              + " existing attribute to attribute type mapping.");
+      replace.setArgName("\"node1:attribute[(type)][=value],attribute1[=value],"
+          + "attribute2  node2:attribute2[=value],attribute3\"");
+      replace.setArgs(1);
+      adminOptions.addOption(replace);
+
+      Option add = new Option(ADD, true,
+          "Adds or updates the node to attributes mapping information"
+              + " at the ResourceManager. Currently supported attribute"
+              + " type is string. And string is the default type too."
+              + " Attribute value if not specified for string type"
+              + " value will be considered as empty string. Added or"
+              + " updated node-attributes should not violate the"
+              + " existing attribute to attribute type mapping.");
+      add.setArgName("\"node1:attribute[(type)][=value],attribute1[=value],"
+          + "attribute2  node2:attribute2[=value],attribute3\"");
+      add.setArgs(1);
+      adminOptions.addOption(add);
+
+      Option remove = new Option(REMOVE, true,
+          "Removes the specified node to attributes mapping"
+              + " information at the ResourceManager");
+      remove.setArgName("\"node1:attribute,attribute1 node2:attribute2\"");
+      remove.setArgs(1);
+      adminOptions.addOption(remove);
+
+      adminOptions.addOption(new Option(FAILUNKNOWNNODES, false,
+          "Can be used optionally along with [add,remove,replace] options. "
+              + "When set, command will fail if specified nodes are unknown."));
+
+      // DEFINE ORDER
+      addOrder(REPLACE);
+      addOrder(ADD);
+      addOrder(REMOVE);
+      addOrder(FAILUNKNOWNNODES);
+
+      return adminOptions;
+    }
+
+    protected ResourceManagerAdministrationProtocol createAdminProtocol()
+        throws IOException {
+      // Get the current configuration
+      final YarnConfiguration conf = new YarnConfiguration(getConf());
+      return ClientRMProxy
+          .createRMProxy(conf, ResourceManagerAdministrationProtocol.class);
+    }
+
+    public int handleCommand(CommandLine cliParser)
+        throws IOException, YarnException {
+      String operation = null;
+      if (cliParser.hasOption(ADD)) {
+        operation = ADD;
+      } else if (cliParser.hasOption(REMOVE)) {
+        operation = REMOVE;
+      } else if (cliParser.hasOption(REPLACE)) {
+        operation = REPLACE;
       }
-      if (nodeToAttributesStr.indexOf(":") == -1) {
+      if (operation != null) {
+        List<NodeToAttributes> buildNodeLabelsListFromStr =
+            buildNodeLabelsListFromStr(cliParser.getOptionValue(operation),
+                !operation.equals(REPLACE), operation);
+        NodesToAttributesMappingRequest request =
+            NodesToAttributesMappingRequest.newInstance(
+                AttributeMappingOperationType.valueOf(operation.toUpperCase()),
+                buildNodeLabelsListFromStr,
+                cliParser.hasOption(FAILUNKNOWNNODES));
+        ResourceManagerAdministrationProtocol adminProtocol =
+            createAdminProtocol();
+        adminProtocol.mapAttributesToNodes(request);
+      } else {
+        // Handle case for only failOnUnknownNodes passed
         throw new IllegalArgumentException(
-            INVALID_MAPPING_ERR_MSG + nodeToAttributesStr);
+            getOptions().getOption(FAILUNKNOWNNODES).getDescription());
       }
-      String[] nodeToAttributes = nodeToAttributesStr.split(":");
-      Preconditions.checkArgument(!nodeToAttributes[0].trim().isEmpty(),
-          "Node name cannot be empty");
-      String node = nodeToAttributes[0];
-      String[] attributeNameValueType = null;
-      List<NodeAttribute> attributesList = new ArrayList<>();
-      NodeAttributeType attributeType = NodeAttributeType.STRING;
-      String attributeValue;
-      String attributeName;
-      Set<String> attributeNamesMapped = new HashSet<>();
-
-      String attributesStr[];
-      if (nodeToAttributes.length == 2) {
-        // fetching multiple attributes for a node
-        attributesStr = nodeToAttributes[1].split(",");
-        for (String attributeStr : attributesStr) {
-          // get information about each attribute.
-          attributeNameValueType = attributeStr.split("="); // to find name
-                                                            // value
-          Preconditions.checkArgument(
-              !(attributeNameValueType[0] == null
-                  || attributeNameValueType[0].isEmpty()),
-              "Attribute name cannot be null or empty");
-          attributeValue = attributeNameValueType.length > 1
-              ? attributeNameValueType[1] : "";
-          int indexOfOpenBracket = attributeNameValueType[0].indexOf("(");
-          if (indexOfOpenBracket == -1) {
-            attributeName = attributeNameValueType[0];
-          } else if (indexOfOpenBracket == 0) {
-            throw new IllegalArgumentException("Attribute for node " + node
-                + " is not properly configured : " + attributeStr);
-          } else {
-            // attribute type has been explicitly configured
-            int indexOfCloseBracket = attributeNameValueType[0].indexOf(")");
-            if (indexOfCloseBracket == -1
-                || indexOfCloseBracket < indexOfOpenBracket) {
+      return 0;
+    }
+
+    /**
+     * args are expected to be of the format
+     * node1:java(string)=8,ssd(boolean)=false node2:ssd(boolean)=true.
+     */
+    private List<NodeToAttributes> buildNodeLabelsListFromStr(String args,
+        boolean validateForAttributes, String operation) {
+      Map<String, NodeToAttributes> nodeToAttributesMap = new HashMap<>();
+      for (String nodeToAttributesStr : args.split("[ \n]")) {
+        // for each node to attribute mapping
+        nodeToAttributesStr = nodeToAttributesStr.trim();
+        if (nodeToAttributesStr.isEmpty() || nodeToAttributesStr
+            .startsWith("#")) {
+          continue;
+        }
+        if (nodeToAttributesStr.indexOf(":") == -1) {
+          throw new IllegalArgumentException(
+              INVALID_MAPPING_ERR_MSG + nodeToAttributesStr);
+        }
+        String[] nodeToAttributes = nodeToAttributesStr.split(":");
+        Preconditions.checkArgument(!nodeToAttributes[0].trim().isEmpty(),
+            "Node name cannot be empty");
+        String node = nodeToAttributes[0];
+        String[] attributeNameValueType = null;
+        List<NodeAttribute> attributesList = new ArrayList<>();
+        NodeAttributeType attributeType = NodeAttributeType.STRING;
+        String attributeValue;
+        String attributeName;
+        Set<String> attributeNamesMapped = new HashSet<>();
+
+        String[] attributesStr;
+        if (nodeToAttributes.length == 2) {
+          // fetching multiple attributes for a node
+          attributesStr = nodeToAttributes[1].split(",");
+          for (String attributeStr : attributesStr) {
+            // get information about each attribute.
+            attributeNameValueType = attributeStr.split("="); // to find name
+            // value
+            Preconditions.checkArgument(
+                !(attributeNameValueType[0] == null || attributeNameValueType[0]
+                    .isEmpty()), "Attribute name cannot be null or empty");
+            attributeValue = attributeNameValueType.length > 1 ?
+                attributeNameValueType[1] :
+                "";
+            int indexOfOpenBracket = attributeNameValueType[0].indexOf("(");
+            if (indexOfOpenBracket == -1) {
+              attributeName = attributeNameValueType[0];
+            } else if (indexOfOpenBracket == 0) {
               throw new IllegalArgumentException("Attribute for node " + node
-                  + " is not properly Configured : " + attributeStr);
+                  + " is not properly configured : " + attributeStr);
+            } else {
+              // attribute type has been explicitly configured
+              int indexOfCloseBracket = attributeNameValueType[0].indexOf(")");
+              if (indexOfCloseBracket == -1
+                  || indexOfCloseBracket < indexOfOpenBracket) {
+                throw new IllegalArgumentException("Attribute for node " + node
+                    + " is not properly Configured : " + attributeStr);
+              }
+              String attributeTypeStr;
+              attributeName =
+                  attributeNameValueType[0].substring(0, indexOfOpenBracket);
+              attributeTypeStr = attributeNameValueType[0]
+                  .substring(indexOfOpenBracket + 1, indexOfCloseBracket);
+              try {
+                attributeType = NodeAttributeType
+                    .valueOf(attributeTypeStr.trim().toUpperCase());
+              } catch (IllegalArgumentException e) {
+                throw new IllegalArgumentException(
+                    "Invalid Attribute type configuration : " + attributeTypeStr
+                        + " in " + attributeStr);
+              }
             }
-            String attributeTypeStr;
-            attributeName =
-                attributeNameValueType[0].substring(0, indexOfOpenBracket);
-            attributeTypeStr = attributeNameValueType[0]
-                .substring(indexOfOpenBracket + 1, indexOfCloseBracket);
-            try {
-              attributeType = NodeAttributeType
-                  .valueOf(attributeTypeStr.trim().toUpperCase());
-            } catch (IllegalArgumentException e) {
-              throw new IllegalArgumentException(
-                  "Invalid Attribute type configuration : " + attributeTypeStr
-                      + " in " + attributeStr);
+            if (attributeNamesMapped.contains(attributeName)) {
+              throw new IllegalArgumentException("Attribute " + attributeName
+                  + " has been mapped more than once in  : "
+                  + nodeToAttributesStr);
             }
+            // TODO when we support different type of attribute type we need to
+            // cross verify whether input attributes itself is not violating
+            // attribute Name to Type mapping.
+            attributesList.add(NodeAttribute
+                .newInstance(NodeAttribute.PREFIX_CENTRALIZED,
+                    attributeName.trim(), attributeType,
+                    attributeValue.trim()));
           }
-          if (attributeNamesMapped.contains(attributeName)) {
-            throw new IllegalArgumentException("Attribute " + attributeName
-                + " has been mapped more than once in  : "
-                + nodeToAttributesStr);
-          }
-          // TODO when we support different type of attribute type we need to
-          // cross verify whether input attributes itself is not violating
-          // attribute Name to Type mapping.
-          attributesList
-              .add(NodeAttribute.newInstance(NodeAttribute.PREFIX_CENTRALIZED,
-                  attributeName.trim(), attributeType, attributeValue.trim()));
         }
+        if (validateForAttributes) {
+          Preconditions.checkArgument((attributesList.size() > 0),
+              "Attributes cannot be null or empty for Operation [" + operation
+                  + "] on the node " + node);
+        }
+        nodeToAttributesMap
+            .put(node, NodeToAttributes.newInstance(node, attributesList));
       }
-      if (validateForAttributes) {
-        Preconditions.checkArgument((attributesList.size() > 0),
-            "Attributes cannot be null or empty for Operation "
-                + operation.name() + " on the node " + node);
+
+      if (nodeToAttributesMap.isEmpty()) {
+        throw new IllegalArgumentException(NO_MAPPING_ERR_MSG);
       }
-      nodeToAttributesMap
-          .put(node,NodeToAttributes.newInstance(node, attributesList));
+      return Lists.newArrayList(nodeToAttributesMap.values());
     }
 
-    if (nodeToAttributesMap.isEmpty()) {
-      throw new IllegalArgumentException(NO_MAPPING_ERR_MSG);
+    @Override
+    public void setConf(Configuration conf) {
+      if (conf != null) {
+        conf = addSecurityConfiguration(conf);
+      }
+      super.setConf(conf);
+    }
+
+    /**
+     * Add the requisite security principal settings to the given Configuration,
+     * returning a copy.
+     *
+     * @param conf the original config
+     * @return a copy with the security settings added
+     */
+    private Configuration addSecurityConfiguration(Configuration conf) {
+      // Make a copy so we don't mutate it. Also use an YarnConfiguration to
+      // force loading of yarn-site.xml.
+      conf = new YarnConfiguration(conf);
+      conf.set(CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY,
+          conf.get(YarnConfiguration.RM_PRINCIPAL, ""));
+      return conf;
     }
-    return Lists.newArrayList(nodeToAttributesMap.values());
-  }
 
-  public static void main(String[] args) throws Exception {
-    int result = ToolRunner.run(new NodeAttributesCLI(), args);
-    System.exit(result);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
index e9253eb..44e9870 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 
 @Private
 @Unstable
@@ -307,6 +306,18 @@ public class NodeCLI extends YarnCLI {
       Collections.sort(nodeLabelsList);
       nodeReportStr.println(StringUtils.join(nodeLabelsList.iterator(), ','));
 
+      if (nodeReport.getNodeAttributes().size() > 0) {
+        ArrayList nodeAtrs = new ArrayList<>(nodeReport.getNodeAttributes());
+        nodeReportStr.print("\tNode Attributes : ");
+        nodeReportStr.println(nodeAtrs.get(0).toString());
+        for (int index = 1; index < nodeAtrs.size(); index++) {
+          nodeReportStr.println(
+              String.format("\t%18s%s", "", nodeAtrs.get(index).toString()));
+        }
+      } else {
+        nodeReportStr.println("\tNode Attributes : ");
+      }
+
       nodeReportStr.print("\tResource Utilization by Node : ");
       if (nodeReport.getNodeUtilization() != null) {
         nodeReportStr.print("PMem:"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestClusterCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestClusterCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestClusterCLI.java
index 5a0f049..26afe6f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestClusterCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestClusterCLI.java
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.yarn.client.cli;
 
+import org.apache.hadoop.yarn.api.records.NodeAttributeInfo;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
@@ -74,7 +77,32 @@ public class TestClusterCLI {
     pw.close();
     verify(sysOut).println(baos.toString("UTF-8"));
   }
-  
+
+  @Test
+  public void testGetClusterNodeAttributes() throws Exception {
+    YarnClient client = mock(YarnClient.class);
+    when(client.getClusterAttributes()).thenReturn(ImmutableSet
+        .of(NodeAttributeInfo.newInstance(NodeAttributeKey.newInstance("GPU"),
+            NodeAttributeType.STRING), NodeAttributeInfo
+            .newInstance(NodeAttributeKey.newInstance("CPU"),
+                NodeAttributeType.STRING)));
+    ClusterCLI cli = new ClusterCLI();
+    cli.setClient(client);
+    cli.setSysOutPrintStream(sysOut);
+    cli.setSysErrPrintStream(sysErr);
+
+    int rc = cli.run(new String[] {ClusterCLI.CMD,
+        "-" + ClusterCLI.LIST_CLUSTER_ATTRIBUTES});
+    assertEquals(0, rc);
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter pw = new PrintWriter(baos);
+    pw.println("rm.yarn.io/GPU(STRING)");
+    pw.println("rm.yarn.io/CPU(STRING)");
+    pw.close();
+    verify(sysOut).println(baos.toString("UTF-8"));
+  }
+
   @Test
   public void testGetClusterNodeLabelsWithLocalAccess() throws Exception {
     YarnClient client = mock(YarnClient.class);
@@ -157,6 +185,8 @@ public class TestClusterCLI {
     pw.println("                                           option is UNSTABLE, could be");
     pw.println("                                           removed in future releases.");
     pw.println(" -h,--help                                 Displays help for all commands.");
+    pw.println(" -lna,--list-node-attributes               List cluster node-attribute");
+    pw.println("                                           collection");
     pw.println(" -lnl,--list-node-labels                   List cluster node-label");
     pw.println("                                           collection");
     pw.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java
index bbd5ca3..7f48493 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java
@@ -18,6 +18,20 @@
 
 package org.apache.hadoop.yarn.client.cli;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
+
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
+import org.apache.hadoop.yarn.api.records.NodeAttributeInfo;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.api.records.NodeToAttributeValue;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
@@ -29,8 +43,8 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeAttributeType;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -56,75 +70,122 @@ public class TestNodeAttributesCLI {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestNodeAttributesCLI.class);
   private ResourceManagerAdministrationProtocol admin;
-  private NodesToAttributesMappingRequest request;
+  private ApplicationClientProtocol client;
+  private NodesToAttributesMappingRequest nodeToAttrRequest;
   private NodeAttributesCLI nodeAttributesCLI;
   private ByteArrayOutputStream errOutBytes = new ByteArrayOutputStream();
+  private ByteArrayOutputStream sysOutBytes = new ByteArrayOutputStream();
   private String errOutput;
+  private String sysOutput;
 
   @Before
   public void configure() throws IOException, YarnException {
+
     admin = mock(ResourceManagerAdministrationProtocol.class);
+    client = mock(ApplicationClientProtocol.class);
 
     when(admin.mapAttributesToNodes(any(NodesToAttributesMappingRequest.class)))
         .thenAnswer(new Answer<NodesToAttributesMappingResponse>() {
           @Override
           public NodesToAttributesMappingResponse answer(
               InvocationOnMock invocation) throws Throwable {
-            request =
+            nodeToAttrRequest =
                 (NodesToAttributesMappingRequest) invocation.getArguments()[0];
             return NodesToAttributesMappingResponse.newInstance();
           }
         });
 
-    nodeAttributesCLI = new NodeAttributesCLI(new Configuration()) {
+    nodeAttributesCLI = new NodeAttributesCLI() {
       @Override
-      protected ResourceManagerAdministrationProtocol createAdminProtocol()
-          throws IOException {
-        return admin;
+      protected AdminCommandHandler getAdminCommandHandler() {
+        return new AdminCommandHandler() {
+          @Override
+          protected ResourceManagerAdministrationProtocol createAdminProtocol()
+              throws IOException {
+            return admin;
+          }
+        };
       }
-    };
 
+      @Override
+      protected ClientCommandHandler getClientCommandHandler() {
+        ClientCommandHandler handler = new ClientCommandHandler() {
+          @Override
+          protected ApplicationClientProtocol createApplicationProtocol()
+              throws IOException {
+            return client;
+          }
+        };
+        handler.setSysOut(new PrintStream(sysOutBytes));
+        return handler;
+      }
+    };
     nodeAttributesCLI.setErrOut(new PrintStream(errOutBytes));
   }
 
   @Test
   public void testHelp() throws Exception {
-    String[] args = new String[] { "-help", "-replace" };
+    String[] args = new String[] {"-help", "-replace"};
     assertTrue("It should have succeeded help for replace", 0 == runTool(args));
-    assertOutputContains(
-        "-replace <\"node1:attribute[(type)][=value],attribute1"
-            + "[=value],attribute2  node2:attribute2[=value],attribute3\"> :");
-    assertOutputContains("Replace the node to attributes mapping information at"
+    assertErrorContains("-replace <\"node1:attribute[(type)][=value],attribute1"
+        + "[=value],attribute2  node2:attribute2[=value],attribute3\">");
+    assertErrorContains("Replace the node to attributes mapping information at"
         + " the ResourceManager with the new mapping. Currently supported"
         + " attribute type. And string is the default type too. Attribute value"
         + " if not specified for string type value will be considered as empty"
         + " string. Replaced node-attributes should not violate the existing"
         + " attribute to attribute type mapping.");
 
-    args = new String[] { "-help", "-remove" };
+    args = new String[] {"-help", "-remove"};
     assertTrue("It should have succeeded help for replace", 0 == runTool(args));
-    assertOutputContains(
-        "-remove <\"node1:attribute,attribute1" + " node2:attribute2\"> :");
-    assertOutputContains("Removes the specified node to attributes mapping"
+    assertErrorContains(
+        "-remove <\"node1:attribute,attribute1" + " node2:attribute2\">");
+    assertErrorContains("Removes the specified node to attributes mapping"
         + " information at the ResourceManager");
 
-    args = new String[] { "-help", "-add" };
+    args = new String[] {"-help", "-add"};
     assertTrue("It should have succeeded help for replace", 0 == runTool(args));
-    assertOutputContains("-add <\"node1:attribute[(type)][=value],"
-        + "attribute1[=value],attribute2  node2:attribute2[=value],attribute3\">"
-        + " :");
-    assertOutputContains("Adds or updates the node to attributes mapping"
+    assertErrorContains("-add <\"node1:attribute[(type)][=value],"
+        + "attribute1[=value],attribute2  node2:attribute2[=value],"
+        + "attribute3\">");
+    assertErrorContains("Adds or updates the node to attributes mapping"
         + " information at the ResourceManager. Currently supported attribute"
         + " type is string. And string is the default type too. Attribute value"
         + " if not specified for string type value will be considered as empty"
         + " string. Added or updated node-attributes should not violate the"
         + " existing attribute to attribute type mapping.");
 
-    args = new String[] { "-help", "-failOnUnknownNodes" };
+    args = new String[] {"-help", "-failOnUnknownNodes"};
     assertTrue("It should have succeeded help for replace", 0 == runTool(args));
-    assertOutputContains("-failOnUnknownNodes :");
-    assertOutputContains("Can be used optionally along with other options. When"
-        + " its set, it will fail if specified nodes are unknown.");
+    assertErrorContains("-failOnUnknownNodes");
+    assertErrorContains("Can be used optionally along with [add,remove,"
+        + "replace] options. When set, command will fail if specified nodes "
+        + "are unknown.");
+
+    args = new String[] {"-help", "-list"};
+    assertTrue("It should have succeeded help for replace", 0 == runTool(args));
+    assertErrorContains("-list");
+    assertErrorContains("List all attributes in cluster");
+
+    args = new String[] {"-help", "-nodes"};
+    assertTrue("It should have succeeded help for replace", 0 == runTool(args));
+    assertErrorContains("-nodes");
+    assertErrorContains(
+        "Works with [list] to specify node hostnames whose mappings "
+            + "are required to be displayed.");
+
+    args = new String[] {"-help", "-attributes"};
+    assertTrue("It should have succeeded help for replace", 0 == runTool(args));
+    assertErrorContains("-attributes");
+    assertErrorContains(
+        "Works with [attributestonodes] to specify attributes whose mapping "
+            + "are required to be displayed.");
+
+    args = new String[] {"-help", "-attributestonodes"};
+    assertTrue("It should have succeeded help for replace", 0 == runTool(args));
+    assertErrorContains("-attributestonodes");
+    assertErrorContains("Displays mapping of attributes to nodes and attribute "
+        + "values grouped by attributes");
   }
 
   @Test
@@ -133,62 +194,62 @@ public class TestNodeAttributesCLI {
     // failure scenarios
     // --------------------------------
     // parenthesis not match
-    String[] args = new String[] { "-replace", "x(" };
+    String[] args = new String[] {"-replace", "x("};
     assertTrue("It should have failed as no node is specified",
         0 != runTool(args));
     assertFailureMessageContains(NodeAttributesCLI.INVALID_MAPPING_ERR_MSG);
 
     // parenthesis not match
-    args = new String[] { "-replace", "x:(=abc" };
+    args = new String[] {"-replace", "x:(=abc"};
     assertTrue(
         "It should have failed as no closing parenthesis is not specified",
         0 != runTool(args));
     assertFailureMessageContains(
         "Attribute for node x is not properly configured : (=abc");
 
-    args = new String[] { "-replace", "x:()=abc" };
+    args = new String[] {"-replace", "x:()=abc"};
     assertTrue("It should have failed as no type specified inside parenthesis",
         0 != runTool(args));
     assertFailureMessageContains(
         "Attribute for node x is not properly configured : ()=abc");
 
-    args = new String[] { "-replace", ":x(string)" };
+    args = new String[] {"-replace", ":x(string)"};
     assertTrue("It should have failed as no node is specified",
         0 != runTool(args));
     assertFailureMessageContains("Node name cannot be empty");
 
     // Not expected key=value specifying inner parenthesis
-    args = new String[] { "-replace", "x:(key=value)" };
+    args = new String[] {"-replace", "x:(key=value)"};
     assertTrue(0 != runTool(args));
     assertFailureMessageContains(
         "Attribute for node x is not properly configured : (key=value)");
 
     // Should fail as no attributes specified
-    args = new String[] { "-replace" };
+    args = new String[] {"-replace"};
     assertTrue("Should fail as no attribute mappings specified",
         0 != runTool(args));
-    assertFailureMessageContains(NodeAttributesCLI.NO_MAPPING_ERR_MSG);
+    assertFailureMessageContains(NodeAttributesCLI.MISSING_ARGUMENT);
 
     // no labels, should fail
-    args = new String[] { "-replace", "-failOnUnknownNodes",
-        "x:key(string)=value,key2=val2" };
+    args = new String[] {"-replace", "-failOnUnknownNodes",
+        "x:key(string)=value,key2=val2"};
     assertTrue("Should fail as no attribute mappings specified for replace",
         0 != runTool(args));
-    assertFailureMessageContains(NodeAttributesCLI.NO_MAPPING_ERR_MSG);
+    assertFailureMessageContains(NodeAttributesCLI.MISSING_ARGUMENT);
 
     // no labels, should fail
-    args = new String[] { "-replace", " " };
+    args = new String[] {"-replace", " "};
     assertTrue(0 != runTool(args));
     assertFailureMessageContains(NodeAttributesCLI.NO_MAPPING_ERR_MSG);
 
-    args = new String[] { "-replace", ", " };
+    args = new String[] {"-replace", ", "};
     assertTrue(0 != runTool(args));
     assertFailureMessageContains(NodeAttributesCLI.INVALID_MAPPING_ERR_MSG);
     // --------------------------------
     // success scenarios
     // --------------------------------
-    args = new String[] { "-replace",
-        "x:key(string)=value,key2=val2 y:key2=val23,key3 z:key4" };
+    args = new String[] {"-replace",
+        "x:key(string)=value,key2=val2 y:key2=val23,key3 z:key4"};
     assertTrue("Should not fail as attribute has been properly mapped",
         0 == runTool(args));
     List<NodeToAttributes> nodeAttributesList = new ArrayList<>();
@@ -221,10 +282,10 @@ public class TestNodeAttributesCLI {
         .add(NodeAttribute.newInstance("key4", NodeAttributeType.STRING, ""));
     nodeAttributesList.add(NodeToAttributes.newInstance("z", attributes));
 
-    NodesToAttributesMappingRequest expected =
-        NodesToAttributesMappingRequest.newInstance(
-            AttributeMappingOperationType.REPLACE, nodeAttributesList, false);
-    assertTrue(request.equals(expected));
+    NodesToAttributesMappingRequest expected = NodesToAttributesMappingRequest
+        .newInstance(AttributeMappingOperationType.REPLACE, nodeAttributesList,
+            false);
+    assertTrue(nodeToAttrRequest.equals(expected));
   }
 
   @Test
@@ -233,16 +294,17 @@ public class TestNodeAttributesCLI {
     // failure scenarios
     // --------------------------------
     // parenthesis not match
-    String[] args = new String[] { "-remove", "x:" };
+    String[] args = new String[] {"-remove", "x:"};
     assertTrue("It should have failed as no node is specified",
         0 != runTool(args));
     assertFailureMessageContains(
-        "Attributes cannot be null or empty for Operation REMOVE on the node x");
+        "Attributes cannot be null or empty for Operation [remove] on the "
+            + "node x");
     // --------------------------------
     // success scenarios
     // --------------------------------
     args =
-        new String[] { "-remove", "x:key2,key3 z:key4", "-failOnUnknownNodes" };
+        new String[] {"-remove", "x:key2,key3 z:key4", "-failOnUnknownNodes"};
     assertTrue("Should not fail as attribute has been properly mapped",
         0 == runTool(args));
     List<NodeToAttributes> nodeAttributesList = new ArrayList<>();
@@ -259,10 +321,10 @@ public class TestNodeAttributesCLI {
         .add(NodeAttribute.newInstance("key4", NodeAttributeType.STRING, ""));
     nodeAttributesList.add(NodeToAttributes.newInstance("z", attributes));
 
-    NodesToAttributesMappingRequest expected =
-        NodesToAttributesMappingRequest.newInstance(
-            AttributeMappingOperationType.REMOVE, nodeAttributesList, true);
-    assertTrue(request.equals(expected));
+    NodesToAttributesMappingRequest expected = NodesToAttributesMappingRequest
+        .newInstance(AttributeMappingOperationType.REMOVE, nodeAttributesList,
+            true);
+    assertTrue(nodeToAttrRequest.equals(expected));
   }
 
   @Test
@@ -271,16 +333,16 @@ public class TestNodeAttributesCLI {
     // failure scenarios
     // --------------------------------
     // parenthesis not match
-    String[] args = new String[] { "-add", "x:" };
+    String[] args = new String[] {"-add", "x:"};
     assertTrue("It should have failed as no node is specified",
         0 != runTool(args));
     assertFailureMessageContains(
-        "Attributes cannot be null or empty for Operation ADD on the node x");
+        "Attributes cannot be null or empty for Operation [add] on the node x");
     // --------------------------------
     // success scenarios
     // --------------------------------
-    args = new String[] { "-add", "x:key2=123,key3=abc z:key4(string)",
-        "-failOnUnknownNodes" };
+    args = new String[] {"-add", "x:key2=123,key3=abc z:key4(string)",
+        "-failOnUnknownNodes"};
     assertTrue("Should not fail as attribute has been properly mapped",
         0 == runTool(args));
     List<NodeToAttributes> nodeAttributesList = new ArrayList<>();
@@ -297,16 +359,16 @@ public class TestNodeAttributesCLI {
         .add(NodeAttribute.newInstance("key4", NodeAttributeType.STRING, ""));
     nodeAttributesList.add(NodeToAttributes.newInstance("z", attributes));
 
-    NodesToAttributesMappingRequest expected =
-        NodesToAttributesMappingRequest.newInstance(
-            AttributeMappingOperationType.ADD, nodeAttributesList, true);
-    assertTrue(request.equals(expected));
+    NodesToAttributesMappingRequest expected = NodesToAttributesMappingRequest
+        .newInstance(AttributeMappingOperationType.ADD, nodeAttributesList,
+            true);
+    assertTrue(nodeToAttrRequest.equals(expected));
 
     // --------------------------------
     // with Duplicate mappings for a host
     // --------------------------------
-    args = new String[] { "-add", "x:key2=123,key3=abc x:key4(string)",
-        "-failOnUnknownNodes" };
+    args = new String[] {"-add", "x:key2=123,key3=abc x:key4(string)",
+        "-failOnUnknownNodes"};
     assertTrue("Should not fail as attribute has been properly mapped",
         0 == runTool(args));
     nodeAttributesList = new ArrayList<>();
@@ -315,32 +377,161 @@ public class TestNodeAttributesCLI {
         .add(NodeAttribute.newInstance("key4", NodeAttributeType.STRING, ""));
     nodeAttributesList.add(NodeToAttributes.newInstance("x", attributes));
 
-    expected =
-        NodesToAttributesMappingRequest.newInstance(
-            AttributeMappingOperationType.ADD, nodeAttributesList, true);
-    assertTrue(request.equals(expected));
+    expected = NodesToAttributesMappingRequest
+        .newInstance(AttributeMappingOperationType.ADD, nodeAttributesList,
+            true);
+    assertTrue(nodeToAttrRequest.equals(expected));
+  }
+
+  @Test
+  public void testListAttributes() throws Exception {
+
+    // GetClusterNodeAttributesRequest
+    when(client
+        .getClusterNodeAttributes(any(GetClusterNodeAttributesRequest.class)))
+        .thenAnswer(new Answer<GetClusterNodeAttributesResponse>() {
+          @Override
+          public GetClusterNodeAttributesResponse answer(
+              InvocationOnMock invocation) throws Throwable {
+            GetClusterNodeAttributesRequest nodeAttrReq =
+                (GetClusterNodeAttributesRequest) invocation.getArguments()[0];
+            return GetClusterNodeAttributesResponse.newInstance(ImmutableSet
+                .of(NodeAttributeInfo
+                    .newInstance(NodeAttributeKey.newInstance("GPU"),
+                        NodeAttributeType.STRING)));
+          }
+        });
+
+    // --------------------------------
+    // Success scenarios
+    // --------------------------------
+    String[] args = new String[] {"-list"};
+    assertTrue("It should be success since it list all attributes",
+        0 == runTool(args));
+    assertSysOutContains("Attribute\t           Type",
+        "rm.yarn.io/GPU\t         STRING");
+  }
+
+  @Test
+  public void testNodeToAttributes() throws Exception {
+    // GetNodesToAttributesRequest response
+    when(client.getNodesToAttributes(any(GetNodesToAttributesRequest.class)))
+        .thenAnswer(new Answer<GetNodesToAttributesResponse>() {
+          @Override
+          public GetNodesToAttributesResponse answer(
+              InvocationOnMock invocation) throws Throwable {
+            GetNodesToAttributesRequest nodeToAttributes =
+                (GetNodesToAttributesRequest) invocation.getArguments()[0];
+            return GetNodesToAttributesResponse.newInstance(
+                ImmutableMap.<String, Set<NodeAttribute>>builder()
+                    .put("hostname", ImmutableSet.of(NodeAttribute
+                        .newInstance("GPU", NodeAttributeType.STRING, "ARM")))
+                    .build());
+          }
+        });
+    // --------------------------------
+    // Failure scenarios
+    // --------------------------------
+    String[] args = new String[] {"-nodetoattributes", "-nodes"};
+    assertTrue("It should not success since nodes are not specified",
+        0 != runTool(args));
+    assertErrorContains(NodeAttributesCLI.INVALID_COMMAND_USAGE);
+
+    // Missing argument for nodes
+    args = new String[] {"-nodestoattributes", "-nodes"};
+    assertTrue("It should not success since nodes are not specified",
+        0 != runTool(args));
+    assertErrorContains(NodeAttributesCLI.MISSING_ARGUMENT);
+
+    // --------------------------------
+    // Success with hostname param
+    // --------------------------------
+    args = new String[] {"-nodestoattributes", "-nodes", "hostname"};
+    assertTrue("Should return hostname to attributed list", 0 == runTool(args));
+    assertSysOutContains("hostname");
+  }
+
+  @Test
+  public void testAttributesToNodes() throws Exception {
+    // GetAttributesToNodesResponse response
+    when(client.getAttributesToNodes(any(GetAttributesToNodesRequest.class)))
+        .thenAnswer(new Answer<GetAttributesToNodesResponse>() {
+          @Override
+          public GetAttributesToNodesResponse answer(
+              InvocationOnMock invocation) throws Throwable {
+            GetAttributesToNodesRequest attrToNodes =
+                (GetAttributesToNodesRequest) invocation.getArguments()[0];
+            return GetAttributesToNodesResponse.newInstance(
+                ImmutableMap.<NodeAttributeKey,
+                    List<NodeToAttributeValue>>builder()
+                    .put(NodeAttributeKey.newInstance("GPU"), ImmutableList
+                        .of(NodeToAttributeValue.newInstance("host1", "ARM")))
+                    .build());
+          }
+        });
+    // --------------------------------
+    // Success scenarios
+    // --------------------------------
+    String[] args = new String[] {"-attributestonodes"};
+    assertTrue("It should be success since it list all attributes",
+        0 == runTool(args));
+    assertSysOutContains("Hostname\tAttribute-value", "rm.yarn.io/GPU :",
+        "host1\t            ARM");
+
+    // --------------------------------
+    // fail scenario argument filter missing
+    // --------------------------------
+    args = new String[] {"-attributestonodes", "-attributes"};
+    assertTrue(
+        "It should not success since attributes for filter are not specified",
+        0 != runTool(args));
+    assertErrorContains(NodeAttributesCLI.MISSING_ARGUMENT);
+
+    // --------------------------------
+    // fail scenario argument filter missing
+    // --------------------------------
+    args = new String[] {"-attributestonodes", "-attributes", "fail/da/fail"};
+    assertTrue("It should not success since attributes format is not correct",
+        0 != runTool(args));
+    assertErrorContains(
+        "Attribute format not correct. Should be <[prefix]/[name]> "
+            + ":fail/da/fail");
   }
 
   private void assertFailureMessageContains(String... messages) {
-    assertOutputContains(messages);
-    assertOutputContains(NodeAttributesCLI.USAGE_YARN_NODE_ATTRIBUTES);
+    assertErrorContains(messages);
+    assertErrorContains(NodeAttributesCLI.USAGE_YARN_NODE_ATTRIBUTES);
   }
 
-  private void assertOutputContains(String... messages) {
+  private void assertErrorContains(String... messages) {
     for (String message : messages) {
       if (!errOutput.contains(message)) {
-        fail("Expected output to contain '" + message
-            + "' but err_output was:\n" + errOutput);
+        fail(
+            "Expected output to contain '" + message + "' but err_output was:\n"
+                + errOutput);
+      }
+    }
+  }
+
+  private void assertSysOutContains(String... messages) {
+    for (String message : messages) {
+      if (!sysOutput.contains(message)) {
+        fail(
+            "Expected output to contain '" + message + "' but sys_output was:\n"
+                + sysOutput);
       }
     }
   }
 
   private int runTool(String... args) throws Exception {
     errOutBytes.reset();
+    sysOutBytes.reset();
     LOG.info("Running: NodeAttributesCLI " + Joiner.on(" ").join(args));
     int ret = nodeAttributesCLI.run(args);
     errOutput = new String(errOutBytes.toByteArray(), Charsets.UTF_8);
+    sysOutput = new String(sysOutBytes.toByteArray(), Charsets.UTF_8);
     LOG.info("Err_output:\n" + errOutput);
+    LOG.info("Sys_output:\n" + sysOutput);
     return ret;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index 20c9603..a600895 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.yarn.client.cli;
 
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyInt;
@@ -1544,8 +1546,8 @@ public class TestYarnCLI {
   public void testNodeStatus() throws Exception {
     NodeId nodeId = NodeId.newInstance("host0", 0);
     NodeCLI cli = new NodeCLI();
-    when(client.getNodeReports()).thenReturn(
-                    getNodeReports(3, NodeState.RUNNING, false));
+    when(client.getNodeReports())
+        .thenReturn(getNodeReports(3, NodeState.RUNNING, false, false, false));
     cli.setClient(client);
     cli.setSysOutPrintStream(sysOut);
     cli.setSysErrPrintStream(sysErr);
@@ -1568,6 +1570,8 @@ public class TestYarnCLI {
     pw.println("\tCPU-Used : 0 vcores");
     pw.println("\tCPU-Capacity : 0 vcores");
     pw.println("\tNode-Labels : a,b,c,x,y,z");
+    pw.println("\tNode Attributes : rm.yarn.io/GPU(STRING)=ARM");
+    pw.println("\t                  rm.yarn.io/CPU(STRING)=ARM");
     pw.println("\tResource Utilization by Node : PMem:2048 MB, VMem:4096 MB, VCores:8.0");
     pw.println("\tResource Utilization by Containers : PMem:1024 MB, VMem:2048 MB, VCores:4.0");
     pw.close();
@@ -1604,6 +1608,7 @@ public class TestYarnCLI {
     pw.println("\tCPU-Used : 0 vcores");
     pw.println("\tCPU-Capacity : 0 vcores");
     pw.println("\tNode-Labels : ");
+    pw.println("\tNode Attributes : ");
     pw.println("\tResource Utilization by Node : PMem:2048 MB, VMem:4096 MB, VCores:8.0");
     pw.println("\tResource Utilization by Containers : PMem:1024 MB, VMem:2048 MB, VCores:4.0");
     pw.close();
@@ -1616,8 +1621,8 @@ public class TestYarnCLI {
   public void testNodeStatusWithEmptyResourceUtilization() throws Exception {
     NodeId nodeId = NodeId.newInstance("host0", 0);
     NodeCLI cli = new NodeCLI();
-    when(client.getNodeReports()).thenReturn(
-                    getNodeReports(3, NodeState.RUNNING, false, true));
+    when(client.getNodeReports())
+        .thenReturn(getNodeReports(3, NodeState.RUNNING, false, true, true));
     cli.setClient(client);
     cli.setSysOutPrintStream(sysOut);
     cli.setSysErrPrintStream(sysErr);
@@ -1640,6 +1645,7 @@ public class TestYarnCLI {
     pw.println("\tCPU-Used : 0 vcores");
     pw.println("\tCPU-Capacity : 0 vcores");
     pw.println("\tNode-Labels : a,b,c,x,y,z");
+    pw.println("\tNode Attributes : ");
     pw.println("\tResource Utilization by Node : ");
     pw.println("\tResource Utilization by Containers : ");
     pw.close();
@@ -2049,18 +2055,20 @@ public class TestYarnCLI {
     cli.run(new String[] { "application" });
     verify(sysErr).println("Invalid Command Usage : ");
   }
-  
+
   private List<NodeReport> getNodeReports(int noOfNodes, NodeState state) {
-    return getNodeReports(noOfNodes, state, true, false);
+    return getNodeReports(noOfNodes, state, true, false, true);
   }
 
   private List<NodeReport> getNodeReports(int noOfNodes, NodeState state,
-      boolean emptyNodeLabel) {
-    return getNodeReports(noOfNodes, state, emptyNodeLabel, false);
+      boolean emptyNodeLabel, boolean emptyAttributes) {
+    return getNodeReports(noOfNodes, state, emptyNodeLabel, false,
+        emptyAttributes);
   }
 
   private List<NodeReport> getNodeReports(int noOfNodes, NodeState state,
-      boolean emptyNodeLabel, boolean emptyResourceUtilization) {
+      boolean emptyNodeLabel, boolean emptyResourceUtilization,
+      boolean emptyAttributes) {
     List<NodeReport> nodeReports = new ArrayList<NodeReport>();
 
     for (int i = 0; i < noOfNodes; i++) {
@@ -2082,6 +2090,11 @@ public class TestYarnCLI {
         nodeReport.setAggregatedContainersUtilization(containersUtilization);
         nodeReport.setNodeUtilization(nodeUtilization);
       }
+      if (!emptyAttributes) {
+        nodeReport.setNodeAttributes(ImmutableSet.of(NodeAttribute
+                .newInstance("GPU", NodeAttributeType.STRING, "ARM"),
+            NodeAttribute.newInstance("CPU", NodeAttributeType.STRING, "ARM")));
+      }
       nodeReports.add(nodeReport);
     }
     return nodeReports;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributeInfoPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributeInfoPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributeInfoPBImpl.java
index bff6335..e2db568 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributeInfoPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributeInfoPBImpl.java
@@ -130,14 +130,18 @@ public class NodeAttributeInfoPBImpl extends NodeAttributeInfo {
     }
     if (obj instanceof NodeAttributeInfo) {
       NodeAttributeInfo other = (NodeAttributeInfo) obj;
-      getAttributeKey().equals(other.getAttributeKey());
-      return true;
+      return getAttributeKey().equals(other.getAttributeKey());
     }
     return false;
   }
 
   @Override
   public String toString() {
-    return getAttributeKey().toString() + ":Type-" + getAttributeType();
+    StringBuilder strBuilder = new StringBuilder();
+    NodeAttributeKey key = this.getAttributeKey();
+    strBuilder.append(key.getAttributePrefix()).append("/")
+        .append(key.getAttributeName()).append("(")
+        .append(this.getAttributeType()).append(")");
+    return strBuilder.toString();
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[26/29] hadoop git commit: YARN-8103. Add CLI interface to query node attributes. Contributed by Bibin A Chundatt.

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
index 0002567..689e2e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
@@ -152,15 +152,19 @@ public class NodeAttributePBImpl extends NodeAttribute {
     }
     if (obj instanceof NodeAttribute) {
       NodeAttribute other = (NodeAttribute) obj;
-      getAttributeKey().equals(other.getAttributeKey());
-      return true;
+      return getAttributeKey().equals(other.getAttributeKey());
     }
     return false;
   }
 
   @Override
   public String toString() {
-    return getAttributeKey().toString() + ":Value-" + getAttributeValue()
-        + ":Type-" + getAttributeType();
+    StringBuilder strBuilder = new StringBuilder();
+    NodeAttributeKey key = this.getAttributeKey();
+    strBuilder.append(key.getAttributePrefix()).append("/")
+        .append(key.getAttributeName()).append("(")
+        .append(this.getAttributeType()).append(")=")
+        .append(this.getAttributeValue());
+    return strBuilder.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeReportPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeReportPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeReportPBImpl.java
index ced588d..7d5a06a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeReportPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeReportPBImpl.java
@@ -18,17 +18,21 @@
 
 package org.apache.hadoop.yarn.api.records.impl.pb;
 
+import java.util.ArrayList;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.NodeUpdateType;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceUtilization;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProtoOrBuilder;
@@ -50,6 +54,7 @@ public class NodeReportPBImpl extends NodeReport {
   private ResourceUtilization containersUtilization = null;
   private ResourceUtilization nodeUtilization = null;
   Set<String> labels;
+  private Set<NodeAttribute> nodeAttributes;
 
   public NodeReportPBImpl() {
     builder = NodeReportProto.newBuilder();
@@ -268,6 +273,14 @@ public class NodeReportPBImpl extends NodeReport {
       builder.clearNodeLabels();
       builder.addAllNodeLabels(this.labels);
     }
+    if (this.nodeAttributes != null) {
+      builder.clearNodeAttributes();
+      List<NodeAttributeProto> attrList = new ArrayList<>();
+      for (NodeAttribute attr : this.nodeAttributes) {
+        attrList.add(convertToProtoFormat(attr));
+      }
+      builder.addAllNodeAttributes(attrList);
+    }
     if (this.nodeUtilization != null
         && !((ResourceUtilizationPBImpl) this.nodeUtilization).getProto()
             .equals(builder.getNodeUtilization())) {
@@ -306,7 +319,16 @@ public class NodeReportPBImpl extends NodeReport {
   private NodeIdProto convertToProtoFormat(NodeId nodeId) {
     return ((NodeIdPBImpl) nodeId).getProto();
   }
-  
+
+  private NodeAttributeProto convertToProtoFormat(NodeAttribute nodeAttr) {
+    return ((NodeAttributePBImpl) nodeAttr).getProto();
+  }
+
+  private NodeAttributePBImpl convertFromProtoFormat(
+      NodeAttributeProto nodeAttr) {
+    return new NodeAttributePBImpl(nodeAttr);
+  }
+
   private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
     return new ResourcePBImpl(p);
   }
@@ -427,4 +449,24 @@ public class NodeReportPBImpl extends NodeReport {
     }
     builder.setNodeUpdateType(ProtoUtils.convertToProtoFormat(nodeUpdateType));
   }
+
+  @Override
+  public void setNodeAttributes(Set<NodeAttribute> nodeAttrs) {
+    maybeInitBuilder();
+    builder.clearNodeAttributes();
+    this.nodeAttributes = nodeAttrs;
+  }
+
+  @Override
+  public Set<NodeAttribute> getNodeAttributes() {
+    if (nodeAttributes != null) {
+      return nodeAttributes;
+    }
+    NodeReportProtoOrBuilder p = viaProto ? proto : builder;
+    this.nodeAttributes = new HashSet<>();
+    for (NodeAttributeProto nattrProto : p.getNodeAttributesList()) {
+      nodeAttributes.add(convertFromProtoFormat(nattrProto));
+    }
+    return nodeAttributes;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
index b6145c9..b9f35a5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -199,7 +200,7 @@ public class BuilderUtils {
       NodeUpdateType nodeUpdateType) {
     return newNodeReport(nodeId, nodeState, httpAddress, rackName, used,
         capability, numContainers, healthReport, lastHealthReportTime,
-        nodeLabels, null, null, decommissioningTimeout, nodeUpdateType);
+        nodeLabels, null, null, decommissioningTimeout, nodeUpdateType, null);
   }
 
   public static NodeReport newNodeReport(NodeId nodeId, NodeState nodeState,
@@ -207,7 +208,7 @@ public class BuilderUtils {
       int numContainers, String healthReport, long lastHealthReportTime,
       Set<String> nodeLabels, ResourceUtilization containersUtilization,
       ResourceUtilization nodeUtilization, Integer decommissioningTimeout,
-      NodeUpdateType nodeUpdateType) {
+      NodeUpdateType nodeUpdateType, Set<NodeAttribute> attrs) {
     NodeReport nodeReport = recordFactory.newRecordInstance(NodeReport.class);
     nodeReport.setNodeId(nodeId);
     nodeReport.setNodeState(nodeState);
@@ -223,6 +224,7 @@ public class BuilderUtils {
     nodeReport.setNodeUtilization(nodeUtilization);
     nodeReport.setDecommissioningTimeout(decommissioningTimeout);
     nodeReport.setNodeUpdateType(nodeUpdateType);
+    nodeReport.setNodeAttributes(attrs);
     return nodeReport;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index 77f8f66..601917a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -983,12 +983,11 @@ public class AdminService extends CompositeService implements
     List<NodeToAttributes> nodesToAttributes = request.getNodesToAttributes();
     boolean failOnUnknownNodes = request.getFailOnUnknownNodes();
 
-    Map<String, Set<NodeAttribute>> nodeAttributeMapping =
-        validateAndFetch(nodesToAttributes, failOnUnknownNodes);
-
     NodeAttributesManager nodeAttributesManager =
         rm.getRMContext().getNodeAttributesManager();
     try {
+      Map<String, Set<NodeAttribute>> nodeAttributeMapping =
+          validateAndFetch(nodesToAttributes, failOnUnknownNodes);
       switch (request.getOperation()) {
       case ADD:
         nodeAttributesManager.addNodeAttributes(nodeAttributeMapping);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index ad796f6..4075332 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -1050,8 +1050,9 @@ public class ClientRMService extends AbstractService implements
     if (schedulerNodeReport != null) {
       used = schedulerNodeReport.getUsedResource();
       numContainers = schedulerNodeReport.getNumContainers();
-    } 
+    }
 
+    Set<NodeAttribute> attrs = rmNode.getAllNodeAttributes();
     NodeReport report =
         BuilderUtils.newNodeReport(rmNode.getNodeID(), rmNode.getState(),
             rmNode.getHttpAddress(), rmNode.getRackName(), used,
@@ -1059,7 +1060,7 @@ public class ClientRMService extends AbstractService implements
             rmNode.getHealthReport(), rmNode.getLastHealthReportTime(),
             rmNode.getNodeLabels(), rmNode.getAggregatedContainersUtilization(),
             rmNode.getNodeUtilization(), rmNode.getDecommissioningTimeout(),
-            null);
+            null, attrs);
 
     return report;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index cbb5ecf..9c4d594 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -673,10 +673,6 @@ public class ResourceTrackerService extends AbstractService implements
         this.rmContext.getNodeAttributesManager()
             .replaceNodeAttributes(NodeAttribute.PREFIX_DISTRIBUTED,
                 ImmutableMap.of(nodeId.getHost(), nodeAttributes));
-
-        // Update node attributes to RMNode
-        rmNode.setNodeAttributes(NodeAttribute.PREFIX_DISTRIBUTED,
-            nodeAttributes);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
index 33e5ef4..c77d29c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
@@ -197,15 +197,8 @@ public interface RMNode {
    */
   RMContext getRMContext();
 
- /**
-   * Sets node attributes per prefix.
-   * @param prefix node attribute prefix
-   * @param nodeAttributes node attributes
-   */
-  void setNodeAttributes(String prefix, Set<NodeAttribute> nodeAttributes);
-
   /**
-   * @return all node attributes grouped by their prefix as a map.
+   * @return all node attributes as a Set.
    */
-  Map<String, Set<NodeAttribute>> getAllNodeAttributes();
+  Set<NodeAttribute> getAllNodeAttributes();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
index e52b621..65a0c20 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
@@ -59,7 +59,9 @@ import org.apache.hadoop.yarn.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.nodelabels.AttributeValue;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
@@ -186,9 +188,6 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   private NodeHeartbeatResponse latestNodeHeartBeatResponse = recordFactory
       .newRecordInstance(NodeHeartbeatResponse.class);
 
-  // Node attributes, store by prefix
-  private Map<String, Set<NodeAttribute>> nodeAttributes = new HashMap<>();
-
   private static final StateMachineFactory<RMNodeImpl,
                                            NodeState,
                                            RMNodeEventType,
@@ -1552,13 +1551,10 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   }
 
   @Override
-  public void setNodeAttributes(String prefix,
-      Set<NodeAttribute> nodeAttributeSet) {
-    this.nodeAttributes.put(prefix, nodeAttributeSet);
-  }
-
-  @Override
-  public Map<String, Set<NodeAttribute>> getAllNodeAttributes() {
-    return this.nodeAttributes;
+  public Set<NodeAttribute> getAllNodeAttributes() {
+    NodeAttributesManager attrMgr = context.getNodeAttributesManager();
+    Map<NodeAttribute, AttributeValue> nodeattrs =
+        attrMgr.getAttributesForNode(hostName);
+    return nodeattrs.keySet();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/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
----------------------------------------------------------------------
diff --git a/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 b/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
index ecf66b9..7e5d20a 100644
--- a/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
+++ b/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
@@ -116,16 +116,11 @@ public class NodeInfo {
     }
 
     // add attributes
-    Map<String, Set<NodeAttribute>> nodeAttributes =
-        ni.getAllNodeAttributes();
+    Set<NodeAttribute> attrs = ni.getAllNodeAttributes();
     nodeAttributesInfo = new NodeAttributesInfo();
-    if (nodeAttributes != null) {
-      for (Set<NodeAttribute> attrs : nodeAttributes.values()) {
-        for (NodeAttribute attribute : attrs) {
-          NodeAttributeInfo info = new NodeAttributeInfo(attribute);
-          this.nodeAttributesInfo.addNodeAttributeInfo(info);
-        }
-      }
+    for (NodeAttribute attribute : attrs) {
+      NodeAttributeInfo info = new NodeAttributeInfo(attribute);
+      this.nodeAttributesInfo.addNodeAttributeInfo(info);
     }
 
     // add allocation tags

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
index b6c0cc1..7c822b4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
@@ -292,8 +292,8 @@ public class MockNodes {
     }
 
     @Override
-    public Map<String, Set<NodeAttribute>> getAllNodeAttributes() {
-      return null;
+    public Set<NodeAttribute> getAllNodeAttributes() {
+      return Collections.emptySet();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb08543c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
index 90945c2..a1b95b8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
@@ -1611,9 +1611,10 @@ public class TestRMAdminService {
     try {
       rm.adminService.mapAttributesToNodes(request);
       fail("host5 is not a valid node, It should have failed");
-    } catch (Exception ex) {
+    } catch (YarnException ex) {
       Assert.assertEquals("Exception Message is not as desired",
-          " Following nodes does not exist : [host5]", ex.getMessage());
+          " Following nodes does not exist : [host5]",
+          ex.getCause().getMessage());
     }
 
     request =
@@ -1633,10 +1634,10 @@ public class TestRMAdminService {
       // against hostname hence the message as : nodes does not exist.
       rm.adminService.mapAttributesToNodes(request);
       fail("host with the port should fail as only hostnames are validated");
-    } catch (Exception ex) {
+    } catch (YarnException ex) {
       Assert.assertEquals("Exception Message is not as desired",
           " Following nodes does not exist : [host4:8889, host2:8889]",
-          ex.getMessage());
+          ex.getCause().getMessage());
     }
 
     request =
@@ -1669,11 +1670,10 @@ public class TestRMAdminService {
     try {
       rm.adminService.mapAttributesToNodes(request);
       fail("This operation should fail as prefix should be \"nm.yarn.io\".");
-    } catch (Exception ex) {
+    } catch (YarnException ex) {
       Assert.assertEquals("Exception Message is not as desired",
           "Invalid Attribute Mapping for the node host5. Prefix should be "
-              + "rm.yarn.io",
-          ex.getMessage());
+              + "rm.yarn.io", ex.getCause().getMessage());
     }
 
     rm.close();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[13/29] hadoop git commit: YARN-7892. Revisit NodeAttribute class structure. Contributed by Naganarasimha G R.

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 9f3e925..682d6ec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.yarn.api;
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.commons.lang3.Range;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
@@ -112,7 +113,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestP
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
-import org.apache.hadoop.yarn.api.records.CollectorInfo;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -120,6 +120,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ApplicationTimeout;
+import org.apache.hadoop.yarn.api.records.CollectorInfo;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
@@ -131,9 +132,12 @@ import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.api.records.NodeAttributeInfo;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeToAttributeValue;
 import org.apache.hadoop.yarn.api.records.PreemptionContainer;
 import org.apache.hadoop.yarn.api.records.PreemptionContract;
 import org.apache.hadoop.yarn.api.records.PreemptionMessage;
@@ -152,8 +156,8 @@ import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.ReservationRequests;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceAllocationRequest;
-import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceSizing;
@@ -183,10 +187,13 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ExecutionTypeRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributeKeyPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributeInfoPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeLabelPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeReportPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeToAttributeValuePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.PreemptionContainerPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.PreemptionContractPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.PreemptionMessagePBImpl;
@@ -222,10 +229,14 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ContainerRetryContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeKeyProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeInfoProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeLabelProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeToAttributeValueProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeToAttributesProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionContainerProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionContractProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionMessageProto;
@@ -243,7 +254,6 @@ import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StrictPreemptionContractProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.URLProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnClusterMetricsProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.NodeToAttributesProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddToClusterNodeLabelsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddToClusterNodeLabelsResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.CheckForDecommissioningNodesRequestProto;
@@ -274,6 +284,7 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllResourceProfilesResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptsRequestProto;
@@ -304,6 +315,8 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProto;
@@ -328,9 +341,6 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationResponseProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllResourceProfilesResponseProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileResponseProto;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl;
@@ -366,7 +376,6 @@ import org.junit.Ignore;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableSet;
-import java.util.Arrays;
 
 /**
  * Test class for YARN API protocol records.
@@ -450,10 +459,12 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
     generateByNewInstance(SchedulingRequest.class);
     generateByNewInstance(RejectedSchedulingRequest.class);
     //for Node attribute support
+    generateByNewInstance(NodeAttributeKey.class);
     generateByNewInstance(NodeAttribute.class);
     generateByNewInstance(NodeToAttributes.class);
+    generateByNewInstance(NodeToAttributeValue.class);
+    generateByNewInstance(NodeAttributeInfo.class);
     generateByNewInstance(NodesToAttributesMappingRequest.class);
- 
  }
 
   @Test
@@ -1250,11 +1261,29 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
   }
 
   @Test
+  public void testNodeAttributeKeyPBImpl() throws Exception {
+    validatePBImplRecord(NodeAttributeKeyPBImpl.class,
+        NodeAttributeKeyProto.class);
+  }
+
+  @Test
+  public void testNodeToAttributeValuePBImpl() throws Exception {
+    validatePBImplRecord(NodeToAttributeValuePBImpl.class,
+        NodeToAttributeValueProto.class);
+  }
+
+  @Test
   public void testNodeAttributePBImpl() throws Exception {
     validatePBImplRecord(NodeAttributePBImpl.class, NodeAttributeProto.class);
   }
 
   @Test
+  public void testNodeAttributeInfoPBImpl() throws Exception {
+    validatePBImplRecord(NodeAttributeInfoPBImpl.class,
+        NodeAttributeInfoProto.class);
+  }
+
+  @Test
   public void testNodeToAttributesPBImpl() throws Exception {
     validatePBImplRecord(NodeToAttributesPBImpl.class,
         NodeToAttributesProto.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java
index d4384b4..bad74d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeAttributesProvider.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
 import org.apache.hadoop.yarn.api.records.NodeAttributeType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.junit.BeforeClass;
@@ -120,7 +121,8 @@ public class TestConfigurationNodeAttributesProvider {
     while(times>0) {
       Set<NodeAttribute> current = spyProvider.getDescriptors();
       Assert.assertEquals(1, current.size());
-      String attributeName = current.iterator().next().getAttributeName();
+      String attributeName =
+          current.iterator().next().getAttributeKey().getAttributeName();
       if ("host".equals(attributeName)){
         numOfOldValue++;
       } else if ("os".equals(attributeName)) {
@@ -173,7 +175,7 @@ public class TestConfigurationNodeAttributesProvider {
       GenericTestUtils.waitFor(() -> {
         Set<NodeAttribute> attributes = spyProvider.getDescriptors();
         return "os".equalsIgnoreCase(attributes
-            .iterator().next().getAttributeName());
+            .iterator().next().getAttributeKey().getAttributeName());
       }, 500, 1000);
     } catch (Exception e) {
       // Make sure we get the timeout exception.
@@ -204,21 +206,22 @@ public class TestConfigurationNodeAttributesProvider {
     Iterator<NodeAttribute> ait = attributes.iterator();
 
     while(ait.hasNext()) {
-      NodeAttribute at = ait.next();
+      NodeAttribute attr = ait.next();
+      NodeAttributeKey at = attr.getAttributeKey();
       if (at.getAttributeName().equals("hostname")) {
         Assert.assertEquals("hostname", at.getAttributeName());
         Assert.assertEquals(NodeAttribute.PREFIX_DISTRIBUTED,
             at.getAttributePrefix());
         Assert.assertEquals(NodeAttributeType.STRING,
-            at.getAttributeType());
-        Assert.assertEquals("host1234", at.getAttributeValue());
+            attr.getAttributeType());
+        Assert.assertEquals("host1234", attr.getAttributeValue());
       } else if (at.getAttributeName().equals("uptime")) {
         Assert.assertEquals("uptime", at.getAttributeName());
         Assert.assertEquals(NodeAttribute.PREFIX_DISTRIBUTED,
             at.getAttributePrefix());
         Assert.assertEquals(NodeAttributeType.STRING,
-            at.getAttributeType());
-        Assert.assertEquals("321543", at.getAttributeValue());
+            attr.getAttributeType());
+        Assert.assertEquals("321543", attr.getAttributeValue());
       } else {
         Assert.fail("Unexpected attribute");
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java
index f764626..3e2e161 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java
@@ -129,7 +129,7 @@ public class TestScriptBasedNodeAttributesProvider {
         .getDescriptors().iterator();
     while (it.hasNext()) {
       NodeAttribute att = it.next();
-      switch (att.getAttributeName()) {
+      switch (att.getAttributeKey().getAttributeName()) {
       case "host":
         Assert.assertEquals(NodeAttributeType.STRING, att.getAttributeType());
         Assert.assertEquals("host1234", att.getAttributeValue());
@@ -143,7 +143,8 @@ public class TestScriptBasedNodeAttributesProvider {
         Assert.assertEquals("10.0.0.1", att.getAttributeValue());
         break;
       default:
-        Assert.fail("Unexpected attribute name " + att.getAttributeName());
+        Assert.fail("Unexpected attribute name "
+            + att.getAttributeKey().getAttributeName());
         break;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index 12f8aaf..77f8f66 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -1035,7 +1035,7 @@ public class AdminService extends CompositeService implements
       List<NodeAttribute> nodeAttributes = nodeToAttributes.getNodeAttributes();
       if (!nodeAttributes.stream()
           .allMatch(nodeAttribute -> NodeAttribute.PREFIX_CENTRALIZED
-              .equals(nodeAttribute.getAttributePrefix()))) {
+              .equals(nodeAttribute.getAttributeKey().getAttributePrefix()))) {
         throw new IOException("Invalid Attribute Mapping for the node " + node
             + ". Prefix should be " + NodeAttribute.PREFIX_CENTRALIZED);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index 3f24355..ad796f6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -29,12 +29,14 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
 
 import org.apache.commons.cli.UnrecognizedOptionException;
 import org.apache.commons.lang3.Range;
@@ -134,8 +136,11 @@ import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.api.records.NodeAttributeInfo;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.NodeToAttributeValue;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
@@ -155,6 +160,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.nodelabels.AttributeValue;
 import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
@@ -1851,9 +1857,23 @@ public class ClientRMService extends AbstractService implements
       GetAttributesToNodesRequest request) throws YarnException, IOException {
     NodeAttributesManager attributesManager =
         rmContext.getNodeAttributesManager();
-    GetAttributesToNodesResponse response = GetAttributesToNodesResponse
-        .newInstance(attributesManager
-            .getAttributesToNodes(request.getNodeAttributes()));
+    Map<NodeAttributeKey, List<NodeToAttributeValue>> attrToNodesWithStrVal =
+        new HashMap<>();
+    Map<NodeAttributeKey, Map<String, AttributeValue>> attributesToNodes =
+        attributesManager.getAttributesToNodes(request.getNodeAttributes());
+    for (Map.Entry<NodeAttributeKey, Map<String, AttributeValue>> attrib :
+          attributesToNodes.entrySet()) {
+      Map<String, AttributeValue> nodesToVal = attrib.getValue();
+      List<NodeToAttributeValue> nodeToAttrValList = new ArrayList<>();
+      for (Map.Entry<String, AttributeValue> nodeToVal : nodesToVal
+          .entrySet()) {
+        nodeToAttrValList.add(NodeToAttributeValue
+            .newInstance(nodeToVal.getKey(), nodeToVal.getValue().getValue()));
+      }
+      attrToNodesWithStrVal.put(attrib.getKey(), nodeToAttrValList);
+    }
+    GetAttributesToNodesResponse response =
+        GetAttributesToNodesResponse.newInstance(attrToNodesWithStrVal);
     return response;
   }
 
@@ -1865,8 +1885,11 @@ public class ClientRMService extends AbstractService implements
         rmContext.getNodeAttributesManager();
     Set<NodeAttribute> attributes =
         attributesManager.getClusterNodeAttributes(null);
+
     GetClusterNodeAttributesResponse response =
-        GetClusterNodeAttributesResponse.newInstance(attributes);
+        GetClusterNodeAttributesResponse.newInstance(
+            attributes.stream().map(attr -> NodeAttributeInfo.newInstance(attr))
+                .collect(Collectors.toSet()));
     return response;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index 4f4400f..cbb5ecf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -660,7 +660,7 @@ public class ResourceTrackerService extends AbstractService implements
       // Validate attributes
       if (!nodeAttributes.stream().allMatch(
           nodeAttribute -> NodeAttribute.PREFIX_DISTRIBUTED
-              .equals(nodeAttribute.getAttributePrefix()))) {
+              .equals(nodeAttribute.getAttributeKey().getAttributePrefix()))) {
         // All attributes must be in same prefix: nm.yarn.io.
         // Since we have the checks in NM to make sure attributes reported
         // in HB are with correct prefix, so it should not reach here.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
index 1645602..6b176d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
@@ -19,21 +19,20 @@
 package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentHashMap.KeySetView;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
-import java.util.ArrayList;
-import java.util.List;
 
 import com.google.common.base.Strings;
 import org.apache.commons.lang3.StringUtils;
@@ -42,6 +41,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -58,6 +58,8 @@ import org.apache.hadoop.yarn.nodelabels.StringAttributeValue;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
 
+import com.google.common.base.Strings;
+
 /**
  * Manager holding the attributes to Labels.
  */
@@ -75,8 +77,8 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
 
   // TODO may be we can have a better collection here.
   // this will be updated to get the attributeName to NM mapping
-  private ConcurrentHashMap<NodeAttribute, RMNodeAttribute> clusterAttributes =
-      new ConcurrentHashMap<>();
+  private ConcurrentHashMap<NodeAttributeKey, RMNodeAttribute> clusterAttributes
+      = new ConcurrentHashMap<>();
 
   // hostname -> (Map (attributeName -> NodeAttribute))
   // Instead of NodeAttribute, plan to have it in future as AttributeValue
@@ -149,7 +151,7 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   private void internalUpdateAttributesOnNodes(
       Map<String, Map<NodeAttribute, AttributeValue>> nodeAttributeMapping,
       AttributeMappingOperationType op,
-      Map<NodeAttribute, RMNodeAttribute> newAttributesToBeAdded,
+      Map<NodeAttributeKey, RMNodeAttribute> newAttributesToBeAdded,
       String attributePrefix) {
     try {
       writeLock.lock();
@@ -210,13 +212,14 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   private void removeNodeFromAttributes(String nodeHost,
       Set<NodeAttribute> attributeMappings) {
     for (NodeAttribute rmAttribute : attributeMappings) {
-      RMNodeAttribute host = clusterAttributes.get(rmAttribute);
+      RMNodeAttribute host =
+          clusterAttributes.get(rmAttribute.getAttributeKey());
       if (host != null) {
         host.removeNode(nodeHost);
         // If there is no other host has such attribute,
         // remove it from the global mapping.
         if (host.getAssociatedNodeIds().isEmpty()) {
-          clusterAttributes.remove(rmAttribute);
+          clusterAttributes.remove(rmAttribute.getAttributeKey());
         }
       }
     }
@@ -224,12 +227,16 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
 
   private void addNodeToAttribute(String nodeHost,
       Map<NodeAttribute, AttributeValue> attributeMappings) {
-    for (NodeAttribute attribute : attributeMappings.keySet()) {
-      RMNodeAttribute rmNodeAttribute = clusterAttributes.get(attribute);
+    for (Entry<NodeAttribute, AttributeValue> attributeEntry : attributeMappings
+        .entrySet()) {
+
+      RMNodeAttribute rmNodeAttribute =
+          clusterAttributes.get(attributeEntry.getKey().getAttributeKey());
       if (rmNodeAttribute != null) {
-        rmNodeAttribute.addNode(nodeHost);
+        rmNodeAttribute.addNode(nodeHost, attributeEntry.getValue());
       } else {
-        clusterAttributes.put(attribute, new RMNodeAttribute(attribute));
+        clusterAttributes.put(attributeEntry.getKey().getAttributeKey(),
+            new RMNodeAttribute(attributeEntry.getKey()));
       }
     }
   }
@@ -257,7 +264,7 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
    */
   protected Map<String, Map<NodeAttribute, AttributeValue>> validate(
       Map<String, Set<NodeAttribute>> nodeAttributeMapping,
-      Map<NodeAttribute, RMNodeAttribute> newAttributesToBeAdded,
+      Map<NodeAttributeKey, RMNodeAttribute> newAttributesToBeAdded,
       boolean isRemoveOperation) throws IOException {
     Map<String, Map<NodeAttribute, AttributeValue>> nodeToAttributesMap =
         new TreeMap<>();
@@ -274,19 +281,21 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
 
       // validate for attributes
       for (NodeAttribute attribute : nodeToAttrMappingEntry.getValue()) {
-        String attributeName = attribute.getAttributeName().trim();
+        NodeAttributeKey attributeKey = attribute.getAttributeKey();
+        String attributeName = attributeKey.getAttributeName().trim();
         NodeLabelUtil.checkAndThrowLabelName(attributeName);
         NodeLabelUtil
-            .checkAndThrowAttributePrefix(attribute.getAttributePrefix());
+            .checkAndThrowAttributePrefix(attributeKey.getAttributePrefix());
 
         // ensure trimmed values are set back
-        attribute.setAttributeName(attributeName);
-        attribute.setAttributePrefix(attribute.getAttributePrefix().trim());
+        attributeKey.setAttributeName(attributeName);
+        attributeKey
+            .setAttributePrefix(attributeKey.getAttributePrefix().trim());
 
         // verify for type against prefix/attributeName
         if (validateForAttributeTypeMismatch(isRemoveOperation, attribute,
             newAttributesToBeAdded)) {
-          newAttributesToBeAdded.put(attribute,
+          newAttributesToBeAdded.put(attribute.getAttributeKey(),
               new RMNodeAttribute(attribute));
         }
         // TODO type based value setting needs to be done using a factory
@@ -310,9 +319,11 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
    */
   private boolean validateForAttributeTypeMismatch(boolean isRemoveOperation,
       NodeAttribute attribute,
-      Map<NodeAttribute, RMNodeAttribute> newAttributes)
+      Map<NodeAttributeKey, RMNodeAttribute> newAttributes)
       throws IOException {
-    if (isRemoveOperation && !clusterAttributes.containsKey(attribute)) {
+    NodeAttributeKey attributeKey = attribute.getAttributeKey();
+    if (isRemoveOperation
+        && !clusterAttributes.containsKey(attributeKey)) {
       // no need to validate anything as its remove operation and attribute
       // doesn't exist.
       return false; // no need to add as its remove operation
@@ -320,10 +331,10 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
       // already existing or attribute is mapped to another Node in the
       // current command, then check whether the attribute type is matching
       NodeAttribute existingAttribute =
-          (clusterAttributes.containsKey((attribute))
-              ? clusterAttributes.get(attribute).getAttribute()
-              : (newAttributes.containsKey(attribute)
-                  ? newAttributes.get(attribute).getAttribute()
+          (clusterAttributes.containsKey(attributeKey)
+              ? clusterAttributes.get(attributeKey).getAttribute()
+              : (newAttributes.containsKey(attributeKey)
+                  ? newAttributes.get(attributeKey).getAttribute()
                   : null));
       if (existingAttribute == null) {
         return true;
@@ -331,7 +342,7 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
           .getAttributeType()) {
         throw new IOException("Attribute name - type is not matching with "
             + "already configured mapping for the attribute "
-            + attribute.getAttributeName() + " existing : "
+            + attributeKey + " existing : "
             + existingAttribute.getAttributeType() + ", new :"
             + attribute.getAttributeType());
       }
@@ -347,37 +358,39 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   }
 
   @Override
-  public Set<NodeAttribute> getClusterNodeAttributes(Set<String> prefix) {
+  public Set<NodeAttribute> getClusterNodeAttributes(
+      Set<String> prefix) {
     Set<NodeAttribute> attributes = new HashSet<>();
-    KeySetView<NodeAttribute, RMNodeAttribute> allAttributes =
-        clusterAttributes.keySet();
+    Set<Entry<NodeAttributeKey, RMNodeAttribute>> allAttributes =
+        clusterAttributes.entrySet();
     // Return all if prefix is not given.
-    if (prefix == null || prefix.isEmpty()) {
-      attributes.addAll(allAttributes);
-      return attributes;
-    }
+    boolean forAllPrefix = prefix == null || prefix.isEmpty();
     // Try search attributes by prefix and return valid ones.
-    Iterator<NodeAttribute> iterator = allAttributes.iterator();
+    Iterator<Entry<NodeAttributeKey, RMNodeAttribute>> iterator =
+        allAttributes.iterator();
     while (iterator.hasNext()) {
-      NodeAttribute current = iterator.next();
-      if (prefix.contains(current.getAttributePrefix())) {
-        attributes.add(current);
+      Entry<NodeAttributeKey, RMNodeAttribute> current = iterator.next();
+      NodeAttributeKey attrID = current.getKey();
+      RMNodeAttribute rmAttr = current.getValue();
+      if (forAllPrefix || prefix.contains(attrID.getAttributePrefix())) {
+        attributes.add(rmAttr.getAttribute());
       }
     }
     return attributes;
   }
 
   @Override
-  public Map<NodeAttribute, Set<String>> getAttributesToNodes(
-      Set<NodeAttribute> attributes) {
+  public Map<NodeAttributeKey, Map<String, AttributeValue>> getAttributesToNodes(
+      Set<NodeAttributeKey> attributes) {
     try {
       readLock.lock();
       boolean fetchAllAttributes = (attributes == null || attributes.isEmpty());
-      Map<NodeAttribute, Set<String>> attributesToNodes = new HashMap<>();
-      for (Entry<NodeAttribute, RMNodeAttribute> attributeEntry :
+      Map<NodeAttributeKey, Map<String, AttributeValue>> attributesToNodes =
+          new HashMap<>();
+      for (Entry<NodeAttributeKey, RMNodeAttribute> attributeEntry :
           clusterAttributes.entrySet()) {
-        if (fetchAllAttributes || attributes
-            .contains(attributeEntry.getKey())) {
+        if (fetchAllAttributes
+            || attributes.contains(attributeEntry.getKey())) {
           attributesToNodes.put(attributeEntry.getKey(),
               attributeEntry.getValue().getAssociatedNodeIds());
         }
@@ -391,8 +404,8 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   public Resource getResourceByAttribute(NodeAttribute attribute) {
     try {
       readLock.lock();
-      return clusterAttributes.containsKey(attribute)
-          ? clusterAttributes.get(attribute).getResource()
+      return clusterAttributes.containsKey(attribute.getAttributeKey())
+          ? clusterAttributes.get(attribute.getAttributeKey()).getResource()
           : Resource.newInstance(0, 0);
     } finally {
       readLock.unlock();
@@ -425,7 +438,8 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
           attrs = new ArrayList<>();
           for (Entry<NodeAttribute, AttributeValue> nodeAttr : v.attributes
               .entrySet()) {
-            if (prefix.contains(nodeAttr.getKey().getAttributePrefix())) {
+            if (prefix.contains(
+                nodeAttr.getKey().getAttributeKey().getAttributePrefix())) {
               attrs.add(nodeAttr.getKey());
             }
           }
@@ -473,7 +487,7 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
       }
       host.activateNode(resource);
       for (NodeAttribute attribute : host.getAttributes().keySet()) {
-        clusterAttributes.get(attribute).removeNode(resource);
+        clusterAttributes.get(attribute.getAttributeKey()).removeNode(resource);
       }
     } finally {
       writeLock.unlock();
@@ -485,7 +499,8 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
       writeLock.lock();
       Host host = nodeCollections.get(nodeId.getHost());
       for (NodeAttribute attribute : host.getAttributes().keySet()) {
-        clusterAttributes.get(attribute).removeNode(host.getResource());
+        clusterAttributes.get(attribute.getAttributeKey())
+            .removeNode(host.getResource());
       }
       host.deactivateNode();
     } finally {
@@ -531,7 +546,8 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
             this.attributes.entrySet().iterator();
         while (it.hasNext()) {
           Entry<NodeAttribute, AttributeValue> current = it.next();
-          if (prefix.equals(current.getKey().getAttributePrefix())) {
+          if (prefix.equals(
+              current.getKey().getAttributeKey().getAttributePrefix())) {
             it.remove();
           }
         }
@@ -659,7 +675,7 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
       Map<String, Set<NodeAttribute>> nodeAttributeMapping,
       AttributeMappingOperationType mappingType, String attributePrefix)
       throws IOException {
-    Map<NodeAttribute, RMNodeAttribute> newAttributesToBeAdded =
+    Map<NodeAttributeKey, RMNodeAttribute> newAttributesToBeAdded =
         new HashMap<>();
     Map<String, Map<NodeAttribute, AttributeValue>> validMapping =
         validate(nodeAttributeMapping, newAttributesToBeAdded, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeLabelsUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeLabelsUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeLabelsUtils.java
index 1645d13..93b901e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeLabelsUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeLabelsUtils.java
@@ -21,9 +21,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 
 /**
@@ -56,4 +58,21 @@ public final class NodeLabelsUtils {
       throw new IOException(msg);
     }
   }
+
+  /**
+   * Returns a set of node attributes whose name exists in the provided
+   * <code>attributeNames</code> list.
+   *
+   * @param attributeNames For this given list of attribute names get the
+   *          cluster NodeAttributes
+   * @param clusterNodeAttributes set of node Attributes
+   * @return set of Node Attributes which maps to the give attributes names
+   */
+  public static Set <NodeAttribute> getNodeAttributesByName(
+      Set<String> attributeNames, Set<NodeAttribute> clusterNodeAttributes) {
+    return clusterNodeAttributes.stream()
+        .filter(attribute -> attributeNames
+            .contains(attribute.getAttributeKey().getAttributeName()))
+        .collect(Collectors.toSet());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeAttributeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeAttributeInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeAttributeInfo.java
index bbc2ec3..8384312 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeAttributeInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeAttributeInfo.java
@@ -41,8 +41,8 @@ public class NodeAttributeInfo {
   }
 
   public NodeAttributeInfo(NodeAttribute nodeAttribute) {
-    this.prefix = nodeAttribute.getAttributePrefix();
-    this.name = nodeAttribute.getAttributeName();
+    this.prefix = nodeAttribute.getAttributeKey().getAttributePrefix();
+    this.name = nodeAttribute.getAttributeKey().getAttributeName();
     this.type = nodeAttribute.getAttributeType().toString();
     this.value = nodeAttribute.getAttributeValue();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index 95ad35a..39892f2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -18,16 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
-import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
-
-import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
-import org.apache.hadoop.yarn.api.records.NodeAttribute;
-import org.apache.hadoop.yarn.api.records.NodeAttributeType;
-import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
@@ -78,6 +68,10 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
@@ -88,6 +82,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
@@ -117,10 +113,15 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeInfo;
+import org.apache.hadoop.yarn.api.records.NodeAttributeKey;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.NodeToAttributeValue;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueConfigurations;
@@ -142,6 +143,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
@@ -163,7 +165,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
-
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
@@ -176,11 +177,11 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
 
 public class TestClientRMService {
 
@@ -2048,11 +2049,12 @@ public class TestClientRMService {
         GetClusterNodeAttributesRequest.newInstance();
     GetClusterNodeAttributesResponse response =
         client.getClusterNodeAttributes(request);
-    Set<NodeAttribute> attributes = response.getNodeAttributes();
+    Set<NodeAttributeInfo> attributes = response.getNodeAttributes();
     Assert.assertEquals("Size not correct", 3, attributes.size());
-    Assert.assertTrue(attributes.contains(gpu));
-    Assert.assertTrue(attributes.contains(os));
-    Assert.assertTrue(attributes.contains(docker));
+    Assert.assertTrue(attributes.contains(NodeAttributeInfo.newInstance(gpu)));
+    Assert.assertTrue(attributes.contains(NodeAttributeInfo.newInstance(os)));
+    Assert
+        .assertTrue(attributes.contains(NodeAttributeInfo.newInstance(docker)));
     rpc.stopProxy(client, conf);
     rm.close();
   }
@@ -2071,17 +2073,17 @@ public class TestClientRMService {
     NodeAttributesManager mgr = rm.getRMContext().getNodeAttributesManager();
     String node1 = "host1";
     String node2 = "host2";
-    NodeAttribute gpu = NodeAttribute
-        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "GPU",
-            NodeAttributeType.STRING, "nvida");
-    NodeAttribute os = NodeAttribute
-        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "OS",
+    NodeAttribute gpu =
+        NodeAttribute.newInstance(NodeAttribute.PREFIX_CENTRALIZED, "GPU",
+            NodeAttributeType.STRING, "nvidia");
+    NodeAttribute os =
+        NodeAttribute.newInstance(NodeAttribute.PREFIX_CENTRALIZED, "OS",
             NodeAttributeType.STRING, "windows64");
-    NodeAttribute docker = NodeAttribute
-        .newInstance(NodeAttribute.PREFIX_DISTRIBUTED, "DOCKER",
+    NodeAttribute docker =
+        NodeAttribute.newInstance(NodeAttribute.PREFIX_DISTRIBUTED, "DOCKER",
             NodeAttributeType.STRING, "docker0");
-    NodeAttribute dist = NodeAttribute
-        .newInstance(NodeAttribute.PREFIX_DISTRIBUTED, "VERSION",
+    NodeAttribute dist =
+        NodeAttribute.newInstance(NodeAttribute.PREFIX_DISTRIBUTED, "VERSION",
             NodeAttributeType.STRING, "3_0_2");
     Map<String, Set<NodeAttribute>> nodes = new HashMap<>();
     nodes.put(node1, ImmutableSet.of(gpu, os, dist));
@@ -2099,35 +2101,55 @@ public class TestClientRMService {
         GetAttributesToNodesRequest.newInstance();
     GetAttributesToNodesResponse response =
         client.getAttributesToNodes(request);
-    Map<NodeAttribute, Set<String>> attrs = response.getAttributesToNodes();
+    Map<NodeAttributeKey, List<NodeToAttributeValue>> attrs =
+        response.getAttributesToNodes();
     Assert.assertEquals(response.getAttributesToNodes().size(), 4);
-    Assert.assertEquals(attrs.get(dist).size(), 2);
-    Assert.assertEquals(attrs.get(os).size(), 1);
-    Assert.assertEquals(attrs.get(gpu).size(), 1);
-    Assert.assertTrue(attrs.get(dist).contains(node1));
-    Assert.assertTrue(attrs.get(dist).contains(node2));
-    Assert.assertTrue(attrs.get(docker).contains(node2));
-
-    GetAttributesToNodesRequest request2 =
-        GetAttributesToNodesRequest.newInstance(ImmutableSet.of(docker));
+    Assert.assertEquals(attrs.get(dist.getAttributeKey()).size(), 2);
+    Assert.assertEquals(attrs.get(os.getAttributeKey()).size(), 1);
+    Assert.assertEquals(attrs.get(gpu.getAttributeKey()).size(), 1);
+    Assert.assertTrue(findHostnameAndValInMapping(node1, "3_0_2",
+        attrs.get(dist.getAttributeKey())));
+    Assert.assertTrue(findHostnameAndValInMapping(node2, "3_0_2",
+        attrs.get(dist.getAttributeKey())));
+    Assert.assertTrue(findHostnameAndValInMapping(node2, "docker0",
+        attrs.get(docker.getAttributeKey())));
+
+    GetAttributesToNodesRequest request2 = GetAttributesToNodesRequest
+        .newInstance(ImmutableSet.of(docker.getAttributeKey()));
     GetAttributesToNodesResponse response2 =
         client.getAttributesToNodes(request2);
-    Map<NodeAttribute, Set<String>> attrs2 = response2.getAttributesToNodes();
-    Assert.assertEquals(response2.getAttributesToNodes().size(), 1);
-    Assert.assertTrue(attrs.get(docker).contains(node2));
+    Map<NodeAttributeKey, List<NodeToAttributeValue>> attrs2 =
+        response2.getAttributesToNodes();
+    Assert.assertEquals(attrs2.size(), 1);
+    Assert.assertTrue(findHostnameAndValInMapping(node2, "docker0",
+        attrs2.get(docker.getAttributeKey())));
 
     GetAttributesToNodesRequest request3 =
-        GetAttributesToNodesRequest.newInstance(ImmutableSet.of(docker, os));
+        GetAttributesToNodesRequest.newInstance(
+            ImmutableSet.of(docker.getAttributeKey(), os.getAttributeKey()));
     GetAttributesToNodesResponse response3 =
         client.getAttributesToNodes(request3);
-    Map<NodeAttribute, Set<String>> attrs3 = response3.getAttributesToNodes();
-    Assert.assertEquals(response3.getAttributesToNodes().size(), 2);
-    Assert.assertTrue(attrs.get(os).contains(node1));
-    Assert.assertTrue(attrs.get(docker).contains(node2));
+    Map<NodeAttributeKey, List<NodeToAttributeValue>> attrs3 =
+        response3.getAttributesToNodes();
+    Assert.assertEquals(attrs3.size(), 2);
+    Assert.assertTrue(findHostnameAndValInMapping(node1, "windows64",
+        attrs3.get(os.getAttributeKey())));
+    Assert.assertTrue(findHostnameAndValInMapping(node2, "docker0",
+        attrs3.get(docker.getAttributeKey())));
     rpc.stopProxy(client, conf);
     rm.close();
   }
 
+  private boolean findHostnameAndValInMapping(String hostname, String attrVal,
+      List<NodeToAttributeValue> mappingVals) {
+    for (NodeToAttributeValue value : mappingVals) {
+      if (value.getHostname().equals(hostname)) {
+        return attrVal.equals(value.getAttributeValue());
+      }
+    }
+    return false;
+  }
+
   @Test(timeout = 120000)
   public void testGetNodesToAttributes() throws IOException, YarnException {
     MockRM rm = new MockRM() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
index adb7fe0..e40b3c0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
@@ -880,7 +880,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
         .getAttributesForNode(nodeId.getHost());
     Assert.assertEquals(1, attrs.size());
     NodeAttribute na = attrs.keySet().iterator().next();
-    Assert.assertEquals("host", na.getAttributeName());
+    Assert.assertEquals("host", na.getAttributeKey().getAttributeName());
     Assert.assertEquals("host2", na.getAttributeValue());
     Assert.assertEquals(NodeAttributeType.STRING, na.getAttributeType());
 
@@ -900,7 +900,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
     attrs = attributeManager.getAttributesForNode(nodeId.getHost());
     Assert.assertEquals(1, attrs.size());
     na = attrs.keySet().iterator().next();
-    Assert.assertEquals("host", na.getAttributeName());
+    Assert.assertEquals("host", na.getAttributeKey().getAttributeName());
     Assert.assertEquals("host3", na.getAttributeValue());
     Assert.assertEquals(NodeAttributeType.STRING, na.getAttributeType());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
index e2ee8b4..502f9d0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
@@ -253,8 +253,10 @@ public class TestFileSystemNodeAttributeStore {
 
   public void checkNodeAttributeEqual(NodeAttribute atr1, NodeAttribute atr2) {
     Assert.assertEquals(atr1.getAttributeType(), atr2.getAttributeType());
-    Assert.assertEquals(atr1.getAttributeName(), atr2.getAttributeName());
-    Assert.assertEquals(atr1.getAttributePrefix(), atr2.getAttributePrefix());
+    Assert.assertEquals(atr1.getAttributeKey().getAttributeName(),
+        atr2.getAttributeKey().getAttributeName());
+    Assert.assertEquals(atr1.getAttributeKey().getAttributePrefix(),
+        atr2.getAttributeKey().getAttributePrefix());
     Assert.assertEquals(atr1.getAttributeValue(), atr2.getAttributeValue());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
index b8c5bc9..9bc9388 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
@@ -344,11 +344,12 @@ public class TestNodeAttributesManager {
     clusterAttributes = attributesManager.getClusterNodeAttributes(
         Sets.newHashSet(NodeAttribute.PREFIX_DISTRIBUTED));
     Assert.assertEquals(1, clusterAttributes.size());
-    NodeAttribute att = clusterAttributes.iterator().next();
-    Assert.assertEquals("dist-node-attribute-v2_0", att.getAttributeName());
+    NodeAttribute attr = clusterAttributes.iterator().next();
+    Assert.assertEquals("dist-node-attribute-v2_0",
+        attr.getAttributeKey().getAttributeName());
     Assert.assertEquals(NodeAttribute.PREFIX_DISTRIBUTED,
-        att.getAttributePrefix());
-    Assert.assertEquals("dist_v3_0", att.getAttributeValue());
+        attr.getAttributeKey().getAttributePrefix());
+    Assert.assertEquals("dist_v3_0", attr.getAttributeValue());
 
     // Replace all attributes
     toReplaceMap.put(HOSTNAMES[0],

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cf6a9a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
index ceabe65..3d9b394 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
@@ -849,6 +849,6 @@ public class FederationClientInterceptor
   @Override
   public GetNodesToAttributesResponse getNodesToAttributes(
       GetNodesToAttributesRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
+    throw new NotImplementedException("Code is not implemented");
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[21/29] hadoop git commit: YARN-7988. Refactor FSNodeLabelStore code for Node Attributes store support. Contributed by Bibin A Chundatt.

Posted by su...@apache.org.
YARN-7988. Refactor FSNodeLabelStore code for Node Attributes store support. Contributed by Bibin A Chundatt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6f4bc49c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6f4bc49c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6f4bc49c

Branch: refs/heads/trunk
Commit: 6f4bc49c6d153e03edcc9b3a796fb1ccc95334b0
Parents: 3b3b6ef
Author: Sunil G <su...@apache.org>
Authored: Thu Mar 29 17:12:08 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../nodelabels/CommonNodeLabelsManager.java     |   8 +-
 .../nodelabels/FileSystemNodeLabelsStore.java   | 268 +++----------------
 .../hadoop/yarn/nodelabels/NodeLabelsStore.java |  32 +--
 .../NonAppendableFSNodeLabelStore.java          |  46 ++--
 .../nodelabels/store/AbstractFSNodeStore.java   | 213 +++++++++++++++
 .../yarn/nodelabels/store/FSStoreOpHandler.java | 108 ++++++++
 .../hadoop/yarn/nodelabels/store/StoreOp.java   |  49 ++++
 .../nodelabels/store/op/AddClusterLabelOp.java  |  73 +++++
 .../nodelabels/store/op/FSNodeStoreLogOp.java   |  35 +++
 .../nodelabels/store/op/NodeLabelMirrorOp.java  |  82 ++++++
 .../yarn/nodelabels/store/op/NodeToLabelOp.java |  75 ++++++
 .../store/op/RemoveClusterLabelOp.java          |  75 ++++++
 .../yarn/nodelabels/store/package-info.java     |  21 ++
 ...emoveFromClusterNodeLabelsRequestPBImpl.java |  21 +-
 .../DummyCommonNodeLabelsManager.java           |   8 +-
 .../TestFileSystemNodeLabelsStore.java          |  16 +-
 .../nodelabels/NullRMNodeLabelsManager.java     |   7 +
 .../webapp/TestRMWebServicesNodeLabels.java     |   2 +-
 18 files changed, 845 insertions(+), 294 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
index b5f4757..19254c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
@@ -234,7 +234,10 @@ public class CommonNodeLabelsManager extends AbstractService {
     return initNodeLabelStoreInProgress;
   }
 
-  boolean isCentralizedConfiguration() {
+  /**
+   * @return true if node label configuration type is not distributed.
+   */
+  public boolean isCentralizedConfiguration() {
     return isCentralizedNodeLabelConfiguration;
   }
 
@@ -245,8 +248,7 @@ public class CommonNodeLabelsManager extends AbstractService {
                 conf.getClass(YarnConfiguration.FS_NODE_LABELS_STORE_IMPL_CLASS,
                     FileSystemNodeLabelsStore.class, NodeLabelsStore.class),
                 conf);
-    this.store.setNodeLabelsManager(this);
-    this.store.init(conf);
+    this.store.init(conf, this);
     this.store.recover();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
index 0ec4ea4..e11e6f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
@@ -18,275 +18,89 @@
 
 package org.apache.hadoop.yarn.nodelabels;
 
-import java.io.EOFException;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddToClusterNodeLabelsRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RemoveFromClusterNodeLabelsRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ReplaceLabelsOnNodeRequestProto;
-import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RemoveFromClusterNodeLabelsRequestPBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOnNodeRequestPBImpl;
+import org.apache.hadoop.yarn.nodelabels.store.AbstractFSNodeStore;
+import org.apache.hadoop.yarn.nodelabels.store.FSStoreOpHandler;
 
-import com.google.common.collect.Sets;
+import org.apache.hadoop.yarn.nodelabels.store.op.AddClusterLabelOp;
+import org.apache.hadoop.yarn.nodelabels.store.FSStoreOpHandler.StoreType;
+import org.apache.hadoop.yarn.nodelabels.store.op.NodeToLabelOp;
+import org.apache.hadoop.yarn.nodelabels.store.op.RemoveClusterLabelOp;
 
-public class FileSystemNodeLabelsStore extends NodeLabelsStore {
-  protected static final Log LOG = LogFactory.getLog(FileSystemNodeLabelsStore.class);
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class FileSystemNodeLabelsStore
+    extends AbstractFSNodeStore<CommonNodeLabelsManager>
+    implements NodeLabelsStore {
+  protected static final Log LOG =
+      LogFactory.getLog(FileSystemNodeLabelsStore.class);
 
   protected static final String DEFAULT_DIR_NAME = "node-labels";
   protected static final String MIRROR_FILENAME = "nodelabel.mirror";
   protected static final String EDITLOG_FILENAME = "nodelabel.editlog";
-  
-  protected enum SerializedLogType {
-    ADD_LABELS, NODE_TO_LABELS, REMOVE_LABELS
+
+  FileSystemNodeLabelsStore() {
+    super(StoreType.NODE_LABEL_STORE);
   }
 
-  Path fsWorkingPath;
-  FileSystem fs;
-  private FSDataOutputStream editlogOs;
-  private Path editLogPath;
-  
   private String getDefaultFSNodeLabelsRootDir() throws IOException {
     // default is in local: /tmp/hadoop-yarn-${user}/node-labels/
-    return "file:///tmp/hadoop-yarn-"
-        + UserGroupInformation.getCurrentUser().getShortUserName() + "/"
-        + DEFAULT_DIR_NAME;
+    return "file:///tmp/hadoop-yarn-" + UserGroupInformation.getCurrentUser()
+        .getShortUserName() + "/" + DEFAULT_DIR_NAME;
   }
 
   @Override
-  public void init(Configuration conf) throws Exception {
-    fsWorkingPath =
-        new Path(conf.get(YarnConfiguration.FS_NODE_LABELS_STORE_ROOT_DIR,
-            getDefaultFSNodeLabelsRootDir()));
-
-    setFileSystem(conf);
-
-    // mkdir of root dir path
-    fs.mkdirs(fsWorkingPath);
+  public void init(Configuration conf, CommonNodeLabelsManager mgr)
+      throws Exception {
+    StoreSchema schema = new StoreSchema(EDITLOG_FILENAME, MIRROR_FILENAME);
+    initStore(conf, new Path(
+        conf.get(YarnConfiguration.FS_NODE_LABELS_STORE_ROOT_DIR,
+            getDefaultFSNodeLabelsRootDir())), schema, mgr);
   }
 
   @Override
   public void close() throws IOException {
-    IOUtils.cleanup(LOG, fs, editlogOs);
-  }
-
-  void setFileSystem(Configuration conf) throws IOException {
-    Configuration confCopy = new Configuration(conf);
-    fs = fsWorkingPath.getFileSystem(confCopy);
-
-    // if it's local file system, use RawLocalFileSystem instead of
-    // LocalFileSystem, the latter one doesn't support append.
-    if (fs.getScheme().equals("file")) {
-      fs = ((LocalFileSystem)fs).getRaw();
-    }
-  }
-  
-  private void ensureAppendEditlogFile() throws IOException {
-    editlogOs = fs.append(editLogPath);
-  }
-  
-  private void ensureCloseEditlogFile() throws IOException {
-    editlogOs.close();
+    super.closeFSStore();
   }
 
   @Override
-  public void updateNodeToLabelsMappings(
-      Map<NodeId, Set<String>> nodeToLabels) throws IOException {
-    try {
-      ensureAppendEditlogFile();
-      editlogOs.writeInt(SerializedLogType.NODE_TO_LABELS.ordinal());
-      ((ReplaceLabelsOnNodeRequestPBImpl) ReplaceLabelsOnNodeRequest
-          .newInstance(nodeToLabels)).getProto().writeDelimitedTo(editlogOs);
-    } finally {
-      ensureCloseEditlogFile();
-    }
+  public void updateNodeToLabelsMappings(Map<NodeId, Set<String>> nodeToLabels)
+      throws IOException {
+    NodeToLabelOp op = new NodeToLabelOp();
+    writeToLog(op.setNodeToLabels(nodeToLabels));
   }
 
   @Override
   public void storeNewClusterNodeLabels(List<NodeLabel> labels)
       throws IOException {
-    try {
-      ensureAppendEditlogFile();
-      editlogOs.writeInt(SerializedLogType.ADD_LABELS.ordinal());
-      ((AddToClusterNodeLabelsRequestPBImpl) AddToClusterNodeLabelsRequest
-          .newInstance(labels)).getProto().writeDelimitedTo(editlogOs);
-    } finally {
-      ensureCloseEditlogFile();
-    }
+    AddClusterLabelOp op = new AddClusterLabelOp();
+    writeToLog(op.setLabels(labels));
   }
 
   @Override
   public void removeClusterNodeLabels(Collection<String> labels)
       throws IOException {
-    try {
-      ensureAppendEditlogFile();
-      editlogOs.writeInt(SerializedLogType.REMOVE_LABELS.ordinal());
-      ((RemoveFromClusterNodeLabelsRequestPBImpl) RemoveFromClusterNodeLabelsRequest.newInstance(Sets
-          .newHashSet(labels.iterator()))).getProto().writeDelimitedTo(editlogOs);
-    } finally {
-      ensureCloseEditlogFile();
-    }
-  }
-  
-  protected void loadFromMirror(Path newMirrorPath, Path oldMirrorPath)
-      throws IOException {
-    // If mirror.new exists, read from mirror.new,
-    FSDataInputStream is = null;
-    try {
-      is = fs.open(newMirrorPath);
-    } catch (FileNotFoundException e) {
-      try {
-        is = fs.open(oldMirrorPath);
-      } catch (FileNotFoundException ignored) {
-
-      }
-    }
-    if (null != is) {
-      List<NodeLabel> labels = new AddToClusterNodeLabelsRequestPBImpl(
-          AddToClusterNodeLabelsRequestProto.parseDelimitedFrom(is))
-              .getNodeLabels();
-      mgr.addToCluserNodeLabels(labels);
-
-      if (mgr.isCentralizedConfiguration()) {
-        // Only load node to labels mapping while using centralized configuration
-        Map<NodeId, Set<String>> nodeToLabels =
-            new ReplaceLabelsOnNodeRequestPBImpl(
-                ReplaceLabelsOnNodeRequestProto.parseDelimitedFrom(is))
-                  .getNodeToLabels();
-        mgr.replaceLabelsOnNode(nodeToLabels);
-      }
-      is.close();
-    }
+    RemoveClusterLabelOp op = new RemoveClusterLabelOp();
+    writeToLog(op.setLabels(labels));
   }
 
   /* (non-Javadoc)
-   * @see org.apache.hadoop.yarn.nodelabels.NodeLabelsStore#recover(boolean)
-   */
-  @Override
-  public void recover() throws YarnException,
-      IOException {
-    /*
-     * Steps of recover
-     * 1) Read from last mirror (from mirror or mirror.old)
-     * 2) Read from last edit log, and apply such edit log
-     * 3) Write new mirror to mirror.writing
-     * 4) Rename mirror to mirror.old
-     * 5) Move mirror.writing to mirror
-     * 6) Remove mirror.old
-     * 7) Remove edit log and create a new empty edit log 
+     * @see org.apache.hadoop.yarn.nodelabels.NodeLabelsStore#recover(boolean)
      */
-    
-    // Open mirror from serialized file
-    Path mirrorPath = new Path(fsWorkingPath, MIRROR_FILENAME);
-    Path oldMirrorPath = new Path(fsWorkingPath, MIRROR_FILENAME + ".old");
-    
-    loadFromMirror(mirrorPath, oldMirrorPath);
-
-    // Open and process editlog
-    editLogPath = new Path(fsWorkingPath, EDITLOG_FILENAME);
-    FSDataInputStream is;
-    try {
-      is = fs.open(editLogPath);
-    } catch (FileNotFoundException e) {
-      is = null;
-    }
-    if (null != is) {
-
-      while (true) {
-        try {
-          // read edit log one by one
-          SerializedLogType type = SerializedLogType.values()[is.readInt()];
-          
-          switch (type) {
-          case ADD_LABELS: {
-            List<NodeLabel> labels =
-                new AddToClusterNodeLabelsRequestPBImpl(
-                    AddToClusterNodeLabelsRequestProto.parseDelimitedFrom(is))
-                    .getNodeLabels();
-            mgr.addToCluserNodeLabels(labels);
-            break;
-          }
-          case REMOVE_LABELS: {
-            Collection<String> labels =
-                RemoveFromClusterNodeLabelsRequestProto.parseDelimitedFrom(is)
-                    .getNodeLabelsList();
-            mgr.removeFromClusterNodeLabels(labels);
-            break;
-          }
-          case NODE_TO_LABELS: {
-            Map<NodeId, Set<String>> map =
-                new ReplaceLabelsOnNodeRequestPBImpl(
-                    ReplaceLabelsOnNodeRequestProto.parseDelimitedFrom(is))
-                    .getNodeToLabels();
-            if (mgr.isCentralizedConfiguration()) {
-              /*
-               * In case of Distributed NodeLabels setup,
-               * ignoreNodeToLabelsMappings will be set to true and recover will
-               * be invoked. As RM will collect the node labels from NM through
-               * registration/HB
-               */
-              mgr.replaceLabelsOnNode(map);
-            }
-            break;
-          }
-          }
-        } catch (EOFException e) {
-          // EOF hit, break
-          break;
-        }
-      }
-      is.close();
-    }
-
-    // Serialize current mirror to mirror.writing
-    Path writingMirrorPath = new Path(fsWorkingPath, MIRROR_FILENAME + ".writing");
-    FSDataOutputStream os = fs.create(writingMirrorPath, true);
-    ((AddToClusterNodeLabelsRequestPBImpl) AddToClusterNodeLabelsRequestPBImpl
-        .newInstance(mgr.getClusterNodeLabels())).getProto().writeDelimitedTo(os);
-    ((ReplaceLabelsOnNodeRequestPBImpl) ReplaceLabelsOnNodeRequest
-        .newInstance(mgr.getNodeLabels())).getProto().writeDelimitedTo(os);
-    os.close();
-    
-    // Move mirror to mirror.old
-    if (fs.exists(mirrorPath)) {
-      fs.delete(oldMirrorPath, false);
-      fs.rename(mirrorPath, oldMirrorPath);
-    }
-    
-    // move mirror.writing to mirror
-    fs.rename(writingMirrorPath, mirrorPath);
-    fs.delete(writingMirrorPath, false);
-    
-    // remove mirror.old
-    fs.delete(oldMirrorPath, false);
-    
-    // create a new editlog file
-    editlogOs = fs.create(editLogPath, true);
-    editlogOs.close();
-    
-    LOG.info("Finished write mirror at:" + mirrorPath.toString());
-    LOG.info("Finished create editlog file at:" + editLogPath.toString());
+  @Override
+  public void recover() throws YarnException, IOException {
+    super.recoverFromStore();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
index aacb920..e4efd68 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
@@ -30,25 +30,27 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 
-public abstract class NodeLabelsStore implements Closeable {
-  protected CommonNodeLabelsManager mgr;
-  
+/**
+ * Interface class for Node label store.
+ */
+public interface NodeLabelsStore extends Closeable {
+
   /**
-   * Store node {@literal ->} label
+   * Store node {@literal ->} label.
    */
-  public abstract void updateNodeToLabelsMappings(
+  void updateNodeToLabelsMappings(
       Map<NodeId, Set<String>> nodeToLabels) throws IOException;
 
   /**
-   * Store new labels
+   * Store new labels.
    */
-  public abstract void storeNewClusterNodeLabels(List<NodeLabel> label)
+  void storeNewClusterNodeLabels(List<NodeLabel> label)
       throws IOException;
 
   /**
-   * Remove labels
+   * Remove labels.
    */
-  public abstract void removeClusterNodeLabels(Collection<String> labels)
+  void removeClusterNodeLabels(Collection<String> labels)
       throws IOException;
 
   /**
@@ -56,16 +58,14 @@ public abstract class NodeLabelsStore implements Closeable {
    * ignoreNodeToLabelsMappings is true then node to labels mappings should not
    * be recovered. In case of Distributed NodeLabels setup
    * ignoreNodeToLabelsMappings will be set to true and recover will be invoked
-   * as RM will collect the node labels from NM through registration/HB
+   * as RM will collect the node labels from NM through registration/HB.
    *
    * @throws IOException
    * @throws YarnException
    */
-  public abstract void recover() throws IOException, YarnException;
-  
-  public void init(Configuration conf) throws Exception {}
+  void recover() throws IOException, YarnException;
+
+  void init(Configuration conf, CommonNodeLabelsManager mgr)
+      throws Exception;
 
-  public void setNodeLabelsManager(CommonNodeLabelsManager mgr) {
-    this.mgr = mgr;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NonAppendableFSNodeLabelStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NonAppendableFSNodeLabelStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NonAppendableFSNodeLabelStore.java
index 989f027..29bfff9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NonAppendableFSNodeLabelStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NonAppendableFSNodeLabelStore.java
@@ -18,13 +18,6 @@
 
 package org.apache.hadoop.yarn.nodelabels;
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -32,11 +25,19 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOnNodeRequestPBImpl;
+import org.apache.hadoop.yarn.nodelabels.store.FSStoreOpHandler;
+import org.apache.hadoop.yarn.nodelabels.store.StoreOp;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+/**
+ * Store implementation for Non Appendable File Store
+ */
 public class NonAppendableFSNodeLabelStore extends FileSystemNodeLabelsStore {
   protected static final Log
       LOG = LogFactory.getLog(NonAppendableFSNodeLabelStore.class);
@@ -52,7 +53,7 @@ public class NonAppendableFSNodeLabelStore extends FileSystemNodeLabelsStore {
     Path newMirrorPath = new Path(fsWorkingPath, MIRROR_FILENAME + ".new");
     Path oldMirrorPath = new Path(fsWorkingPath, MIRROR_FILENAME);
     loadFromMirror(newMirrorPath, oldMirrorPath);
-    
+
     // if new mirror exists, remove old mirror and rename new mirror
     if (fs.exists(newMirrorPath)) {
       // remove old mirror
@@ -91,29 +92,18 @@ public class NonAppendableFSNodeLabelStore extends FileSystemNodeLabelsStore {
   }
 
   private void writeNewMirror() throws IOException {
-    ReentrantReadWriteLock.ReadLock readLock = mgr.readLock;
+    ReentrantReadWriteLock.ReadLock readLock = manager.readLock;
     try {
       // Acquire readlock to make sure we get cluster node labels and
       // node-to-labels mapping atomically.
       readLock.lock();
-      List<NodeLabel> nodeLabels = mgr.getClusterNodeLabels();
-      Map<NodeId, Set<String>> nodeToLabels = mgr.getNodeLabels();
-      
       // Write mirror to mirror.new.tmp file
-      Path newTmpPath = new Path(fsWorkingPath, MIRROR_FILENAME + ".new.tmp"); 
-      FSDataOutputStream os = fs
-          .create(newTmpPath, true);
-      ((AddToClusterNodeLabelsRequestPBImpl) AddToClusterNodeLabelsRequest
-          .newInstance(nodeLabels)).getProto().writeDelimitedTo(os);
-
-      if (mgr.isCentralizedConfiguration()) {
-        // Only save node-to-labels mapping while using centralized configuration
-        ((ReplaceLabelsOnNodeRequestPBImpl) ReplaceLabelsOnNodeRequest
-            .newInstance(nodeToLabels)).getProto().writeDelimitedTo(os);
+      Path newTmpPath = new Path(fsWorkingPath, MIRROR_FILENAME + ".new.tmp");
+      try (FSDataOutputStream os = fs.create(newTmpPath, true)) {
+        StoreOp op = FSStoreOpHandler.getMirrorOp(getStoreType());
+        op.write(os, manager);
       }
       
-      os.close();
-      
       // Rename mirror.new.tmp to mirror.new (will remove .new if it's existed)
       Path newPath = new Path(fsWorkingPath, MIRROR_FILENAME + ".new"); 
       fs.delete(newPath, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java
new file mode 100644
index 0000000..a47cacf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java
@@ -0,0 +1,213 @@
+/**
+ * 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.nodelabels.store;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.yarn.nodelabels.store.op.FSNodeStoreLogOp;
+import org.apache.hadoop.yarn.nodelabels.store.FSStoreOpHandler.StoreType;
+
+import java.io.EOFException;
+import java.io.IOException;
+
+/**
+ * Abstract class for File System based store.
+ *
+ * @param <M> manager filesystem store.Currently nodelabel will use
+ *           CommonNodeLabelManager.
+ */
+public abstract class AbstractFSNodeStore<M> {
+
+  protected static final Log LOG = LogFactory.getLog(AbstractFSNodeStore.class);
+
+  private StoreType storeType;
+  private FSDataOutputStream editlogOs;
+
+  private Path editLogPath;
+  private StoreSchema schema;
+
+  protected M manager;
+  protected Path fsWorkingPath;
+  protected FileSystem fs;
+
+  public AbstractFSNodeStore(StoreType storeType) {
+    this.storeType = storeType;
+  }
+
+  protected void initStore(Configuration conf, Path fsStorePath,
+      StoreSchema schma, M mgr) throws IOException {
+    this.schema = schma;
+    this.fsWorkingPath = fsStorePath;
+    this.manager = mgr;
+    initFileSystem(conf);
+    // mkdir of root dir path
+    fs.mkdirs(fsWorkingPath);
+
+  }
+
+  /**
+   * Filesystem store schema define the log name and mirror name.
+   */
+  public static class StoreSchema {
+    private String editLogName;
+    private String mirrorName;
+
+    public StoreSchema(String editLogName, String mirrorName) {
+      this.editLogName = editLogName;
+      this.mirrorName = mirrorName;
+    }
+  }
+
+  public void initFileSystem(Configuration conf) throws IOException {
+    Configuration confCopy = new Configuration(conf);
+    fs = fsWorkingPath.getFileSystem(confCopy);
+    // if it's local file system, use RawLocalFileSystem instead of
+    // LocalFileSystem, the latter one doesn't support append.
+    if (fs.getScheme().equals("file")) {
+      fs = ((LocalFileSystem) fs).getRaw();
+    }
+  }
+
+  protected void writeToLog(FSNodeStoreLogOp op) throws IOException {
+    try {
+      ensureAppendEditLogFile();
+      editlogOs.writeInt(op.getOpCode());
+      op.write(editlogOs, manager);
+    } finally {
+      ensureCloseEditlogFile();
+    }
+  }
+
+  protected void ensureAppendEditLogFile() throws IOException {
+    editlogOs = fs.append(editLogPath);
+  }
+
+  protected void ensureCloseEditlogFile() throws IOException {
+    editlogOs.close();
+  }
+
+  protected void loadFromMirror(Path newMirrorPath, Path oldMirrorPath)
+      throws IOException {
+    // If mirror.new exists, read from mirror.new
+    Path mirrorToRead = fs.exists(newMirrorPath) ?
+        newMirrorPath :
+        fs.exists(oldMirrorPath) ? oldMirrorPath : null;
+    if (mirrorToRead != null) {
+      try (FSDataInputStream is = fs.open(mirrorToRead)) {
+        StoreOp op = FSStoreOpHandler.getMirrorOp(storeType);
+        op.recover(is, manager);
+      }
+    }
+  }
+
+  protected StoreType getStoreType() {
+    return storeType;
+  }
+
+  public Path getFsWorkingPath() {
+    return fsWorkingPath;
+  }
+
+  protected void recoverFromStore() throws IOException {
+        /*
+     * Steps of recover
+     * 1) Read from last mirror (from mirror or mirror.old)
+     * 2) Read from last edit log, and apply such edit log
+     * 3) Write new mirror to mirror.writing
+     * 4) Rename mirror to mirror.old
+     * 5) Move mirror.writing to mirror
+     * 6) Remove mirror.old
+     * 7) Remove edit log and create a new empty edit log
+     */
+
+    // Open mirror from serialized file
+    Path mirrorPath = new Path(fsWorkingPath, schema.mirrorName);
+    Path oldMirrorPath = new Path(fsWorkingPath, schema.mirrorName + ".old");
+
+    loadFromMirror(mirrorPath, oldMirrorPath);
+
+    // Open and process editlog
+    editLogPath = new Path(fsWorkingPath, schema.editLogName);
+
+    loadManagerFromEditLog(editLogPath);
+
+    // Serialize current mirror to mirror.writing
+    Path writingMirrorPath =
+        new Path(fsWorkingPath, schema.mirrorName + ".writing");
+
+    try(FSDataOutputStream os = fs.create(writingMirrorPath, true)){
+      StoreOp op = FSStoreOpHandler.getMirrorOp(storeType);
+      op.write(os, manager);
+    }
+    // Move mirror to mirror.old
+    if (fs.exists(mirrorPath)) {
+      fs.delete(oldMirrorPath, false);
+      fs.rename(mirrorPath, oldMirrorPath);
+    }
+
+    // move mirror.writing to mirror
+    fs.rename(writingMirrorPath, mirrorPath);
+    fs.delete(writingMirrorPath, false);
+
+    // remove mirror.old
+    fs.delete(oldMirrorPath, false);
+
+    // create a new editlog file
+    editlogOs = fs.create(editLogPath, true);
+    editlogOs.close();
+
+    LOG.info("Finished write mirror at:" + mirrorPath.toString());
+    LOG.info("Finished create editlog file at:" + editLogPath.toString());
+  }
+
+  protected void loadManagerFromEditLog(Path editLogPath) throws IOException {
+    if (!fs.exists(editLogPath)) {
+      return;
+    }
+    try (FSDataInputStream is = fs.open(editLogPath)) {
+      while (true) {
+        try {
+          StoreOp storeOp = FSStoreOpHandler.get(is.readInt(),storeType);
+          storeOp.recover(is, manager);
+        } catch (EOFException e) {
+          // EOF hit, break
+          break;
+        }
+      }
+    }
+  }
+
+  public FileSystem getFs() {
+    return fs;
+  }
+
+  public void setFs(FileSystem fs) {
+    this.fs = fs;
+  }
+
+  protected void closeFSStore() {
+    IOUtils.closeStreams(fs, editlogOs);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java
new file mode 100644
index 0000000..0f7f53d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java
@@ -0,0 +1,108 @@
+/**
+ * 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.nodelabels.store;
+
+import static org.apache.hadoop.yarn.nodelabels.store.FSStoreOpHandler
+    .StoreType.NODE_LABEL_STORE;
+import org.apache.hadoop.yarn.nodelabels.store.op.AddClusterLabelOp;
+import org.apache.hadoop.yarn.nodelabels.store.op.FSNodeStoreLogOp;
+import org.apache.hadoop.yarn.nodelabels.store.op.NodeLabelMirrorOp;
+import org.apache.hadoop.yarn.nodelabels.store.op.NodeToLabelOp;
+import org.apache.hadoop.yarn.nodelabels.store.op.RemoveClusterLabelOp;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * File system store op handler.
+ */
+public class FSStoreOpHandler {
+
+  private static Map<StoreType, Map<Integer, Class<? extends FSNodeStoreLogOp>>>
+      editLogOp;
+  private static Map<StoreType, Class<? extends FSNodeStoreLogOp>> mirrorOp;
+
+  public enum StoreType {
+    NODE_LABEL_STORE,
+    NODE_LABEL_ATTRIBUTE;
+  }
+
+  static {
+    editLogOp = new HashMap<>();
+    mirrorOp = new HashMap<>();
+
+    // registerLog edit log operation
+    registerLog(NODE_LABEL_STORE, AddClusterLabelOp.OPCODE, AddClusterLabelOp.class);
+    registerLog(NODE_LABEL_STORE, NodeToLabelOp.OPCODE, NodeToLabelOp.class);
+    registerLog(NODE_LABEL_STORE, RemoveClusterLabelOp.OPCODE, RemoveClusterLabelOp.class);
+
+    // registerLog Mirror op
+
+    registerMirror(NODE_LABEL_STORE, NodeLabelMirrorOp.class);
+  }
+
+  private static void registerMirror(StoreType type,
+      Class<? extends FSNodeStoreLogOp> clazz) {
+    mirrorOp.put(type, clazz);
+  }
+
+  private static void registerLog(StoreType type, int opcode,
+      Class<? extends FSNodeStoreLogOp> clazz) {
+    Map<Integer, Class<? extends FSNodeStoreLogOp>> ops = editLogOp.get(type);
+    Integer code = Integer.valueOf(opcode);
+    if (ops == null) {
+      Map<Integer, Class<? extends FSNodeStoreLogOp>> newOps = new HashMap<>();
+      newOps.put(code, clazz);
+      editLogOp.put(type, newOps);
+    } else {
+      ops.put(code, clazz);
+    }
+  }
+
+  /**
+   * Get mirror operation of store Type.
+   *
+   * @param storeType
+   * @return instance of FSNodeStoreLogOp.
+   */
+  public static FSNodeStoreLogOp getMirrorOp(StoreType storeType) {
+    return newInstance(mirrorOp.get(storeType));
+  }
+
+  /**
+   * Will return StoreOp instance basead on opCode and StoreType.
+   * @param opCode
+   * @param storeType
+   * @return instance of FSNodeStoreLogOp.
+   */
+  public static FSNodeStoreLogOp get(int opCode, StoreType storeType) {
+    return newInstance(editLogOp.get(storeType).get(opCode));
+  }
+
+  private static <T extends FSNodeStoreLogOp> T newInstance(Class<T> clazz) {
+    FSNodeStoreLogOp instance = null;
+    if (clazz != null) {
+      try {
+        instance = clazz.newInstance();
+      } catch (Exception ex) {
+        throw new RuntimeException("Failed to instantiate " + clazz, ex);
+      }
+    }
+    return (T) instance;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/StoreOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/StoreOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/StoreOp.java
new file mode 100644
index 0000000..c26e1dc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/StoreOp.java
@@ -0,0 +1,49 @@
+/**
+ * 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.nodelabels.store;
+
+import java.io.IOException;
+
+/**
+ * Define the interface for store activity.
+ * Used by for FileSystem based operation.
+ *
+ * @param <W> write to be done to
+ * @param <R> read to be done from
+ * @param <M> manager used
+ */
+public interface StoreOp<W, R, M> {
+
+  /**
+   * Write operation to persistent storage
+   *
+   * @param write write to be done to
+   * @param mgr manager used by store
+   * @throws IOException
+   */
+  void write(W write, M mgr) throws IOException;
+
+  /**
+   * Read and populate StoreOp
+   *
+   * @param read read to be done from
+   * @param mgr  manager used by store
+   * @throws IOException
+   */
+  void recover(R read, M mgr) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/AddClusterLabelOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/AddClusterLabelOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/AddClusterLabelOp.java
new file mode 100644
index 0000000..ce736aa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/AddClusterLabelOp.java
@@ -0,0 +1,73 @@
+/**
+ * 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.nodelabels.store.op;
+
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos;
+import org.apache.hadoop.yarn.server.api.protocolrecords
+    .AddToClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb
+    .AddToClusterNodeLabelsRequestPBImpl;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+
+/**
+ * Add label operation for file system.
+ */
+public class AddClusterLabelOp
+    extends FSNodeStoreLogOp<CommonNodeLabelsManager> {
+
+  private List<NodeLabel> labels;
+
+  public static final int OPCODE = 0;
+
+  @Override
+  public void write(OutputStream os, CommonNodeLabelsManager mgr)
+      throws IOException {
+    ((AddToClusterNodeLabelsRequestPBImpl) AddToClusterNodeLabelsRequest
+        .newInstance(labels)).getProto().writeDelimitedTo(os);
+  }
+
+  @Override
+  public void recover(InputStream is, CommonNodeLabelsManager mgr)
+      throws IOException {
+    labels = new AddToClusterNodeLabelsRequestPBImpl(
+        YarnServerResourceManagerServiceProtos
+            .AddToClusterNodeLabelsRequestProto
+            .parseDelimitedFrom(is)).getNodeLabels();
+    mgr.addToCluserNodeLabels(labels);
+  }
+
+  public AddClusterLabelOp setLabels(List<NodeLabel> labels) {
+    this.labels = labels;
+    return this;
+  }
+
+  public List<NodeLabel> getLabels() {
+    return labels;
+  }
+
+  @Override
+  public int getOpCode() {
+    return OPCODE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/FSNodeStoreLogOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/FSNodeStoreLogOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/FSNodeStoreLogOp.java
new file mode 100644
index 0000000..cd739c0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/FSNodeStoreLogOp.java
@@ -0,0 +1,35 @@
+/**
+ * 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.nodelabels.store.op;
+
+import org.apache.hadoop.yarn.nodelabels.store.StoreOp;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Defines all FileSystem editlog operation. All node label and attribute
+ * store write or read operation will be defined in this class.
+ *
+ * @param <M> Manager used for each operation.
+ */
+public abstract class FSNodeStoreLogOp<M>
+    implements StoreOp<OutputStream, InputStream, M> {
+
+  public abstract int getOpCode();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeLabelMirrorOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeLabelMirrorOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeLabelMirrorOp.java
new file mode 100644
index 0000000..3ec837b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeLabelMirrorOp.java
@@ -0,0 +1,82 @@
+/**
+ * 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.nodelabels.store.op;
+
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos;
+import org.apache.hadoop.yarn.server.api.protocolrecords
+    .ReplaceLabelsOnNodeRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb
+    .AddToClusterNodeLabelsRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb
+    .ReplaceLabelsOnNodeRequestPBImpl;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class NodeLabelMirrorOp
+    extends FSNodeStoreLogOp<CommonNodeLabelsManager> {
+
+  public NodeLabelMirrorOp() {
+    super();
+  }
+
+  @Override
+  public void write(OutputStream os, CommonNodeLabelsManager mgr)
+      throws IOException {
+    ((AddToClusterNodeLabelsRequestPBImpl) AddToClusterNodeLabelsRequestPBImpl
+        .newInstance(mgr.getClusterNodeLabels())).getProto()
+        .writeDelimitedTo(os);
+    if (mgr.isCentralizedConfiguration()) {
+      ((ReplaceLabelsOnNodeRequestPBImpl) ReplaceLabelsOnNodeRequest
+          .newInstance(mgr.getNodeLabels())).getProto().writeDelimitedTo(os);
+    }
+  }
+
+  @Override
+  public void recover(InputStream is, CommonNodeLabelsManager mgr)
+      throws IOException {
+    List<NodeLabel> labels = new AddToClusterNodeLabelsRequestPBImpl(
+        YarnServerResourceManagerServiceProtos
+            .AddToClusterNodeLabelsRequestProto
+            .parseDelimitedFrom(is)).getNodeLabels();
+    mgr.addToCluserNodeLabels(labels);
+
+    if (mgr.isCentralizedConfiguration()) {
+      // Only load node to labels mapping while using centralized
+      // configuration
+      Map<NodeId, Set<String>> nodeToLabels =
+          new ReplaceLabelsOnNodeRequestPBImpl(
+              YarnServerResourceManagerServiceProtos
+                  .ReplaceLabelsOnNodeRequestProto
+                  .parseDelimitedFrom(is)).getNodeToLabels();
+      mgr.replaceLabelsOnNode(nodeToLabels);
+    }
+  }
+
+  @Override
+  public int getOpCode() {
+    return -1;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeToLabelOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeToLabelOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeToLabelOp.java
new file mode 100644
index 0000000..0e1e398
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeToLabelOp.java
@@ -0,0 +1,75 @@
+/**
+ * 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.nodelabels.store.op;
+
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos;
+import org.apache.hadoop.yarn.server.api.protocolrecords
+    .ReplaceLabelsOnNodeRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb
+    .ReplaceLabelsOnNodeRequestPBImpl;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Node to label mapping store operation for label.
+ */
+public class NodeToLabelOp
+    extends FSNodeStoreLogOp<CommonNodeLabelsManager> {
+
+  private Map<NodeId, Set<String>> nodeToLabels;
+  public static final int OPCODE = 1;
+
+  @Override
+  public void write(OutputStream os, CommonNodeLabelsManager mgr)
+      throws IOException {
+    ((ReplaceLabelsOnNodeRequestPBImpl) ReplaceLabelsOnNodeRequest
+        .newInstance(nodeToLabels)).getProto().writeDelimitedTo(os);
+  }
+
+  @Override
+  public void recover(InputStream is, CommonNodeLabelsManager mgr)
+      throws IOException {
+    nodeToLabels = new ReplaceLabelsOnNodeRequestPBImpl(
+        YarnServerResourceManagerServiceProtos.ReplaceLabelsOnNodeRequestProto
+            .parseDelimitedFrom(is)).getNodeToLabels();
+    if (mgr.isCentralizedConfiguration()) {
+      mgr.replaceLabelsOnNode(nodeToLabels);
+    }
+  }
+
+  public NodeToLabelOp setNodeToLabels(
+      Map<NodeId, Set<String>> nodeToLabels) {
+    this.nodeToLabels = nodeToLabels;
+    return this;
+  }
+
+  public Map<NodeId, Set<String>> getNodeToLabels() {
+    return nodeToLabels;
+  }
+
+  @Override
+  public int getOpCode() {
+    return OPCODE;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/RemoveClusterLabelOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/RemoveClusterLabelOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/RemoveClusterLabelOp.java
new file mode 100644
index 0000000..4f6d4bd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/RemoveClusterLabelOp.java
@@ -0,0 +1,75 @@
+/**
+ * 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.nodelabels.store.op;
+
+import com.google.common.collect.Sets;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos;
+import org.apache.hadoop.yarn.server.api.protocolrecords
+    .RemoveFromClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb
+    .RemoveFromClusterNodeLabelsRequestPBImpl;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Collection;
+
+/**
+ * Remove label from cluster log store operation.
+ */
+public class RemoveClusterLabelOp
+    extends FSNodeStoreLogOp<CommonNodeLabelsManager> {
+
+  private Collection<String> labels;
+
+  public static final int OPCODE = 2;
+
+  @Override
+  public void write(OutputStream os, CommonNodeLabelsManager mgr)
+      throws IOException {
+    ((RemoveFromClusterNodeLabelsRequestPBImpl)
+        RemoveFromClusterNodeLabelsRequest
+        .newInstance(Sets.newHashSet(labels.iterator()))).getProto()
+        .writeDelimitedTo(os);
+  }
+
+  @Override
+  public void recover(InputStream is, CommonNodeLabelsManager mgr)
+      throws IOException {
+    labels =
+        YarnServerResourceManagerServiceProtos
+            .RemoveFromClusterNodeLabelsRequestProto
+            .parseDelimitedFrom(is).getNodeLabelsList();
+    mgr.removeFromClusterNodeLabels(labels);
+  }
+
+  public RemoveClusterLabelOp setLabels(Collection<String> labels) {
+    this.labels = labels;
+    return this;
+  }
+
+  public Collection<String> getLabels() {
+    return labels;
+  }
+
+  @Override
+  public int getOpCode() {
+    return OPCODE;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/package-info.java
new file mode 100644
index 0000000..0444807
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.nodelabels.store;
+import org.apache.hadoop.classification.InterfaceAudience;
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoveFromClusterNodeLabelsRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoveFromClusterNodeLabelsRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoveFromClusterNodeLabelsRequestPBImpl.java
index a935840..f633804 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoveFromClusterNodeLabelsRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoveFromClusterNodeLabelsRequestPBImpl.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * 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.
@@ -21,12 +21,15 @@ package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RemoveFromClusterNodeLabelsRequestProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RemoveFromClusterNodeLabelsRequestProto;
-import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos
+    .RemoveFromClusterNodeLabelsRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos
+    .RemoveFromClusterNodeLabelsRequestProto;
+import org.apache.hadoop.yarn.server.api.protocolrecords
+    .RemoveFromClusterNodeLabelsRequest;
 
-public class RemoveFromClusterNodeLabelsRequestPBImpl extends
-    RemoveFromClusterNodeLabelsRequest {
+public class RemoveFromClusterNodeLabelsRequestPBImpl
+    extends RemoveFromClusterNodeLabelsRequest {
   Set<String> labels;
   RemoveFromClusterNodeLabelsRequestProto proto =
       RemoveFromClusterNodeLabelsRequestProto.getDefaultInstance();
@@ -102,7 +105,7 @@ public class RemoveFromClusterNodeLabelsRequestPBImpl extends
     assert false : "hashCode not designed";
     return 0;
   }
-  
+
   @Override
   public boolean equals(Object other) {
     if (other == null)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java
index 64c74c2..61373dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java
@@ -44,6 +44,12 @@ public class DummyCommonNodeLabelsManager extends CommonNodeLabelsManager {
       }
 
       @Override
+      public void init(Configuration conf, CommonNodeLabelsManager mgr)
+          throws Exception {
+
+      }
+
+      @Override
       public void removeClusterNodeLabels(Collection<String> labels)
           throws IOException {
         lastRemovedlabels = labels;
@@ -65,8 +71,6 @@ public class DummyCommonNodeLabelsManager extends CommonNodeLabelsManager {
         // do nothing 
       }
     };
-
-    this.store.setNodeLabelsManager(this);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
index ed2f4aa..93c039a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
@@ -96,7 +96,7 @@ public class TestFileSystemNodeLabelsStore extends NodeLabelTestBase {
     if (mgr.store instanceof FileSystemNodeLabelsStore) {
       FileSystemNodeLabelsStore fsStore =
           ((FileSystemNodeLabelsStore) mgr.store);
-      fsStore.fs.delete(fsStore.fsWorkingPath, true);
+      fsStore.getFs().delete(fsStore.getFsWorkingPath(), true);
     }
     mgr.stop();
   }
@@ -342,12 +342,12 @@ public class TestFileSystemNodeLabelsStore extends NodeLabelTestBase {
   public void testRootMkdirOnInitStore() throws Exception {
     final FileSystem mockFs = Mockito.mock(FileSystem.class);
     FileSystemNodeLabelsStore mockStore = new FileSystemNodeLabelsStore() {
-      void setFileSystem(Configuration conf) throws IOException {
-        fs = mockFs;
+      public void initFileSystem(Configuration config) throws IOException {
+        setFs(mockFs);
       }
     };
-    mockStore.setNodeLabelsManager(mgr);
-    mockStore.fs = mockFs;
+
+    mockStore.setFs(mockFs);
     verifyMkdirsCount(mockStore, true, 1);
     verifyMkdirsCount(mockStore, false, 2);
     verifyMkdirsCount(mockStore, true, 3);
@@ -357,10 +357,10 @@ public class TestFileSystemNodeLabelsStore extends NodeLabelTestBase {
   private void verifyMkdirsCount(FileSystemNodeLabelsStore store,
                                  boolean existsRetVal, int expectedNumOfCalls)
       throws Exception {
-    Mockito.when(store.fs.exists(Mockito.any(
+    Mockito.when(store.getFs().exists(Mockito.any(
         Path.class))).thenReturn(existsRetVal);
-    store.init(conf);
-    Mockito.verify(store.fs,Mockito.times(
+    store.init(conf, mgr);
+    Mockito.verify(store.getFs(),Mockito.times(
         expectedNumOfCalls)).mkdirs(Mockito.any(Path
         .class));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java
index bb0b45f..b8f3fae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.nodelabels.NodeLabelsStore;
 
 public class NullRMNodeLabelsManager extends RMNodeLabelsManager {
@@ -46,6 +47,12 @@ public class NullRMNodeLabelsManager extends RMNodeLabelsManager {
       }
 
       @Override
+      public void init(Configuration conf, CommonNodeLabelsManager mgr)
+          throws Exception {
+
+      }
+
+      @Override
       public void removeClusterNodeLabels(Collection<String> labels)
           throws IOException {
         // do nothing

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f4bc49c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
index 5e6fd4e..21df698 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
@@ -605,7 +605,7 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
     String expectedmessage =
         "java.io.IOException: label name should only contains"
             + " {0-9, a-z, A-Z, -, _} and should not started with"
-            + " {-,_}, now it is=a&";
+            + " {-,_}, now it is= a&";
     validateJsonExceptionContent(response, expectedmessage);
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/29] hadoop git commit: YARN-8351. Node attribute manager logs are flooding RM logs. Contributed by Weiwei Yang.

Posted by su...@apache.org.
YARN-8351. Node attribute manager logs are flooding RM logs. Contributed by Weiwei Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/76183428
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/76183428
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/76183428

Branch: refs/heads/trunk
Commit: 76183428b76524818fc250ae97bbf973dc910e1c
Parents: 8cf6a9a
Author: Sunil G <su...@apache.org>
Authored: Fri May 25 16:08:26 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../resourcemanager/nodelabels/NodeAttributesManagerImpl.java    | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/76183428/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
index 6b176d6..fac2dfd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
@@ -196,7 +196,9 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
         logMsg.append("] ,");
       }
 
-      LOG.info(logMsg);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(logMsg);
+      }
 
       if (null != dispatcher && NodeAttribute.PREFIX_CENTRALIZED
           .equals(attributePrefix)) {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/29] hadoop git commit: YARN-7757. Refactor NodeLabelsProvider to be more generic and reusable for node attributes providers. Contributed by Weiwei Yang.

Posted by su...@apache.org.
YARN-7757. Refactor NodeLabelsProvider to be more generic and reusable for node attributes providers. Contributed by Weiwei Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d312b5cf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d312b5cf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d312b5cf

Branch: refs/heads/trunk
Commit: d312b5cf9ffff016432ec5a7e64b81ab7a8c31f3
Parents: 4458b27
Author: Naganarasimha <na...@apache.org>
Authored: Mon Feb 5 05:47:02 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  33 +++
 .../src/main/resources/yarn-default.xml         |  38 ++++
 .../nodemanager/NodeStatusUpdaterImpl.java      |   4 +-
 .../AbstractNodeDescriptorsProvider.java        | 197 ++++++++++++++++
 .../nodelabels/AbstractNodeLabelsProvider.java  | 149 -------------
 .../ConfigurationNodeLabelsProvider.java        |  13 +-
 .../nodelabels/NodeAttributesProvider.java      |  34 +++
 .../nodelabels/NodeDescriptorsProvider.java     |  45 ++++
 .../nodelabels/NodeDescriptorsScriptRunner.java |  84 +++++++
 .../nodelabels/NodeLabelsProvider.java          |  22 +-
 .../ScriptBasedNodeAttributesProvider.java      | 129 +++++++++++
 .../ScriptBasedNodeLabelsProvider.java          | 126 +++--------
 .../nodemanager/nodelabels/package-info.java    |  28 +++
 .../TestNodeStatusUpdaterForLabels.java         |  39 ++--
 .../TestConfigurationNodeLabelsProvider.java    |  26 ++-
 .../TestScriptBasedNodeAttributesProvider.java  | 223 +++++++++++++++++++
 .../TestScriptBasedNodeLabelsProvider.java      |  18 +-
 17 files changed, 910 insertions(+), 298 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index af31fb3..5bae5f4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -3542,6 +3542,9 @@ public class YarnConfiguration extends Configuration {
   private static final String NM_NODE_LABELS_PREFIX = NM_PREFIX
       + "node-labels.";
 
+  private static final String NM_NODE_ATTRIBUTES_PREFIX = NM_PREFIX
+      + "node-attributes.";
+
   public static final String NM_NODE_LABELS_PROVIDER_CONFIG =
       NM_NODE_LABELS_PREFIX + "provider";
 
@@ -3552,6 +3555,9 @@ public class YarnConfiguration extends Configuration {
   private static final String NM_NODE_LABELS_PROVIDER_PREFIX =
       NM_NODE_LABELS_PREFIX + "provider.";
 
+  private static final String NM_NODE_ATTRIBUTES_PROVIDER_PREFIX =
+      NM_NODE_ATTRIBUTES_PREFIX + "provider.";
+
   public static final String NM_NODE_LABELS_RESYNC_INTERVAL =
       NM_NODE_LABELS_PREFIX + "resync-interval-ms";
 
@@ -3623,6 +3629,33 @@ public class YarnConfiguration extends Configuration {
       NM_SCRIPT_BASED_NODE_LABELS_PROVIDER_PREFIX + "opts";
 
   /**
+   * Node attribute provider fetch attributes interval and timeout.
+   */
+  public static final String NM_NODE_ATTRIBUTES_PROVIDER_FETCH_INTERVAL_MS =
+      NM_NODE_ATTRIBUTES_PROVIDER_PREFIX + "fetch-interval-ms";
+
+  public static final long
+      DEFAULT_NM_NODE_ATTRIBUTES_PROVIDER_FETCH_INTERVAL_MS = 10 * 60 * 1000;
+
+  public static final String NM_NODE_ATTRIBUTES_PROVIDER_FETCH_TIMEOUT_MS =
+      NM_NODE_ATTRIBUTES_PROVIDER_PREFIX + "fetch-timeout-ms";
+
+  public static final long DEFAULT_NM_NODE_ATTRIBUTES_PROVIDER_FETCH_TIMEOUT_MS
+      = DEFAULT_NM_NODE_ATTRIBUTES_PROVIDER_FETCH_INTERVAL_MS * 2;
+
+  /**
+   * Script to collect node attributes.
+   */
+  private static final String NM_SCRIPT_BASED_NODE_ATTRIBUTES_PROVIDER_PREFIX =
+      NM_NODE_ATTRIBUTES_PROVIDER_PREFIX + "script.";
+
+  public static final String NM_SCRIPT_BASED_NODE_ATTRIBUTES_PROVIDER_PATH =
+      NM_SCRIPT_BASED_NODE_ATTRIBUTES_PROVIDER_PREFIX + "path";
+
+  public static final String NM_SCRIPT_BASED_NODE_ATTRIBUTES_PROVIDER_OPTS =
+      NM_SCRIPT_BASED_NODE_ATTRIBUTES_PROVIDER_PREFIX + "opts";
+
+  /*
    * Support to view apps for given user in secure cluster.
    * @deprecated This field is deprecated for {@link #FILTER_ENTITY_LIST_BY_USER}
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 70dbf6e..65b2a6d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -2901,6 +2901,44 @@
     <value>1800000</value>
   </property>
 
+  <!-- Distributed Node Attributes Configuration -->
+  <property>
+    <description>
+      The node attribute script NM runs to collect node attributes.
+      Script output Line starting with "NODE_ATTRIBUTE:" will be
+      considered as a record of node attribute, attribute name, type
+      and value should be delimited by comma. Each of such lines
+      will be parsed to a node attribute.
+    </description>
+    <name>yarn.nodemanager.node-attributes.provider.script.path</name>
+  </property>
+
+  <property>
+    <description>
+      Command arguments passed to the node attribute script.
+    </description>
+    <name>yarn.nodemanager.node-attributes.provider.script.opts</name>
+  </property>
+
+  <property>
+    <description>
+      Time interval that determines how long NM fetches node attributes
+      from a given provider. If -1 is configured then node labels are
+      retrieved from provider only during initialization. Defaults to 10 mins.
+    </description>
+    <name>yarn.nodemanager.node-attributes.provider.fetch-interval-ms</name>
+    <value>600000</value>
+  </property>
+
+  <property>
+    <description>
+      Timeout period after which NM will interrupt the node attribute
+      provider script which queries node attributes. Defaults to 20 mins.
+    </description>
+    <name>yarn.nodemanager.node-attributes.provider.fetch-timeout-ms</name>
+    <value>1200000</value>
+  </property>
+
   <property>
     <description>
     Timeout in seconds for YARN node graceful decommission.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index faf7adb..33096bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -932,7 +932,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
     @Override
     public Set<NodeLabel> getNodeLabelsForRegistration() {
-      Set<NodeLabel> nodeLabels = nodeLabelsProvider.getNodeLabels();
+      Set<NodeLabel> nodeLabels = nodeLabelsProvider.getDescriptors();
       nodeLabels = (null == nodeLabels)
           ? CommonNodeLabelsManager.EMPTY_NODELABEL_SET : nodeLabels;
       previousNodeLabels = nodeLabels;
@@ -967,7 +967,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     @Override
     public Set<NodeLabel> getNodeLabelsForHeartbeat() {
       Set<NodeLabel> nodeLabelsForHeartbeat =
-          nodeLabelsProvider.getNodeLabels();
+          nodeLabelsProvider.getDescriptors();
       // if the provider returns null then consider empty labels are set
       nodeLabelsForHeartbeat = (nodeLabelsForHeartbeat == null)
           ? CommonNodeLabelsManager.EMPTY_NODELABEL_SET

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/AbstractNodeDescriptorsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/AbstractNodeDescriptorsProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/AbstractNodeDescriptorsProvider.java
new file mode 100644
index 0000000..088c9cb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/AbstractNodeDescriptorsProvider.java
@@ -0,0 +1,197 @@
+/**
+ * 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.nodemanager.nodelabels;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.Collections;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+
+/**
+ * Provides base implementation of NodeDescriptorsProvider with Timer and
+ * expects subclass to provide TimerTask which can fetch node descriptors.
+ */
+public abstract class AbstractNodeDescriptorsProvider<T>
+    extends AbstractService implements NodeDescriptorsProvider<T> {
+  public static final long DISABLE_NODE_DESCRIPTORS_PROVIDER_FETCH_TIMER = -1;
+
+  // Delay after which timer task are triggered to fetch node descriptors.
+  // Default interval is -1 means it is an one time task, each implementation
+  // will override this value from configuration.
+  private long intervalTime = -1;
+
+  // Timer used to schedule node descriptors fetching
+  private Timer scheduler;
+
+  protected Lock readLock = null;
+  protected Lock writeLock = null;
+
+  protected TimerTask timerTask;
+
+  private Set<T> nodeDescriptors = Collections
+      .unmodifiableSet(new HashSet<>(0));
+
+  public AbstractNodeDescriptorsProvider(String name) {
+    super(name);
+  }
+
+  public long getIntervalTime() {
+    return intervalTime;
+  }
+
+  public void setIntervalTime(long intervalMS) {
+    this.intervalTime = intervalMS;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+    readLock = readWriteLock.readLock();
+    writeLock = readWriteLock.writeLock();
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    timerTask = createTimerTask();
+    timerTask.run();
+    long taskInterval = getIntervalTime();
+    if (taskInterval != DISABLE_NODE_DESCRIPTORS_PROVIDER_FETCH_TIMER) {
+      scheduler =
+          new Timer("DistributedNodeDescriptorsRunner-Timer", true);
+      // Start the timer task and then periodically at the configured interval
+      // time. Illegal values for intervalTime is handled by timer api
+      scheduler.schedule(timerTask, taskInterval, taskInterval);
+    }
+    super.serviceStart();
+  }
+
+  /**
+   * terminate the timer
+   * @throws Exception
+   */
+  @Override
+  protected void serviceStop() throws Exception {
+    if (scheduler != null) {
+      scheduler.cancel();
+    }
+    cleanUp();
+    super.serviceStop();
+  }
+
+  /**
+   * method for subclasses to cleanup.
+   */
+  protected abstract void cleanUp() throws Exception ;
+
+  /**
+   * @return Returns output from provider.
+   */
+  @Override
+  public Set<T> getDescriptors() {
+    readLock.lock();
+    try {
+      return this.nodeDescriptors;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public void setDescriptors(Set<T> descriptorsSet) {
+    writeLock.lock();
+    try {
+      this.nodeDescriptors = descriptorsSet;
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Method used to determine if or not node descriptors fetching script is
+   * configured and whether it is fit to run. Returns true if following
+   * conditions are met:
+   *
+   * <ol>
+   * <li>Path to the script is not empty</li>
+   * <li>The script file exists</li>
+   * </ol>
+   *
+   * @throws IOException
+   */
+  protected void verifyConfiguredScript(String scriptPath)
+      throws IOException {
+    boolean invalidConfiguration;
+    if (scriptPath == null
+        || scriptPath.trim().isEmpty()) {
+      invalidConfiguration = true;
+    } else {
+      File f = new File(scriptPath);
+      invalidConfiguration = !f.exists() || !FileUtil.canExecute(f);
+    }
+    if (invalidConfiguration) {
+      throw new IOException(
+          "Node descriptors provider script \"" + scriptPath
+              + "\" is not configured properly. Please check whether"
+              + " the script path exists, owner and the access rights"
+              + " are suitable for NM process to execute it");
+    }
+  }
+
+  static Set<NodeLabel> convertToNodeLabelSet(String partitionNodeLabel) {
+    if (null == partitionNodeLabel) {
+      return null;
+    }
+    Set<NodeLabel> labels = new HashSet<NodeLabel>();
+    labels.add(NodeLabel.newInstance(partitionNodeLabel));
+    return labels;
+  }
+
+  /**
+   * Used only by tests to access the timer task directly
+   *
+   * @return the timer task
+   */
+  TimerTask getTimerTask() {
+    return timerTask;
+  }
+
+  @VisibleForTesting
+  public Timer getScheduler() {
+    return this.scheduler;
+  }
+
+  /**
+   * Creates a timer task which be scheduled periodically by the provider,
+   * and the task is responsible to update node descriptors to the provider.
+   * @return a timer task.
+   */
+  public abstract TimerTask createTimerTask();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/AbstractNodeLabelsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/AbstractNodeLabelsProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/AbstractNodeLabelsProvider.java
deleted file mode 100644
index c810654..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/AbstractNodeLabelsProvider.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/**
- * 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.nodemanager.nodelabels;
-
-import java.util.HashSet;
-import java.util.Set;
-import java.util.Timer;
-import java.util.TimerTask;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.records.NodeLabel;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
-
-/**
- * Provides base implementation of NodeLabelsProvider with Timer and expects
- * subclass to provide TimerTask which can fetch NodeLabels
- */
-public abstract class AbstractNodeLabelsProvider extends AbstractService
-    implements NodeLabelsProvider {
-  public static final long DISABLE_NODE_LABELS_PROVIDER_FETCH_TIMER = -1;
-
-  // Delay after which timer task are triggered to fetch NodeLabels
-  protected long intervalTime;
-
-  // Timer used to schedule node labels fetching
-  protected Timer nodeLabelsScheduler;
-
-  public static final String NODE_LABELS_SEPRATOR = ",";
-
-  protected Lock readLock = null;
-  protected Lock writeLock = null;
-
-  protected TimerTask timerTask;
-
-  protected Set<NodeLabel> nodeLabels =
-      CommonNodeLabelsManager.EMPTY_NODELABEL_SET;
-
-
-  public AbstractNodeLabelsProvider(String name) {
-    super(name);
-  }
-
-  @Override
-  protected void serviceInit(Configuration conf) throws Exception {
-    this.intervalTime =
-        conf.getLong(YarnConfiguration.NM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS,
-            YarnConfiguration.DEFAULT_NM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS);
-
-    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
-    readLock = readWriteLock.readLock();
-    writeLock = readWriteLock.writeLock();
-    super.serviceInit(conf);
-  }
-
-  @Override
-  protected void serviceStart() throws Exception {
-    timerTask = createTimerTask();
-    timerTask.run();
-    if (intervalTime != DISABLE_NODE_LABELS_PROVIDER_FETCH_TIMER) {
-      nodeLabelsScheduler =
-          new Timer("DistributedNodeLabelsRunner-Timer", true);
-      // Start the timer task and then periodically at the configured interval
-      // time. Illegal values for intervalTime is handled by timer api
-      nodeLabelsScheduler.scheduleAtFixedRate(timerTask, intervalTime,
-          intervalTime);
-    }
-    super.serviceStart();
-  }
-
-  /**
-   * terminate the timer
-   * @throws Exception
-   */
-  @Override
-  protected void serviceStop() throws Exception {
-    if (nodeLabelsScheduler != null) {
-      nodeLabelsScheduler.cancel();
-    }
-    cleanUp();
-    super.serviceStop();
-  }
-
-  /**
-   * method for subclasses to cleanup.
-   */
-  protected abstract void cleanUp() throws Exception ;
-
-  /**
-   * @return Returns output from provider.
-   */
-  @Override
-  public Set<NodeLabel> getNodeLabels() {
-    readLock.lock();
-    try {
-      return nodeLabels;
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  protected void setNodeLabels(Set<NodeLabel> nodeLabelsSet) {
-    writeLock.lock();
-    try {
-      nodeLabels = nodeLabelsSet;
-    } finally {
-      writeLock.unlock();
-    }
-  }
-
-  static Set<NodeLabel> convertToNodeLabelSet(String partitionNodeLabel) {
-    if (null == partitionNodeLabel) {
-      return null;
-    }
-    Set<NodeLabel> labels = new HashSet<NodeLabel>();
-    labels.add(NodeLabel.newInstance(partitionNodeLabel));
-    return labels;
-  }
-
-  /**
-   * Used only by tests to access the timer task directly
-   *
-   * @return the timer task
-   */
-  TimerTask getTimerTask() {
-    return timerTask;
-  }
-
-  public abstract TimerTask createTimerTask();
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeLabelsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeLabelsProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeLabelsProvider.java
index 7490cc2..1c6af8e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeLabelsProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ConfigurationNodeLabelsProvider.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 /**
  * Provides Node's Labels by constantly monitoring the configuration.
  */
-public class ConfigurationNodeLabelsProvider extends AbstractNodeLabelsProvider {
+public class ConfigurationNodeLabelsProvider extends NodeLabelsProvider {
 
   private static final Logger LOG =
        LoggerFactory.getLogger(ConfigurationNodeLabelsProvider.class);
@@ -38,11 +38,20 @@ public class ConfigurationNodeLabelsProvider extends AbstractNodeLabelsProvider
     super("Configuration Based NodeLabels Provider");
   }
 
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    long taskInterval = conf.getLong(
+        YarnConfiguration.NM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS,
+        YarnConfiguration.DEFAULT_NM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS);
+    this.setIntervalTime(taskInterval);
+    super.serviceInit(conf);
+  }
+
   private void updateNodeLabelsFromConfig(Configuration conf)
       throws IOException {
     String configuredNodePartition =
         conf.get(YarnConfiguration.NM_PROVIDER_CONFIGURED_NODE_PARTITION, null);
-    setNodeLabels(convertToNodeLabelSet(configuredNodePartition));
+    setDescriptors(convertToNodeLabelSet(configuredNodePartition));
   }
 
   private class ConfigurationMonitorTimerTask extends TimerTask {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeAttributesProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeAttributesProvider.java
new file mode 100644
index 0000000..8240024
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeAttributesProvider.java
@@ -0,0 +1,34 @@
+/**
+ * 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.nodemanager.nodelabels;
+
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+
+import java.util.Set;
+
+/**
+ * Abstract class which will be responsible for fetching the node attributes.
+ *
+ */
+public abstract class NodeAttributesProvider
+    extends AbstractNodeDescriptorsProvider<NodeAttribute> {
+
+  public NodeAttributesProvider(String name) {
+    super(name);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeDescriptorsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeDescriptorsProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeDescriptorsProvider.java
new file mode 100644
index 0000000..51608b5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeDescriptorsProvider.java
@@ -0,0 +1,45 @@
+/**
+ * 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.nodemanager.nodelabels;
+
+import java.util.Set;
+
+/**
+ * Interface which will be responsible for fetching node descriptors,
+ * a node descriptor could be a
+ * {@link org.apache.hadoop.yarn.api.records.NodeLabel} or a
+ * {@link org.apache.hadoop.yarn.api.records.NodeAttribute}.
+ */
+public interface NodeDescriptorsProvider<T> {
+
+  /**
+   * Provides the descriptors. The provider is expected to give same
+   * descriptors continuously until there is a change.
+   * If null is returned then an empty set is assumed by the caller.
+   *
+   * @return Set of node descriptors applicable for a node
+   */
+  Set<T> getDescriptors();
+
+  /**
+   * Sets a set of descriptors to the provider.
+   * @param descriptors node descriptors.
+   */
+  void setDescriptors(Set<T> descriptors);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeDescriptorsScriptRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeDescriptorsScriptRunner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeDescriptorsScriptRunner.java
new file mode 100644
index 0000000..6365f3b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeDescriptorsScriptRunner.java
@@ -0,0 +1,84 @@
+/**
+ * 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.nodemanager.nodelabels;
+
+import org.apache.hadoop.util.Shell;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.TimerTask;
+
+/**
+ * A node descriptors script runner periodically runs a script,
+ * parses the output to collect desired descriptors, and then
+ * post these descriptors to the given {@link NodeDescriptorsProvider}.
+ * @param <T> a certain type of descriptor.
+ */
+public abstract class NodeDescriptorsScriptRunner<T> extends TimerTask {
+
+  private final static Logger LOG = LoggerFactory
+      .getLogger(NodeDescriptorsScriptRunner.class);
+
+  private final Shell.ShellCommandExecutor exec;
+  private final NodeDescriptorsProvider provider;
+
+  public NodeDescriptorsScriptRunner(String scriptPath,
+      String[] scriptArgs, long scriptTimeout,
+      NodeDescriptorsProvider ndProvider) {
+    ArrayList<String> execScript = new ArrayList<>();
+    execScript.add(scriptPath);
+    if (scriptArgs != null) {
+      execScript.addAll(Arrays.asList(scriptArgs));
+    }
+    this.provider = ndProvider;
+    this.exec = new Shell.ShellCommandExecutor(
+        execScript.toArray(new String[execScript.size()]), null, null,
+        scriptTimeout);
+  }
+
+  @Override
+  public void run() {
+    try {
+      exec.execute();
+      provider.setDescriptors(parseOutput(exec.getOutput()));
+    } catch (Exception e) {
+      if (exec.isTimedOut()) {
+        LOG.warn("Node Labels script timed out, Caught exception : "
+            + e.getMessage(), e);
+      } else {
+        LOG.warn("Execution of Node Labels script failed, Caught exception : "
+            + e.getMessage(), e);
+      }
+    }
+  }
+
+  public void cleanUp() {
+    if (exec != null) {
+      Process p = exec.getProcess();
+      if (p != null) {
+        p.destroy();
+      }
+    }
+  }
+
+  abstract Set<T> parseOutput(String scriptOutput) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeLabelsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeLabelsProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeLabelsProvider.java
index 0c076ef..9610568 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeLabelsProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeLabelsProvider.java
@@ -18,22 +18,16 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.nodelabels;
 
-import java.util.Set;
-
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 
 /**
- * Interface which will be responsible for fetching the labels
- * 
+ * Abstract class which will be responsible for fetching the node labels.
+ *
  */
-public interface NodeLabelsProvider {
+public abstract class NodeLabelsProvider
+    extends AbstractNodeDescriptorsProvider<NodeLabel>{
 
-  /**
-   * Provides the labels. LabelProvider is expected to give same Labels
-   * continuously until there is a change in labels. 
-   * If null is returned then Empty label set is assumed by the caller.
-   * 
-   * @return Set of node label strings applicable for a node
-   */
-  public abstract Set<NodeLabel> getNodeLabels();
-}
\ No newline at end of file
+  public NodeLabelsProvider(String name) {
+    super(name);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java
new file mode 100644
index 0000000..06771ba
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java
@@ -0,0 +1,129 @@
+/**
+ * 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.nodemanager.nodelabels;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.TimerTask;
+
+import static org.apache.hadoop.yarn.conf.YarnConfiguration
+    .NM_SCRIPT_BASED_NODE_ATTRIBUTES_PROVIDER_PATH;
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.
+    NM_SCRIPT_BASED_NODE_ATTRIBUTES_PROVIDER_OPTS;
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.
+    NM_NODE_ATTRIBUTES_PROVIDER_FETCH_INTERVAL_MS;
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.
+    NM_NODE_ATTRIBUTES_PROVIDER_FETCH_TIMEOUT_MS;
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.
+    DEFAULT_NM_NODE_ATTRIBUTES_PROVIDER_FETCH_INTERVAL_MS;
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.
+    DEFAULT_NM_NODE_ATTRIBUTES_PROVIDER_FETCH_TIMEOUT_MS;
+
+/**
+ * Node attribute provider that periodically runs a script to collect
+ * node attributes.
+ */
+public class ScriptBasedNodeAttributesProvider extends NodeAttributesProvider{
+
+  private static final String NODE_ATTRIBUTE_PATTERN = "NODE_ATTRIBUTE:";
+  private static final String NODE_ATTRIBUTE_DELIMITER = ",";
+
+  private NodeAttributeScriptRunner runner;
+
+  public ScriptBasedNodeAttributesProvider() {
+    super(ScriptBasedNodeAttributesProvider.class.getName());
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    String nodeAttributeProviderScript = conf.get(
+        NM_SCRIPT_BASED_NODE_ATTRIBUTES_PROVIDER_PATH);
+    long scriptTimeout = conf.getLong(
+        NM_NODE_ATTRIBUTES_PROVIDER_FETCH_TIMEOUT_MS,
+        DEFAULT_NM_NODE_ATTRIBUTES_PROVIDER_FETCH_TIMEOUT_MS);
+    String[] scriptArgs = conf.getStrings(
+        NM_SCRIPT_BASED_NODE_ATTRIBUTES_PROVIDER_OPTS,
+        new String[] {});
+    verifyConfiguredScript(nodeAttributeProviderScript);
+
+    long intervalTime = conf.getLong(
+        NM_NODE_ATTRIBUTES_PROVIDER_FETCH_INTERVAL_MS,
+        DEFAULT_NM_NODE_ATTRIBUTES_PROVIDER_FETCH_INTERVAL_MS);
+    this.setIntervalTime(intervalTime);
+
+    this.runner = new NodeAttributeScriptRunner(nodeAttributeProviderScript,
+        scriptArgs, scriptTimeout, this);
+  }
+
+  @Override
+  protected void cleanUp() throws Exception {
+    runner.cleanUp();
+  }
+
+  @Override
+  public TimerTask createTimerTask() {
+    return runner;
+  }
+
+  private static class NodeAttributeScriptRunner extends
+      NodeDescriptorsScriptRunner<NodeAttribute> {
+
+    NodeAttributeScriptRunner(String scriptPath, String[] scriptArgs,
+        long scriptTimeout, ScriptBasedNodeAttributesProvider provider) {
+      super(scriptPath, scriptArgs, scriptTimeout, provider);
+    }
+
+    @Override
+    Set<NodeAttribute> parseOutput(String scriptOutput) throws IOException {
+      Set<NodeAttribute> attributeSet = new HashSet<>();
+      // TODO finalize format
+
+      // each line is a record of ndoe attribute like following:
+      // NODE_ATTRIBUTE:ATTRIBUTE_NAME,ATTRIBUTE_TYPE,ATTRIBUTE_VALUE
+      String[] splits = scriptOutput.split("\n");
+      for (String line : splits) {
+        String trimmedLine = line.trim();
+        if (trimmedLine.startsWith(NODE_ATTRIBUTE_PATTERN)) {
+          String nodeAttribute = trimmedLine
+              .substring(NODE_ATTRIBUTE_PATTERN.length());
+          String[] attributeStrs = nodeAttribute
+              .split(NODE_ATTRIBUTE_DELIMITER);
+          if (attributeStrs.length != 3) {
+            throw new IOException("Malformed output, expecting format "
+                + NODE_ATTRIBUTE_PATTERN + ":" + "ATTRIBUTE_NAME"
+                + NODE_ATTRIBUTE_DELIMITER + "ATTRIBUTE_TYPE"
+                + NODE_ATTRIBUTE_DELIMITER + "ATTRIBUTE_VALUE; but get "
+                + nodeAttribute);
+          }
+          NodeAttribute na = NodeAttribute
+              .newInstance(attributeStrs[0],
+                  NodeAttributeType.valueOf(attributeStrs[1]),
+                  attributeStrs[2]);
+          attributeSet.add(na);
+        }
+      }
+      return attributeSet;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeLabelsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeLabelsProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeLabelsProvider.java
index 32f180a..c867de9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeLabelsProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeLabelsProvider.java
@@ -18,19 +18,11 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.nodelabels;
 
-import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Set;
-import java.util.Timer;
 import java.util.TimerTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
@@ -40,20 +32,12 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
  * pattern which will be used to search node label partition from the out put of
  * the NodeLabels provider script
  */
-public class ScriptBasedNodeLabelsProvider extends AbstractNodeLabelsProvider {
-  /** Absolute path to the node labels script. */
-  private String nodeLabelsScriptPath;
-
-  /** Time after which the script should be timed out */
-  private long scriptTimeout;
-
-  /** ShellCommandExecutor used to execute monitoring script */
-  ShellCommandExecutor shexec = null;
+public class ScriptBasedNodeLabelsProvider extends NodeLabelsProvider {
 
   /** Pattern used for searching in the output of the node labels script */
   public static final String NODE_LABEL_PARTITION_PATTERN = "NODE_PARTITION:";
 
-  private String[] scriptArgs;
+  private NodeDescriptorsScriptRunner runner;
 
   public ScriptBasedNodeLabelsProvider() {
     super(ScriptBasedNodeLabelsProvider.class.getName());
@@ -64,48 +48,24 @@ public class ScriptBasedNodeLabelsProvider extends AbstractNodeLabelsProvider {
    */
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
-    super.serviceInit(conf);
-    this.nodeLabelsScriptPath =
+    String nodeLabelsScriptPath =
         conf.get(YarnConfiguration.NM_SCRIPT_BASED_NODE_LABELS_PROVIDER_PATH);
-    this.scriptTimeout =
+    long scriptTimeout =
         conf.getLong(YarnConfiguration.NM_NODE_LABELS_PROVIDER_FETCH_TIMEOUT_MS,
             YarnConfiguration.DEFAULT_NM_NODE_LABELS_PROVIDER_FETCH_TIMEOUT_MS);
-    scriptArgs = conf.getStrings(
+    String[] scriptArgs = conf.getStrings(
         YarnConfiguration.NM_SCRIPT_BASED_NODE_LABELS_PROVIDER_SCRIPT_OPTS,
         new String[] {});
+    verifyConfiguredScript(nodeLabelsScriptPath);
 
-    verifyConfiguredScript();
-  }
+    long taskInterval = conf.getLong(
+        YarnConfiguration.NM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS,
+        YarnConfiguration.DEFAULT_NM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS);
+    this.setIntervalTime(taskInterval);
+    this.runner = new NodeLabelScriptRunner(nodeLabelsScriptPath, scriptArgs,
+            scriptTimeout, this);
 
-  /**
-   * Method used to determine if or not node labels fetching script is
-   * configured and whether it is fit to run. Returns true if following
-   * conditions are met:
-   *
-   * <ol>
-   * <li>Path to Node Labels fetch script is not empty</li>
-   * <li>Node Labels fetch script file exists</li>
-   * </ol>
-   *
-   * @throws IOException
-   */
-  private void verifyConfiguredScript()
-      throws IOException {
-    boolean invalidConfiguration = false;
-    if (nodeLabelsScriptPath == null
-        || nodeLabelsScriptPath.trim().isEmpty()) {
-      invalidConfiguration = true;
-    } else {
-      File f = new File(nodeLabelsScriptPath);
-      invalidConfiguration = !f.exists() || !FileUtil.canExecute(f);
-    }
-    if (invalidConfiguration) {
-      throw new IOException(
-          "Distributed Node labels provider script \"" + nodeLabelsScriptPath
-              + "\" is not configured properly. Please check whether the script "
-              + "path exists, owner and the access rights are suitable for NM "
-              + "process to execute it");
-    }
+    super.serviceInit(conf);
   }
 
   /**
@@ -113,53 +73,19 @@ public class ScriptBasedNodeLabelsProvider extends AbstractNodeLabelsProvider {
    */
   @Override
   public void cleanUp() {
-    if (shexec != null) {
-      Process p = shexec.getProcess();
-      if (p != null) {
-        p.destroy();
-      }
+    if (runner != null) {
+      runner.cleanUp();
     }
   }
 
-  @Override
-  public TimerTask createTimerTask() {
-    return new NodeLabelsScriptRunner();
-  }
-
-  /**
-   * Class which is used by the {@link Timer} class to periodically execute the
-   * node labels script.
-   */
-  private class NodeLabelsScriptRunner extends TimerTask {
+  // A script runner periodically runs a script to get node labels,
+  // and sets these labels to the given provider.
+  private static class NodeLabelScriptRunner extends
+      NodeDescriptorsScriptRunner<NodeLabel> {
 
-    private final Logger LOG =
-        LoggerFactory.getLogger(NodeLabelsScriptRunner.class);
-
-    public NodeLabelsScriptRunner() {
-      ArrayList<String> execScript = new ArrayList<String>();
-      execScript.add(nodeLabelsScriptPath);
-      if (scriptArgs != null) {
-        execScript.addAll(Arrays.asList(scriptArgs));
-      }
-      shexec = new ShellCommandExecutor(
-          execScript.toArray(new String[execScript.size()]), null, null,
-          scriptTimeout);
-    }
-
-    @Override
-    public void run() {
-      try {
-        shexec.execute();
-        setNodeLabels(fetchLabelsFromScriptOutput(shexec.getOutput()));
-      } catch (Exception e) {
-        if (shexec.isTimedOut()) {
-          LOG.warn("Node Labels script timed out, Caught exception : "
-              + e.getMessage(), e);
-        } else {
-          LOG.warn("Execution of Node Labels script failed, Caught exception : "
-              + e.getMessage(), e);
-        }
-      }
+    NodeLabelScriptRunner(String scriptPath, String[] scriptArgs,
+        long scriptTimeout, ScriptBasedNodeLabelsProvider provider) {
+      super(scriptPath, scriptArgs, scriptTimeout, provider);
     }
 
     /**
@@ -170,7 +96,8 @@ public class ScriptBasedNodeLabelsProvider extends AbstractNodeLabelsProvider {
      * @return true if output string has error pattern in it.
      * @throws IOException
      */
-    private Set<NodeLabel> fetchLabelsFromScriptOutput(String scriptOutput)
+    @Override
+    Set<NodeLabel> parseOutput(String scriptOutput)
         throws IOException {
       String nodePartitionLabel = null;
       String[] splits = scriptOutput.split("\n");
@@ -184,4 +111,9 @@ public class ScriptBasedNodeLabelsProvider extends AbstractNodeLabelsProvider {
       return convertToNodeLabelSet(nodePartitionLabel);
     }
   }
+
+  @Override
+  public TimerTask createTimerTask() {
+    return runner;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/package-info.java
new file mode 100644
index 0000000..190022f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * 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 to encapsulate classes used to handle node labels and node
+ * attributes in NM.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.nodemanager.nodelabels;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
index 257e18c..7ef23cb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
@@ -26,6 +26,7 @@ import java.io.IOException;
 import java.lang.Thread.State;
 import java.nio.ByteBuffer;
 import java.util.Set;
+import java.util.TimerTask;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.net.ServerSocketUtil;
@@ -179,17 +180,27 @@ public class TestNodeStatusUpdaterForLabels extends NodeLabelTestBase {
     }
   }
 
-  public static class DummyNodeLabelsProvider implements NodeLabelsProvider {
+  public static class DummyNodeLabelsProvider extends NodeLabelsProvider {
 
-    private Set<NodeLabel> nodeLabels = CommonNodeLabelsManager.EMPTY_NODELABEL_SET;
+    public DummyNodeLabelsProvider() {
+      super("DummyNodeLabelsProvider");
+      // disable the fetch timer.
+      setIntervalTime(-1);
+    }
 
     @Override
-    public synchronized Set<NodeLabel> getNodeLabels() {
-      return nodeLabels;
+    protected void cleanUp() throws Exception {
+      // fake implementation, nothing to cleanup
     }
 
-    synchronized void setNodeLabels(Set<NodeLabel> nodeLabels) {
-      this.nodeLabels = nodeLabels;
+    @Override
+    public TimerTask createTimerTask() {
+      return new TimerTask() {
+        @Override
+        public void run() {
+          setDescriptors(CommonNodeLabelsManager.EMPTY_NODELABEL_SET);
+        }
+      };
     }
   }
 
@@ -241,18 +252,18 @@ public class TestNodeStatusUpdaterForLabels extends NodeLabelTestBase {
     resourceTracker.resetNMHeartbeatReceiveFlag();
     nm.start();
     resourceTracker.waitTillRegister();
-    assertNLCollectionEquals(dummyLabelsProviderRef.getNodeLabels(),
+    assertNLCollectionEquals(dummyLabelsProviderRef.getDescriptors(),
         resourceTracker.labels);
 
     resourceTracker.waitTillHeartbeat();// wait till the first heartbeat
     resourceTracker.resetNMHeartbeatReceiveFlag();
 
     // heartbeat with updated labels
-    dummyLabelsProviderRef.setNodeLabels(toNodeLabelSet("P"));
+    dummyLabelsProviderRef.setDescriptors(toNodeLabelSet("P"));
 
     sendOutofBandHeartBeat();
     resourceTracker.waitTillHeartbeat();
-    assertNLCollectionEquals(dummyLabelsProviderRef.getNodeLabels(),
+    assertNLCollectionEquals(dummyLabelsProviderRef.getDescriptors(),
         resourceTracker.labels);
     resourceTracker.resetNMHeartbeatReceiveFlag();
 
@@ -265,7 +276,7 @@ public class TestNodeStatusUpdaterForLabels extends NodeLabelTestBase {
         resourceTracker.labels);
 
     // provider return with null labels
-    dummyLabelsProviderRef.setNodeLabels(null);
+    dummyLabelsProviderRef.setDescriptors(null);
     sendOutofBandHeartBeat();
     resourceTracker.waitTillHeartbeat();
     assertNotNull(
@@ -279,7 +290,7 @@ public class TestNodeStatusUpdaterForLabels extends NodeLabelTestBase {
     // so that every sec 1 heartbeat is send.
     int nullLabels = 0;
     int nonNullLabels = 0;
-    dummyLabelsProviderRef.setNodeLabels(toNodeLabelSet("P1"));
+    dummyLabelsProviderRef.setDescriptors(toNodeLabelSet("P1"));
     for (int i = 0; i < 5; i++) {
       sendOutofBandHeartBeat();
       resourceTracker.waitTillHeartbeat();
@@ -331,19 +342,19 @@ public class TestNodeStatusUpdaterForLabels extends NodeLabelTestBase {
         };
       }
     };
-    dummyLabelsProviderRef.setNodeLabels(toNodeLabelSet("P"));
+
     YarnConfiguration conf = createNMConfigForDistributeNodeLabels();
     conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "0.0.0.0:"
         + ServerSocketUtil.getPort(8040, 10));
-
     nm.init(conf);
     resourceTracker.resetNMHeartbeatReceiveFlag();
     nm.start();
+    dummyLabelsProviderRef.setDescriptors(toNodeLabelSet("P"));
     resourceTracker.waitTillHeartbeat();// wait till the first heartbeat
     resourceTracker.resetNMHeartbeatReceiveFlag();
 
     // heartbeat with invalid labels
-    dummyLabelsProviderRef.setNodeLabels(toNodeLabelSet("_.P"));
+    dummyLabelsProviderRef.setDescriptors(toNodeLabelSet("_.P"));
 
     sendOutofBandHeartBeat();
     resourceTracker.waitTillHeartbeat();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeLabelsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeLabelsProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeLabelsProvider.java
index 6b4d883..2acd9b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeLabelsProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestConfigurationNodeLabelsProvider.java
@@ -98,32 +98,34 @@ public class TestConfigurationNodeLabelsProvider extends NodeLabelTestBase {
     // test for ensuring labels are set during initialization of the class
     nodeLabelsProvider.start();
     assertNLCollectionEquals(toNodeLabelSet("A"),
-        nodeLabelsProvider.getNodeLabels());
+        nodeLabelsProvider.getDescriptors());
 
     // test for valid Modification
     TimerTask timerTask = nodeLabelsProvider.getTimerTask();
     modifyConf("X");
     timerTask.run();
     assertNLCollectionEquals(toNodeLabelSet("X"),
-        nodeLabelsProvider.getNodeLabels());
+        nodeLabelsProvider.getDescriptors());
   }
 
   @Test
   public void testConfigForNoTimer() throws Exception {
     Configuration conf = new Configuration();
     modifyConf("A");
-    conf.setLong(YarnConfiguration.NM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS,
-        AbstractNodeLabelsProvider.DISABLE_NODE_LABELS_PROVIDER_FETCH_TIMER);
+    conf.setLong(YarnConfiguration
+            .NM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS,
+        AbstractNodeDescriptorsProvider
+            .DISABLE_NODE_DESCRIPTORS_PROVIDER_FETCH_TIMER);
     nodeLabelsProvider.init(conf);
     nodeLabelsProvider.start();
     Assert
-        .assertNull(
-            "Timer is not expected to be created when interval is configured as -1",
-            nodeLabelsProvider.nodeLabelsScheduler);
-    // Ensure that even though timer is not run, node labels are fetched at least once so
-    // that NM registers/updates Labels with RM
+        .assertNull("Timer is not expected to be"
+                + " created when interval is configured as -1",
+            nodeLabelsProvider.getScheduler());
+    // Ensure that even though timer is not run, node labels
+    // are fetched at least once so that NM registers/updates Labels with RM
     assertNLCollectionEquals(toNodeLabelSet("A"),
-        nodeLabelsProvider.getNodeLabels());
+        nodeLabelsProvider.getDescriptors());
   }
 
   @Test
@@ -138,11 +140,11 @@ public class TestConfigurationNodeLabelsProvider extends NodeLabelTestBase {
     // least once so
     // that NM registers/updates Labels with RM
     assertNLCollectionEquals(toNodeLabelSet("A"),
-        nodeLabelsProvider.getNodeLabels());
+        nodeLabelsProvider.getDescriptors());
     modifyConf("X");
     Thread.sleep(1500);
     assertNLCollectionEquals(toNodeLabelSet("X"),
-        nodeLabelsProvider.getNodeLabels());
+        nodeLabelsProvider.getDescriptors());
 
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java
new file mode 100644
index 0000000..58d2d20
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java
@@ -0,0 +1,223 @@
+/**
+ * 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.nodemanager.nodelabels;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Test cases for script based node attributes provider.
+ */
+public class TestScriptBasedNodeAttributesProvider {
+
+  private static File testRootDir = new File("target",
+      TestScriptBasedNodeAttributesProvider.class.getName() + "-localDir")
+      .getAbsoluteFile();
+
+  private final File nodeAttributeScript =
+      new File(testRootDir, Shell.appendScriptExtension("attributeScript"));
+
+  private ScriptBasedNodeAttributesProvider nodeAttributesProvider;
+
+  @Before
+  public void setup() {
+    testRootDir.mkdirs();
+    nodeAttributesProvider = new ScriptBasedNodeAttributesProvider();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (testRootDir.exists()) {
+      FileContext.getLocalFSFileContext()
+          .delete(new Path(testRootDir.getAbsolutePath()), true);
+    }
+    if (nodeAttributesProvider != null) {
+      nodeAttributesProvider.stop();
+    }
+  }
+
+  private Configuration getConfForNodeAttributeScript() {
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.NM_SCRIPT_BASED_NODE_ATTRIBUTES_PROVIDER_PATH,
+        nodeAttributeScript.getAbsolutePath());
+    // set bigger interval so that test cases can be run
+    conf.setLong(
+        YarnConfiguration.NM_NODE_ATTRIBUTES_PROVIDER_FETCH_INTERVAL_MS,
+        1000);
+    conf.setLong(
+        YarnConfiguration.NM_NODE_ATTRIBUTES_PROVIDER_FETCH_TIMEOUT_MS,
+        1000);
+    return conf;
+  }
+
+  private void writeNodeAttributeScriptFile(String scriptStr,
+      boolean setExecutable) throws IOException {
+    PrintWriter pw = null;
+    try {
+      FileUtil.setWritable(nodeAttributeScript, true);
+      FileUtil.setReadable(nodeAttributeScript, true);
+      pw = new PrintWriter(new FileOutputStream(nodeAttributeScript));
+      pw.println(scriptStr);
+      pw.flush();
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.fail();
+    } finally {
+      if (null != pw) {
+        pw.close();
+      }
+    }
+    FileUtil.setExecutable(nodeAttributeScript, setExecutable);
+  }
+
+  @Test
+  public void testNodeAttributeScriptProvider()
+      throws IOException, InterruptedException {
+    String simpleScript = "echo NODE_ATTRIBUTE:host,STRING,host1234\n "
+        + "echo NODE_ATTRIBUTE:os,STRING,redhat_6_3\n "
+        + "echo NODE_ATTRIBUTE:ip,STRING,10.0.0.1";
+    writeNodeAttributeScriptFile(simpleScript, true);
+
+    nodeAttributesProvider.init(getConfForNodeAttributeScript());
+    nodeAttributesProvider.start();
+
+    try {
+      GenericTestUtils.waitFor(
+          () -> nodeAttributesProvider.getDescriptors().size() == 3,
+          500, 3000);
+    } catch (TimeoutException e) {
+      Assert.fail("Expecting node attributes size is 3, but got "
+          + nodeAttributesProvider.getDescriptors().size());
+    }
+
+    Iterator<NodeAttribute> it = nodeAttributesProvider
+        .getDescriptors().iterator();
+    while (it.hasNext()) {
+      NodeAttribute att = it.next();
+      switch (att.getAttributeName()) {
+      case "host":
+        Assert.assertEquals(NodeAttributeType.STRING, att.getAttributeType());
+        Assert.assertEquals("host1234", att.getAttributeValue());
+        break;
+      case "os":
+        Assert.assertEquals(NodeAttributeType.STRING, att.getAttributeType());
+        Assert.assertEquals("redhat_6_3", att.getAttributeValue());
+        break;
+      case "ip":
+        Assert.assertEquals(NodeAttributeType.STRING, att.getAttributeType());
+        Assert.assertEquals("10.0.0.1", att.getAttributeValue());
+        break;
+      default:
+        Assert.fail("Unexpected attribute name " + att.getAttributeName());
+        break;
+      }
+    }
+  }
+
+  @Test
+  public void testInvalidScriptOutput()
+      throws IOException, InterruptedException {
+    // Script output doesn't have correct prefix.
+    String scriptContent = "echo host,STRING,host1234";
+    writeNodeAttributeScriptFile(scriptContent, true);
+
+    nodeAttributesProvider.init(getConfForNodeAttributeScript());
+    nodeAttributesProvider.start();
+
+    try {
+      GenericTestUtils.waitFor(
+          () -> nodeAttributesProvider.getDescriptors().size() == 1,
+          500, 3000);
+      Assert.fail("This test should timeout because the provide is unable"
+          + " to parse any attributes from the script output.");
+    } catch (TimeoutException e) {
+      Assert.assertEquals(0, nodeAttributesProvider
+          .getDescriptors().size());
+    }
+  }
+
+  @Test
+  public void testMalformedScriptOutput() throws Exception{
+    // Script output has correct prefix but each line is malformed.
+    String scriptContent =
+        "echo NODE_ATTRIBUTE:host,STRING,host1234,a_extra_column";
+    writeNodeAttributeScriptFile(scriptContent, true);
+
+    nodeAttributesProvider.init(getConfForNodeAttributeScript());
+    nodeAttributesProvider.start();
+
+    // There should be no attributes found, and we should
+    // see Malformed output warnings in the log
+    try {
+      GenericTestUtils
+          .waitFor(() -> nodeAttributesProvider
+                  .getDescriptors().size() == 1,
+              500, 3000);
+      Assert.fail("This test should timeout because the provide is unable"
+          + " to parse any attributes from the script output.");
+    } catch (TimeoutException e) {
+      Assert.assertEquals(0, nodeAttributesProvider
+          .getDescriptors().size());
+    }
+  }
+
+  @Test
+  public void testFetchInterval() throws Exception {
+    // The script returns the pid (as an attribute) each time runs this script
+    String simpleScript = "echo NODE_ATTRIBUTE:pid,STRING,$$";
+    writeNodeAttributeScriptFile(simpleScript, true);
+
+    nodeAttributesProvider.init(getConfForNodeAttributeScript());
+    nodeAttributesProvider.start();
+
+    // Wait for at most 3 seconds until we get at least 1
+    // different attribute value.
+    Set<String> resultSet = new HashSet<>();
+    GenericTestUtils.waitFor(() -> {
+      Set<NodeAttribute> attributes =
+          nodeAttributesProvider.getDescriptors();
+      if (attributes != null) {
+        Assert.assertEquals(1, attributes.size());
+        resultSet.add(attributes.iterator().next().getAttributeValue());
+        return resultSet.size() > 1;
+      } else {
+        return false;
+      }
+    }, 500, 3000);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d312b5cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeLabelsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeLabelsProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeLabelsProvider.java
index 1e98547..87d100f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeLabelsProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeLabelsProvider.java
@@ -151,19 +151,21 @@ public class TestScriptBasedNodeLabelsProvider extends NodeLabelTestBase {
   @Test
   public void testConfigForNoTimer() throws Exception {
     Configuration conf = getConfForNodeLabelScript();
-    conf.setLong(YarnConfiguration.NM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS,
-        AbstractNodeLabelsProvider.DISABLE_NODE_LABELS_PROVIDER_FETCH_TIMER);
+    conf.setLong(YarnConfiguration
+            .NM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS,
+        AbstractNodeDescriptorsProvider
+            .DISABLE_NODE_DESCRIPTORS_PROVIDER_FETCH_TIMER);
     String normalScript = "echo NODE_PARTITION:X86";
     writeNodeLabelsScriptFile(normalScript, true);
     nodeLabelsProvider.init(conf);
     nodeLabelsProvider.start();
     Assert.assertNull(
         "Timer is not expected to be created when interval is configured as -1",
-        nodeLabelsProvider.nodeLabelsScheduler);
+        nodeLabelsProvider.getScheduler());
     // Ensure that even though timer is not run script is run at least once so
     // that NM registers/updates Labels with RM
     assertNLCollectionEquals(toNodeLabelSet("X86"),
-        nodeLabelsProvider.getNodeLabels());
+        nodeLabelsProvider.getDescriptors());
   }
 
   @Test
@@ -185,25 +187,25 @@ public class TestScriptBasedNodeLabelsProvider extends NodeLabelTestBase {
     Assert.assertNull(
         "Node Label Script runner should return null when script doesnt "
             + "give any Labels output",
-        nodeLabelsProvider.getNodeLabels());
+        nodeLabelsProvider.getDescriptors());
 
     writeNodeLabelsScriptFile(normalScript, true);
     timerTask.run();
     assertNLCollectionEquals(toNodeLabelSet("Windows"),
-        nodeLabelsProvider.getNodeLabels());
+        nodeLabelsProvider.getDescriptors());
 
     // multiple lines with partition tag then the last line's partition info
     // needs to be taken.
     writeNodeLabelsScriptFile(scrptWithMultipleLinesHavingNodeLabels, true);
     timerTask.run();
     assertNLCollectionEquals(toNodeLabelSet("JDK1_6"),
-        nodeLabelsProvider.getNodeLabels());
+        nodeLabelsProvider.getDescriptors());
 
     // timeout script.
     writeNodeLabelsScriptFile(timeOutScript, true);
     timerTask.run();
 
     Assert.assertNotEquals("Node Labels should not be set after timeout ",
-        toNodeLabelSet("ALL"), nodeLabelsProvider.getNodeLabels());
+        toNodeLabelSet("ALL"), nodeLabelsProvider.getDescriptors());
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[29/29] hadoop git commit: YARN-8739. Fix jenkins issues for Node Attributes branch. Contributed by Sunil Govindan.

Posted by su...@apache.org.
YARN-8739. Fix jenkins issues for Node Attributes branch. Contributed by Sunil Govindan.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c44088ac
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c44088ac
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c44088ac

Branch: refs/heads/trunk
Commit: c44088ac190e515b099183aeed4f9d6f8bee7da6
Parents: 1e7d6e5
Author: bibinchundatt <bi...@apache.org>
Authored: Sun Sep 2 18:44:18 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:01 2018 +0530

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ha/HAAdmin.java   |  5 ++++-
 .../hadoop/yarn/sls/scheduler/RMNodeWrapper.java  |  2 +-
 .../GetAttributesToNodesResponse.java             |  5 +++--
 .../ResourceManagerAdministrationProtocol.java    |  2 +-
 .../NodesToAttributesMappingResponse.java         |  4 ++++
 .../apache/hadoop/yarn/client/api/YarnClient.java |  3 ++-
 .../pb/GetAttributesToNodesResponsePBImpl.java    |  8 +++++---
 .../nodelabels/FileSystemNodeLabelsStore.java     |  4 +++-
 .../yarn/nodelabels/NodeAttributesManager.java    |  3 ++-
 .../nodelabels/NonAppendableFSNodeLabelStore.java |  2 +-
 .../hadoop/yarn/nodelabels/RMNodeLabel.java       |  3 ++-
 .../nodelabels/store/AbstractFSNodeStore.java     |  8 ++++----
 .../yarn/nodelabels/store/FSStoreOpHandler.java   | 18 +++++++++++++-----
 .../hadoop/yarn/nodelabels/store/StoreOp.java     |  4 ++--
 .../nodelabels/store/op/AddClusterLabelOp.java    |  4 ++--
 .../nodelabels/store/op/NodeLabelMirrorOp.java    |  3 +++
 .../yarn/nodelabels/store/op/NodeToLabelOp.java   |  4 ++--
 .../nodelabels/store/op/RemoveClusterLabelOp.java |  4 ++--
 .../impl/pb/NodeToAttributesPBImpl.java           |  3 +++
 .../pb/NodesToAttributesMappingRequestPBImpl.java |  3 +++
 .../NodesToAttributesMappingResponsePBImpl.java   |  3 +++
 .../RemoveFromClusterNodeLabelsRequestPBImpl.java |  3 +++
 .../apache/hadoop/yarn/api/TestPBImplRecords.java |  2 +-
 .../nodelabels/TestFileSystemNodeLabelsStore.java |  2 +-
 .../hadoop/yarn/nodelabels/TestNodeLabelUtil.java |  8 ++++++--
 .../yarn/server/MockResourceManagerFacade.java    | 11 ++++++-----
 .../nodelabels/NodeAttributesProvider.java        |  2 --
 .../TestNodeStatusUpdaterForLabels.java           |  3 +++
 .../resourcemanager/ResourceTrackerService.java   |  1 -
 .../nodelabels/NodeAttributesManagerImpl.java     |  7 ++++---
 .../nodelabels/NodeAttributesStoreEvent.java      |  3 ++-
 .../yarn/server/resourcemanager/MockNodes.java    |  2 +-
 .../TestFileSystemNodeAttributeStore.java         |  3 +++
 33 files changed, 95 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
index 8c92bd0..e955979 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
@@ -574,7 +574,10 @@ public abstract class HAAdmin extends Configured implements Tool {
     }
     return 0;
   }
-  
+
+  /**
+   * UsageInfo class holds args and help details.
+   */
   public static class UsageInfo {
     public final String args;
     public final String help;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
index 248b634..c73fb15 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java
@@ -207,7 +207,7 @@ public class RMNodeWrapper implements RMNode {
   public Map<String, Long> getAllocationTagsWithCount() {
     return node.getAllocationTagsWithCount();
   }
-  
+
   @Override
   public Set<NodeAttribute> getAllNodeAttributes() {
     return node.getAllNodeAttributes();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java
index 9bd529f..c83785f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetAttributesToNodesResponse.java
@@ -54,12 +54,13 @@ public abstract class GetAttributesToNodesResponse {
 
   /**
    * Get mapping of NodeAttributeKey to its associated mapping of list of
-   * NodeToAttributeValuenode to attribute value.
+   * NodeToAttributeValue associated with attribute.
    *
    * @return Map<NodeAttributeKey, List<NodeToAttributeValue>> node attributes
    *         to list of NodeToAttributeValuenode.
    */
   @Public
   @Evolving
-  public abstract Map<NodeAttributeKey, List<NodeToAttributeValue>> getAttributesToNodes();
+  public abstract Map<NodeAttributeKey,
+      List<NodeToAttributeValue>> getAttributesToNodes();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
index 58bb270..4777cf8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
@@ -150,7 +150,7 @@ public interface ResourceManagerAdministrationProtocol extends GetUserMappingsPr
 
   @Private
   @Idempotent
-  public NodesToAttributesMappingResponse mapAttributesToNodes(
+  NodesToAttributesMappingResponse mapAttributesToNodes(
       NodesToAttributesMappingRequest request) throws YarnException,
       IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingResponse.java
index 8e44adf..10081e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingResponse.java
@@ -20,6 +20,10 @@ package org.apache.hadoop.yarn.server.api.protocolrecords;
 
 import org.apache.hadoop.yarn.util.Records;
 
+/**
+ * NodesToAttributesMappingResponse holds response object for attribute
+ * mapping.
+ */
 public class NodesToAttributesMappingResponse {
   public static NodesToAttributesMappingResponse newInstance() {
     return Records.newRecord(NodesToAttributesMappingResponse.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
index f51b2f9..59fa6a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
@@ -936,7 +936,8 @@ public abstract class YarnClient extends AbstractService {
    */
   @Public
   @Unstable
-  public abstract Map<NodeAttributeKey, List<NodeToAttributeValue>> getAttributesToNodes(
+  public abstract Map<NodeAttributeKey,
+      List<NodeToAttributeValue>> getAttributesToNodes(
       Set<NodeAttributeKey> attributes) throws YarnException, IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
index 11999bdc..0d4c6e0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
@@ -100,8 +100,9 @@ public class GetAttributesToNodesResponsePBImpl
     Iterable<AttributeToNodesProto> iterable =
         () -> new Iterator<AttributeToNodesProto>() {
 
-          private Iterator<Map.Entry<NodeAttributeKey, List<NodeToAttributeValue>>> iter =
-              attributesToNodes.entrySet().iterator();
+          private Iterator<Map.Entry<NodeAttributeKey,
+              List<NodeToAttributeValue>>> iter = attributesToNodes.entrySet()
+              .iterator();
 
           @Override
           public void remove() {
@@ -198,7 +199,8 @@ public class GetAttributesToNodesResponsePBImpl
   }
 
   @Override
-  public Map<NodeAttributeKey, List<NodeToAttributeValue>> getAttributesToNodes() {
+  public Map<NodeAttributeKey,
+      List<NodeToAttributeValue>> getAttributesToNodes() {
     initAttributesToNodes();
     return this.attributesToNodes;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
index e11e6f8..6c459c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.nodelabels.store.AbstractFSNodeStore;
-import org.apache.hadoop.yarn.nodelabels.store.FSStoreOpHandler;
 
 import org.apache.hadoop.yarn.nodelabels.store.op.AddClusterLabelOp;
 import org.apache.hadoop.yarn.nodelabels.store.FSStoreOpHandler.StoreType;
@@ -41,6 +40,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+/**
+ * FileSystemNodeLabelsStore for storing node labels.
+ */
 public class FileSystemNodeLabelsStore
     extends AbstractFSNodeStore<CommonNodeLabelsManager>
     implements NodeLabelsStore {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
index 20f72d9..ca04e8d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
@@ -96,7 +96,8 @@ public abstract class NodeAttributesManager extends AbstractService {
    * @return a Map of attributeKeys to a map of hostnames to its attribute
    *         values.
    */
-  public abstract Map<NodeAttributeKey, Map<String, AttributeValue>> getAttributesToNodes(
+  public abstract Map<NodeAttributeKey,
+      Map<String, AttributeValue>> getAttributesToNodes(
       Set<NodeAttributeKey> attributes);
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NonAppendableFSNodeLabelStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NonAppendableFSNodeLabelStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NonAppendableFSNodeLabelStore.java
index 29bfff9..6747037 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NonAppendableFSNodeLabelStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NonAppendableFSNodeLabelStore.java
@@ -36,7 +36,7 @@ import java.util.Set;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 /**
- * Store implementation for Non Appendable File Store
+ * Store implementation for Non Appendable File Store.
  */
 public class NonAppendableFSNodeLabelStore extends FileSystemNodeLabelsStore {
   protected static final Log

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
index a8d151c..5e755f4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
@@ -30,7 +30,8 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 /**
  * Partition representation in RM.
  */
-public class RMNodeLabel extends AbstractLabel implements Comparable<RMNodeLabel> {
+public class RMNodeLabel extends AbstractLabel
+    implements Comparable<RMNodeLabel> {
   private boolean exclusive;
   private NodeLabel nodeLabel;
   private Set<NodeId> nodeIds;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java
index 216fc79..7127d11 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/AbstractFSNodeStore.java
@@ -182,14 +182,14 @@ public abstract class AbstractFSNodeStore<M> {
     LOG.info("Finished create editlog file at:" + editLogPath.toString());
   }
 
-  protected void loadManagerFromEditLog(Path editLogPath) throws IOException {
-    if (!fs.exists(editLogPath)) {
+  protected void loadManagerFromEditLog(Path editPath) throws IOException {
+    if (!fs.exists(editPath)) {
       return;
     }
-    try (FSDataInputStream is = fs.open(editLogPath)) {
+    try (FSDataInputStream is = fs.open(editPath)) {
       while (true) {
         try {
-          StoreOp storeOp = FSStoreOpHandler.get(is.readInt(),storeType);
+          StoreOp storeOp = FSStoreOpHandler.get(is.readInt(), storeType);
           storeOp.recover(is, manager);
         } catch (EOFException e) {
           // EOF hit, break

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java
index a626537..59a1860 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/FSStoreOpHandler.java
@@ -41,6 +41,9 @@ public class FSStoreOpHandler {
       editLogOp;
   private static Map<StoreType, Class<? extends FSNodeStoreLogOp>> mirrorOp;
 
+  /**
+   * Store Type enum to hold label and attribute.
+   */
   public enum StoreType {
     NODE_LABEL_STORE,
     NODE_ATTRIBUTE
@@ -53,14 +56,19 @@ public class FSStoreOpHandler {
     // registerLog edit log operation
 
     //Node Label Operations
-    registerLog(NODE_LABEL_STORE, AddClusterLabelOp.OPCODE, AddClusterLabelOp.class);
+    registerLog(NODE_LABEL_STORE, AddClusterLabelOp.OPCODE,
+        AddClusterLabelOp.class);
     registerLog(NODE_LABEL_STORE, NodeToLabelOp.OPCODE, NodeToLabelOp.class);
-    registerLog(NODE_LABEL_STORE, RemoveClusterLabelOp.OPCODE, RemoveClusterLabelOp.class);
+    registerLog(NODE_LABEL_STORE, RemoveClusterLabelOp.OPCODE,
+        RemoveClusterLabelOp.class);
 
     //NodeAttibute operation
-    registerLog(NODE_ATTRIBUTE, AddNodeToAttributeLogOp.OPCODE, AddNodeToAttributeLogOp.class);
-    registerLog(NODE_ATTRIBUTE, RemoveNodeToAttributeLogOp.OPCODE, RemoveNodeToAttributeLogOp.class);
-    registerLog(NODE_ATTRIBUTE, ReplaceNodeToAttributeLogOp.OPCODE, ReplaceNodeToAttributeLogOp.class);
+    registerLog(NODE_ATTRIBUTE, AddNodeToAttributeLogOp.OPCODE,
+        AddNodeToAttributeLogOp.class);
+    registerLog(NODE_ATTRIBUTE, RemoveNodeToAttributeLogOp.OPCODE,
+        RemoveNodeToAttributeLogOp.class);
+    registerLog(NODE_ATTRIBUTE, ReplaceNodeToAttributeLogOp.OPCODE,
+        ReplaceNodeToAttributeLogOp.class);
 
     // registerLog Mirror op
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/StoreOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/StoreOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/StoreOp.java
index c26e1dc..e0b26da 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/StoreOp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/StoreOp.java
@@ -30,7 +30,7 @@ import java.io.IOException;
 public interface StoreOp<W, R, M> {
 
   /**
-   * Write operation to persistent storage
+   * Write operation to persistent storage.
    *
    * @param write write to be done to
    * @param mgr manager used by store
@@ -39,7 +39,7 @@ public interface StoreOp<W, R, M> {
   void write(W write, M mgr) throws IOException;
 
   /**
-   * Read and populate StoreOp
+   * Read and populate StoreOp.
    *
    * @param read read to be done from
    * @param mgr  manager used by store

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/AddClusterLabelOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/AddClusterLabelOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/AddClusterLabelOp.java
index ce736aa..f0259b3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/AddClusterLabelOp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/AddClusterLabelOp.java
@@ -57,8 +57,8 @@ public class AddClusterLabelOp
     mgr.addToCluserNodeLabels(labels);
   }
 
-  public AddClusterLabelOp setLabels(List<NodeLabel> labels) {
-    this.labels = labels;
+  public AddClusterLabelOp setLabels(List<NodeLabel> nodeLabels) {
+    this.labels = nodeLabels;
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeLabelMirrorOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeLabelMirrorOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeLabelMirrorOp.java
index 3ec837b..9500486 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeLabelMirrorOp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeLabelMirrorOp.java
@@ -35,6 +35,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+/**
+ * NodeLabel Mirror Op class.
+ */
 public class NodeLabelMirrorOp
     extends FSNodeStoreLogOp<CommonNodeLabelsManager> {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeToLabelOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeToLabelOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeToLabelOp.java
index 0e1e398..df08ffc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeToLabelOp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/NodeToLabelOp.java
@@ -59,8 +59,8 @@ public class NodeToLabelOp
   }
 
   public NodeToLabelOp setNodeToLabels(
-      Map<NodeId, Set<String>> nodeToLabels) {
-    this.nodeToLabels = nodeToLabels;
+      Map<NodeId, Set<String>> nodeToLabelsList) {
+    this.nodeToLabels = nodeToLabelsList;
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/RemoveClusterLabelOp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/RemoveClusterLabelOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/RemoveClusterLabelOp.java
index 4f6d4bd..2fc4ac3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/RemoveClusterLabelOp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/store/op/RemoveClusterLabelOp.java
@@ -59,8 +59,8 @@ public class RemoveClusterLabelOp
     mgr.removeFromClusterNodeLabels(labels);
   }
 
-  public RemoveClusterLabelOp setLabels(Collection<String> labels) {
-    this.labels = labels;
+  public RemoveClusterLabelOp setLabels(Collection<String> nodeLabels) {
+    this.labels = nodeLabels;
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
index 7204914..94b967b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
@@ -28,6 +28,9 @@ import org.apache.hadoop.yarn.proto.YarnProtos.NodeToAttributesProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeToAttributesProtoOrBuilder;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
 
+/**
+ * Proto class for Node to attributes mapping.
+ */
 public class NodeToAttributesPBImpl extends NodeToAttributes {
   private NodeToAttributesProto proto =
       NodeToAttributesProto.getDefaultInstance();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
index 6cb9a97..c1a86b3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
@@ -29,6 +29,9 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperati
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
 
+/**
+ * Proto class for node to attributes mapping request.
+ */
 public class NodesToAttributesMappingRequestPBImpl
     extends NodesToAttributesMappingRequest {
   private NodesToAttributesMappingRequestProto proto =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingResponsePBImpl.java
index 0554ef8..955c3cb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingResponsePBImpl.java
@@ -21,6 +21,9 @@ package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingResponseProto;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 
+/**
+ * Proto class for node to attributes mapping response.
+ */
 public class NodesToAttributesMappingResponsePBImpl
     extends NodesToAttributesMappingResponse {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoveFromClusterNodeLabelsRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoveFromClusterNodeLabelsRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoveFromClusterNodeLabelsRequestPBImpl.java
index f633804..afabcd9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoveFromClusterNodeLabelsRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RemoveFromClusterNodeLabelsRequestPBImpl.java
@@ -28,6 +28,9 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos
 import org.apache.hadoop.yarn.server.api.protocolrecords
     .RemoveFromClusterNodeLabelsRequest;
 
+/**
+ * Proto class to handlde RemoveFromClusterNodeLabels request.
+ */
 public class RemoveFromClusterNodeLabelsRequestPBImpl
     extends RemoveFromClusterNodeLabelsRequest {
   Set<String> labels;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 682d6ec..cdb6c46 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -465,7 +465,7 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
     generateByNewInstance(NodeToAttributeValue.class);
     generateByNewInstance(NodeAttributeInfo.class);
     generateByNewInstance(NodesToAttributesMappingRequest.class);
- }
+  }
 
   @Test
   public void testAllocateRequestPBImpl() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
index 93c039a..36dbc2b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
@@ -360,7 +360,7 @@ public class TestFileSystemNodeLabelsStore extends NodeLabelTestBase {
     Mockito.when(store.getFs().exists(Mockito.any(
         Path.class))).thenReturn(existsRetVal);
     store.init(conf, mgr);
-    Mockito.verify(store.getFs(),Mockito.times(
+    Mockito.verify(store.getFs(), Mockito.times(
         expectedNumOfCalls)).mkdirs(Mockito.any(Path
         .class));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestNodeLabelUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestNodeLabelUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestNodeLabelUtil.java
index d43da4f..afdfcbb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestNodeLabelUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestNodeLabelUtil.java
@@ -20,12 +20,16 @@ package org.apache.hadoop.yarn.nodelabels;
 import static org.junit.Assert.fail;
 import org.junit.Test;
 
+/**
+ * Test class to verify node label util ops.
+ */
 public class TestNodeLabelUtil {
 
   @Test
   public void testAttributeValueAddition() {
     String[] values =
-        new String[] {"1_8", "1.8", "ABZ", "ABZ", "az", "a-z","a_z", "123456789"};
+        new String[]{"1_8", "1.8", "ABZ", "ABZ", "az", "a-z", "a_z",
+            "123456789"};
     for (String val : values) {
       try {
         NodeLabelUtil.checkAndThrowAttributeValue(val);
@@ -34,7 +38,7 @@ public class TestNodeLabelUtil {
       }
     }
 
-    String[] invalidVals = new String[] {"_18","1,8","1/5",".15","1\\5"};
+    String[] invalidVals = new String[]{"_18", "1,8", "1/5", ".15", "1\\5"};
     for (String val : invalidVals) {
       try {
         NodeLabelUtil.checkAndThrowAttributeValue(val);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
index d2cced6..958b1f1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
@@ -189,8 +189,8 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
 
   private HashSet<ApplicationId> applicationMap = new HashSet<>();
   private HashSet<ApplicationId> keepContainerOnUams = new HashSet<>();
-  private HashMap<ApplicationAttemptId, List<ContainerId>> applicationContainerIdMap =
-      new HashMap<ApplicationAttemptId, List<ContainerId>>();
+  private HashMap<ApplicationAttemptId,
+      List<ContainerId>> applicationContainerIdMap = new HashMap<>();
   private AtomicInteger containerIndex = new AtomicInteger(0);
   private Configuration conf;
   private int subClusterId;
@@ -495,7 +495,7 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     if (request.getApplicationSubmissionContext().getUnmanagedAM()
         || request.getApplicationSubmissionContext()
             .getKeepContainersAcrossApplicationAttempts()) {
-	keepContainerOnUams.add(appId);
+      keepContainerOnUams.add(appId);
     }
     return SubmitApplicationResponse.newInstance();
   }
@@ -921,8 +921,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   }
 
   @Override
-  public NodesToAttributesMappingResponse mapAttributesToNodes(NodesToAttributesMappingRequest request)
-    throws YarnException, IOException {
+  public NodesToAttributesMappingResponse mapAttributesToNodes(
+      NodesToAttributesMappingRequest request)
+      throws YarnException, IOException {
     return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeAttributesProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeAttributesProvider.java
index 8240024..2a41800 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeAttributesProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/NodeAttributesProvider.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.yarn.server.nodemanager.nodelabels;
 
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
 
-import java.util.Set;
-
 /**
  * Abstract class which will be responsible for fetching the node attributes.
  *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
index 3e2d963..a86ca3e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdaterForLabels.java
@@ -180,6 +180,9 @@ public class TestNodeStatusUpdaterForLabels extends NodeLabelTestBase {
     }
   }
 
+  /**
+   * A dummy NodeLabelsProvider class for tests.
+   */
   public static class DummyNodeLabelsProvider extends NodeLabelsProvider {
 
     public DummyNodeLabelsProvider() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index 9c4d594..b67172e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -515,7 +515,6 @@ public class ResourceTrackerService extends AbstractService implements
      * 4. Send healthStatus to RMNode
      * 5. Update node's labels if distributed Node Labels configuration is enabled
      */
-
     NodeId nodeId = remoteNodeStatus.getNodeId();
 
     // 1. Check if it's a valid (i.e. not excluded) node, if not, see if it is

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
index 906f41e..81f42d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
@@ -163,8 +163,8 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
       StringBuilder logMsg = new StringBuilder(op.name());
       logMsg.append(" attributes on nodes:");
       // do update labels from nodes
-      for (Entry<String, Map<NodeAttribute, AttributeValue>> entry : nodeAttributeMapping
-          .entrySet()) {
+      for (Entry<String, Map<NodeAttribute, AttributeValue>> entry :
+          nodeAttributeMapping.entrySet()) {
         String nodeHost = entry.getKey();
         Map<NodeAttribute, AttributeValue> attributes = entry.getValue();
 
@@ -400,7 +400,8 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   }
 
   @Override
-  public Map<NodeAttributeKey, Map<String, AttributeValue>> getAttributesToNodes(
+  public Map<NodeAttributeKey,
+      Map<String, AttributeValue>> getAttributesToNodes(
       Set<NodeAttributeKey> attributes) {
     try {
       readLock.lock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesStoreEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesStoreEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesStoreEvent.java
index d04e8cf..707981f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesStoreEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesStoreEvent.java
@@ -41,7 +41,8 @@ public class NodeAttributesStoreEvent
     this.operation = operation;
   }
 
-  public Map<String, Map<NodeAttribute, AttributeValue>> getNodeAttributeMappingList() {
+  public Map<String,
+      Map<NodeAttribute, AttributeValue>> getNodeAttributeMappingList() {
     return nodeAttributeMapping;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
index 7c822b4..a871993 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
@@ -285,7 +285,7 @@ public class MockNodes {
     public Map<String, Long> getAllocationTagsWithCount() {
       return null;
     }
-    
+
     public void setNodeAttributes(String prefix,
         Set<NodeAttribute> nodeAttributes) {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c44088ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
index 502f9d0..07ea20d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
@@ -36,6 +36,9 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
+/**
+ * Test class for FileSystemNodeAttributeStore.
+ */
 public class TestFileSystemNodeAttributeStore {
 
   private MockNodeAttrbuteManager mgr = null;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/29] hadoop git commit: YARN-8117. Fix TestRMWebServicesNodes test failure. Contributed by Bibin A Chundatt.

Posted by su...@apache.org.
YARN-8117. Fix TestRMWebServicesNodes test failure. Contributed by Bibin A Chundatt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a6590c1f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a6590c1f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a6590c1f

Branch: refs/heads/trunk
Commit: a6590c1f1f7cacd3843265f6e6227f1221205865
Parents: 901e852
Author: bibinchundatt <bi...@apache.org>
Authored: Thu Apr 5 14:09:50 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6590c1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java
index 7ea7e81..72b7a45 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodes.java
@@ -740,7 +740,7 @@ public class TestRMWebServicesNodes extends JerseyTestBase {
 
   public void verifyNodeInfo(JSONObject nodeInfo, RMNode nm)
       throws JSONException, Exception {
-    assertEquals("incorrect number of elements", 19, nodeInfo.length());
+    assertEquals("incorrect number of elements", 20, nodeInfo.length());
 
     JSONObject resourceInfo = nodeInfo.getJSONObject("resourceUtilization");
     verifyNodeInfoGeneric(nm, nodeInfo.getString("state"),


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/29] hadoop git commit: YARN-8033. CLI Integration with NodeAttributesManagerImpl. Contributed by Naganarasimha G R.

Posted by su...@apache.org.
YARN-8033. CLI Integration with NodeAttributesManagerImpl. Contributed by Naganarasimha G R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/901e8523
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/901e8523
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/901e8523

Branch: refs/heads/trunk
Commit: 901e85238da08129374d44af80716b07f0b912e6
Parents: 89b3ebd
Author: bibinchundatt <bi...@apache.org>
Authored: Sun Apr 1 19:24:00 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/api/records/NodeAttribute.java  |   3 +-
 .../yarn/client/cli/NodeAttributesCLI.java      |  18 +-
 .../yarn/client/cli/TestNodeAttributesCLI.java  |  18 ++
 .../server/resourcemanager/AdminService.java    |  97 +++++++++++
 .../resourcemanager/TestRMAdminService.java     | 170 +++++++++++++++++--
 5 files changed, 282 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/901e8523/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
index 4f6846b..25ac9ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
@@ -45,13 +45,12 @@ import org.apache.hadoop.yarn.util.Records;
 @Unstable
 public abstract class NodeAttribute {
 
-  public static final String DEFAULT_PREFIX = "";
   public static final String PREFIX_DISTRIBUTED = "nm.yarn.io";
   public static final String PREFIX_CENTRALIZED = "rm.yarn.io";
 
   public static NodeAttribute newInstance(String attributeName,
       NodeAttributeType attributeType, String attributeValue) {
-    return newInstance(DEFAULT_PREFIX, attributeName, attributeType,
+    return newInstance(PREFIX_CENTRALIZED, attributeName, attributeType,
         attributeValue);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/901e8523/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java
index 2eff155..df5a57d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.yarn.client.cli;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -50,6 +52,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappin
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
 
 /**
  * CLI to map attributes to Nodes.
@@ -311,7 +314,7 @@ public class NodeAttributesCLI extends Configured implements Tool {
    */
   private List<NodeToAttributes> buildNodeLabelsMapFromStr(String args,
       boolean validateForAttributes, AttributeMappingOperationType operation) {
-    List<NodeToAttributes> nodeToAttributesList = new ArrayList<>();
+    Map<String,NodeToAttributes> nodeToAttributesMap = new HashMap<>();
     for (String nodeToAttributesStr : args.split("[ \n]")) {
       // for each node to attribute mapping
       nodeToAttributesStr = nodeToAttributesStr.trim();
@@ -384,8 +387,9 @@ public class NodeAttributesCLI extends Configured implements Tool {
           // TODO when we support different type of attribute type we need to
           // cross verify whether input attributes itself is not violating
           // attribute Name to Type mapping.
-          attributesList.add(NodeAttribute.newInstance(attributeName.trim(),
-              attributeType, attributeValue.trim()));
+          attributesList
+              .add(NodeAttribute.newInstance(NodeAttribute.PREFIX_CENTRALIZED,
+                  attributeName.trim(), attributeType, attributeValue.trim()));
         }
       }
       if (validateForAttributes) {
@@ -393,14 +397,14 @@ public class NodeAttributesCLI extends Configured implements Tool {
             "Attributes cannot be null or empty for Operation "
                 + operation.name() + " on the node " + node);
       }
-      nodeToAttributesList
-          .add(NodeToAttributes.newInstance(node, attributesList));
+      nodeToAttributesMap
+          .put(node,NodeToAttributes.newInstance(node, attributesList));
     }
 
-    if (nodeToAttributesList.isEmpty()) {
+    if (nodeToAttributesMap.isEmpty()) {
       throw new IllegalArgumentException(NO_MAPPING_ERR_MSG);
     }
-    return nodeToAttributesList;
+    return Lists.newArrayList(nodeToAttributesMap.values());
   }
 
   public static void main(String[] args) throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/901e8523/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java
index cc92a93..bbd5ca3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java
@@ -301,6 +301,24 @@ public class TestNodeAttributesCLI {
         NodesToAttributesMappingRequest.newInstance(
             AttributeMappingOperationType.ADD, nodeAttributesList, true);
     assertTrue(request.equals(expected));
+
+    // --------------------------------
+    // with Duplicate mappings for a host
+    // --------------------------------
+    args = new String[] { "-add", "x:key2=123,key3=abc x:key4(string)",
+        "-failOnUnknownNodes" };
+    assertTrue("Should not fail as attribute has been properly mapped",
+        0 == runTool(args));
+    nodeAttributesList = new ArrayList<>();
+    attributes = new ArrayList<>();
+    attributes
+        .add(NodeAttribute.newInstance("key4", NodeAttributeType.STRING, ""));
+    nodeAttributesList.add(NodeToAttributes.newInstance("x", attributes));
+
+    expected =
+        NodesToAttributesMappingRequest.newInstance(
+            AttributeMappingOperationType.ADD, nodeAttributesList, true);
+    assertTrue(request.equals(expected));
   }
 
   private void assertFailureMessageContains(String... messages) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/901e8523/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index bff3f2c..12f8aaf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -23,6 +23,8 @@ import java.io.InputStream;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -50,6 +52,7 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.conf.HAUtil;
@@ -59,6 +62,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.security.ConfiguredYarnAuthorizer;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
@@ -66,6 +70,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsR
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
@@ -968,7 +973,99 @@ public class AdminService extends CompositeService implements
   public NodesToAttributesMappingResponse mapAttributesToNodes(
       NodesToAttributesMappingRequest request)
       throws YarnException, IOException {
+
+    final String operation = "mapAttributesToNodes";
+    final String msg = "Map Attributes to Nodes";
+    UserGroupInformation user = checkAcls(operation);
+    checkRMStatus(user.getShortUserName(), operation, msg);
+
+
+    List<NodeToAttributes> nodesToAttributes = request.getNodesToAttributes();
+    boolean failOnUnknownNodes = request.getFailOnUnknownNodes();
+
+    Map<String, Set<NodeAttribute>> nodeAttributeMapping =
+        validateAndFetch(nodesToAttributes, failOnUnknownNodes);
+
+    NodeAttributesManager nodeAttributesManager =
+        rm.getRMContext().getNodeAttributesManager();
+    try {
+      switch (request.getOperation()) {
+      case ADD:
+        nodeAttributesManager.addNodeAttributes(nodeAttributeMapping);
+        break;
+      case REMOVE:
+        nodeAttributesManager.removeNodeAttributes(nodeAttributeMapping);
+        break;
+      case REPLACE:
+        nodeAttributesManager.replaceNodeAttributes(
+            NodeAttribute.PREFIX_CENTRALIZED, nodeAttributeMapping);
+        break;
+      default:
+        throw new IOException("Invalid operation " + request.getOperation()
+            + " specified in the mapAttributesToNodes request ");
+
+      }
+    } catch (IOException ioe) {
+      throw logAndWrapException(ioe, user.getShortUserName(), operation, msg);
+    }
+    RMAuditLogger.logSuccess(user.getShortUserName(), operation,
+        "AdminService");
     return recordFactory
         .newRecordInstance(NodesToAttributesMappingResponse.class);
   }
+
+  /**
+   * @param nodesToAttributesMapping input to be validated
+   * @param failOnUnknownNodes indicates to fail if the nodes are not available.
+   * @return the map of Node host name to set of NodeAttributes
+   * @throws IOException if validation fails for node existence or the attribute
+   *           has a wrong prefix
+   */
+  private Map<String, Set<NodeAttribute>> validateAndFetch(
+      List<NodeToAttributes> nodesToAttributesMapping,
+      boolean failOnUnknownNodes) throws IOException {
+    Map<String, Set<NodeAttribute>> attributeMapping = new HashMap<>();
+    List<String> invalidNodes = new ArrayList<>();
+    for (NodeToAttributes nodeToAttributes : nodesToAttributesMapping) {
+      String node = nodeToAttributes.getNode();
+      if (!validateForInvalidNode(node, failOnUnknownNodes)) {
+        invalidNodes.add(node);
+        continue;
+      }
+      List<NodeAttribute> nodeAttributes = nodeToAttributes.getNodeAttributes();
+      if (!nodeAttributes.stream()
+          .allMatch(nodeAttribute -> NodeAttribute.PREFIX_CENTRALIZED
+              .equals(nodeAttribute.getAttributePrefix()))) {
+        throw new IOException("Invalid Attribute Mapping for the node " + node
+            + ". Prefix should be " + NodeAttribute.PREFIX_CENTRALIZED);
+      }
+      attributeMapping.put(node, new HashSet<>(nodeAttributes));
+    }
+    if (!invalidNodes.isEmpty()) {
+      String message = " Following nodes does not exist : " + invalidNodes;
+      LOG.error(message);
+      throw new IOException(message);
+    }
+    return attributeMapping;
+  }
+
+  /**
+   * @param node
+   * @return true if valid else false;
+   */
+  private boolean validateForInvalidNode(String node,
+      boolean failOnUnknownNodes) {
+    if (!failOnUnknownNodes) {
+      return true;
+    }
+    // both active and inactive nodes are recognized as known nodes
+    boolean isKnown = rm.getRMContext().getRMNodes().keySet().stream()
+        .anyMatch(activeNode -> activeNode.getHost().equals(node));
+
+    if (!isKnown) {
+      isKnown = rm.getRMContext().getInactiveRMNodes().keySet().stream()
+          .anyMatch(inactiveNode -> inactiveNode.getHost().equals(node));
+    }
+    return isKnown;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/901e8523/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
index f0484e1..90945c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
@@ -53,6 +53,8 @@ import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.records.DecommissionType;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -60,6 +62,9 @@ import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesRequest;
@@ -85,11 +90,14 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddToClusterNodeLabelsRequestProto;
 
 import static org.junit.Assert.assertTrue;
@@ -1203,21 +1211,7 @@ public class TestRMAdminService {
 
     ((RMContextImpl) rm.getRMContext())
         .setHAServiceState(HAServiceState.ACTIVE);
-    Map<NodeId, RMNode> rmNodes = rm.getRMContext().getRMNodes();
-    rmNodes.put(NodeId.newInstance("host1", 1111),
-        new RMNodeImpl(null, rm.getRMContext(), "host1", 0, 0, null, null,
-                null));
-    rmNodes.put(NodeId.newInstance("host2", 2222),
-            new RMNodeImpl(null, rm.getRMContext(), "host2", 0, 0, null, null,
-                null));
-    rmNodes.put(NodeId.newInstance("host3", 3333),
-            new RMNodeImpl(null, rm.getRMContext(), "host3", 0, 0, null, null,
-                null));
-    Map<NodeId, RMNode> rmInactiveNodes = rm.getRMContext()
-        .getInactiveRMNodes();
-    rmInactiveNodes.put(NodeId.newInstance("host4", 4444),
-        new RMNodeImpl(null, rm.getRMContext(), "host4", 0, 0, null, null,
-                null));
+    setActiveAndInactiveNodes(rm);
     RMNodeLabelsManager labelMgr = rm.rmContext.getNodeLabelManager();
 
     // by default, distributed configuration for node label is disabled, this
@@ -1552,4 +1546,150 @@ public class TestRMAdminService {
     Assert.assertTrue(
         response.getNodeLabelList().containsAll(Arrays.asList(labelX, labelY)));
   }
+
+  @Test(timeout = 30000)
+  public void testMapAttributesToNodes() throws Exception, YarnException {
+    // 1. Need to test for the Invalid Node
+    // 1.1. Need to test for active nodes
+    // 1.2. Need to test for Inactive nodes
+    // 1.3. Test with Single Node invalid
+    // 1.4. Need to test with port (should fail)
+    // 1.5. Test with unknown node when failOnUnknownNodes is false
+
+    // also test : 3. Ensure Appropriate manager Method call is done
+    rm = new MockRM();
+
+    NodeAttributesManager spiedAttributesManager =
+        Mockito.spy(rm.getRMContext().getNodeAttributesManager());
+    rm.getRMContext().setNodeAttributesManager(spiedAttributesManager);
+
+    ((RMContextImpl) rm.getRMContext())
+        .setHAServiceState(HAServiceState.ACTIVE);
+    setActiveAndInactiveNodes(rm);
+    // by default, distributed configuration for node label is disabled, this
+    // should pass
+    NodesToAttributesMappingRequest request =
+        NodesToAttributesMappingRequest
+            .newInstance(AttributeMappingOperationType.ADD,
+                ImmutableList.of(NodeToAttributes.newInstance("host1",
+                    ImmutableList.of(NodeAttribute.newInstance(
+                        NodeAttribute.PREFIX_CENTRALIZED, "x",
+                        NodeAttributeType.STRING, "dfasdf")))),
+                true);
+    try {
+      rm.adminService.mapAttributesToNodes(request);
+    } catch (Exception ex) {
+      fail("should not fail on known node in active state" + ex.getMessage());
+    }
+    Mockito.verify(spiedAttributesManager, Mockito.times(1))
+        .addNodeAttributes(Mockito.anyMap());
+
+    request =
+        NodesToAttributesMappingRequest
+            .newInstance(AttributeMappingOperationType.REMOVE,
+                ImmutableList.of(NodeToAttributes.newInstance("host4",
+                    ImmutableList.of(NodeAttribute.newInstance(
+                        NodeAttribute.PREFIX_CENTRALIZED, "x",
+                        NodeAttributeType.STRING, "dfasdf")))),
+                true);
+    try {
+      rm.adminService.mapAttributesToNodes(request);
+    } catch (Exception ex) {
+      fail("should not fail on known node in inactive state" + ex.getMessage());
+    }
+    Mockito.verify(spiedAttributesManager, Mockito.times(1))
+        .removeNodeAttributes(Mockito.anyMap());
+
+    request =
+        NodesToAttributesMappingRequest
+            .newInstance(AttributeMappingOperationType.ADD,
+                ImmutableList.of(NodeToAttributes.newInstance("host5",
+                    ImmutableList.of(NodeAttribute.newInstance(
+                        NodeAttribute.PREFIX_CENTRALIZED, "x",
+                        NodeAttributeType.STRING, "dfasdf")))),
+                true);
+    try {
+      rm.adminService.mapAttributesToNodes(request);
+      fail("host5 is not a valid node, It should have failed");
+    } catch (Exception ex) {
+      Assert.assertEquals("Exception Message is not as desired",
+          " Following nodes does not exist : [host5]", ex.getMessage());
+    }
+
+    request =
+        NodesToAttributesMappingRequest
+            .newInstance(AttributeMappingOperationType.ADD, ImmutableList.of(
+                NodeToAttributes.newInstance("host4:8889",
+                    ImmutableList.of(NodeAttribute.newInstance(
+                        NodeAttribute.PREFIX_CENTRALIZED, "x",
+                        NodeAttributeType.STRING, "dfasdf"))),
+                NodeToAttributes.newInstance("host2:8889",
+                    ImmutableList.of(NodeAttribute.newInstance(
+                        NodeAttribute.PREFIX_CENTRALIZED, "x",
+                        NodeAttributeType.STRING, "dfasdf")))),
+                true);
+    try {
+      // port if added in CLI it fails in the client itself. Here we just check
+      // against hostname hence the message as : nodes does not exist.
+      rm.adminService.mapAttributesToNodes(request);
+      fail("host with the port should fail as only hostnames are validated");
+    } catch (Exception ex) {
+      Assert.assertEquals("Exception Message is not as desired",
+          " Following nodes does not exist : [host4:8889, host2:8889]",
+          ex.getMessage());
+    }
+
+    request =
+        NodesToAttributesMappingRequest
+            .newInstance(AttributeMappingOperationType.REPLACE,
+                ImmutableList.of(NodeToAttributes.newInstance("host5",
+                    ImmutableList.of(NodeAttribute.newInstance(
+                        NodeAttribute.PREFIX_CENTRALIZED, "x",
+                        NodeAttributeType.STRING, "dfasdf")))),
+                false);
+    try {
+      rm.adminService.mapAttributesToNodes(request);
+    } catch (Exception ex) {
+      fail("This operation should not fail as failOnUnknownNodes is false : "
+          + ex.getMessage());
+    }
+    Mockito.verify(spiedAttributesManager, Mockito.times(1))
+        .replaceNodeAttributes(Mockito.eq(NodeAttribute.PREFIX_CENTRALIZED),
+            Mockito.anyMap());
+
+    // 2. fail on invalid prefix
+    request =
+        NodesToAttributesMappingRequest
+            .newInstance(AttributeMappingOperationType.ADD,
+                ImmutableList.of(NodeToAttributes.newInstance("host5",
+                    ImmutableList.of(NodeAttribute.newInstance(
+                        NodeAttribute.PREFIX_DISTRIBUTED, "x",
+                        NodeAttributeType.STRING, "dfasdf")))),
+                false);
+    try {
+      rm.adminService.mapAttributesToNodes(request);
+      fail("This operation should fail as prefix should be \"nm.yarn.io\".");
+    } catch (Exception ex) {
+      Assert.assertEquals("Exception Message is not as desired",
+          "Invalid Attribute Mapping for the node host5. Prefix should be "
+              + "rm.yarn.io",
+          ex.getMessage());
+    }
+
+    rm.close();
+  }
+
+  private void setActiveAndInactiveNodes(ResourceManager resourceManager) {
+    Map<NodeId, RMNode> rmNodes = resourceManager.getRMContext().getRMNodes();
+    rmNodes.put(NodeId.newInstance("host1", 1111), new RMNodeImpl(null,
+        resourceManager.getRMContext(), "host1", 0, 0, null, null, null));
+    rmNodes.put(NodeId.newInstance("host2", 2222), new RMNodeImpl(null,
+        resourceManager.getRMContext(), "host2", 0, 0, null, null, null));
+    rmNodes.put(NodeId.newInstance("host3", 3333), new RMNodeImpl(null,
+        resourceManager.getRMContext(), "host3", 0, 0, null, null, null));
+    Map<NodeId, RMNode> rmInactiveNodes =
+        resourceManager.getRMContext().getInactiveRMNodes();
+    rmInactiveNodes.put(NodeId.newInstance("host4", 4444), new RMNodeImpl(null,
+        resourceManager.getRMContext(), "host4", 0, 0, null, null, null));
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[23/29] hadoop git commit: YARN-7863. Modify placement constraints to support node attributes. Contributed by Sunil Govindan.

Posted by su...@apache.org.
YARN-7863. Modify placement constraints to support node attributes. Contributed by Sunil Govindan.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/67ae81f0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/67ae81f0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/67ae81f0

Branch: refs/heads/trunk
Commit: 67ae81f0e0ac7f107261ee15f2eb4d189e3b1983
Parents: 8c94739
Author: Naganarasimha <na...@apache.org>
Authored: Mon Aug 27 10:27:33 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:01 2018 +0530

----------------------------------------------------------------------
 .../yarn/api/records/NodeAttributeOpCode.java   |  43 +++++++
 .../yarn/api/resource/PlacementConstraint.java  |  40 +++++-
 .../yarn/api/resource/PlacementConstraints.java |  19 +++
 .../constraint/PlacementConstraintParser.java   | 112 +++++++++++++++--
 .../src/main/proto/yarn_protos.proto            |   7 ++
 .../resource/TestPlacementConstraintParser.java |  61 ++++++++-
 .../distributedshell/ApplicationMaster.java     |  35 ++++--
 .../applications/distributedshell/Client.java   |   5 +-
 .../distributedshell/PlacementSpec.java         |  19 ++-
 .../PlacementConstraintFromProtoConverter.java  |  10 +-
 .../pb/PlacementConstraintToProtoConverter.java |  11 ++
 .../server/resourcemanager/ResourceManager.java |   7 +-
 .../nodelabels/NodeAttributesManagerImpl.java   |  24 +++-
 .../scheduler/SchedulerNode.java                |  11 ++
 .../scheduler/capacity/CapacityScheduler.java   |  36 +++++-
 .../constraint/PlacementConstraintsUtil.java    | 126 ++++++++++++++++---
 .../NodeAttributesUpdateSchedulerEvent.java     |  41 ++++++
 .../scheduler/event/SchedulerEventType.java     |   1 +
 .../LocalityAppPlacementAllocator.java          |   4 +
 19 files changed, 567 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeOpCode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeOpCode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeOpCode.java
new file mode 100644
index 0000000..76db063
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeOpCode.java
@@ -0,0 +1,43 @@
+/**
+* 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+/**
+ * Enumeration of various node attribute op codes.
+ */
+@Public
+@Evolving
+public enum NodeAttributeOpCode {
+  /**
+   * Default as No OP.
+   */
+  NO_OP,
+  /**
+   * EQUALS op code for Attribute.
+   */
+  EQ,
+
+  /**
+   * NOT EQUALS op code for Attribute.
+   */
+  NE
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
index 0fe8273..79196fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
@@ -29,6 +29,7 @@ import java.util.Iterator;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.NodeAttributeOpCode;
 
 /**
  * {@code PlacementConstraint} represents a placement constraint for a resource
@@ -155,13 +156,22 @@ public class PlacementConstraint {
     private int minCardinality;
     private int maxCardinality;
     private Set<TargetExpression> targetExpressions;
+    private NodeAttributeOpCode attributeOpCode;
 
     public SingleConstraint(String scope, int minCardinality,
-        int maxCardinality, Set<TargetExpression> targetExpressions) {
+        int maxCardinality, NodeAttributeOpCode opCode,
+        Set<TargetExpression> targetExpressions) {
       this.scope = scope;
       this.minCardinality = minCardinality;
       this.maxCardinality = maxCardinality;
       this.targetExpressions = targetExpressions;
+      this.attributeOpCode = opCode;
+    }
+
+    public SingleConstraint(String scope, int minCardinality,
+        int maxCardinality, Set<TargetExpression> targetExpressions) {
+      this(scope, minCardinality, maxCardinality, NodeAttributeOpCode.NO_OP,
+          targetExpressions);
     }
 
     public SingleConstraint(String scope, int minC, int maxC,
@@ -169,6 +179,13 @@ public class PlacementConstraint {
       this(scope, minC, maxC, new HashSet<>(Arrays.asList(targetExpressions)));
     }
 
+    public SingleConstraint(String scope, int minC, int maxC,
+        NodeAttributeOpCode opCode,
+        TargetExpression... targetExpressions) {
+      this(scope, minC, maxC, opCode,
+          new HashSet<>(Arrays.asList(targetExpressions)));
+    }
+
     /**
      * Get the scope of the constraint.
      *
@@ -205,6 +222,15 @@ public class PlacementConstraint {
       return targetExpressions;
     }
 
+    /**
+     * Get the NodeAttributeOpCode of the constraint.
+     *
+     * @return nodeAttribute Op Code
+     */
+    public NodeAttributeOpCode getNodeAttributeOpCode() {
+      return attributeOpCode;
+    }
+
     @Override
     public boolean equals(Object o) {
       if (this == o) {
@@ -225,6 +251,10 @@ public class PlacementConstraint {
       if (!getScope().equals(that.getScope())) {
         return false;
       }
+      if (getNodeAttributeOpCode() != null && !getNodeAttributeOpCode()
+          .equals(that.getNodeAttributeOpCode())) {
+        return false;
+      }
       return getTargetExpressions().equals(that.getTargetExpressions());
     }
 
@@ -233,6 +263,7 @@ public class PlacementConstraint {
       int result = getScope().hashCode();
       result = 31 * result + getMinCardinality();
       result = 31 * result + getMaxCardinality();
+      result = 31 * result + getNodeAttributeOpCode().hashCode();
       result = 31 * result + getTargetExpressions().hashCode();
       return result;
     }
@@ -259,6 +290,13 @@ public class PlacementConstraint {
               .append(getScope()).append(",")
               .append(targetExpr)
               .toString());
+        } else if (min == -1 && max == -1) {
+          // node attribute
+          targetConstraints.add(new StringBuilder()
+              .append(getScope()).append(",")
+              .append(getNodeAttributeOpCode()).append(",")
+              .append(targetExpr)
+              .toString());
         } else {
           // cardinality
           targetConstraints.add(new StringBuilder()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
index d22a6bd..73fa328 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraints.java
@@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.AllocationTagNamespaceType;
+import org.apache.hadoop.yarn.api.records.NodeAttributeOpCode;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
@@ -86,6 +87,24 @@ public final class PlacementConstraints {
   }
 
   /**
+   * Creates a constraint that requires allocations to be placed on nodes that
+   * belong to a scope (e.g., node or rack) that satisfy any of the
+   * target expressions based on node attribute op code.
+   *
+   * @param scope the scope within which the target expressions should not be
+   *          true
+   * @param opCode Node Attribute code which could be equals, not equals.
+   * @param targetExpressions the expressions that need to not be true within
+   *          the scope
+   * @return the resulting placement constraint
+   */
+  public static AbstractConstraint targetNodeAttribute(String scope,
+      NodeAttributeOpCode opCode,
+      TargetExpression... targetExpressions) {
+    return new SingleConstraint(scope, -1, -1, opCode, targetExpressions);
+  }
+
+  /**
    * Creates a constraint that restricts the number of allocations within a
    * given scope (e.g., node or rack).
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/constraint/PlacementConstraintParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/constraint/PlacementConstraintParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/constraint/PlacementConstraintParser.java
index 2926c9d..93fd706 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/constraint/PlacementConstraintParser.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/constraint/PlacementConstraintParser.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.util.constraint;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.records.NodeAttributeOpCode;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
@@ -44,11 +45,12 @@ import java.util.regex.Pattern;
 @InterfaceStability.Unstable
 public final class PlacementConstraintParser {
 
+  public static final char EXPRESSION_VAL_DELIM = ',';
   private static final char EXPRESSION_DELIM = ':';
   private static final char KV_SPLIT_DELIM = '=';
-  private static final char EXPRESSION_VAL_DELIM = ',';
   private static final char BRACKET_START = '(';
   private static final char BRACKET_END = ')';
+  private static final String KV_NE_DELIM = "!=";
   private static final String IN = "in";
   private static final String NOT_IN = "notin";
   private static final String AND = "and";
@@ -350,6 +352,91 @@ public final class PlacementConstraintParser {
   }
 
   /**
+   * Constraint parser used to parse a given target expression.
+   */
+  public static class NodeConstraintParser extends ConstraintParser {
+
+    public NodeConstraintParser(String expression) {
+      super(new BaseStringTokenizer(expression,
+          String.valueOf(EXPRESSION_VAL_DELIM)));
+    }
+
+    @Override
+    public AbstractConstraint parse()
+        throws PlacementConstraintParseException {
+      PlacementConstraint.AbstractConstraint placementConstraints = null;
+      String attributeName = "";
+      NodeAttributeOpCode opCode = NodeAttributeOpCode.EQ;
+      String scope = SCOPE_NODE;
+
+      Set<String> constraintEntities = new TreeSet<>();
+      while (hasMoreTokens()) {
+        String currentTag = nextToken();
+        StringTokenizer attributeKV = getAttributeOpCodeTokenizer(currentTag);
+
+        // Usually there will be only one k=v pair. However in case when
+        // multiple values are present for same attribute, it will also be
+        // coming as next token. for example, java=1.8,1.9 or python!=2.
+        if (attributeKV.countTokens() > 1) {
+          opCode = getAttributeOpCode(currentTag);
+          attributeName = attributeKV.nextToken();
+          currentTag = attributeKV.nextToken();
+        }
+        constraintEntities.add(currentTag);
+      }
+
+      if(attributeName.isEmpty()) {
+        throw new PlacementConstraintParseException(
+            "expecting valid expression like k=v or k!=v, but get "
+                + constraintEntities);
+      }
+
+      PlacementConstraint.TargetExpression target = null;
+      if (!constraintEntities.isEmpty()) {
+        target = PlacementConstraints.PlacementTargets
+            .nodeAttribute(attributeName,
+                constraintEntities
+                    .toArray(new String[constraintEntities.size()]));
+      }
+
+      placementConstraints = PlacementConstraints
+          .targetNodeAttribute(scope, opCode, target);
+      return placementConstraints;
+    }
+
+    private StringTokenizer getAttributeOpCodeTokenizer(String currentTag) {
+      StringTokenizer attributeKV = new StringTokenizer(currentTag,
+          KV_NE_DELIM);
+
+      // Try with '!=' delim as well.
+      if (attributeKV.countTokens() < 2) {
+        attributeKV = new StringTokenizer(currentTag,
+            String.valueOf(KV_SPLIT_DELIM));
+      }
+      return attributeKV;
+    }
+
+    /**
+     * Below conditions are validated.
+     * java=8   : OpCode = EQUALS
+     * java!=8  : OpCode = NEQUALS
+     * @param currentTag tag
+     * @return Attribute op code.
+     */
+    private NodeAttributeOpCode getAttributeOpCode(String currentTag)
+        throws PlacementConstraintParseException {
+      if (currentTag.contains(KV_NE_DELIM)) {
+        return NodeAttributeOpCode.NE;
+      } else if (currentTag.contains(String.valueOf(KV_SPLIT_DELIM))) {
+        return NodeAttributeOpCode.EQ;
+      }
+      throw new PlacementConstraintParseException(
+          "expecting valid expression like k=v or k!=v, but get "
+              + currentTag);
+    }
+  }
+
+  /**
    * Constraint parser used to parse a given target expression, such as
    * "NOTIN, NODE, foo, bar".
    */
@@ -363,20 +450,23 @@ public final class PlacementConstraintParser {
     @Override
     public AbstractConstraint parse()
         throws PlacementConstraintParseException {
-      PlacementConstraint.AbstractConstraint placementConstraints;
+      PlacementConstraint.AbstractConstraint placementConstraints = null;
       String op = nextToken();
       if (op.equalsIgnoreCase(IN) || op.equalsIgnoreCase(NOT_IN)) {
         String scope = nextToken();
         scope = parseScope(scope);
 
-        Set<String> allocationTags = new TreeSet<>();
+        Set<String> constraintEntities = new TreeSet<>();
         while(hasMoreTokens()) {
           String tag = nextToken();
-          allocationTags.add(tag);
+          constraintEntities.add(tag);
+        }
+        PlacementConstraint.TargetExpression target = null;
+        if(!constraintEntities.isEmpty()) {
+          target = PlacementConstraints.PlacementTargets.allocationTag(
+              constraintEntities
+                  .toArray(new String[constraintEntities.size()]));
         }
-        PlacementConstraint.TargetExpression target =
-            PlacementConstraints.PlacementTargets.allocationTag(
-                allocationTags.toArray(new String[allocationTags.size()]));
         if (op.equalsIgnoreCase(IN)) {
           placementConstraints = PlacementConstraints
               .targetIn(scope, target);
@@ -551,6 +641,11 @@ public final class PlacementConstraintParser {
         constraintOptional = Optional.ofNullable(jp.tryParse());
       }
       if (!constraintOptional.isPresent()) {
+        NodeConstraintParser np =
+            new NodeConstraintParser(constraintStr);
+        constraintOptional = Optional.ofNullable(np.tryParse());
+      }
+      if (!constraintOptional.isPresent()) {
         throw new PlacementConstraintParseException(
             "Invalid constraint expression " + constraintStr);
       }
@@ -584,12 +679,13 @@ public final class PlacementConstraintParser {
    */
   public static Map<SourceTags, PlacementConstraint> parsePlacementSpec(
       String expression) throws PlacementConstraintParseException {
+    // Continue handling for application tag based constraint otherwise.
     // Respect insertion order.
     Map<SourceTags, PlacementConstraint> result = new LinkedHashMap<>();
     PlacementConstraintParser.ConstraintTokenizer tokenizer =
         new PlacementConstraintParser.MultipleConstraintsTokenizer(expression);
     tokenizer.validate();
-    while(tokenizer.hasMoreElements()) {
+    while (tokenizer.hasMoreElements()) {
       String specStr = tokenizer.nextElement();
       // each spec starts with sourceAllocationTag=numOfContainers and
       // followed by a constraint expression.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 10b36c7..5fe2cc9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -646,11 +646,18 @@ message PlacementConstraintProto {
   optional CompositePlacementConstraintProto compositeConstraint = 2;
 }
 
+enum NodeAttributeOpCodeProto {
+  NO_OP = 1;
+  EQ = 2;
+  NE = 3;
+}
+
 message SimplePlacementConstraintProto {
   required string scope = 1;
   repeated PlacementConstraintTargetProto targetExpressions = 2;
   optional int32 minCardinality = 3;
   optional int32 maxCardinality = 4;
+  optional NodeAttributeOpCodeProto attributeOpCode = 5;
 }
 
 message PlacementConstraintTargetProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintParser.java
index a69571c..9806ba4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintParser.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintParser.java
@@ -22,6 +22,8 @@ import com.google.common.collect.Sets;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
+
+import org.apache.hadoop.yarn.api.records.NodeAttributeOpCode;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
@@ -38,8 +40,14 @@ import org.apache.hadoop.yarn.util.constraint.PlacementConstraintParser.Multiple
 import org.apache.hadoop.yarn.util.constraint.PlacementConstraintParser.SourceTagsTokenizer;
 import org.apache.hadoop.yarn.util.constraint.PlacementConstraintParser.ConstraintTokenizer;
 
-import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.*;
 import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.and;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.cardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.or;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetNodeAttribute;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetNotIn;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -443,4 +451,55 @@ public class TestPlacementConstraintParser {
           + constrainExpr + ", caused by: " + e.getMessage());
     }
   }
+
+  @Test
+  public void testParseNodeAttributeSpec()
+      throws PlacementConstraintParseException {
+    Map<SourceTags, PlacementConstraint> result;
+    PlacementConstraint.AbstractConstraint expectedPc1, expectedPc2;
+    PlacementConstraint actualPc1, actualPc2;
+
+    // A single node attribute constraint
+    result = PlacementConstraintParser
+        .parsePlacementSpec("xyz=4,rm.yarn.io/foo=true");
+    Assert.assertEquals(1, result.size());
+    TargetExpression target = PlacementTargets
+        .nodeAttribute("rm.yarn.io/foo", "true");
+    expectedPc1 = targetNodeAttribute("node", NodeAttributeOpCode.EQ, target);
+
+    actualPc1 = result.values().iterator().next();
+    Assert.assertEquals(expectedPc1, actualPc1.getConstraintExpr());
+
+    // A single node attribute constraint
+    result = PlacementConstraintParser
+        .parsePlacementSpec("xyz=3,rm.yarn.io/foo!=abc");
+    Assert.assertEquals(1, result.size());
+    target = PlacementTargets
+        .nodeAttribute("rm.yarn.io/foo", "abc");
+    expectedPc1 = targetNodeAttribute("node", NodeAttributeOpCode.NE, target);
+
+    actualPc1 = result.values().iterator().next();
+    Assert.assertEquals(expectedPc1, actualPc1.getConstraintExpr());
+
+    actualPc1 = result.values().iterator().next();
+    Assert.assertEquals(expectedPc1, actualPc1.getConstraintExpr());
+
+    // A single node attribute constraint
+    result = PlacementConstraintParser
+        .parsePlacementSpec(
+            "xyz=1,rm.yarn.io/foo!=abc:zxy=1,rm.yarn.io/bar=true");
+    Assert.assertEquals(2, result.size());
+    target = PlacementTargets
+        .nodeAttribute("rm.yarn.io/foo", "abc");
+    expectedPc1 = targetNodeAttribute("node", NodeAttributeOpCode.NE, target);
+    target = PlacementTargets
+        .nodeAttribute("rm.yarn.io/bar", "true");
+    expectedPc2 = targetNodeAttribute("node", NodeAttributeOpCode.EQ, target);
+
+    Iterator<PlacementConstraint> valueIt = result.values().iterator();
+    actualPc1 = valueIt.next();
+    actualPc2 = valueIt.next();
+    Assert.assertEquals(expectedPc1, actualPc1.getConstraintExpr());
+    Assert.assertEquals(expectedPc2, actualPc2.getConstraintExpr());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
index ecf07b1..09a796e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
@@ -523,9 +523,13 @@ public class ApplicationMaster {
 
     if (cliParser.hasOption("placement_spec")) {
       String placementSpec = cliParser.getOptionValue("placement_spec");
-      LOG.info("Placement Spec received [{}]", placementSpec);
-      parsePlacementSpecs(placementSpec);
+      String decodedSpec = getDecodedPlacementSpec(placementSpec);
+      LOG.info("Placement Spec received [{}]", decodedSpec);
+
+      this.numTotalContainers = 0;
+      parsePlacementSpecs(decodedSpec);
       LOG.info("Total num containers requested [{}]", numTotalContainers);
+
       if (numTotalContainers == 0) {
         throw new IllegalArgumentException(
             "Cannot run distributed shell with no containers");
@@ -694,23 +698,25 @@ public class ApplicationMaster {
     return true;
   }
 
-  private void parsePlacementSpecs(String placementSpecifications) {
-    // Client sends placement spec in encoded format
-    Base64.Decoder decoder = Base64.getDecoder();
-    byte[] decodedBytes = decoder.decode(
-        placementSpecifications.getBytes(StandardCharsets.UTF_8));
-    String decodedSpec = new String(decodedBytes, StandardCharsets.UTF_8);
-    LOG.info("Decode placement spec: " + decodedSpec);
+  private void parsePlacementSpecs(String decodedSpec) {
     Map<String, PlacementSpec> pSpecs =
         PlacementSpec.parse(decodedSpec);
     this.placementSpecs = new HashMap<>();
-    this.numTotalContainers = 0;
     for (PlacementSpec pSpec : pSpecs.values()) {
-      this.numTotalContainers += pSpec.numContainers;
+      this.numTotalContainers += pSpec.getNumContainers();
       this.placementSpecs.put(pSpec.sourceTag, pSpec);
     }
   }
 
+  private String getDecodedPlacementSpec(String placementSpecifications) {
+    Base64.Decoder decoder = Base64.getDecoder();
+    byte[] decodedBytes = decoder.decode(
+        placementSpecifications.getBytes(StandardCharsets.UTF_8));
+    String decodedSpec = new String(decodedBytes, StandardCharsets.UTF_8);
+    LOG.info("Decode placement spec: " + decodedSpec);
+    return decodedSpec;
+  }
+
   /**
    * Helper function to print usage
    *
@@ -798,6 +804,7 @@ public class ApplicationMaster {
         }
       }
     }
+
     RegisterApplicationMasterResponse response = amRMClient
         .registerApplicationMaster(appMasterHostname, appMasterRpcPort,
             appMasterTrackingUrl, placementConstraintMap);
@@ -845,14 +852,18 @@ public class ApplicationMaster {
     // Keep looping until all the containers are launched and shell script
     // executed on them ( regardless of success/failure).
     if (this.placementSpecs == null) {
+      LOG.info("placementSpecs null");
       for (int i = 0; i < numTotalContainersToRequest; ++i) {
         ContainerRequest containerAsk = setupContainerAskForRM();
         amRMClient.addContainerRequest(containerAsk);
       }
     } else {
+      LOG.info("placementSpecs to create req:" + placementSpecs);
       List<SchedulingRequest> schedReqs = new ArrayList<>();
       for (PlacementSpec pSpec : this.placementSpecs.values()) {
-        for (int i = 0; i < pSpec.numContainers; i++) {
+        LOG.info("placementSpec :" + pSpec + ", container:" + pSpec
+            .getNumContainers());
+        for (int i = 0; i < pSpec.getNumContainers(); i++) {
           SchedulingRequest sr = setupSchedulingRequest(pSpec);
           schedReqs.add(sr);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
index 9da9288..e8b69fe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
@@ -103,7 +103,7 @@ import org.slf4j.LoggerFactory;
  * the provided shell command on a set of containers. </p>
  * 
  * <p>This client is meant to act as an example on how to write yarn-based applications. </p>
- * 
+ *
  * <p> To submit an application, a client first needs to connect to the <code>ResourceManager</code> 
  * aka ApplicationsManager or ASM via the {@link ApplicationClientProtocol}. The {@link ApplicationClientProtocol} 
  * provides a way for the client to get access to cluster information and to request for a
@@ -192,6 +192,8 @@ public class Client {
 
   // Placement specification
   private String placementSpec = "";
+  // Node Attribute specification
+  private String nodeAttributeSpec = "";
   // log4j.properties file 
   // if available, add to local resources and set into classpath 
   private String log4jPropFile = "";	
@@ -448,6 +450,7 @@ public class Client {
       // Check if it is parsable
       PlacementSpec.parse(this.placementSpec);
     }
+
     appName = cliParser.getOptionValue("appname", "DistributedShell");
     amPriority = Integer.parseInt(cliParser.getOptionValue("priority", "0"));
     amQueue = cliParser.getOptionValue("queue", "default");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/PlacementSpec.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/PlacementSpec.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/PlacementSpec.java
index 2909259..ceaa37d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/PlacementSpec.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/PlacementSpec.java
@@ -37,8 +37,8 @@ public class PlacementSpec {
       LoggerFactory.getLogger(PlacementSpec.class);
 
   public final String sourceTag;
-  public final int numContainers;
   public final PlacementConstraint constraint;
+  private int numContainers;
 
   public PlacementSpec(String sourceTag, int numContainers,
       PlacementConstraint constraint) {
@@ -47,6 +47,22 @@ public class PlacementSpec {
     this.constraint = constraint;
   }
 
+  /**
+   * Get the number of container for this spec.
+   * @return container count
+   */
+  public int getNumContainers() {
+    return numContainers;
+  }
+
+  /**
+   * Set number of containers for this spec.
+   * @param numContainers number of containers.
+   */
+  public void setNumContainers(int numContainers) {
+    this.numContainers = numContainers;
+  }
+
   // Placement specification should be of the form:
   // PlacementSpec => ""|KeyVal;PlacementSpec
   // KeyVal => SourceTag=Constraint
@@ -71,6 +87,7 @@ public class PlacementSpec {
   public static Map<String, PlacementSpec> parse(String specs)
       throws IllegalArgumentException {
     LOG.info("Parsing Placement Specs: [{}]", specs);
+
     Map<String, PlacementSpec> pSpecs = new HashMap<>();
     Map<SourceTags, PlacementConstraint> parsed;
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java
index 926b6fa..447905e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintFromProtoConverter.java
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.NodeAttributeOpCode;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
@@ -37,6 +38,7 @@ import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementConstraint;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeOpCodeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintTargetProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.SimplePlacementConstraintProto;
@@ -73,7 +75,8 @@ public class PlacementConstraintFromProtoConverter {
     }
 
     return new SingleConstraint(proto.getScope(), proto.getMinCardinality(),
-        proto.getMaxCardinality(), targets);
+        proto.getMaxCardinality(),
+        convertFromProtoFormat(proto.getAttributeOpCode()), targets);
   }
 
   private TargetExpression convert(PlacementConstraintTargetProto proto) {
@@ -113,4 +116,9 @@ public class PlacementConstraintFromProtoConverter {
     return new TimedPlacementConstraint(pConstraint, proto.getSchedulingDelay(),
         ProtoUtils.convertFromProtoFormat(proto.getDelayUnit()));
   }
+
+  private static NodeAttributeOpCode convertFromProtoFormat(
+      NodeAttributeOpCodeProto p) {
+    return NodeAttributeOpCode.valueOf(p.name());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java
index 7816e18..30f7741 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/pb/PlacementConstraintToProtoConverter.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.api.pb;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.NodeAttributeOpCode;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
@@ -34,6 +35,7 @@ import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TimedPlacementCon
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.CompositePlacementConstraintProto.CompositeType;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeOpCodeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PlacementConstraintTargetProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.SimplePlacementConstraintProto;
@@ -72,6 +74,10 @@ public class PlacementConstraintToProtoConverter
     }
     sb.setMinCardinality(constraint.getMinCardinality());
     sb.setMaxCardinality(constraint.getMaxCardinality());
+    if (constraint.getNodeAttributeOpCode() != null) {
+      sb.setAttributeOpCode(
+          convertToProtoFormat(constraint.getNodeAttributeOpCode()));
+    }
     if (constraint.getTargetExpressions() != null) {
       for (TargetExpression target : constraint.getTargetExpressions()) {
         sb.addTargetExpressions(
@@ -171,4 +177,9 @@ public class PlacementConstraintToProtoConverter
 
     return tb.build();
   }
+
+  private static NodeAttributeOpCodeProto convertToProtoFormat(
+      NodeAttributeOpCode p) {
+    return NodeAttributeOpCodeProto.valueOf(p.name());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 81ef337..16f019f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -519,9 +519,10 @@ public class ResourceManager extends CompositeService
     return new RMNodeLabelsManager();
   }
 
-  protected NodeAttributesManager createNodeAttributesManager()
-      throws InstantiationException, IllegalAccessException {
-    return new NodeAttributesManagerImpl();
+  protected NodeAttributesManager createNodeAttributesManager() {
+    NodeAttributesManagerImpl namImpl = new NodeAttributesManagerImpl();
+    namImpl.setRMContext(rmContext);
+    return namImpl;
   }
 
   protected AllocationTagsManager createAllocationTagsManager() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
index fac2dfd..9111d0f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
@@ -57,6 +57,8 @@ import org.apache.hadoop.yarn.nodelabels.RMNodeAttribute;
 import org.apache.hadoop.yarn.nodelabels.StringAttributeValue;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAttributesUpdateSchedulerEvent;
 
 import com.google.common.base.Strings;
 
@@ -92,6 +94,7 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
 
   private final ReadLock readLock;
   private final WriteLock writeLock;
+  private RMContext rmContext = null;
 
   public NodeAttributesManagerImpl() {
     super("NodeAttributesManagerImpl");
@@ -131,7 +134,7 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
   }
 
   protected void initNodeAttributeStore(Configuration conf) throws Exception {
-    this.store =getAttributeStoreClass(conf);
+    this.store = getAttributeStoreClass(conf);
     this.store.init(conf, this);
     this.store.recover();
   }
@@ -206,6 +209,21 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
             .handle(new NodeAttributesStoreEvent(nodeAttributeMapping, op));
       }
 
+      // Map used to notify RM
+      Map<String, Set<NodeAttribute>> newNodeToAttributesMap =
+          new HashMap<String, Set<NodeAttribute>>();
+      nodeAttributeMapping.forEach((k, v) -> {
+        Host node = nodeCollections.get(k);
+        newNodeToAttributesMap.put(k, node.attributes.keySet());
+      });
+
+      // Notify RM
+      if (rmContext != null && rmContext.getDispatcher() != null) {
+        LOG.info("Updated NodeAttribute event to RM:" + newNodeToAttributesMap
+            .values());
+        rmContext.getDispatcher().getEventHandler().handle(
+            new NodeAttributesUpdateSchedulerEvent(newNodeToAttributesMap));
+      }
     } finally {
       writeLock.unlock();
     }
@@ -703,4 +721,8 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
       store.close();
     }
   }
+
+  public void setRMContext(RMContext context) {
+    this.rmContext  = context;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index 59771fd..b35aeba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceUtilization;
@@ -79,6 +80,8 @@ public abstract class SchedulerNode {
 
   private volatile Set<String> labels = null;
 
+  private volatile Set<NodeAttribute> nodeAttributes = null;
+
   // Last updated time
   private volatile long lastHeartbeatMonotonicTime;
 
@@ -503,6 +506,14 @@ public abstract class SchedulerNode {
     return getNodeID().hashCode();
   }
 
+  public Set<NodeAttribute> getNodeAttributes() {
+    return nodeAttributes;
+  }
+
+  public void updateNodeAttributes(Set<NodeAttribute> attributes) {
+    this.nodeAttributes = attributes;
+  }
+
   private static class ContainerInfo {
     private final RMContainer container;
     private boolean launchedOnNode;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index 81dcf86..a1d3f60 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -137,6 +138,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedS
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAttributesUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeLabelsUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent;
@@ -1767,6 +1769,14 @@ public class CapacityScheduler extends
       updateNodeLabelsAndQueueResource(labelUpdateEvent);
     }
     break;
+    case NODE_ATTRIBUTES_UPDATE:
+    {
+      NodeAttributesUpdateSchedulerEvent attributeUpdateEvent =
+          (NodeAttributesUpdateSchedulerEvent) event;
+
+      updateNodeAttributes(attributeUpdateEvent);
+    }
+    break;
     case NODE_UPDATE:
     {
       NodeUpdateSchedulerEvent nodeUpdatedEvent = (NodeUpdateSchedulerEvent)event;
@@ -1900,6 +1910,30 @@ public class CapacityScheduler extends
     }
   }
 
+  private void updateNodeAttributes(
+      NodeAttributesUpdateSchedulerEvent attributeUpdateEvent) {
+    try {
+      writeLock.lock();
+      for (Entry<String, Set<NodeAttribute>> entry : attributeUpdateEvent
+          .getUpdatedNodeToAttributes().entrySet()) {
+        String hostname = entry.getKey();
+        Set<NodeAttribute> attributes = entry.getValue();
+        List<NodeId> nodeIds = nodeTracker.getNodeIdsByResourceName(hostname);
+        updateAttributesOnNode(nodeIds, attributes);
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private void updateAttributesOnNode(List<NodeId> nodeIds,
+      Set<NodeAttribute> attributes) {
+    nodeIds.forEach((k) -> {
+      SchedulerNode node = nodeTracker.getNode(k);
+      node.updateNodeAttributes(attributes);
+    });
+  }
+
   /**
    * Process node labels update.
    */
@@ -2768,7 +2802,7 @@ public class CapacityScheduler extends
               schedulingRequest, schedulerNode,
               rmContext.getPlacementConstraintManager(),
               rmContext.getAllocationTagsManager())) {
-            LOG.debug("Failed to allocate container for application "
+            LOG.info("Failed to allocate container for application "
                 + appAttempt.getApplicationId() + " on node "
                 + schedulerNode.getNodeName()
                 + " because this allocation violates the"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java
index f47e1d4..ccd334c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java
@@ -24,8 +24,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.SchedulingRequest;
+import org.apache.hadoop.yarn.api.records.*;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
@@ -114,22 +113,92 @@ public final class PlacementConstraintsUtil {
             || maxScopeCardinality <= desiredMaxCardinality);
   }
 
-  private static boolean canSatisfyNodePartitionConstraintExpresssion(
-      TargetExpression targetExpression, SchedulerNode schedulerNode) {
+  private static boolean canSatisfyNodeConstraintExpresssion(
+      SingleConstraint sc, TargetExpression targetExpression,
+      SchedulerNode schedulerNode) {
     Set<String> values = targetExpression.getTargetValues();
-    if (values == null || values.isEmpty()) {
-      return schedulerNode.getPartition().equals(
-          RMNodeLabelsManager.NO_LABEL);
-    } else{
-      String nodePartition = values.iterator().next();
-      if (!nodePartition.equals(schedulerNode.getPartition())) {
+
+    if (targetExpression.getTargetKey().equals(NODE_PARTITION)) {
+      if (values == null || values.isEmpty()) {
+        return schedulerNode.getPartition()
+            .equals(RMNodeLabelsManager.NO_LABEL);
+      } else {
+        String nodePartition = values.iterator().next();
+        if (!nodePartition.equals(schedulerNode.getPartition())) {
+          return false;
+        }
+      }
+    } else {
+      NodeAttributeOpCode opCode = sc.getNodeAttributeOpCode();
+      // compare attributes.
+      String inputAttribute = values.iterator().next();
+      NodeAttribute requestAttribute = getNodeConstraintFromRequest(
+          targetExpression.getTargetKey(), inputAttribute);
+      if (requestAttribute == null) {
+        return true;
+      }
+
+      if (schedulerNode.getNodeAttributes() == null ||
+          !schedulerNode.getNodeAttributes().contains(requestAttribute)) {
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("Incoming requestAttribute:" + requestAttribute
+              + "is not present in " + schedulerNode.getNodeID());
+        }
+        return false;
+      }
+      boolean found = false;
+      for (Iterator<NodeAttribute> it = schedulerNode.getNodeAttributes()
+          .iterator(); it.hasNext();) {
+        NodeAttribute nodeAttribute = it.next();
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Starting to compare Incoming requestAttribute :"
+              + requestAttribute
+              + " with requestAttribute value= " + requestAttribute
+              .getAttributeValue()
+              + ", stored nodeAttribute value=" + nodeAttribute
+              .getAttributeValue());
+        }
+        if (requestAttribute.equals(nodeAttribute)) {
+          if (isOpCodeMatches(requestAttribute, nodeAttribute, opCode)) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(
+                  "Incoming requestAttribute:" + requestAttribute
+                      + " matches with node:" + schedulerNode.getNodeID());
+            }
+            found = true;
+            return found;
+          }
+        }
+      }
+      if (!found) {
+        if(LOG.isDebugEnabled()) {
+          LOG.info("skip this node:" + schedulerNode.getNodeID()
+              + " for requestAttribute:" + requestAttribute);
+        }
         return false;
       }
     }
-
     return true;
   }
 
+  private static boolean isOpCodeMatches(NodeAttribute requestAttribute,
+      NodeAttribute nodeAttribute, NodeAttributeOpCode opCode) {
+    boolean retCode = false;
+    switch (opCode) {
+    case EQ:
+      retCode = requestAttribute.getAttributeValue()
+          .equals(nodeAttribute.getAttributeValue());
+      break;
+    case NE:
+      retCode = !(requestAttribute.getAttributeValue()
+          .equals(nodeAttribute.getAttributeValue()));
+      break;
+    default:
+      break;
+    }
+    return retCode;
+  }
+
   private static boolean canSatisfySingleConstraint(ApplicationId applicationId,
       SingleConstraint singleConstraint, SchedulerNode schedulerNode,
       AllocationTagsManager tagsManager)
@@ -146,10 +215,12 @@ public final class PlacementConstraintsUtil {
             singleConstraint, currentExp, schedulerNode, tagsManager)) {
           return false;
         }
-      } else if (currentExp.getTargetType().equals(TargetType.NODE_ATTRIBUTE)
-          && currentExp.getTargetKey().equals(NODE_PARTITION)) {
-        // This is a node partition expression, check it.
-        canSatisfyNodePartitionConstraintExpresssion(currentExp, schedulerNode);
+      } else if (currentExp.getTargetType().equals(TargetType.NODE_ATTRIBUTE)) {
+        // This is a node attribute expression, check it.
+        if (!canSatisfyNodeConstraintExpresssion(singleConstraint, currentExp,
+            schedulerNode)) {
+          return false;
+        }
       }
     }
     // return true if all targetExpressions are satisfied
@@ -203,6 +274,11 @@ public final class PlacementConstraintsUtil {
       AllocationTagsManager atm)
       throws InvalidAllocationTagsQueryException {
     if (constraint == null) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug(
+            "Constraint is found empty during constraint validation for app:"
+                + appId);
+      }
       return true;
     }
 
@@ -263,4 +339,24 @@ public final class PlacementConstraintsUtil {
         pcm.getMultilevelConstraint(applicationId, sourceTags, pc),
         schedulerNode, atm);
   }
+
+  private static NodeAttribute getNodeConstraintFromRequest(String attrKey,
+      String attrString) {
+    NodeAttribute nodeAttribute = null;
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Incoming node attribute: " + attrKey + "=" + attrString);
+    }
+
+    // Input node attribute could be like 1.8
+    String[] name = attrKey.split("/");
+    if (name == null || name.length == 1) {
+      nodeAttribute = NodeAttribute
+          .newInstance(attrKey, NodeAttributeType.STRING, attrString);
+    } else {
+      nodeAttribute = NodeAttribute
+          .newInstance(name[0], name[1], NodeAttributeType.STRING, attrString);
+    }
+
+    return nodeAttribute;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/NodeAttributesUpdateSchedulerEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/NodeAttributesUpdateSchedulerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/NodeAttributesUpdateSchedulerEvent.java
new file mode 100644
index 0000000..cdc0b69
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/NodeAttributesUpdateSchedulerEvent.java
@@ -0,0 +1,41 @@
+/**
+ * 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.scheduler.event;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+
+/**
+ * Event handler class for Node Attributes which sends events to Scheduler.
+ */
+public class NodeAttributesUpdateSchedulerEvent extends SchedulerEvent {
+  private Map<String, Set<NodeAttribute>> nodeToAttributes;
+
+  public NodeAttributesUpdateSchedulerEvent(
+      Map<String, Set<NodeAttribute>> newNodeToAttributesMap) {
+    super(SchedulerEventType.NODE_ATTRIBUTES_UPDATE);
+    this.nodeToAttributes = newNodeToAttributesMap;
+  }
+
+  public Map<String, Set<NodeAttribute>> getUpdatedNodeToAttributes() {
+    return nodeToAttributes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
index b107cf4..869bf0ed9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/SchedulerEventType.java
@@ -26,6 +26,7 @@ public enum SchedulerEventType {
   NODE_UPDATE,
   NODE_RESOURCE_UPDATE,
   NODE_LABELS_UPDATE,
+  NODE_ATTRIBUTES_UPDATE,
 
   // Source: RMApp
   APP_ADDED,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67ae81f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
index 9d30e90..4557350 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
@@ -396,6 +396,10 @@ public class LocalityAppPlacementAllocator <N extends SchedulerNode>
       SchedulingMode schedulingMode) {
     // We will only look at node label = nodeLabelToLookAt according to
     // schedulingMode and partition of node.
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("precheckNode is invoked for " + schedulerNode.getNodeID() + ","
+          + schedulingMode);
+    }
     String nodePartitionToLookAt;
     if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
       nodePartitionToLookAt = schedulerNode.getPartition();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[24/29] hadoop git commit: YARN-7865. Node attributes documentation. Contributed by Naganarasimha G R.

Posted by su...@apache.org.
YARN-7865. Node attributes documentation. Contributed by Naganarasimha G R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1e7d6e55
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1e7d6e55
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1e7d6e55

Branch: refs/heads/trunk
Commit: 1e7d6e55a59e2d5265d0c99e7bb1856c6849d662
Parents: 5219435
Author: Weiwei Yang <ww...@apache.org>
Authored: Fri Aug 31 17:52:26 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:01 2018 +0530

----------------------------------------------------------------------
 hadoop-project/src/site/site.xml                |   1 +
 .../src/site/markdown/NodeAttributes.md         | 156 +++++++++++++++++++
 2 files changed, 157 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e7d6e55/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index 40df7c5..b40dbfc 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -142,6 +142,7 @@
       <item name="ResourceManager HA" href="hadoop-yarn/hadoop-yarn-site/ResourceManagerHA.html"/>
       <item name="Resource Model" href="hadoop-yarn/hadoop-yarn-site/ResourceModel.html"/>
       <item name="Node Labels" href="hadoop-yarn/hadoop-yarn-site/NodeLabel.html"/>
+      <item name="Node Attributes" href="hadoop-yarn/hadoop-yarn-site/NodeAttributes.html"/>
       <item name="Web Application Proxy" href="hadoop-yarn/hadoop-yarn-site/WebApplicationProxy.html"/>
       <item name="Timeline Server" href="hadoop-yarn/hadoop-yarn-site/TimelineServer.html"/>
       <item name="Timeline Service V.2" href="hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html"/>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e7d6e55/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeAttributes.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeAttributes.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeAttributes.md
new file mode 100644
index 0000000..5128004
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeAttributes.md
@@ -0,0 +1,156 @@
+<!---
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+YARN Node Attributes
+===============
+
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
+
+Overview
+--------
+
+Node Attribute is a way to describe the attributes of a Node without resource guarantees. This could be used by applications to pick up the right nodes for their container to be placed based on expression of multitude of these attributes.
+
+Features
+--------
+
+The salient features of ```Node Attributes``` is as follows:
+
+* A Node can be associated with multiple attributes.
+* Value can be associated with a attribute tagged to a node. String type values are only supported currently.
+* Unlike Node Labels, Node Attributes need not be specified explicitly at the cluster level, but there are API's to list the attributes available at the cluster level.
+* As its non tangible resource, its not associated with any queue and thus queue resource planning and authorisation is not required for attributes.
+* Similar to the allocation tags, Applications will be able to request containers using expressions containing one or more of these attributes using *Placement Constraints*.
+* Equals (=) and Not Equals (!=) are the only supported operators in the expression. AND & OR can also be used as part of attribute expression.
+* Node attribute constraints are hard limits, that says the allocation can only be made if the node satisfies the node attribute constraint. In another word, the request keeps pending until it finds a valid node satisfying the constraint. There is no relax policy at present.
+* Operability
+    * Node Attributes and its mapping to nodes can be recovered across RM restart
+    * Update node attributes - admin can add, remove and replace attributes on nodes when RM is running
+* Mapping of NM to node attributes can be done in two ways,
+    * **Centralised :** Node to attributes mapping can be done through RM exposed CLI or RPC (REST is yet to be supported).
+    * **Distributed :** Node to attributes mapping will be set by a configured Node Attributes Provider in NM. We have two different providers in YARN: *Script* based provider and *Configuration* based provider. In case of script, NM can be configured with a script path and the script can emit the attribute(s) of the node. In case of config, node Attributes can be directly configured in the NM's yarn-site.xml. In both of these options dynamic refresh of the attribute mapping is supported.
+
+* Unlike labels, attributes can be mapped to a node from both Centralised and Distributed modes at the same time. There will be no clashes as attributes are identified with different prefix in different modes. In case of **Centralized** attributes are identified by prefix *"rm.yarn.io"* and in case of **Distributed** attributes are identified by prefix *"nm.yarn.io"*. This implies attributes are uniquely identified by *prefix* and *name*.
+
+Configuration
+-------------
+
+###Setting up ResourceManager for Node Attributes
+
+Unlike Node Labels, Node Attributes need not be explicitly enabled as it will always exist and would have no impact in terms of performance or compatibility even if feature is not used.
+
+Setup following properties in ```yarn-site.xml```
+
+Property  | Value | Default Value
+--- | ---- | ----
+yarn.node-attribute.fs-store.root-dir  | path where centralized attribute mappings are stored | file:///tmp/hadoop-yarn-${user}/node-attribute/
+yarn.node-attribute.fs-store.impl.class | Configured class needs to extend org.apache.hadoop.yarn.nodelabels.NodeAttributeStore | FileSystemNodeAttributeStore
+
+
+Notes:
+
+* Make sure ```yarn.node-attribute.fs-store.root-dir``` is created with resource manager process user and ```ResourceManager``` has permission to access it. (Typically from “yarn” user)
+* If user want to store node attributes to local file system of RM, paths like `file:///home/yarn/node-attributes` can be used else if in hdfs  paths like `hdfs://namenode:port/path/to/store/node-attributes/` can be used.
+
+###Centralised Node Attributes mapping.
+
+Three options are supported to map attributes to node in **Centralised** approach:
+
+* **add**
+    Executing ```yarn nodeattributes -add “node1:attribute[(type)][=value],attribute2  node2:attribute2[=value],attribute3``` adds attributes to the nodes without impacting already existing mapping on the node(s).
+
+* **remove**
+    Executing ```yarn nodeattributes -remove “node1:attribute,attribute1 node2:attribute2"``` removes attributes to the nodes without impacting already existing mapping on the node(s).
+
+* **replace**
+    Executing ```yarn nodeattributes -replace “node1:attribute[(type)][=value],attribute1[=value],attribute2  node2:attribute2[=value],attribute3""``` replaces the existing attributes to the nodes with the one configured as part of this command.
+
+Notes:
+
+* Ports need **not** be mentioned, attributes are mapped to all the NM instances in the node.
+* *Space* is the delimiter for multiple node-Attribute mapping pair
+* *","* is used as delimiter for multiple attributes of a node.
+* *"type"* defaults to string if not specified which is the only type currently supported.
+* All the above 3 operations can be performed only by admin user.
+
+###Distributed Node Attributes mapping.
+
+Configuring attributes to nodes in **Distributed** mode
+
+Property  | Value
+----- | ------
+yarn.nodemanager.node-attributes.provider | Administrators can configure the provider for the node attributes by configuring this parameter in NM. Administrators can configure *"config"*, *"script"* or the *class name* of the provider. Configured  class needs to extend *org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeAttributesProvider*. If *"config"* is configured, then *"ConfigurationNodeAttributesProvider"* and if *"script"* is configured, then *"ScriptBasedNodeAttributesProvider"* will be used.
+yarn.nodemanager.node-attributes.provider.fetch-interval-ms  | When *"yarn.nodemanager.node-attributes.provider"* is configured with *"config"*, *"script"* or the *configured class* extends NodeAttributesProvider, then periodically node attributes are retrieved from the node attributes provider. This configuration is to define the interval period. If -1 is configured, then node attributes are retrieved from provider only during initialisation. Defaults to 10 mins.
+yarn.nodemanager.node-attributes.provider.fetch-timeout-ms | When *"yarn.nodemanager.node-attributes.provider"* is configured with *"script"*, then this configuration provides the timeout period after which it will interrupt the script which queries the node attributes. Defaults to 20 mins.
+yarn.nodemanager.node-attributes.provider.script.path | The node attribute script NM runs to collect node attributes. Lines in the script output starting with "NODE_ATTRIBUTE:" will be considered as a record of node attribute, attribute name, type and value should be delimited by comma. Each of such lines will be parsed to a node attribute.
+yarn.nodemanager.node-attributes.provider.script.opts | The arguments to pass to the node attribute script.
+yarn.nodemanager.node-attributes.provider.configured-node-attributes |  When "yarn.nodemanager.node-attributes.provider" is configured with "config" then ConfigurationNodeAttributesProvider fetches node attributes from this parameter.
+
+Specifying node attributes for application
+-------------------------------------
+
+Applications can use Placement Constraint APIs to specify node attribute request as mentioned in [Placement Constraint documentation](./PlacementConstraints.html).
+
+Here is an example for creating a Scheduling Request object with NodeAttribute expression:
+
+
+    //expression : AND(python!=3:java=1.8)
+    SchedulingRequest schedulingRequest =
+        SchedulingRequest.newBuilder().executionType(
+            ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
+            .allocationRequestId(10L).priority(Priority.newInstance(1))
+            .placementConstraintExpression(
+                PlacementConstraints.and(
+                    PlacementConstraints
+                        .targetNodeAttribute(PlacementConstraints.NODE,
+                            NodeAttributeOpCode.NE,
+                            PlacementConstraints.PlacementTargets
+                                .nodeAttribute("python", "3")),
+                    PlacementConstraints
+                        .targetNodeAttribute(PlacementConstraints.NODE,
+                            NodeAttributeOpCode.EQ,
+                            PlacementConstraints.PlacementTargets
+                                .nodeAttribute("java", "1.8")))
+                    .build()).resourceSizing(
+            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
+            .build();
+
+The above SchedulingRequest requests for 1 container on nodes that must satisfy following constraints:
+1. Node attribute *`rm.yarn.io/python`* doesn't exist on the node or it exist but its value is not equal to 3
+2. Node attribute *`rm.yarn.io/java`* must exist on the node and its value is equal to 1.8
+
+
+Monitoring
+----------
+
+###Monitoring through REST
+
+As part of *`http://rm-http-address:port/ws/v1/cluster/nodes/{nodeid}`* REST output attributes and its values mapped to the given node can be got.
+
+###Monitoring through web UI
+
+Yet to be supported
+
+###Monitoring through commandline
+
+* Use `yarn cluster --list-node-attributes` to get all the attributes in the cluster
+* Use `yarn nodeattributes -list` to get attributes in the cluster
+* Use `yarn nodeattributes -attributestonodes  -attributes <Attributes>` to list for each attribute, all the mapped nodes and the attribute value configured for each node. Optionally we can specify for the specified attributes using *-attributes*.
+* Use `yarn nodeattributes -nodestoattributes -nodes <Host Names>` to list all the attributes and its value mapped to a node. Optionally we can specify for the specified node using *-nodes*.
+* Node status/detail got from `yarn node -status` will list all the attributes and its value associated with the node.
+
+Useful links
+------------
+
+*  [Placement Constraint documentation](./PlacementConstraints.html), if you need more understanding about how to configure Placement Constraints.


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[15/29] hadoop git commit: YARN-8104. Add API to fetch node to attribute mapping. Contributed by Bibin A Chundatt.

Posted by su...@apache.org.
YARN-8104. Add API to fetch node to attribute mapping. Contributed by Bibin A Chundatt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5dc7d6e0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5dc7d6e0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5dc7d6e0

Branch: refs/heads/trunk
Commit: 5dc7d6e0f3f46277f0615d61755970a7aa28b8d7
Parents: 0a01b13
Author: Naganarasimha <na...@apache.org>
Authored: Fri Apr 20 07:31:53 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../hadoop/mapred/ResourceMgrDelegate.java      |   6 +
 .../hadoop/mapred/TestClientRedirect.java       |   8 +
 .../yarn/api/ApplicationClientProtocol.java     |  18 ++
 .../GetNodesToAttributesRequest.java            |  65 +++++++
 .../GetNodesToAttributesResponse.java           |  63 +++++++
 .../main/proto/applicationclient_protocol.proto |   1 +
 ..._server_resourcemanager_service_protos.proto |   5 -
 .../src/main/proto/yarn_protos.proto            |   5 +
 .../src/main/proto/yarn_service_protos.proto    |  10 +-
 .../hadoop/yarn/client/api/YarnClient.java      |  19 ++
 .../yarn/client/api/impl/YarnClientImpl.java    |   9 +
 .../ApplicationClientProtocolPBClientImpl.java  |  18 ++
 .../ApplicationClientProtocolPBServiceImpl.java |  21 +++
 .../pb/GetAttributesToNodesResponsePBImpl.java  |   6 +-
 .../pb/GetNodesToAttributesRequestPBImpl.java   | 132 ++++++++++++++
 .../pb/GetNodesToAttributesResponsePBImpl.java  | 181 +++++++++++++++++++
 .../yarn/nodelabels/NodeAttributesManager.java  |   9 +
 .../impl/pb/NodeToAttributesPBImpl.java         |   4 +-
 .../NodesToAttributesMappingRequestPBImpl.java  |   2 +-
 .../hadoop/yarn/api/TestPBImplRecords.java      |  16 +-
 .../yarn/server/MockResourceManagerFacade.java  |   8 +
 .../server/resourcemanager/ClientRMService.java |  13 ++
 .../nodelabels/NodeAttributesManagerImpl.java   |  24 +++
 .../resourcemanager/TestClientRMService.java    |  86 +++++++++
 .../DefaultClientRequestInterceptor.java        |   8 +
 .../clientrm/FederationClientInterceptor.java   |   8 +
 .../router/clientrm/RouterClientRMService.java  |   9 +
 .../PassThroughClientRequestInterceptor.java    |   8 +
 28 files changed, 749 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
index 4f96a6b..1a7f308 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
@@ -551,4 +551,10 @@ public class ResourceMgrDelegate extends YarnClient {
       Set<NodeAttribute> attributes) throws YarnException, IOException {
     return client.getAttributesToNodes(attributes);
   }
+
+  @Override
+  public Map<String, Set<NodeAttribute>> getNodeToAttributes(
+      Set<String> hostNames) throws YarnException, IOException {
+    return client.getNodeToAttributes(hostNames);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
index 23a1a85..5972f65 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
@@ -104,6 +104,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
@@ -538,6 +540,12 @@ public class TestClientRedirect {
         throws YarnException, IOException {
       return null;
     }
+
+    @Override
+    public GetNodesToAttributesResponse getNodesToAttributes(
+        GetNodesToAttributesRequest request) throws YarnException, IOException {
+      return null;
+    }
   }
 
   class HistoryService extends AMService implements HSClientProtocol {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
index 8661a78..941a688 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
@@ -43,6 +43,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
@@ -679,4 +681,20 @@ public interface ApplicationClientProtocol extends ApplicationBaseProtocol {
   GetClusterNodeAttributesResponse getClusterNodeAttributes(
       GetClusterNodeAttributesRequest request)
       throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by client to get node to attributes mappings.
+   * in existing cluster.
+   * </p>
+   *
+   * @param request request to get nodes to attributes mapping.
+   * @return nodes to attributes mappings.
+   * @throws YarnException if any error happens inside YARN.
+   * @throws IOException
+   */
+  @Public
+  @Unstable
+  GetNodesToAttributesResponse getNodesToAttributes(
+      GetNodesToAttributesRequest request) throws YarnException, IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesRequest.java
new file mode 100644
index 0000000..8e91bca
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesRequest.java
@@ -0,0 +1,65 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.Set;
+
+/**
+ * <p>
+ * The request from clients to get nodes to attributes mapping
+ * in the cluster from the <code>ResourceManager</code>.
+ * </p>
+ *
+ * @see ApplicationClientProtocol#getNodesToAttributes
+ * (GetNodesToAttributesRequest)
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class GetNodesToAttributesRequest {
+
+  public static GetNodesToAttributesRequest newInstance(Set<String> hostNames) {
+    GetNodesToAttributesRequest request =
+        Records.newRecord(GetNodesToAttributesRequest.class);
+    request.setHostNames(hostNames);
+    return request;
+  }
+
+  /**
+   * Set hostnames for which mapping is required.
+   *
+   * @param hostnames
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public abstract void setHostNames(Set<String> hostnames);
+
+  /**
+   * Get hostnames for which mapping is required.
+   *
+   * @return Set<String> of hostnames.
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public abstract Set<String> getHostNames();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesResponse.java
new file mode 100644
index 0000000..acc07bb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesResponse.java
@@ -0,0 +1,63 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * <p>
+ * The response sent by the <code>ResourceManager</code> to a client requesting
+ * nodes to attributes mapping.
+ * </p>
+ *
+ * @see ApplicationClientProtocol#getNodesToAttributes
+ * (GetNodesToAttributesRequest)
+ */
+@Public
+@Evolving
+public abstract class GetNodesToAttributesResponse {
+
+  public static GetNodesToAttributesResponse newInstance(
+      Map<String, Set<NodeAttribute>> map) {
+    GetNodesToAttributesResponse response =
+        Records.newRecord(GetNodesToAttributesResponse.class);
+    response.setNodeToAttributes(map);
+    return response;
+  }
+
+  @Public
+  @Evolving
+  public abstract void setNodeToAttributes(Map<String, Set<NodeAttribute>> map);
+
+  /**
+   * Get hostnames to NodeAttributes mapping.
+   *
+   * @return Map<String, Set<NodeAttribute>> host to attributes.
+   */
+  @Public
+  @Evolving
+  public abstract Map<String, Set<NodeAttribute>> getNodeToAttributes();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
index eeb884c..fdd4bc5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
@@ -66,4 +66,5 @@ service ApplicationClientProtocolService {
   rpc getResourceTypeInfo(GetAllResourceTypeInfoRequestProto) returns (GetAllResourceTypeInfoResponseProto);
   rpc getClusterNodeAttributes (GetClusterNodeAttributesRequestProto) returns (GetClusterNodeAttributesResponseProto);
   rpc getAttributesToNodes (GetAttributesToNodesRequestProto) returns (GetAttributesToNodesResponseProto);
+  rpc getNodesToAttributes (GetNodesToAttributesRequestProto) returns (GetNodesToAttributesResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
index 5b93aec..d37e36a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
@@ -144,11 +144,6 @@ message NodesToAttributesMappingRequestProto {
   optional bool failOnUnknownNodes = 3;
 }
 
-message NodeToAttributesProto {
-  optional string node = 1;
-  repeated NodeAttributeProto nodeAttributes = 2;
-}
-
 message NodesToAttributesMappingResponseProto {
 }
 //////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 2b796ff..5576ee6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -388,6 +388,11 @@ message AttributeToNodesProto {
   repeated string hostnames = 2;
 }
 
+message NodeToAttributesProto {
+  optional string node = 1;
+  repeated NodeAttributeProto nodeAttributes = 2;
+}
+
 enum ContainerTypeProto {
   APPLICATION_MASTER = 1;
   TASK = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index 084457b..439780b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -272,7 +272,15 @@ message GetAttributesToNodesRequestProto {
 }
 
 message GetAttributesToNodesResponseProto {
-  repeated AttributeToNodesProto attributeToNodes = 1;
+  repeated AttributeToNodesProto attributesToNodes = 1;
+}
+
+message GetNodesToAttributesRequestProto {
+  repeated string hostnames = 1;
+}
+
+message GetNodesToAttributesResponseProto {
+  repeated NodeToAttributesProto nodesToAttributes = 1;
 }
 
 message UpdateApplicationPriorityRequestProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
index ca0b7b7..0099845 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java
@@ -934,4 +934,23 @@ public abstract class YarnClient extends AbstractService {
   @Unstable
   public abstract Map<NodeAttribute, Set<String>> getAttributesToNodes(
       Set<NodeAttribute> attributes) throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by client to get all node to attribute mapping in
+   * existing cluster.
+   * </p>
+   *
+   * @param hostNames HostNames for which host to attributes mapping has to
+   *                  be retrived.If empty or null is set then will return
+   *                  all nodes to attributes mapping in cluster.
+   * @return Node to attribute mappings
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Public
+  @Unstable
+  public abstract Map<String, Set<NodeAttribute>> getNodeToAttributes(
+      Set<String> hostNames) throws YarnException, IOException;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
index 2c7496e..a08d35d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
@@ -69,6 +69,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
@@ -994,4 +995,12 @@ public class YarnClientImpl extends YarnClient {
         GetAttributesToNodesRequest.newInstance(attributes);
     return rmClient.getAttributesToNodes(request).getAttributesToNodes();
   }
+
+  @Override
+  public Map<String, Set<NodeAttribute>> getNodeToAttributes(
+      Set<String> hostNames) throws YarnException, IOException {
+    GetNodesToAttributesRequest request =
+        GetNodesToAttributesRequest.newInstance(hostNames);
+    return rmClient.getNodesToAttributes(request).getNodeToAttributes();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
index 4cf0548..1bebbe2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
@@ -65,6 +65,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
@@ -133,6 +135,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationReque
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewReservationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewReservationResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToAttributesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToAttributesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToLabelsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueInfoRequestPBImpl;
@@ -710,4 +714,18 @@ public class ApplicationClientProtocolPBClientImpl implements ApplicationClientP
       return null;
     }
   }
+
+  @Override
+  public GetNodesToAttributesResponse getNodesToAttributes(
+      GetNodesToAttributesRequest request) throws YarnException, IOException {
+    YarnServiceProtos.GetNodesToAttributesRequestProto requestProto =
+        ((GetNodesToAttributesRequestPBImpl) request).getProto();
+    try {
+      return new GetNodesToAttributesResponsePBImpl(
+          proxy.getNodesToAttributes(null, requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
index 8e53f08..2c296cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
@@ -98,6 +99,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationReque
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewReservationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewReservationResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToAttributesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToAttributesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToLabelsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueInfoRequestPBImpl;
@@ -193,6 +196,7 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileRequestP
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterNodeAttributesResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAttributesToNodesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNodesToAttributesResponseProto;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -738,4 +742,21 @@ public class ApplicationClientProtocolPBServiceImpl implements ApplicationClient
       throw new ServiceException(ie);
     }
   }
+
+  @Override
+  public GetNodesToAttributesResponseProto getNodesToAttributes(
+      RpcController controller,
+      YarnServiceProtos.GetNodesToAttributesRequestProto proto)
+      throws ServiceException {
+    GetNodesToAttributesRequestPBImpl req =
+        new GetNodesToAttributesRequestPBImpl(proto);
+    try {
+      GetNodesToAttributesResponse resp = real.getNodesToAttributes(req);
+      return ((GetNodesToAttributesResponsePBImpl) resp).getProto();
+    } catch (YarnException ye) {
+      throw new ServiceException(ye);
+    } catch (IOException ie) {
+      throw new ServiceException(ie);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
index ab6204e..175c10e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetAttributesToNodesResponsePBImpl.java
@@ -66,7 +66,7 @@ public class GetAttributesToNodesResponsePBImpl
     }
     YarnServiceProtos.GetAttributesToNodesResponseProtoOrBuilder p =
         viaProto ? proto : builder;
-    List<AttributeToNodesProto> list = p.getAttributeToNodesList();
+    List<AttributeToNodesProto> list = p.getAttributesToNodesList();
     this.attributesToNodes = new HashMap<>();
 
     for (AttributeToNodesProto c : list) {
@@ -87,7 +87,7 @@ public class GetAttributesToNodesResponsePBImpl
 
   private void addAttributesToNodesToProto() {
     maybeInitBuilder();
-    builder.clearAttributeToNodes();
+    builder.clearAttributesToNodes();
     if (attributesToNodes == null) {
       return;
     }
@@ -119,7 +119,7 @@ public class GetAttributesToNodesResponsePBImpl
             return iter.hasNext();
           }
         };
-    builder.addAllAttributeToNodes(iterable);
+    builder.addAllAttributesToNodes(iterable);
   }
 
   private NodeAttributePBImpl convertFromProtoFormat(NodeAttributeProto p) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesRequestPBImpl.java
new file mode 100644
index 0000000..0d9b722
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesRequestPBImpl.java
@@ -0,0 +1,132 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import com.google.protobuf.TextFormat;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNodesToAttributesRequestProto;
+
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Request to get hostname to attributes mapping.
+ */
+public class GetNodesToAttributesRequestPBImpl
+    extends GetNodesToAttributesRequest {
+
+  private GetNodesToAttributesRequestProto proto =
+      GetNodesToAttributesRequestProto.getDefaultInstance();
+  private GetNodesToAttributesRequestProto.Builder builder = null;
+
+  private Set<String> hostNames = null;
+  private boolean viaProto = false;
+
+  public GetNodesToAttributesRequestPBImpl() {
+    builder = GetNodesToAttributesRequestProto.newBuilder();
+  }
+
+  public GetNodesToAttributesRequestPBImpl(
+      GetNodesToAttributesRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetNodesToAttributesRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (hostNames != null && !hostNames.isEmpty()) {
+      builder.clearHostnames();
+      builder.addAllHostnames(hostNames);
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public void setHostNames(Set<String> hostnames) {
+    maybeInitBuilder();
+    if (hostNames == null) {
+      builder.clearHostnames();
+    }
+    this.hostNames = hostnames;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder =
+          YarnServiceProtos.GetNodesToAttributesRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public Set<String> getHostNames() {
+    initNodeToAttributes();
+    return this.hostNames;
+  }
+
+  private void initNodeToAttributes() {
+    if (this.hostNames != null) {
+      return;
+    }
+    YarnServiceProtos.GetNodesToAttributesRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<String> hostNamesList = p.getHostnamesList();
+    this.hostNames = new HashSet<>();
+    this.hostNames.addAll(hostNamesList);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesResponsePBImpl.java
new file mode 100644
index 0000000..1114d14
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesResponsePBImpl.java
@@ -0,0 +1,181 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Nodes to attributes request response.
+ */
+public class GetNodesToAttributesResponsePBImpl
+    extends GetNodesToAttributesResponse {
+
+  private YarnServiceProtos.GetNodesToAttributesResponseProto proto =
+      YarnServiceProtos.GetNodesToAttributesResponseProto.getDefaultInstance();
+  private YarnServiceProtos.GetNodesToAttributesResponseProto.Builder builder =
+      null;
+  private boolean viaProto = false;
+
+  private Map<String, Set<NodeAttribute>> nodesToAttributes;
+
+  public GetNodesToAttributesResponsePBImpl() {
+    this.builder =
+        YarnServiceProtos.GetNodesToAttributesResponseProto.newBuilder();
+  }
+
+  public GetNodesToAttributesResponsePBImpl(
+      YarnServiceProtos.GetNodesToAttributesResponseProto proto) {
+    this.proto = proto;
+    this.viaProto = true;
+  }
+
+  private void initNodesToAttributes() {
+    if (this.nodesToAttributes != null) {
+      return;
+    }
+    YarnServiceProtos.GetNodesToAttributesResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<YarnProtos.NodeToAttributesProto> list = p.getNodesToAttributesList();
+    this.nodesToAttributes = new HashMap<>();
+    for (YarnProtos.NodeToAttributesProto c : list) {
+      HashSet<NodeAttribute> attributes = new HashSet<>();
+      for (YarnProtos.NodeAttributeProto nodeAttrProto : c
+          .getNodeAttributesList()) {
+        attributes.add(new NodeAttributePBImpl(nodeAttrProto));
+      }
+      nodesToAttributes.put(c.getNode(), attributes);
+    }
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder =
+          YarnServiceProtos.GetNodesToAttributesResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void addNodesToAttributesToProto() {
+    maybeInitBuilder();
+    builder.clearNodesToAttributes();
+    if (nodesToAttributes == null) {
+      return;
+    }
+    Iterable<YarnProtos.NodeToAttributesProto> iterable =
+        () -> new Iterator<YarnProtos.NodeToAttributesProto>() {
+
+          private Iterator<Map.Entry<String, Set<NodeAttribute>>> iter =
+              nodesToAttributes.entrySet().iterator();
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+
+          @Override
+          public YarnProtos.NodeToAttributesProto next() {
+            Map.Entry<String, Set<NodeAttribute>> now = iter.next();
+            Set<YarnProtos.NodeAttributeProto> protoSet = new HashSet<>();
+            for (NodeAttribute nodeAttribute : now.getValue()) {
+              protoSet.add(convertToProtoFormat(nodeAttribute));
+            }
+            return YarnProtos.NodeToAttributesProto.newBuilder()
+                .setNode(now.getKey()).addAllNodeAttributes(protoSet).build();
+          }
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+        };
+    builder.addAllNodesToAttributes(iterable);
+  }
+
+  private NodeAttributePBImpl convertFromProtoFormat(
+      YarnProtos.NodeAttributeProto p) {
+    return new NodeAttributePBImpl(p);
+  }
+
+  private YarnProtos.NodeAttributeProto convertToProtoFormat(NodeAttribute t) {
+    return ((NodeAttributePBImpl) t).getProto();
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.nodesToAttributes != null) {
+      addNodesToAttributesToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  public YarnServiceProtos.GetNodesToAttributesResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    assert false : "hashCode not designed";
+    return 0;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public void setNodeToAttributes(Map<String, Set<NodeAttribute>> map) {
+    initNodesToAttributes();
+    nodesToAttributes.clear();
+    nodesToAttributes.putAll(map);
+  }
+
+  @Override
+  public Map<String, Set<NodeAttribute>> getNodeToAttributes() {
+    initNodesToAttributes();
+    return nodesToAttributes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
index 3816051..79c53e2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeAttributesManager.java
@@ -113,6 +113,15 @@ public abstract class NodeAttributesManager extends AbstractService {
   public abstract List<NodeToAttributes> getNodeToAttributes(
       Set<String> prefix);
 
+  /**
+   * Get all node to Attributes mapping.
+   *
+   * @return Map<String, Set<NodeAttribute>> nodesToAttributes matching
+   * filter.If empty or null is passed as argument will return all.
+   */
+  public abstract Map<String, Set<NodeAttribute>> getNodesToAttributes(
+      Set<String> hostNames);
+
   // futuristic
   // public set<NodeId> getNodesMatchingExpression(String nodeLabelExp);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
index 7b52d03..7204914 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
@@ -24,8 +24,8 @@ import java.util.List;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodeToAttributesProto;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodeToAttributesProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeToAttributesProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeToAttributesProtoOrBuilder;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
 
 public class NodeToAttributesPBImpl extends NodeToAttributes {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
index b319b26..6cb9a97 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AttributeMappingOperationTypeProto;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodeToAttributesProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeToAttributesProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingRequestProtoOrBuilder;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 9397dd8..9f3e925 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -75,6 +75,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetLabelsToNodesReques
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetLabelsToNodesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToAttributesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToAttributesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToLabelsRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueInfoRequestPBImpl;
@@ -241,11 +243,11 @@ import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StrictPreemptionContractProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.URLProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnClusterMetricsProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeToAttributesProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddToClusterNodeLabelsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddToClusterNodeLabelsResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.CheckForDecommissioningNodesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.CheckForDecommissioningNodesResponseProto;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodeToAttributesProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsResponseProto;
@@ -1287,4 +1289,16 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
     validatePBImplRecord(GetClusterNodeAttributesResponsePBImpl.class,
         YarnServiceProtos.GetClusterNodeAttributesResponseProto.class);
   }
+
+  @Test
+  public void testGetNodesToAttributesRequestPBImpl() throws Exception {
+    validatePBImplRecord(GetNodesToAttributesRequestPBImpl.class,
+        YarnServiceProtos.GetNodesToAttributesRequestProto.class);
+  }
+
+  @Test
+  public void testGetNodesToAttributesResponsePBImpl() throws Exception {
+    validatePBImplRecord(GetNodesToAttributesResponsePBImpl.class,
+        YarnServiceProtos.GetNodesToAttributesResponseProto.class);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
index 47b51f8c..d2cced6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
@@ -75,6 +75,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
@@ -913,6 +915,12 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   }
 
   @Override
+  public GetNodesToAttributesResponse getNodesToAttributes(
+      GetNodesToAttributesRequest request) throws YarnException, IOException {
+    return null;
+  }
+
+  @Override
   public NodesToAttributesMappingResponse mapAttributesToNodes(NodesToAttributesMappingRequest request)
     throws YarnException, IOException {
     return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index 05c11cf..3f24355 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -88,6 +88,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
@@ -1868,6 +1870,17 @@ public class ClientRMService extends AbstractService implements
     return response;
   }
 
+  @Override
+  public GetNodesToAttributesResponse getNodesToAttributes(
+      GetNodesToAttributesRequest request) throws YarnException, IOException {
+    NodeAttributesManager attributesManager =
+        rmContext.getNodeAttributesManager();
+    GetNodesToAttributesResponse response = GetNodesToAttributesResponse
+        .newInstance(
+            attributesManager.getNodesToAttributes(request.getHostNames()));
+    return response;
+  }
+
   @VisibleForTesting
   public void setDisplayPerUserApps(boolean displayPerUserApps) {
     this.filterAppsByUser = displayPerUserApps;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
index 09671f1..1645602 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
@@ -438,6 +438,30 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
     }
   }
 
+  @Override
+  public Map<String, Set<NodeAttribute>> getNodesToAttributes(
+      Set<String> hostNames) {
+    try {
+      readLock.lock();
+      boolean fetchAllNodes = (hostNames == null || hostNames.isEmpty());
+      Map<String, Set<NodeAttribute>> nodeToAttrs = new HashMap<>();
+      if (fetchAllNodes) {
+        nodeCollections.forEach((key, value) -> nodeToAttrs
+            .put(key, value.getAttributes().keySet()));
+      } else {
+        for (String hostName : hostNames) {
+          Host host = nodeCollections.get(hostName);
+          if (host != null) {
+            nodeToAttrs.put(hostName, host.getAttributes().keySet());
+          }
+        }
+      }
+      return nodeToAttrs;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
   public void activateNode(NodeId nodeId, Resource resource) {
     try {
       writeLock.lock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index 4a6b366..95ad35a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -22,6 +22,9 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetAttributesToNodesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeAttributesResponse;
+
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeAttributeType;
 import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
@@ -2126,6 +2129,89 @@ public class TestClientRMService {
   }
 
   @Test(timeout = 120000)
+  public void testGetNodesToAttributes() throws IOException, YarnException {
+    MockRM rm = new MockRM() {
+      protected ClientRMService createClientRMService() {
+        return new ClientRMService(this.rmContext, scheduler, this.rmAppManager,
+            this.applicationACLsManager, this.queueACLsManager,
+            this.getRMContext().getRMDelegationTokenSecretManager());
+      }
+    };
+    rm.start();
+
+    NodeAttributesManager mgr = rm.getRMContext().getNodeAttributesManager();
+    String node1 = "host1";
+    String node2 = "host2";
+    NodeAttribute gpu = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "GPU",
+            NodeAttributeType.STRING, "nvida");
+    NodeAttribute os = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_CENTRALIZED, "OS",
+            NodeAttributeType.STRING, "windows64");
+    NodeAttribute docker = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_DISTRIBUTED, "DOCKER",
+            NodeAttributeType.STRING, "docker0");
+    NodeAttribute dist = NodeAttribute
+        .newInstance(NodeAttribute.PREFIX_DISTRIBUTED, "VERSION",
+            NodeAttributeType.STRING, "3_0_2");
+    Map<String, Set<NodeAttribute>> nodes = new HashMap<>();
+    nodes.put(node1, ImmutableSet.of(gpu, os, dist));
+    nodes.put(node2, ImmutableSet.of(docker, dist));
+    mgr.addNodeAttributes(nodes);
+    // Create a client.
+    Configuration conf = new Configuration();
+    YarnRPC rpc = YarnRPC.create(conf);
+    InetSocketAddress rmAddress = rm.getClientRMService().getBindAddress();
+    LOG.info("Connecting to ResourceManager at " + rmAddress);
+    ApplicationClientProtocol client = (ApplicationClientProtocol) rpc
+        .getProxy(ApplicationClientProtocol.class, rmAddress, conf);
+
+    // Specify null for hostnames.
+    GetNodesToAttributesRequest request1 =
+        GetNodesToAttributesRequest.newInstance(null);
+    GetNodesToAttributesResponse response1 =
+        client.getNodesToAttributes(request1);
+    Map<String, Set<NodeAttribute>> hostToAttrs =
+        response1.getNodeToAttributes();
+    Assert.assertEquals(2, hostToAttrs.size());
+
+    Assert.assertTrue(hostToAttrs.get(node2).contains(dist));
+    Assert.assertTrue(hostToAttrs.get(node2).contains(docker));
+    Assert.assertTrue(hostToAttrs.get(node1).contains(dist));
+
+    // Specify particular node
+    GetNodesToAttributesRequest request2 =
+        GetNodesToAttributesRequest.newInstance(ImmutableSet.of(node1));
+    GetNodesToAttributesResponse response2 =
+        client.getNodesToAttributes(request2);
+    hostToAttrs = response2.getNodeToAttributes();
+    Assert.assertEquals(1, response2.getNodeToAttributes().size());
+    Assert.assertTrue(hostToAttrs.get(node1).contains(dist));
+
+    // Test queury with empty set
+    GetNodesToAttributesRequest request3 =
+        GetNodesToAttributesRequest.newInstance(Collections.emptySet());
+    GetNodesToAttributesResponse response3 =
+        client.getNodesToAttributes(request3);
+    hostToAttrs = response3.getNodeToAttributes();
+    Assert.assertEquals(2, hostToAttrs.size());
+
+    Assert.assertTrue(hostToAttrs.get(node2).contains(dist));
+    Assert.assertTrue(hostToAttrs.get(node2).contains(docker));
+    Assert.assertTrue(hostToAttrs.get(node1).contains(dist));
+
+    // test invalid hostname
+    GetNodesToAttributesRequest request4 =
+        GetNodesToAttributesRequest.newInstance(ImmutableSet.of("invalid"));
+    GetNodesToAttributesResponse response4 =
+        client.getNodesToAttributes(request4);
+    hostToAttrs = response4.getNodeToAttributes();
+    Assert.assertEquals(0, hostToAttrs.size());
+    rpc.stopProxy(client, conf);
+    rm.close();
+  }
+
+  @Test(timeout = 120000)
   public void testUpdatePriorityAndKillAppWithZeroClusterResource()
       throws Exception {
     int maxPriority = 10;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java
index f6adb43..4cd4a01 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java
@@ -61,6 +61,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
@@ -344,6 +346,12 @@ public class DefaultClientRequestInterceptor
     return clientRMProxy.getClusterNodeAttributes(request);
   }
 
+  @Override
+  public GetNodesToAttributesResponse getNodesToAttributes(
+      GetNodesToAttributesRequest request) throws YarnException, IOException {
+    return clientRMProxy.getNodesToAttributes(request);
+  }
+
   @VisibleForTesting
   public void setRMClient(ApplicationClientProtocol clientRM) {
     this.clientRMProxy = clientRM;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
index bf006a4..ceabe65 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
@@ -77,6 +77,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
@@ -843,4 +845,10 @@ public class FederationClientInterceptor
       throws YarnException, IOException {
     throw new NotImplementedException("Code is not implemented");
   }
+
+  @Override
+  public GetNodesToAttributesResponse getNodesToAttributes(
+      GetNodesToAttributesRequest request) throws YarnException, IOException {
+    throw new NotImplementedException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java
index 3237dd4..db1f482 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java
@@ -72,6 +72,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
@@ -449,6 +451,13 @@ public class RouterClientRMService extends AbstractService
     return pipeline.getRootInterceptor().getClusterNodeAttributes(request);
   }
 
+  @Override
+  public GetNodesToAttributesResponse getNodesToAttributes(
+      GetNodesToAttributesRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getNodesToAttributes(request);
+  }
+
   @VisibleForTesting
   protected RequestInterceptorChainWrapper getInterceptorChain()
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5dc7d6e0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java
index 96da4c4..a35feae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java
@@ -58,6 +58,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
@@ -305,4 +307,10 @@ public class PassThroughClientRequestInterceptor
       throws YarnException, IOException {
     return getNextInterceptor().getClusterNodeAttributes(request);
   }
+
+  @Override
+  public GetNodesToAttributesResponse getNodesToAttributes(
+      GetNodesToAttributesRequest request) throws YarnException, IOException {
+    return getNextInterceptor().getNodesToAttributes(request);
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[18/29] hadoop git commit: YARN-7856. Validate Node Attributes from NM. Contributed by Weiwei Yang.

Posted by su...@apache.org.
YARN-7856. Validate Node Attributes from NM. Contributed by Weiwei Yang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ffcabd24
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ffcabd24
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ffcabd24

Branch: refs/heads/trunk
Commit: ffcabd24c33fc8c663cc75b57d7562db4c199713
Parents: 2f7712b
Author: Sunil G <su...@apache.org>
Authored: Tue Feb 27 08:15:42 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:00 2018 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/api/records/NodeAttribute.java  |  2 ++
 .../hadoop/yarn/nodelabels/NodeLabelUtil.java   | 31 ++++++++++++++++++++
 .../ScriptBasedNodeAttributesProvider.java      | 25 ++++++++++++++--
 .../TestScriptBasedNodeAttributesProvider.java  | 27 +++++++++++++++++
 4 files changed, 83 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffcabd24/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
index 01c70b2..4f6846b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
@@ -46,6 +46,8 @@ import org.apache.hadoop.yarn.util.Records;
 public abstract class NodeAttribute {
 
   public static final String DEFAULT_PREFIX = "";
+  public static final String PREFIX_DISTRIBUTED = "nm.yarn.io";
+  public static final String PREFIX_CENTRALIZED = "rm.yarn.io";
 
   public static NodeAttribute newInstance(String attributeName,
       NodeAttributeType attributeType, String attributeValue) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffcabd24/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
index d918712..fdfd0ce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelUtil.java
@@ -17,7 +17,11 @@
  */
 package org.apache.hadoop.yarn.nodelabels;
 
+import com.google.common.base.Strings;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+
 import java.io.IOException;
+import java.util.Set;
 import java.util.regex.Pattern;
 
 /**
@@ -94,4 +98,31 @@ public final class NodeLabelUtil {
           + ", now it is= " + prefix);
     }
   }
+
+  /**
+   * Validate if a given set of attributes are valid. Attributes could be
+   * invalid if any of following conditions is met:
+   *
+   * <ul>
+   *   <li>Missing prefix: the attribute doesn't have prefix defined</li>
+   *   <li>Malformed attribute prefix: the prefix is not in valid format</li>
+   * </ul>
+   * @param attributeSet
+   * @throws IOException
+   */
+  public static void validateNodeAttributes(Set<NodeAttribute> attributeSet)
+      throws IOException {
+    if (attributeSet != null && !attributeSet.isEmpty()) {
+      for (NodeAttribute nodeAttribute : attributeSet) {
+        String prefix = nodeAttribute.getAttributePrefix();
+        if (Strings.isNullOrEmpty(prefix)) {
+          throw new IOException("Attribute prefix must be set");
+        }
+        // Verify attribute prefix format.
+        checkAndThrowAttributePrefix(prefix);
+        // Verify attribute name format.
+        checkAndThrowLabelName(nodeAttribute.getAttributeName());
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffcabd24/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java
index 06771ba..4621434 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/ScriptBasedNodeAttributesProvider.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.nodemanager.nodelabels;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.nodelabels.NodeLabelUtil;
 
 import java.io.IOException;
 import java.util.HashSet;
@@ -116,13 +117,33 @@ public class ScriptBasedNodeAttributesProvider extends NodeAttributesProvider{
                 + NODE_ATTRIBUTE_DELIMITER + "ATTRIBUTE_VALUE; but get "
                 + nodeAttribute);
           }
+          // Automatically setup prefix for collected attributes
           NodeAttribute na = NodeAttribute
-              .newInstance(attributeStrs[0],
+              .newInstance(NodeAttribute.PREFIX_DISTRIBUTED,
+                  attributeStrs[0],
                   NodeAttributeType.valueOf(attributeStrs[1]),
                   attributeStrs[2]);
-          attributeSet.add(na);
+
+          // Since a NodeAttribute is identical with another one as long as
+          // their prefix and name are same, to avoid attributes getting
+          // overwritten by ambiguous attribute, make sure it fails in such
+          // case.
+          if (!attributeSet.add(na)) {
+            throw new IOException("Ambiguous node attribute is found: "
+                + na.toString() + ", a same attribute already exists");
+          }
         }
       }
+
+      // Before updating the attributes to the provider,
+      // verify if they are valid
+      try {
+        NodeLabelUtil.validateNodeAttributes(attributeSet);
+      } catch (IOException e) {
+        throw new IOException("Node attributes collected by the script "
+            + "contains some invalidate entries. Detail message: "
+            + e.getMessage());
+      }
       return attributeSet;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffcabd24/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java
index 58d2d20..f764626 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/nodelabels/TestScriptBasedNodeAttributesProvider.java
@@ -220,4 +220,31 @@ public class TestScriptBasedNodeAttributesProvider {
       }
     }, 500, 3000);
   }
+
+  @Test
+  public void testNodeAttributesValidation() throws Exception{
+    // Script output contains ambiguous node attributes
+    String scriptContent = "echo NODE_ATTRIBUTE:host,STRING,host1234\n "
+        + "echo NODE_ATTRIBUTE:host,STRING,host2345\n "
+        + "echo NODE_ATTRIBUTE:ip,STRING,10.0.0.1";
+
+    writeNodeAttributeScriptFile(scriptContent, true);
+
+    nodeAttributesProvider.init(getConfForNodeAttributeScript());
+    nodeAttributesProvider.start();
+
+    // There should be no attributes found, and we should
+    // see Malformed output warnings in the log
+    try {
+      GenericTestUtils
+          .waitFor(() -> nodeAttributesProvider
+                  .getDescriptors().size() == 3,
+              500, 3000);
+      Assert.fail("This test should timeout because the provide is unable"
+          + " to parse any attributes from the script output.");
+    } catch (TimeoutException e) {
+      Assert.assertEquals(0, nodeAttributesProvider
+          .getDescriptors().size());
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[25/29] hadoop git commit: YARN-8740. Clear node attribute path after each test run. Contributed by Bibin A Chundatt.

Posted by su...@apache.org.
YARN-8740. Clear node attribute path after each test run. Contributed by Bibin A Chundatt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5e64e62d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5e64e62d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5e64e62d

Branch: refs/heads/trunk
Commit: 5e64e62deeb467ac2969bc357a24810db855cffa
Parents: c44088a
Author: Sunil G <su...@apache.org>
Authored: Mon Sep 3 12:11:19 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:01:01 2018 +0530

----------------------------------------------------------------------
 .../nodelabels/NodeAttributesManagerImpl.java   | 13 +++--
 .../resourcemanager/NodeAttributeTestUtils.java | 54 ++++++++++++++++++++
 .../resourcemanager/TestClientRMService.java    |  9 ++--
 .../resourcemanager/TestRMAdminService.java     |  3 +-
 .../TestFileSystemNodeAttributeStore.java       | 28 +++++++---
 .../nodelabels/TestNodeAttributesManager.java   |  9 +---
 6 files changed, 93 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e64e62d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
index 81f42d9..5533436 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
 
+import com.google.common.annotations.VisibleForTesting;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -151,7 +152,8 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
     }
   }
 
-  private void internalUpdateAttributesOnNodes(
+  @VisibleForTesting
+  protected void internalUpdateAttributesOnNodes(
       Map<String, Map<NodeAttribute, AttributeValue>> nodeAttributeMapping,
       AttributeMappingOperationType op,
       Map<NodeAttributeKey, RMNodeAttribute> newAttributesToBeAdded,
@@ -195,7 +197,7 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
         logMsg.append(" NM = ");
         logMsg.append(entry.getKey());
         logMsg.append(", attributes=[ ");
-        logMsg.append(StringUtils.join(entry.getValue().entrySet(), ","));
+        logMsg.append(StringUtils.join(entry.getValue().keySet(), ","));
         logMsg.append("] ,");
       }
 
@@ -700,9 +702,10 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
         new HashMap<>();
     Map<String, Map<NodeAttribute, AttributeValue>> validMapping =
         validate(nodeAttributeMapping, newAttributesToBeAdded, false);
-
-    internalUpdateAttributesOnNodes(validMapping, mappingType,
-        newAttributesToBeAdded, attributePrefix);
+    if (validMapping.size() > 0) {
+      internalUpdateAttributesOnNodes(validMapping, mappingType,
+          newAttributesToBeAdded, attributePrefix);
+    }
   }
 
   protected void stopDispatcher() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e64e62d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeAttributeTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeAttributeTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeAttributeTestUtils.java
new file mode 100644
index 0000000..1d12cb7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeAttributeTestUtils.java
@@ -0,0 +1,54 @@
+/**
+ * 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;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+/**
+ * Test Utils for NodeAttribute.
+ */
+public final class NodeAttributeTestUtils {
+
+  private NodeAttributeTestUtils() {
+
+  }
+
+  public static YarnConfiguration getRandomDirConf(Configuration conf)
+      throws IOException {
+    YarnConfiguration newConf;
+    if (conf == null) {
+      newConf = new YarnConfiguration();
+    } else {
+      newConf = new YarnConfiguration(conf);
+    }
+    File tempDir = GenericTestUtils.getRandomizedTestDir();
+    FileUtils.deleteDirectory(tempDir);
+    tempDir.mkdirs();
+    tempDir.deleteOnExit();
+    newConf.set(YarnConfiguration.FS_NODE_ATTRIBUTE_STORE_ROOT_DIR,
+        tempDir.getAbsolutePath());
+    return newConf;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e64e62d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index 39892f2..efabbe2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -2012,7 +2012,8 @@ public class TestClientRMService {
 
   @Test(timeout = 120000)
   public void testGetClusterNodeAttributes() throws IOException, YarnException {
-    MockRM rm = new MockRM() {
+    Configuration newConf = NodeAttributeTestUtils.getRandomDirConf(null);
+    MockRM rm = new MockRM(newConf) {
       protected ClientRMService createClientRMService() {
         return new ClientRMService(this.rmContext, scheduler, this.rmAppManager,
             this.applicationACLsManager, this.queueACLsManager,
@@ -2061,7 +2062,8 @@ public class TestClientRMService {
 
   @Test(timeout = 120000)
   public void testGetAttributesToNodes() throws IOException, YarnException {
-    MockRM rm = new MockRM() {
+    Configuration newConf = NodeAttributeTestUtils.getRandomDirConf(null);
+    MockRM rm = new MockRM(newConf) {
       protected ClientRMService createClientRMService() {
         return new ClientRMService(this.rmContext, scheduler, this.rmAppManager,
             this.applicationACLsManager, this.queueACLsManager,
@@ -2152,7 +2154,8 @@ public class TestClientRMService {
 
   @Test(timeout = 120000)
   public void testGetNodesToAttributes() throws IOException, YarnException {
-    MockRM rm = new MockRM() {
+    Configuration newConf = NodeAttributeTestUtils.getRandomDirConf(null);
+    MockRM rm = new MockRM(newConf) {
       protected ClientRMService createClientRMService() {
         return new ClientRMService(this.rmContext, scheduler, this.rmAppManager,
             this.applicationACLsManager, this.queueACLsManager,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e64e62d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
index a1b95b8..475dd8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
@@ -1557,7 +1557,8 @@ public class TestRMAdminService {
     // 1.5. Test with unknown node when failOnUnknownNodes is false
 
     // also test : 3. Ensure Appropriate manager Method call is done
-    rm = new MockRM();
+    Configuration newConf = NodeAttributeTestUtils.getRandomDirConf(null);
+    rm = new MockRM(newConf);
 
     NodeAttributesManager spiedAttributesManager =
         Mockito.spy(rm.getRMContext().getNodeAttributesManager());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e64e62d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
index 07ea20d..4a7a59f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestFileSystemNodeAttributeStore.java
@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
 
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
 import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeAttribute;
@@ -25,12 +30,12 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.InlineDispatcher;
 import org.apache.hadoop.yarn.nodelabels.AttributeValue;
 import org.apache.hadoop.yarn.nodelabels.NodeAttributeStore;
+import org.apache.hadoop.yarn.server.resourcemanager.NodeAttributeTestUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
@@ -68,12 +73,7 @@ public class TestFileSystemNodeAttributeStore {
     conf = new Configuration();
     conf.setClass(YarnConfiguration.FS_NODE_ATTRIBUTE_STORE_IMPL_CLASS,
         FileSystemNodeAttributeStore.class, NodeAttributeStore.class);
-    File tempDir = File.createTempFile("nattr", ".tmp");
-    tempDir.delete();
-    tempDir.mkdirs();
-    tempDir.deleteOnExit();
-    conf.set(YarnConfiguration.FS_NODE_ATTRIBUTE_STORE_ROOT_DIR,
-        tempDir.getAbsolutePath());
+    conf = NodeAttributeTestUtils.getRandomDirConf(conf);
     mgr.init(conf);
     mgr.start();
   }
@@ -87,6 +87,20 @@ public class TestFileSystemNodeAttributeStore {
   }
 
   @Test(timeout = 10000)
+  public void testEmptyRecoverSkipInternalUdpate() throws Exception {
+    // Stop manager
+    mgr.stop();
+
+    // Start new attribute manager with same path
+    mgr = spy(new MockNodeAttrbuteManager());
+    mgr.init(conf);
+    mgr.start();
+
+    verify(mgr, times(0))
+        .internalUpdateAttributesOnNodes(any(), any(), any(), any());
+  }
+
+  @Test(timeout = 10000)
   public void testRecoverWithMirror() throws Exception {
 
     //------host0----

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e64e62d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
index 9bc9388..b181677 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestNodeAttributesManager.java
@@ -28,12 +28,12 @@ import org.apache.hadoop.yarn.nodelabels.AttributeValue;
 import org.apache.hadoop.yarn.nodelabels.NodeAttributeStore;
 import org.apache.hadoop.yarn.nodelabels.NodeAttributesManager;
 import org.apache.hadoop.yarn.nodelabels.NodeLabelUtil;
+import org.apache.hadoop.yarn.server.resourcemanager.NodeAttributeTestUtils;
 import org.junit.Test;
 import org.junit.Before;
 import org.junit.After;
 import org.junit.Assert;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -57,12 +57,7 @@ public class TestNodeAttributesManager {
     attributesManager = new NodeAttributesManagerImpl();
     conf.setClass(YarnConfiguration.FS_NODE_ATTRIBUTE_STORE_IMPL_CLASS,
         FileSystemNodeAttributeStore.class, NodeAttributeStore.class);
-    File tempDir = File.createTempFile("nattr", ".tmp");
-    tempDir.delete();
-    tempDir.mkdirs();
-    tempDir.deleteOnExit();
-    conf.set(YarnConfiguration.FS_NODE_ATTRIBUTE_STORE_ROOT_DIR,
-        tempDir.getAbsolutePath());
+    conf = NodeAttributeTestUtils.getRandomDirConf(conf);
     attributesManager.init(conf);
     attributesManager.start();
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/29] hadoop git commit: YARN-6856. [YARN-3409] Support CLI for Node Attributes Mapping. Contributed by Naganarasimha G R.

Posted by su...@apache.org.
YARN-6856. [YARN-3409] Support CLI for Node Attributes Mapping. Contributed by Naganarasimha G R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2475fb0a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2475fb0a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2475fb0a

Branch: refs/heads/trunk
Commit: 2475fb0a1e2158aac11829c6e33eae3e02bf33ea
Parents: 1f42ce9
Author: Naganarasimha <na...@apache.org>
Authored: Tue Jan 23 07:18:20 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 12 16:00:59 2018 +0530

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ha/HAAdmin.java |   2 +-
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |   5 +
 .../yarn/client/cli/NodeAttributesCLI.java      | 410 +++++++++++++++++++
 .../yarn/client/cli/TestNodeAttributesCLI.java  | 328 +++++++++++++++
 4 files changed, 744 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2475fb0a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
index 9b7d7ba..8c92bd0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
@@ -575,7 +575,7 @@ public abstract class HAAdmin extends Configured implements Tool {
     return 0;
   }
   
-  protected static class UsageInfo {
+  public static class UsageInfo {
     public final String args;
     public final String help;
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2475fb0a/hadoop-yarn-project/hadoop-yarn/bin/yarn
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn
index 69afe6f..7cd838f 100755
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn
@@ -55,6 +55,7 @@ function hadoop_usage
   hadoop_add_subcommand "timelinereader" client "run the timeline reader server"
   hadoop_add_subcommand "timelineserver" daemon "run the timeline server"
   hadoop_add_subcommand "top" client "view cluster information"
+  hadoop_add_subcommand "node-attributes" "map node to attibutes"
   hadoop_add_subcommand "version" client "print the version"
   hadoop_generate_usage "${HADOOP_SHELL_EXECNAME}" true
 }
@@ -186,6 +187,10 @@ ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}"
       hadoop_add_classpath "$HADOOP_YARN_HOME/$YARN_DIR/timelineservice/lib/*"
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderServer'
     ;;
+	node-attributes)
+      HADOOP_SUBCMD_SUPPORTDAEMONIZATION="false"
+      HADOOP_CLASSNAME='org.apache.hadoop.yarn.client.cli.NodeAttributesCLI'
+	;;
     timelineserver)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer'

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2475fb0a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java
new file mode 100644
index 0000000..2eff155
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeAttributesCLI.java
@@ -0,0 +1,410 @@
+/**
+ * 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.client.cli;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.MissingArgumentException;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.ha.HAAdmin.UsageInfo;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * CLI to map attributes to Nodes.
+ *
+ */
+public class NodeAttributesCLI extends Configured implements Tool {
+
+  protected static final String INVALID_MAPPING_ERR_MSG =
+      "Invalid Node to attribute mapping : ";
+
+  protected static final String USAGE_YARN_NODE_ATTRIBUTES =
+      "Usage: yarn node-attributes ";
+
+  protected static final String NO_MAPPING_ERR_MSG =
+      "No node-to-attributes mappings are specified";
+
+  protected final static Map<String, UsageInfo> NODE_ATTRIB_USAGE =
+      ImmutableMap.<String, UsageInfo>builder()
+          .put("-replace",
+              new UsageInfo(
+                  "<\"node1:attribute[(type)][=value],attribute1[=value],"
+                      + "attribute2  node2:attribute2[=value],attribute3\">",
+                  " Replace the node to attributes mapping information at the"
+                      + " ResourceManager with the new mapping. Currently"
+                      + " supported attribute type. And string is the default"
+                      + " type too. Attribute value if not specified for string"
+                      + " type value will be considered as empty string."
+                      + " Replaced node-attributes should not violate the"
+                      + " existing attribute to attribute type mapping."))
+          .put("-add",
+              new UsageInfo(
+                  "<\"node1:attribute[(type)][=value],attribute1[=value],"
+                      + "attribute2  node2:attribute2[=value],attribute3\">",
+                  " Adds or updates the node to attributes mapping information"
+                      + " at the ResourceManager. Currently supported attribute"
+                      + " type is string. And string is the default type too."
+                      + " Attribute value if not specified for string type"
+                      + " value will be considered as empty string. Added or"
+                      + " updated node-attributes should not violate the"
+                      + " existing attribute to attribute type mapping."))
+          .put("-remove",
+              new UsageInfo("<\"node1:attribute,attribute1 node2:attribute2\">",
+                  " Removes the specified node to attributes mapping"
+                      + " information at the ResourceManager"))
+          .put("-failOnUnknownNodes",
+              new UsageInfo("",
+                  "Can be used optionally along with other options. When its"
+                      + " set, it will fail if specified nodes are unknown."))
+          .build();
+
+  /** Output stream for errors, for use in tests. */
+  private PrintStream errOut = System.err;
+
+  public NodeAttributesCLI() {
+    super();
+  }
+
+  public NodeAttributesCLI(Configuration conf) {
+    super(conf);
+  }
+
+  protected void setErrOut(PrintStream errOut) {
+    this.errOut = errOut;
+  }
+
+  private void printHelpMsg(String cmd) {
+    StringBuilder builder = new StringBuilder();
+    UsageInfo usageInfo = null;
+    if (cmd != null && !(cmd.trim().isEmpty())) {
+      usageInfo = NODE_ATTRIB_USAGE.get(cmd);
+    }
+    if (usageInfo != null) {
+      if (usageInfo.args == null) {
+        builder.append("   " + cmd + ":\n" + usageInfo.help);
+      } else {
+        String space = (usageInfo.args == "") ? "" : " ";
+        builder.append(
+            "   " + cmd + space + usageInfo.args + " :\n" + usageInfo.help);
+      }
+    } else {
+      // help for all commands
+      builder.append("Usage: yarn node-attributes\n");
+      for (Map.Entry<String, UsageInfo> cmdEntry : NODE_ATTRIB_USAGE
+          .entrySet()) {
+        usageInfo = cmdEntry.getValue();
+        builder.append("   " + cmdEntry.getKey() + " " + usageInfo.args
+            + " :\n " + usageInfo.help + "\n");
+      }
+      builder.append("   -help" + " [cmd]\n");
+    }
+    errOut.println(builder);
+  }
+
+  private static void buildIndividualUsageMsg(String cmd,
+      StringBuilder builder) {
+    UsageInfo usageInfo = NODE_ATTRIB_USAGE.get(cmd);
+    if (usageInfo == null) {
+      return;
+    }
+    if (usageInfo.args == null) {
+      builder.append(USAGE_YARN_NODE_ATTRIBUTES + cmd + "\n");
+    } else {
+      String space = (usageInfo.args == "") ? "" : " ";
+      builder.append(
+          USAGE_YARN_NODE_ATTRIBUTES + cmd + space + usageInfo.args + "\n");
+    }
+  }
+
+  private static void buildUsageMsgForAllCmds(StringBuilder builder) {
+    builder.append("Usage: yarn node-attributes\n");
+    for (Map.Entry<String, UsageInfo> cmdEntry : NODE_ATTRIB_USAGE.entrySet()) {
+      UsageInfo usageInfo = cmdEntry.getValue();
+      builder.append("   " + cmdEntry.getKey() + " " + usageInfo.args + "\n");
+    }
+    builder.append("   -help" + " [cmd]\n");
+  }
+
+  /**
+   * Displays format of commands.
+   *
+   * @param cmd The command that is being executed.
+   */
+  private void printUsage(String cmd) {
+    StringBuilder usageBuilder = new StringBuilder();
+    if (NODE_ATTRIB_USAGE.containsKey(cmd)) {
+      buildIndividualUsageMsg(cmd, usageBuilder);
+    } else {
+      buildUsageMsgForAllCmds(usageBuilder);
+    }
+    errOut.println(usageBuilder);
+  }
+
+  private void printUsage() {
+    printUsage("");
+  }
+
+  protected ResourceManagerAdministrationProtocol createAdminProtocol()
+      throws IOException {
+    // Get the current configuration
+    final YarnConfiguration conf = new YarnConfiguration(getConf());
+    return ClientRMProxy.createRMProxy(conf,
+        ResourceManagerAdministrationProtocol.class);
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    if (conf != null) {
+      conf = addSecurityConfiguration(conf);
+    }
+    super.setConf(conf);
+  }
+
+  /**
+   * Add the requisite security principal settings to the given Configuration,
+   * returning a copy.
+   *
+   * @param conf the original config
+   * @return a copy with the security settings added
+   */
+  private static Configuration addSecurityConfiguration(Configuration conf) {
+    // Make a copy so we don't mutate it. Also use an YarnConfiguration to
+    // force loading of yarn-site.xml.
+    conf = new YarnConfiguration(conf);
+    conf.set(CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY,
+        conf.get(YarnConfiguration.RM_PRINCIPAL, ""));
+    return conf;
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    if (args.length < 1) {
+      printUsage();
+      return -1;
+    }
+
+    int exitCode = -1;
+    int i = 0;
+    String cmd = args[i++];
+
+    if ("-help".equals(cmd)) {
+      exitCode = 0;
+      if (args.length >= 2) {
+        printHelpMsg(args[i]);
+      } else {
+        printHelpMsg("");
+      }
+      return exitCode;
+    }
+
+    try {
+      if ("-replace".equals(cmd)) {
+        exitCode = handleNodeAttributeMapping(args,
+            AttributeMappingOperationType.REPLACE);
+      } else if ("-add".equals(cmd)) {
+        exitCode =
+            handleNodeAttributeMapping(args, AttributeMappingOperationType.ADD);
+      } else if ("-remove".equals(cmd)) {
+        exitCode = handleNodeAttributeMapping(args,
+            AttributeMappingOperationType.REMOVE);
+      } else {
+        exitCode = -1;
+        errOut.println(cmd.substring(1) + ": Unknown command");
+        printUsage();
+      }
+    } catch (IllegalArgumentException arge) {
+      exitCode = -1;
+      errOut.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
+      printUsage(cmd);
+    } catch (RemoteException e) {
+      //
+      // This is a error returned by hadoop server. Print
+      // out the first line of the error message, ignore the stack trace.
+      exitCode = -1;
+      try {
+        String[] content;
+        content = e.getLocalizedMessage().split("\n");
+        errOut.println(cmd.substring(1) + ": " + content[0]);
+      } catch (Exception ex) {
+        errOut.println(cmd.substring(1) + ": " + ex.getLocalizedMessage());
+      }
+    } catch (Exception e) {
+      exitCode = -1;
+      errOut.println(cmd.substring(1) + ": " + e.getLocalizedMessage());
+    }
+    return exitCode;
+  }
+
+  private int handleNodeAttributeMapping(String args[],
+      AttributeMappingOperationType operation)
+      throws IOException, YarnException, ParseException {
+    Options opts = new Options();
+    opts.addOption(operation.name().toLowerCase(), true,
+        operation.name().toLowerCase());
+    opts.addOption("failOnUnknownNodes", false, "Fail on unknown nodes.");
+    int exitCode = -1;
+    CommandLine cliParser = null;
+    try {
+      cliParser = new GnuParser().parse(opts, args);
+    } catch (MissingArgumentException ex) {
+      errOut.println(NO_MAPPING_ERR_MSG);
+      printUsage(args[0]);
+      return exitCode;
+    }
+    List<NodeToAttributes> buildNodeLabelsMapFromStr =
+        buildNodeLabelsMapFromStr(
+            cliParser.getOptionValue(operation.name().toLowerCase()),
+            operation != AttributeMappingOperationType.REPLACE, operation);
+    NodesToAttributesMappingRequest request = NodesToAttributesMappingRequest
+        .newInstance(operation, buildNodeLabelsMapFromStr,
+            cliParser.hasOption("failOnUnknownNodes"));
+    ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol();
+    adminProtocol.mapAttributesToNodes(request);
+    return 0;
+  }
+
+  /**
+   * args are expected to be of the format
+   * node1:java(string)=8,ssd(boolean)=false node2:ssd(boolean)=true
+   */
+  private List<NodeToAttributes> buildNodeLabelsMapFromStr(String args,
+      boolean validateForAttributes, AttributeMappingOperationType operation) {
+    List<NodeToAttributes> nodeToAttributesList = new ArrayList<>();
+    for (String nodeToAttributesStr : args.split("[ \n]")) {
+      // for each node to attribute mapping
+      nodeToAttributesStr = nodeToAttributesStr.trim();
+      if (nodeToAttributesStr.isEmpty()
+          || nodeToAttributesStr.startsWith("#")) {
+        continue;
+      }
+      if (nodeToAttributesStr.indexOf(":") == -1) {
+        throw new IllegalArgumentException(
+            INVALID_MAPPING_ERR_MSG + nodeToAttributesStr);
+      }
+      String[] nodeToAttributes = nodeToAttributesStr.split(":");
+      Preconditions.checkArgument(!nodeToAttributes[0].trim().isEmpty(),
+          "Node name cannot be empty");
+      String node = nodeToAttributes[0];
+      String[] attributeNameValueType = null;
+      List<NodeAttribute> attributesList = new ArrayList<>();
+      NodeAttributeType attributeType = NodeAttributeType.STRING;
+      String attributeValue;
+      String attributeName;
+      Set<String> attributeNamesMapped = new HashSet<>();
+
+      String attributesStr[];
+      if (nodeToAttributes.length == 2) {
+        // fetching multiple attributes for a node
+        attributesStr = nodeToAttributes[1].split(",");
+        for (String attributeStr : attributesStr) {
+          // get information about each attribute.
+          attributeNameValueType = attributeStr.split("="); // to find name
+                                                            // value
+          Preconditions.checkArgument(
+              !(attributeNameValueType[0] == null
+                  || attributeNameValueType[0].isEmpty()),
+              "Attribute name cannot be null or empty");
+          attributeValue = attributeNameValueType.length > 1
+              ? attributeNameValueType[1] : "";
+          int indexOfOpenBracket = attributeNameValueType[0].indexOf("(");
+          if (indexOfOpenBracket == -1) {
+            attributeName = attributeNameValueType[0];
+          } else if (indexOfOpenBracket == 0) {
+            throw new IllegalArgumentException("Attribute for node " + node
+                + " is not properly configured : " + attributeStr);
+          } else {
+            // attribute type has been explicitly configured
+            int indexOfCloseBracket = attributeNameValueType[0].indexOf(")");
+            if (indexOfCloseBracket == -1
+                || indexOfCloseBracket < indexOfOpenBracket) {
+              throw new IllegalArgumentException("Attribute for node " + node
+                  + " is not properly Configured : " + attributeStr);
+            }
+            String attributeTypeStr;
+            attributeName =
+                attributeNameValueType[0].substring(0, indexOfOpenBracket);
+            attributeTypeStr = attributeNameValueType[0]
+                .substring(indexOfOpenBracket + 1, indexOfCloseBracket);
+            try {
+              attributeType = NodeAttributeType
+                  .valueOf(attributeTypeStr.trim().toUpperCase());
+            } catch (IllegalArgumentException e) {
+              throw new IllegalArgumentException(
+                  "Invalid Attribute type configuration : " + attributeTypeStr
+                      + " in " + attributeStr);
+            }
+          }
+          if (attributeNamesMapped.contains(attributeName)) {
+            throw new IllegalArgumentException("Attribute " + attributeName
+                + " has been mapped more than once in  : "
+                + nodeToAttributesStr);
+          }
+          // TODO when we support different type of attribute type we need to
+          // cross verify whether input attributes itself is not violating
+          // attribute Name to Type mapping.
+          attributesList.add(NodeAttribute.newInstance(attributeName.trim(),
+              attributeType, attributeValue.trim()));
+        }
+      }
+      if (validateForAttributes) {
+        Preconditions.checkArgument((attributesList.size() > 0),
+            "Attributes cannot be null or empty for Operation "
+                + operation.name() + " on the node " + node);
+      }
+      nodeToAttributesList
+          .add(NodeToAttributes.newInstance(node, attributesList));
+    }
+
+    if (nodeToAttributesList.isEmpty()) {
+      throw new IllegalArgumentException(NO_MAPPING_ERR_MSG);
+    }
+    return nodeToAttributesList;
+  }
+
+  public static void main(String[] args) throws Exception {
+    int result = ToolRunner.run(new NodeAttributesCLI(), args);
+    System.exit(result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2475fb0a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java
new file mode 100644
index 0000000..cc92a93
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestNodeAttributesCLI.java
@@ -0,0 +1,328 @@
+/**
+ * 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.client.cli;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Joiner;
+
+/**
+ * Test class for TestNodeAttributesCLI.
+ */
+public class TestNodeAttributesCLI {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestNodeAttributesCLI.class);
+  private ResourceManagerAdministrationProtocol admin;
+  private NodesToAttributesMappingRequest request;
+  private NodeAttributesCLI nodeAttributesCLI;
+  private ByteArrayOutputStream errOutBytes = new ByteArrayOutputStream();
+  private String errOutput;
+
+  @Before
+  public void configure() throws IOException, YarnException {
+    admin = mock(ResourceManagerAdministrationProtocol.class);
+
+    when(admin.mapAttributesToNodes(any(NodesToAttributesMappingRequest.class)))
+        .thenAnswer(new Answer<NodesToAttributesMappingResponse>() {
+          @Override
+          public NodesToAttributesMappingResponse answer(
+              InvocationOnMock invocation) throws Throwable {
+            request =
+                (NodesToAttributesMappingRequest) invocation.getArguments()[0];
+            return NodesToAttributesMappingResponse.newInstance();
+          }
+        });
+
+    nodeAttributesCLI = new NodeAttributesCLI(new Configuration()) {
+      @Override
+      protected ResourceManagerAdministrationProtocol createAdminProtocol()
+          throws IOException {
+        return admin;
+      }
+    };
+
+    nodeAttributesCLI.setErrOut(new PrintStream(errOutBytes));
+  }
+
+  @Test
+  public void testHelp() throws Exception {
+    String[] args = new String[] { "-help", "-replace" };
+    assertTrue("It should have succeeded help for replace", 0 == runTool(args));
+    assertOutputContains(
+        "-replace <\"node1:attribute[(type)][=value],attribute1"
+            + "[=value],attribute2  node2:attribute2[=value],attribute3\"> :");
+    assertOutputContains("Replace the node to attributes mapping information at"
+        + " the ResourceManager with the new mapping. Currently supported"
+        + " attribute type. And string is the default type too. Attribute value"
+        + " if not specified for string type value will be considered as empty"
+        + " string. Replaced node-attributes should not violate the existing"
+        + " attribute to attribute type mapping.");
+
+    args = new String[] { "-help", "-remove" };
+    assertTrue("It should have succeeded help for replace", 0 == runTool(args));
+    assertOutputContains(
+        "-remove <\"node1:attribute,attribute1" + " node2:attribute2\"> :");
+    assertOutputContains("Removes the specified node to attributes mapping"
+        + " information at the ResourceManager");
+
+    args = new String[] { "-help", "-add" };
+    assertTrue("It should have succeeded help for replace", 0 == runTool(args));
+    assertOutputContains("-add <\"node1:attribute[(type)][=value],"
+        + "attribute1[=value],attribute2  node2:attribute2[=value],attribute3\">"
+        + " :");
+    assertOutputContains("Adds or updates the node to attributes mapping"
+        + " information at the ResourceManager. Currently supported attribute"
+        + " type is string. And string is the default type too. Attribute value"
+        + " if not specified for string type value will be considered as empty"
+        + " string. Added or updated node-attributes should not violate the"
+        + " existing attribute to attribute type mapping.");
+
+    args = new String[] { "-help", "-failOnUnknownNodes" };
+    assertTrue("It should have succeeded help for replace", 0 == runTool(args));
+    assertOutputContains("-failOnUnknownNodes :");
+    assertOutputContains("Can be used optionally along with other options. When"
+        + " its set, it will fail if specified nodes are unknown.");
+  }
+
+  @Test
+  public void testReplace() throws Exception {
+    // --------------------------------
+    // failure scenarios
+    // --------------------------------
+    // parenthesis not match
+    String[] args = new String[] { "-replace", "x(" };
+    assertTrue("It should have failed as no node is specified",
+        0 != runTool(args));
+    assertFailureMessageContains(NodeAttributesCLI.INVALID_MAPPING_ERR_MSG);
+
+    // parenthesis not match
+    args = new String[] { "-replace", "x:(=abc" };
+    assertTrue(
+        "It should have failed as no closing parenthesis is not specified",
+        0 != runTool(args));
+    assertFailureMessageContains(
+        "Attribute for node x is not properly configured : (=abc");
+
+    args = new String[] { "-replace", "x:()=abc" };
+    assertTrue("It should have failed as no type specified inside parenthesis",
+        0 != runTool(args));
+    assertFailureMessageContains(
+        "Attribute for node x is not properly configured : ()=abc");
+
+    args = new String[] { "-replace", ":x(string)" };
+    assertTrue("It should have failed as no node is specified",
+        0 != runTool(args));
+    assertFailureMessageContains("Node name cannot be empty");
+
+    // Not expected key=value specifying inner parenthesis
+    args = new String[] { "-replace", "x:(key=value)" };
+    assertTrue(0 != runTool(args));
+    assertFailureMessageContains(
+        "Attribute for node x is not properly configured : (key=value)");
+
+    // Should fail as no attributes specified
+    args = new String[] { "-replace" };
+    assertTrue("Should fail as no attribute mappings specified",
+        0 != runTool(args));
+    assertFailureMessageContains(NodeAttributesCLI.NO_MAPPING_ERR_MSG);
+
+    // no labels, should fail
+    args = new String[] { "-replace", "-failOnUnknownNodes",
+        "x:key(string)=value,key2=val2" };
+    assertTrue("Should fail as no attribute mappings specified for replace",
+        0 != runTool(args));
+    assertFailureMessageContains(NodeAttributesCLI.NO_MAPPING_ERR_MSG);
+
+    // no labels, should fail
+    args = new String[] { "-replace", " " };
+    assertTrue(0 != runTool(args));
+    assertFailureMessageContains(NodeAttributesCLI.NO_MAPPING_ERR_MSG);
+
+    args = new String[] { "-replace", ", " };
+    assertTrue(0 != runTool(args));
+    assertFailureMessageContains(NodeAttributesCLI.INVALID_MAPPING_ERR_MSG);
+    // --------------------------------
+    // success scenarios
+    // --------------------------------
+    args = new String[] { "-replace",
+        "x:key(string)=value,key2=val2 y:key2=val23,key3 z:key4" };
+    assertTrue("Should not fail as attribute has been properly mapped",
+        0 == runTool(args));
+    List<NodeToAttributes> nodeAttributesList = new ArrayList<>();
+    List<NodeAttribute> attributes = new ArrayList<>();
+    attributes.add(
+        NodeAttribute.newInstance("key", NodeAttributeType.STRING, "value"));
+    attributes.add(
+        NodeAttribute.newInstance("key2", NodeAttributeType.STRING, "val2"));
+    nodeAttributesList.add(NodeToAttributes.newInstance("x", attributes));
+
+    // for node y
+    attributes = new ArrayList<>();
+    attributes.add(
+        NodeAttribute.newInstance("key2", NodeAttributeType.STRING, "val23"));
+    attributes
+        .add(NodeAttribute.newInstance("key3", NodeAttributeType.STRING, ""));
+    nodeAttributesList.add(NodeToAttributes.newInstance("y", attributes));
+
+    // for node y
+    attributes = new ArrayList<>();
+    attributes.add(
+        NodeAttribute.newInstance("key2", NodeAttributeType.STRING, "val23"));
+    attributes
+        .add(NodeAttribute.newInstance("key3", NodeAttributeType.STRING, ""));
+    nodeAttributesList.add(NodeToAttributes.newInstance("y", attributes));
+
+    // for node z
+    attributes = new ArrayList<>();
+    attributes
+        .add(NodeAttribute.newInstance("key4", NodeAttributeType.STRING, ""));
+    nodeAttributesList.add(NodeToAttributes.newInstance("z", attributes));
+
+    NodesToAttributesMappingRequest expected =
+        NodesToAttributesMappingRequest.newInstance(
+            AttributeMappingOperationType.REPLACE, nodeAttributesList, false);
+    assertTrue(request.equals(expected));
+  }
+
+  @Test
+  public void testRemove() throws Exception {
+    // --------------------------------
+    // failure scenarios
+    // --------------------------------
+    // parenthesis not match
+    String[] args = new String[] { "-remove", "x:" };
+    assertTrue("It should have failed as no node is specified",
+        0 != runTool(args));
+    assertFailureMessageContains(
+        "Attributes cannot be null or empty for Operation REMOVE on the node x");
+    // --------------------------------
+    // success scenarios
+    // --------------------------------
+    args =
+        new String[] { "-remove", "x:key2,key3 z:key4", "-failOnUnknownNodes" };
+    assertTrue("Should not fail as attribute has been properly mapped",
+        0 == runTool(args));
+    List<NodeToAttributes> nodeAttributesList = new ArrayList<>();
+    List<NodeAttribute> attributes = new ArrayList<>();
+    attributes
+        .add(NodeAttribute.newInstance("key2", NodeAttributeType.STRING, ""));
+    attributes
+        .add(NodeAttribute.newInstance("key3", NodeAttributeType.STRING, ""));
+    nodeAttributesList.add(NodeToAttributes.newInstance("x", attributes));
+
+    // for node z
+    attributes = new ArrayList<>();
+    attributes
+        .add(NodeAttribute.newInstance("key4", NodeAttributeType.STRING, ""));
+    nodeAttributesList.add(NodeToAttributes.newInstance("z", attributes));
+
+    NodesToAttributesMappingRequest expected =
+        NodesToAttributesMappingRequest.newInstance(
+            AttributeMappingOperationType.REMOVE, nodeAttributesList, true);
+    assertTrue(request.equals(expected));
+  }
+
+  @Test
+  public void testAdd() throws Exception {
+    // --------------------------------
+    // failure scenarios
+    // --------------------------------
+    // parenthesis not match
+    String[] args = new String[] { "-add", "x:" };
+    assertTrue("It should have failed as no node is specified",
+        0 != runTool(args));
+    assertFailureMessageContains(
+        "Attributes cannot be null or empty for Operation ADD on the node x");
+    // --------------------------------
+    // success scenarios
+    // --------------------------------
+    args = new String[] { "-add", "x:key2=123,key3=abc z:key4(string)",
+        "-failOnUnknownNodes" };
+    assertTrue("Should not fail as attribute has been properly mapped",
+        0 == runTool(args));
+    List<NodeToAttributes> nodeAttributesList = new ArrayList<>();
+    List<NodeAttribute> attributes = new ArrayList<>();
+    attributes.add(
+        NodeAttribute.newInstance("key2", NodeAttributeType.STRING, "123"));
+    attributes.add(
+        NodeAttribute.newInstance("key3", NodeAttributeType.STRING, "abc"));
+    nodeAttributesList.add(NodeToAttributes.newInstance("x", attributes));
+
+    // for node z
+    attributes = new ArrayList<>();
+    attributes
+        .add(NodeAttribute.newInstance("key4", NodeAttributeType.STRING, ""));
+    nodeAttributesList.add(NodeToAttributes.newInstance("z", attributes));
+
+    NodesToAttributesMappingRequest expected =
+        NodesToAttributesMappingRequest.newInstance(
+            AttributeMappingOperationType.ADD, nodeAttributesList, true);
+    assertTrue(request.equals(expected));
+  }
+
+  private void assertFailureMessageContains(String... messages) {
+    assertOutputContains(messages);
+    assertOutputContains(NodeAttributesCLI.USAGE_YARN_NODE_ATTRIBUTES);
+  }
+
+  private void assertOutputContains(String... messages) {
+    for (String message : messages) {
+      if (!errOutput.contains(message)) {
+        fail("Expected output to contain '" + message
+            + "' but err_output was:\n" + errOutput);
+      }
+    }
+  }
+
+  private int runTool(String... args) throws Exception {
+    errOutBytes.reset();
+    LOG.info("Running: NodeAttributesCLI " + Joiner.on(" ").join(args));
+    int ret = nodeAttributesCLI.run(args);
+    errOutput = new String(errOutBytes.toByteArray(), Charsets.UTF_8);
+    LOG.info("Err_output:\n" + errOutput);
+    return ret;
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org