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/08/25 15:49:58 UTC

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

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/c190002a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c190002a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c190002a

Branch: refs/heads/YARN-3409
Commit: c190002a099529b829485549266ff8dd99adf8b9
Parents: eed7de2
Author: Naganarasimha <na...@apache.org>
Authored: Fri Apr 20 07:31:53 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:57 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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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/c190002a/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 1320aae..88f442a 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;
@@ -912,6 +914,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/c190002a/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/c190002a/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 510cbaf..67e1f38 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/c190002a/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/c190002a/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/c190002a/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 4a64473..46f0e89 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
@@ -66,6 +66,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;
@@ -749,4 +751,10 @@ public class FederationClientInterceptor
       throws YarnException, IOException {
     throw new NotImplementedException();
   }
+
+  @Override
+  public GetNodesToAttributesResponse getNodesToAttributes(
+      GetNodesToAttributesRequest request) throws YarnException, IOException {
+    throw new NotImplementedException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c190002a/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/c190002a/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