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/07/27 11:32:56 UTC

[01/50] [abbrv] hadoop git commit: HDDS-272. TestBlockDeletingService is failing with DiskOutOfSpaceException. Contributed by Lokesh Jain. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-3409 81e410404 -> acd7729af (forced update)


HDDS-272. TestBlockDeletingService is failing with DiskOutOfSpaceException. Contributed by Lokesh Jain.


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

Branch: refs/heads/YARN-3409
Commit: 773d312f7412d5050c106ed3a1cd0d1934bfa2e0
Parents: ff7c2ed
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Tue Jul 24 21:23:20 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Tue Jul 24 21:23:20 2018 +0530

----------------------------------------------------------------------
 .../container/keyvalue/KeyValueHandler.java     |  2 +-
 .../background/BlockDeletingService.java        |  9 +-
 .../testutils/BlockDeletingServiceTestImpl.java |  3 +-
 .../common/TestBlockDeletingService.java        | 90 ++++++++++++--------
 4 files changed, 60 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/773d312f/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 9aa3df7..d3a1ca4 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -136,7 +136,7 @@ public class KeyValueHandler extends Handler {
             TimeUnit.MILLISECONDS);
     this.blockDeletingService =
         new BlockDeletingService(containerSet, svcInterval, serviceTimeout,
-            config);
+            TimeUnit.MILLISECONDS, config);
     blockDeletingService.start();
     // TODO: Add supoort for different volumeChoosingPolicies.
     volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/773d312f/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
index a3e36f4..4a572ca 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
@@ -88,11 +88,10 @@ public class BlockDeletingService extends BackgroundService{
   // Core pool size for container tasks
   private final static int BLOCK_DELETING_SERVICE_CORE_POOL_SIZE = 10;
 
-  public BlockDeletingService(ContainerSet containerSet,
-      long serviceInterval, long serviceTimeout, Configuration conf) {
-    super("BlockDeletingService", serviceInterval,
-        TimeUnit.MILLISECONDS, BLOCK_DELETING_SERVICE_CORE_POOL_SIZE,
-        serviceTimeout);
+  public BlockDeletingService(ContainerSet containerSet, long serviceInterval,
+      long serviceTimeout, TimeUnit timeUnit, Configuration conf) {
+    super("BlockDeletingService", serviceInterval, timeUnit,
+        BLOCK_DELETING_SERVICE_CORE_POOL_SIZE, serviceTimeout);
     this.containerSet = containerSet;
     containerDeletionPolicy = ReflectionUtils.newInstance(conf.getClass(
         ScmConfigKeys.OZONE_SCM_KEY_VALUE_CONTAINER_DELETION_CHOOSING_POLICY,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/773d312f/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
index a87f655..115b5e2 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/testutils/BlockDeletingServiceTestImpl.java
@@ -44,7 +44,8 @@ public class BlockDeletingServiceTestImpl
 
   public BlockDeletingServiceTestImpl(ContainerSet containerSet,
       int serviceInterval, Configuration conf) {
-    super(containerSet, serviceInterval, SERVICE_TIMEOUT_IN_MILLISECONDS, conf);
+    super(containerSet, serviceInterval, SERVICE_TIMEOUT_IN_MILLISECONDS,
+        TimeUnit.MILLISECONDS, conf);
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/773d312f/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
index 1ddd39a..a6e53c2 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
@@ -45,17 +45,17 @@ import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.apache.hadoop.utils.BackgroundService;
 import org.apache.hadoop.utils.MetadataKeyFilters;
 import org.apache.hadoop.utils.MetadataStore;
+import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.BeforeClass;
-import org.junit.Before;
-import org.junit.After;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
 import java.nio.charset.Charset;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
@@ -72,35 +72,28 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
 /**
  * Tests to test block deleting service.
  */
-// TODO: Fix BlockDeletingService to work with new StorageLayer
 public class TestBlockDeletingService {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(TestBlockDeletingService.class);
 
   private static File testRoot;
-  private static File containersDir;
-  private static File chunksDir;
+  private static String scmId;
+  private static String clusterID;
 
   @BeforeClass
-  public static void init() {
+  public static void init() throws IOException {
     testRoot = GenericTestUtils
         .getTestDir(TestBlockDeletingService.class.getSimpleName());
-    chunksDir = new File(testRoot, "chunks");
-    containersDir = new File(testRoot, "containers");
-  }
-
-  @Before
-  public void setup() throws IOException {
-    if (chunksDir.exists()) {
-      FileUtils.deleteDirectory(chunksDir);
+    if (testRoot.exists()) {
+      FileUtils.cleanDirectory(testRoot);
     }
+    scmId = UUID.randomUUID().toString();
+    clusterID = UUID.randomUUID().toString();
   }
 
-  @After
-  public void cleanup() throws IOException {
-    FileUtils.deleteDirectory(chunksDir);
-    FileUtils.deleteDirectory(containersDir);
+  @AfterClass
+  public static void cleanup() throws IOException {
     FileUtils.deleteDirectory(testRoot);
   }
 
@@ -111,14 +104,15 @@ public class TestBlockDeletingService {
    */
   private void createToDeleteBlocks(ContainerSet containerSet,
       Configuration conf, int numOfContainers, int numOfBlocksPerContainer,
-      int numOfChunksPerBlock, File chunkDir) throws IOException {
+      int numOfChunksPerBlock) throws IOException {
     for (int x = 0; x < numOfContainers; x++) {
+      conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, testRoot.getAbsolutePath());
       long containerID = ContainerTestHelper.getTestContainerID();
       KeyValueContainerData data = new KeyValueContainerData(containerID,
           ContainerTestHelper.CONTAINER_MAX_SIZE_GB);
       Container container = new KeyValueContainer(data, conf);
-      container.create(new VolumeSet(UUID.randomUUID().toString(), conf),
-          new RoundRobinVolumeChoosingPolicy(), UUID.randomUUID().toString());
+      container.create(new VolumeSet(scmId, clusterID, conf),
+          new RoundRobinVolumeChoosingPolicy(), scmId);
       containerSet.addContainer(container);
       data = (KeyValueContainerData) containerSet.getContainer(
           containerID).getContainerData();
@@ -133,7 +127,7 @@ public class TestBlockDeletingService {
         for (int k = 0; k<numOfChunksPerBlock; k++) {
           // offset doesn't matter here
           String chunkName = blockID.getLocalID() + "_chunk_" + k;
-          File chunk = new File(chunkDir, chunkName);
+          File chunk = new File(data.getChunksPath(), chunkName);
           FileUtils.writeStringToFile(chunk, "a chunk",
               Charset.defaultCharset());
           LOG.info("Creating file {}", chunk.getAbsolutePath());
@@ -193,7 +187,7 @@ public class TestBlockDeletingService {
     conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
     ContainerSet containerSet = new ContainerSet();
-    createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
+    createToDeleteBlocks(containerSet, conf, 1, 3, 1);
 
     BlockDeletingServiceTestImpl svc =
         new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
@@ -208,12 +202,13 @@ public class TestBlockDeletingService {
     MetadataStore meta = KeyUtils.getDB(
         (KeyValueContainerData) containerData.get(0), conf);
     Map<Long, Container> containerMap = containerSet.getContainerMap();
+    long transactionId = containerMap.get(containerData.get(0).getContainerID())
+        .getContainerData().getDeleteTransactionId();
 
 
     // Number of deleted blocks in container should be equal to 0 before
     // block delete
-    // TODO : Implement deleteTransactionID in ContainerData.
-//    Assert.assertEquals(0, transactionId);
+    Assert.assertEquals(0, transactionId);
 
     // Ensure there are 3 blocks under deletion and 0 deleted blocks
     Assert.assertEquals(3, getUnderDeletionBlocksCount(meta));
@@ -247,7 +242,7 @@ public class TestBlockDeletingService {
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 10);
     ContainerSet containerSet = new ContainerSet();
     // Create 1 container with 100 blocks
-    createToDeleteBlocks(containerSet, conf, 1, 100, 1, chunksDir);
+    createToDeleteBlocks(containerSet, conf, 1, 100, 1);
 
     BlockDeletingServiceTestImpl service =
         new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
@@ -275,12 +270,13 @@ public class TestBlockDeletingService {
     conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
     ContainerSet containerSet = new ContainerSet();
-    createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
+    createToDeleteBlocks(containerSet, conf, 1, 3, 1);
 
     // set timeout value as 1ns to trigger timeout behavior
     long timeout  = 1;
-    BlockDeletingService svc =
-        new BlockDeletingService(containerSet, 1000, timeout, conf);
+    BlockDeletingService svc = new BlockDeletingService(containerSet,
+        TimeUnit.MILLISECONDS.toNanos(1000), timeout, TimeUnit.NANOSECONDS,
+        conf);
     svc.start();
 
     LogCapturer log = LogCapturer.captureLogs(BackgroundService.LOG);
@@ -299,8 +295,10 @@ public class TestBlockDeletingService {
 
     // test for normal case that doesn't have timeout limitation
     timeout  = 0;
-    createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
-    svc =  new BlockDeletingService(containerSet, 1000, timeout, conf);
+    createToDeleteBlocks(containerSet, conf, 1, 3, 1);
+    svc = new BlockDeletingService(containerSet,
+        TimeUnit.MILLISECONDS.toNanos(1000), timeout, TimeUnit.MILLISECONDS,
+        conf);
     svc.start();
 
     // get container meta data
@@ -347,7 +345,7 @@ public class TestBlockDeletingService {
     conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 1);
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 1);
     ContainerSet containerSet = new ContainerSet();
-    createToDeleteBlocks(containerSet, conf, 2, 1, 10, chunksDir);
+    createToDeleteBlocks(containerSet, conf, 2, 1, 10);
 
     BlockDeletingServiceTestImpl service =
         new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
@@ -357,7 +355,12 @@ public class TestBlockDeletingService {
       GenericTestUtils.waitFor(() -> service.isStarted(), 100, 3000);
       // 1st interval processes 1 container 1 block and 10 chunks
       deleteAndWait(service, 1);
-      Assert.assertEquals(10, chunksDir.listFiles().length);
+      Assert.assertEquals(10, getNumberOfChunksInContainers(containerSet));
+      deleteAndWait(service, 2);
+      deleteAndWait(service, 3);
+      deleteAndWait(service, 4);
+      deleteAndWait(service, 5);
+      Assert.assertEquals(0, getNumberOfChunksInContainers(containerSet));
     } finally {
       service.shutdown();
     }
@@ -383,10 +386,10 @@ public class TestBlockDeletingService {
     conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
     conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
     ContainerSet containerSet = new ContainerSet();
-    createToDeleteBlocks(containerSet, conf, 5, 3, 1, chunksDir);
+    createToDeleteBlocks(containerSet, conf, 5, 3, 1);
 
     // Make sure chunks are created
-    Assert.assertEquals(15, chunksDir.listFiles().length);
+    Assert.assertEquals(15, getNumberOfChunksInContainers(containerSet));
 
     BlockDeletingServiceTestImpl service =
         new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
@@ -399,13 +402,26 @@ public class TestBlockDeletingService {
       // number of containers = 5
       // each interval will at most runDeletingTasks 5 * 2 = 10 blocks
       deleteAndWait(service, 1);
-      Assert.assertEquals(5, chunksDir.listFiles().length);
+      Assert.assertEquals(5, getNumberOfChunksInContainers(containerSet));
 
       // There is only 5 blocks left to runDeletingTasks
       deleteAndWait(service, 2);
-      Assert.assertEquals(0, chunksDir.listFiles().length);
+      Assert.assertEquals(0, getNumberOfChunksInContainers(containerSet));
     } finally {
       service.shutdown();
     }
   }
+
+  private int getNumberOfChunksInContainers(ContainerSet containerSet) {
+    Iterator<Container> containerIterator = containerSet.getContainerIterator();
+    int numChunks = 0;
+    while (containerIterator.hasNext()) {
+      Container container = containerIterator.next();
+      File chunkDir = FileUtils.getFile(
+          ((KeyValueContainerData) container.getContainerData())
+              .getChunksPath());
+      numChunks += chunkDir.listFiles().length;
+    }
+    return numChunks;
+  }
 }


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


[43/50] [abbrv] 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/8ea6d0ee/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/8ea6d0ee/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 4c4e371..4a64473 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
@@ -44,8 +44,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;
@@ -732,4 +736,17 @@ public class FederationClientInterceptor
       GetAllResourceTypeInfoRequest request) throws YarnException, IOException {
     throw new NotImplementedException("Code is not implemented");
   }
+
+  @Override
+  public GetAttributesToNodesResponse getAttributesToNodes(
+      GetAttributesToNodesRequest request) throws YarnException, IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public GetClusterNodeAttributesResponse getClusterNodeAttributes(
+      GetClusterNodeAttributesRequest request)
+      throws YarnException, IOException {
+    throw new NotImplementedException();
+  }
 }

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


[32/50] [abbrv] 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/0771f5b6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0771f5b6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0771f5b6

Branch: refs/heads/YARN-3409
Commit: 0771f5b63ad3ce118cdfe5a4cddfe4628130c0b3
Parents: 8017ef7
Author: Naganarasimha <na...@apache.org>
Authored: Mon Feb 5 05:47:02 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:40:33 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/0771f5b6/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 bbf877f..e6e9198 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
@@ -3519,6 +3519,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";
 
@@ -3529,6 +3532,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";
 
@@ -3600,6 +3606,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/0771f5b6/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 2cc842f..ec955b2 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
@@ -2893,6 +2893,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/0771f5b6/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 8154723..8421eea 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/0771f5b6/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/0771f5b6/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/0771f5b6/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/0771f5b6/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/0771f5b6/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/0771f5b6/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/0771f5b6/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/0771f5b6/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/0771f5b6/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/0771f5b6/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/0771f5b6/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/0771f5b6/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/0771f5b6/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/0771f5b6/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


[21/50] [abbrv] hadoop git commit: HDDS-201. Add name for LeaseManager. Contributed by Sandeep Nemuri.

Posted by su...@apache.org.
HDDS-201. Add name for LeaseManager. Contributed by Sandeep Nemuri.


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

Branch: refs/heads/YARN-3409
Commit: a19229594e48fad9f50dbdb1f0b2fcbf7443ce66
Parents: 9089790
Author: Nanda kumar <na...@apache.org>
Authored: Thu Jul 26 19:00:23 2018 +0530
Committer: Nanda kumar <na...@apache.org>
Committed: Thu Jul 26 19:00:23 2018 +0530

----------------------------------------------------------------------
 .../apache/hadoop/ozone/lease/LeaseManager.java | 14 ++++++++-----
 .../hadoop/ozone/lease/TestLeaseManager.java    | 21 ++++++++++----------
 .../hdds/server/events/TestEventWatcher.java    |  2 +-
 .../hdds/scm/container/ContainerMapping.java    |  4 ++--
 .../hdds/scm/pipelines/PipelineSelector.java    |  4 ++--
 .../scm/server/StorageContainerManager.java     |  3 ++-
 .../replication/TestReplicationManager.java     |  4 ++--
 7 files changed, 28 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java
index b8390dd..756a41a 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/lease/LeaseManager.java
@@ -42,6 +42,7 @@ public class LeaseManager<T> {
   private static final Logger LOG =
       LoggerFactory.getLogger(LeaseManager.class);
 
+  private final String name;
   private final long defaultTimeout;
   private Map<T, Lease<T>> activeLeases;
   private LeaseMonitor leaseMonitor;
@@ -51,10 +52,13 @@ public class LeaseManager<T> {
   /**
    * Creates an instance of lease manager.
    *
+   * @param name
+   *        Name for the LeaseManager instance.
    * @param defaultTimeout
    *        Default timeout in milliseconds to be used for lease creation.
    */
-  public LeaseManager(long defaultTimeout) {
+  public LeaseManager(String name, long defaultTimeout) {
+    this.name = name;
     this.defaultTimeout = defaultTimeout;
   }
 
@@ -62,11 +66,11 @@ public class LeaseManager<T> {
    * Starts the lease manager service.
    */
   public void start() {
-    LOG.debug("Starting LeaseManager service");
+    LOG.debug("Starting {} LeaseManager service", name);
     activeLeases = new ConcurrentHashMap<>();
     leaseMonitor = new LeaseMonitor();
     leaseMonitorThread = new Thread(leaseMonitor);
-    leaseMonitorThread.setName("LeaseManager#LeaseMonitor");
+    leaseMonitorThread.setName(name + "-LeaseManager#LeaseMonitor");
     leaseMonitorThread.setDaemon(true);
     leaseMonitorThread.setUncaughtExceptionHandler((thread, throwable) -> {
       // Let us just restart this thread after logging an error.
@@ -75,7 +79,7 @@ public class LeaseManager<T> {
           thread.toString(), throwable);
       leaseMonitorThread.start();
     });
-    LOG.debug("Starting LeaseManager#LeaseMonitor Thread");
+    LOG.debug("Starting {}-LeaseManager#LeaseMonitor Thread", name);
     leaseMonitorThread.start();
     isRunning = true;
   }
@@ -203,7 +207,7 @@ public class LeaseManager<T> {
     @Override
     public void run() {
       while(monitor) {
-        LOG.debug("LeaseMonitor: checking for lease expiry");
+        LOG.debug("{}-LeaseMonitor: checking for lease expiry", name);
         long sleepTime = Long.MAX_VALUE;
 
         for (T resource : activeLeases.keySet()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lease/TestLeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lease/TestLeaseManager.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lease/TestLeaseManager.java
index 517c1a7..bdc70fc 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lease/TestLeaseManager.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/lease/TestLeaseManager.java
@@ -67,7 +67,7 @@ public class TestLeaseManager {
   public void testLeaseAcquireAndRelease() throws LeaseException {
     //It is assumed that the test case execution won't take more than 5 seconds,
     //if it takes more time increase the defaultTimeout value of LeaseManager.
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     DummyResource resourceTwo = new DummyResource("two");
@@ -93,7 +93,7 @@ public class TestLeaseManager {
 
   @Test
   public void testLeaseAlreadyExist() throws LeaseException {
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     DummyResource resourceTwo = new DummyResource("two");
@@ -113,7 +113,7 @@ public class TestLeaseManager {
 
   @Test
   public void testLeaseNotFound() throws LeaseException, InterruptedException {
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     DummyResource resourceTwo = new DummyResource("two");
@@ -154,7 +154,7 @@ public class TestLeaseManager {
 
   @Test
   public void testCustomLeaseTimeout() throws LeaseException {
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     DummyResource resourceTwo = new DummyResource("two");
@@ -179,7 +179,7 @@ public class TestLeaseManager {
   @Test
   public void testLeaseCallback() throws LeaseException, InterruptedException {
     Map<DummyResource, String> leaseStatus = new HashMap<>();
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     Lease<DummyResource> leaseOne = manager.acquire(resourceOne);
@@ -209,7 +209,7 @@ public class TestLeaseManager {
       throws LeaseException, InterruptedException {
     // Callbacks should not be executed in case of lease release
     Map<DummyResource, String> leaseStatus = new HashMap<>();
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     Lease<DummyResource> leaseOne = manager.acquire(resourceOne);
@@ -231,7 +231,7 @@ public class TestLeaseManager {
   public void testLeaseCallbackWithMultipleLeases()
       throws LeaseException, InterruptedException {
     Map<DummyResource, String> leaseStatus = new HashMap<>();
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     DummyResource resourceTwo = new DummyResource("two");
@@ -302,7 +302,7 @@ public class TestLeaseManager {
 
   @Test
   public void testReuseReleasedLease() throws LeaseException {
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     Lease<DummyResource> leaseOne = manager.acquire(resourceOne);
@@ -324,13 +324,12 @@ public class TestLeaseManager {
   @Test
   public void testReuseTimedOutLease()
       throws LeaseException, InterruptedException {
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     Lease<DummyResource> leaseOne = manager.acquire(resourceOne);
     Assert.assertEquals(leaseOne, manager.get(resourceOne));
     Assert.assertFalse(leaseOne.hasExpired());
-
     // wait for lease to expire
     long sleepTime = leaseOne.getRemainingTime() + 1000;
     try {
@@ -352,7 +351,7 @@ public class TestLeaseManager {
 
   @Test
   public void testRenewLease() throws LeaseException, InterruptedException {
-    LeaseManager<DummyResource> manager = new LeaseManager<>(5000);
+    LeaseManager<DummyResource> manager = new LeaseManager<>("Test", 5000);
     manager.start();
     DummyResource resourceOne = new DummyResource("one");
     Lease<DummyResource> leaseOne = manager.acquire(resourceOne);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventWatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventWatcher.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventWatcher.java
index 786b7b8..b72d2ae 100644
--- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventWatcher.java
+++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventWatcher.java
@@ -46,7 +46,7 @@ public class TestEventWatcher {
   @Before
   public void startLeaseManager() {
     DefaultMetricsSystem.instance();
-    leaseManager = new LeaseManager<>(2000l);
+    leaseManager = new LeaseManager<>("Test", 2000L);
     leaseManager.start();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
index f07d22b..e17fe3d 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
@@ -139,8 +139,8 @@ public class ContainerMapping implements Mapping {
         ScmConfigKeys.OZONE_SCM_CONTAINER_CREATION_LEASE_TIMEOUT,
         ScmConfigKeys.OZONE_SCM_CONTAINER_CREATION_LEASE_TIMEOUT_DEFAULT,
         TimeUnit.MILLISECONDS);
-    LOG.trace("Starting Container Lease Manager.");
-    containerLeaseManager = new LeaseManager<>(containerCreationLeaseTimeout);
+    containerLeaseManager = new LeaseManager<>("ContainerCreation",
+        containerCreationLeaseTimeout);
     containerLeaseManager.start();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
index 08710e7..b1e1dd0 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
@@ -99,8 +99,8 @@ public class PipelineSelector {
         ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT,
         ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT_DEFAULT,
         TimeUnit.MILLISECONDS);
-    LOG.trace("Starting Pipeline Lease Manager.");
-    pipelineLeaseManager = new LeaseManager<>(pipelineCreationLeaseTimeout);
+    pipelineLeaseManager = new LeaseManager<>("PipelineCreation",
+        pipelineCreationLeaseTimeout);
     pipelineLeaseManager.start();
 
     // These are the steady states of a container.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
index f4cd448..165805f 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
@@ -223,7 +223,8 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
         conf.getTimeDuration(ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT,
             HDDS_SCM_WATCHER_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS);
 
-    commandWatcherLeaseManager = new LeaseManager<>(watcherTimeout);
+    commandWatcherLeaseManager = new LeaseManager<>("CommandWatcher",
+        watcherTimeout);
 
     //TODO: support configurable containerPlacement policy
     ContainerPlacementPolicy containerPlacementPolicy =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1922959/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java
index 99ec59f..9aa4b64 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java
@@ -112,7 +112,7 @@ public class TestReplicationManager {
 
     //GIVEN
 
-    LeaseManager<Long> leaseManager = new LeaseManager<>(100000L);
+    LeaseManager<Long> leaseManager = new LeaseManager<>("Test", 100000L);
     try {
       leaseManager.start();
 
@@ -152,7 +152,7 @@ public class TestReplicationManager {
   public void testCommandWatcher() throws InterruptedException, IOException {
 
     Logger.getRootLogger().setLevel(Level.DEBUG);
-    LeaseManager<Long> leaseManager = new LeaseManager<>(1000L);
+    LeaseManager<Long> leaseManager = new LeaseManager<>("Test", 1000L);
 
     try {
       leaseManager.start();


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


[41/50] [abbrv] 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/823000d4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/823000d4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/823000d4

Branch: refs/heads/YARN-3409
Commit: 823000d41dd311ad7db86ea308749ca9d129a7aa
Parents: 1950aee
Author: bibinchundatt <bi...@apache.org>
Authored: Sun Apr 1 19:24:00 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:41:51 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/823000d4/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/823000d4/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/823000d4/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/823000d4/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/823000d4/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


[11/50] [abbrv] hadoop git commit: YARN-8546. Resource leak caused by a reserved container being released more than once under async scheduling. Contributed by Tao Yang.

Posted by su...@apache.org.
YARN-8546. Resource leak caused by a reserved container being released more than once under async scheduling. Contributed by Tao Yang.


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

Branch: refs/heads/YARN-3409
Commit: 5be9f4a5d05c9cb99348719fe35626b1de3055db
Parents: 955f795
Author: Weiwei Yang <ww...@apache.org>
Authored: Wed Jul 25 17:35:27 2018 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Wed Jul 25 17:35:27 2018 +0800

----------------------------------------------------------------------
 .../scheduler/common/fica/FiCaSchedulerApp.java | 15 ++++
 .../TestCapacitySchedulerAsyncScheduling.java   | 89 ++++++++++++++++++++
 2 files changed, 104 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5be9f4a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.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/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index 3b1b82c..9810e98 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -361,6 +361,21 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
         .isEmpty()) {
       for (SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode>
           releaseContainer : allocation.getToRelease()) {
+        // Make sure to-release reserved containers are not outdated
+        if (releaseContainer.getRmContainer().getState()
+            == RMContainerState.RESERVED
+            && releaseContainer.getRmContainer() != releaseContainer
+            .getSchedulerNode().getReservedContainer()) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Failed to accept this proposal because "
+                + "it tries to release an outdated reserved container "
+                + releaseContainer.getRmContainer().getContainerId()
+                + " on node " + releaseContainer.getSchedulerNode().getNodeID()
+                + " whose reserved container is "
+                + releaseContainer.getSchedulerNode().getReservedContainer());
+          }
+          return false;
+        }
         // Only consider non-reserved container (reserved container will
         // not affect available resource of node) on the same node
         if (releaseContainer.getRmContainer().getState()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5be9f4a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAsyncScheduling.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/TestCapacitySchedulerAsyncScheduling.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/TestCapacitySchedulerAsyncScheduling.java
index 338b9f9..c2c1519 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/TestCapacitySchedulerAsyncScheduling.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/TestCapacitySchedulerAsyncScheduling.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -41,6 +42,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -685,6 +687,93 @@ public class TestCapacitySchedulerAsyncScheduling {
     rm.stop();
   }
 
+
+  @Test(timeout = 60000)
+  public void testReleaseOutdatedReservedContainer() throws Exception {
+    /*
+     * Submit a application, reserved container_02 on nm1,
+     * submit two allocate proposals which contain the same reserved
+     * container_02 as to-released container.
+     * First proposal should be accepted, second proposal should be rejected
+     * because it try to release an outdated reserved container
+     */
+    MockRM rm1 = new MockRM();
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+    MockNM nm2 = rm1.registerNode("h2:1234", 8 * GB);
+    MockNM nm3 = rm1.registerNode("h3:1234", 8 * GB);
+    rm1.drainEvents();
+
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+    LeafQueue defaultQueue = (LeafQueue) cs.getQueue("default");
+    SchedulerNode sn1 = cs.getSchedulerNode(nm1.getNodeId());
+    SchedulerNode sn2 = cs.getSchedulerNode(nm2.getNodeId());
+    SchedulerNode sn3 = cs.getSchedulerNode(nm3.getNodeId());
+
+    // launch another app to queue, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(4 * GB, "app", "user", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+    Resource allocateResource = Resources.createResource(5 * GB);
+    am1.allocate("*", (int) allocateResource.getMemorySize(), 3, 0,
+        new ArrayList<ContainerId>(), "");
+    FiCaSchedulerApp schedulerApp1 =
+        cs.getApplicationAttempt(am1.getApplicationAttemptId());
+
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    Assert.assertEquals(1, schedulerApp1.getReservedContainers().size());
+    Assert.assertEquals(9 * GB,
+        defaultQueue.getQueueResourceUsage().getUsed().getMemorySize());
+
+    RMContainer reservedContainer =
+        schedulerApp1.getReservedContainers().get(0);
+    ResourceCommitRequest allocateFromSameReservedContainerProposal1 =
+        createAllocateFromReservedProposal(3, allocateResource, schedulerApp1,
+            sn2, sn1, cs.getRMContext(), reservedContainer);
+    boolean tryCommitResult = cs.tryCommit(cs.getClusterResource(),
+        allocateFromSameReservedContainerProposal1, true);
+    Assert.assertTrue(tryCommitResult);
+    ResourceCommitRequest allocateFromSameReservedContainerProposal2 =
+        createAllocateFromReservedProposal(4, allocateResource, schedulerApp1,
+            sn3, sn1, cs.getRMContext(), reservedContainer);
+    tryCommitResult = cs.tryCommit(cs.getClusterResource(),
+        allocateFromSameReservedContainerProposal2, true);
+    Assert.assertFalse("This proposal should be rejected because "
+        + "it try to release an outdated reserved container", tryCommitResult);
+
+    rm1.close();
+  }
+
+  private ResourceCommitRequest createAllocateFromReservedProposal(
+      int containerId, Resource allocateResource, FiCaSchedulerApp schedulerApp,
+      SchedulerNode allocateNode, SchedulerNode reservedNode,
+      RMContext rmContext, RMContainer reservedContainer) {
+    Container container = Container.newInstance(
+        ContainerId.newContainerId(schedulerApp.getApplicationAttemptId(), containerId),
+        allocateNode.getNodeID(), allocateNode.getHttpAddress(), allocateResource,
+        Priority.newInstance(0), null);
+    RMContainer rmContainer = new RMContainerImpl(container, SchedulerRequestKey
+        .create(ResourceRequest
+            .newInstance(Priority.newInstance(0), "*", allocateResource, 1)),
+        schedulerApp.getApplicationAttemptId(), allocateNode.getNodeID(), "user",
+        rmContext);
+    SchedulerContainer allocateContainer =
+        new SchedulerContainer(schedulerApp, allocateNode, rmContainer, "", true);
+    SchedulerContainer reservedSchedulerContainer =
+        new SchedulerContainer(schedulerApp, reservedNode, reservedContainer, "",
+            false);
+    List<SchedulerContainer> toRelease = new ArrayList<>();
+    toRelease.add(reservedSchedulerContainer);
+    ContainerAllocationProposal allocateFromReservedProposal =
+        new ContainerAllocationProposal(allocateContainer, toRelease, null,
+            NodeType.OFF_SWITCH, NodeType.OFF_SWITCH,
+            SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY, allocateResource);
+    List<ContainerAllocationProposal> allocateProposals = new ArrayList<>();
+    allocateProposals.add(allocateFromReservedProposal);
+    return new ResourceCommitRequest(allocateProposals, null, null);
+  }
+
   private void keepNMHeartbeat(List<MockNM> mockNMs, int interval) {
     if (nmHeartbeatThread != null) {
       nmHeartbeatThread.setShouldStop();


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


[07/50] [abbrv] hadoop git commit: HDFS-13448. HDFS Block Placement - Ignore Locality for First Block Replica (Contributed by BELUGA BEHR via Daniel Templeton)

Posted by su...@apache.org.
HDFS-13448. HDFS Block Placement - Ignore Locality for First Block Replica
(Contributed by BELUGA BEHR via Daniel Templeton)

Change-Id: I965d1cfa642ad24296038b83e3d5c9983545267d


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

Branch: refs/heads/YARN-3409
Commit: 849c45db187224095b13fe297a4d7377fbb9d2cd
Parents: 6bec03c
Author: Daniel Templeton <te...@apache.org>
Authored: Tue Jul 24 15:34:19 2018 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Tue Jul 24 16:05:27 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/CreateFlag.java   |  9 ++-
 .../org/apache/hadoop/hdfs/AddBlockFlag.java    | 11 ++-
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  3 +
 .../hadoop/hdfs/DistributedFileSystem.java      | 11 +++
 .../src/main/proto/ClientNamenodeProtocol.proto |  1 +
 .../BlockPlacementPolicyDefault.java            |  4 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  | 30 +++++---
 .../server/namenode/TestFSDirWriteFileOp.java   | 79 ++++++++++++++++++++
 8 files changed, 134 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java
index 383d65a..c3e088b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java
@@ -116,7 +116,14 @@ public enum CreateFlag {
    * Enforce the file to be a replicated file, no matter what its parent
    * directory's replication or erasure coding policy is.
    */
-  SHOULD_REPLICATE((short) 0x80);
+  SHOULD_REPLICATE((short) 0x80),
+
+  /**
+   * Advise that the first block replica NOT take into account DataNode
+   * locality. The first block replica should be placed randomly within the
+   * cluster. Subsequent block replicas should follow DataNode locality rules.
+   */
+  IGNORE_CLIENT_LOCALITY((short) 0x100);
 
   private final short mode;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AddBlockFlag.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AddBlockFlag.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AddBlockFlag.java
index 6a0805b..b0686d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AddBlockFlag.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AddBlockFlag.java
@@ -36,7 +36,16 @@ public enum AddBlockFlag {
    *
    * @see CreateFlag#NO_LOCAL_WRITE
    */
-  NO_LOCAL_WRITE((short) 0x01);
+  NO_LOCAL_WRITE((short) 0x01),
+
+  /**
+   * Advise that the first block replica NOT take into account DataNode
+   * locality. The first block replica should be placed randomly within the
+   * cluster. Subsequent block replicas should follow DataNode locality rules.
+   *
+   * @see CreateFlag#IGNORE_CLIENT_LOCALITY
+   */
+  IGNORE_CLIENT_LOCALITY((short) 0x02);
 
   private final short mode;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 9734752..e977054 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -201,6 +201,9 @@ public class DFSOutputStream extends FSOutputSummer
     if (flag.contains(CreateFlag.NO_LOCAL_WRITE)) {
       this.addBlockFlags.add(AddBlockFlag.NO_LOCAL_WRITE);
     }
+    if (flag.contains(CreateFlag.IGNORE_CLIENT_LOCALITY)) {
+      this.addBlockFlags.add(AddBlockFlag.IGNORE_CLIENT_LOCALITY);
+    }
     if (progress != null) {
       DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
           +"{}", src);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 82cdd8c..3519c60 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -3205,6 +3205,17 @@ public class DistributedFileSystem extends FileSystem
       return this;
     }
 
+    /**
+     * Advise that the first block replica be written without regard to the
+     * client locality.
+     *
+     * @see CreateFlag for the details.
+     */
+    public HdfsDataOutputStreamBuilder ignoreClientLocality() {
+      getFlags().add(CreateFlag.IGNORE_CLIENT_LOCALITY);
+      return this;
+    }
+
     @VisibleForTesting
     @Override
     protected EnumSet<CreateFlag> getFlags() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index 0f5ce94..e51aeda 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -167,6 +167,7 @@ message AbandonBlockResponseProto { // void response
 
 enum AddBlockFlagProto {
   NO_LOCAL_WRITE = 1; // avoid writing to local node.
+  IGNORE_CLIENT_LOCALITY = 2; // write to a random node
 }
 
 message AddBlockRequestProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
index c94232f..6985f55 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
@@ -280,7 +280,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     if (avoidLocalNode) {
       results = new ArrayList<>(chosenStorage);
       Set<Node> excludedNodeCopy = new HashSet<>(excludedNodes);
-      excludedNodeCopy.add(writer);
+      if (writer != null) {
+        excludedNodeCopy.add(writer);
+      }
       localNode = chooseTarget(numOfReplicas, writer,
           excludedNodeCopy, blocksize, maxNodesPerRack, results,
           avoidStaleNodes, storagePolicy,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 03c349c..2875708 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -269,19 +269,27 @@ class FSDirWriteFileOp {
       BlockManager bm, String src, DatanodeInfo[] excludedNodes,
       String[] favoredNodes, EnumSet<AddBlockFlag> flags,
       ValidateAddBlockResult r) throws IOException {
-    Node clientNode = bm.getDatanodeManager()
-        .getDatanodeByHost(r.clientMachine);
-    if (clientNode == null) {
-      clientNode = getClientNode(bm, r.clientMachine);
-    }
+    Node clientNode = null;
 
-    Set<Node> excludedNodesSet = null;
-    if (excludedNodes != null) {
-      excludedNodesSet = new HashSet<>(excludedNodes.length);
-      Collections.addAll(excludedNodesSet, excludedNodes);
+    boolean ignoreClientLocality = (flags != null
+            && flags.contains(AddBlockFlag.IGNORE_CLIENT_LOCALITY));
+
+    // If client locality is ignored, clientNode remains 'null' to indicate
+    if (!ignoreClientLocality) {
+      clientNode = bm.getDatanodeManager().getDatanodeByHost(r.clientMachine);
+      if (clientNode == null) {
+        clientNode = getClientNode(bm, r.clientMachine);
+      }
     }
-    List<String> favoredNodesList = (favoredNodes == null) ? null
-        : Arrays.asList(favoredNodes);
+
+    Set<Node> excludedNodesSet =
+        (excludedNodes == null) ? new HashSet<>()
+            : new HashSet<>(Arrays.asList(excludedNodes));
+
+    List<String> favoredNodesList =
+        (favoredNodes == null) ? Collections.emptyList()
+            : Arrays.asList(favoredNodes);
+
     // choose targets for the new block to be allocated.
     return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode,
                                     excludedNodesSet, r.blockSize,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/849c45db/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirWriteFileOp.java
new file mode 100644
index 0000000..762fa61
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirWriteFileOp.java
@@ -0,0 +1,79 @@
+/**
+ * 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.hdfs.server.namenode;
+
+import static org.junit.Assert.assertNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyByte;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyList;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anySet;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.EnumSet;
+
+import org.apache.hadoop.hdfs.AddBlockFlag;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.ValidateAddBlockResult;
+import org.apache.hadoop.net.Node;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+public class TestFSDirWriteFileOp {
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testIgnoreClientLocality() throws IOException {
+    ValidateAddBlockResult addBlockResult =
+        new ValidateAddBlockResult(1024L, 3, (byte) 0x01, null, null, null);
+
+    EnumSet<AddBlockFlag> addBlockFlags =
+        EnumSet.of(AddBlockFlag.IGNORE_CLIENT_LOCALITY);
+
+    BlockManager bmMock = mock(BlockManager.class);
+
+    ArgumentCaptor<Node> nodeCaptor = ArgumentCaptor.forClass(Node.class);
+
+    when(bmMock.chooseTarget4NewBlock(anyString(), anyInt(), any(), anySet(),
+        anyLong(), anyList(), anyByte(), any(), any(), any())).thenReturn(null);
+
+    FSDirWriteFileOp.chooseTargetForNewBlock(bmMock, "localhost", null, null,
+        addBlockFlags, addBlockResult);
+
+    // There should be no other interactions with the block manager when the
+    // IGNORE_CLIENT_LOCALITY is passed in because there is no need to discover
+    // the local node requesting the new block
+    verify(bmMock, times(1)).chooseTarget4NewBlock(anyString(), anyInt(),
+        nodeCaptor.capture(), anySet(), anyLong(), anyList(), anyByte(), any(),
+        any(), any());
+
+    verifyNoMoreInteractions(bmMock);
+
+    assertNull(
+        "Source node was assigned a value. Expected 'null' value because "
+            + "chooseTarget was flagged to ignore source node locality",
+        nodeCaptor.getValue());
+  }
+}


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


[48/50] [abbrv] 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/649e014f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/649e014f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/649e014f

Branch: refs/heads/YARN-3409
Commit: 649e014f5646ba6e3031be4990cbefe5a1c62a6b
Parents: 4770a85
Author: Sunil G <su...@apache.org>
Authored: Fri May 25 16:08:26 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:45:20 2018 +0530

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/649e014f/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 328910f..6eb4589 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
@@ -195,7 +195,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


[45/50] [abbrv] 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/1ea02a00
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1ea02a00
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1ea02a00

Branch: refs/heads/YARN-3409
Commit: 1ea02a000af8b24df4db39a9ecc9be309379e41b
Parents: 8ea6d0e
Author: Naganarasimha <na...@apache.org>
Authored: Fri Apr 20 07:31:53 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:45:19 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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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 5ed3090..d3f4081 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
@@ -271,7 +271,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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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 585b484..be816d8 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;
@@ -2122,6 +2125,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/1ea02a00/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/1ea02a00/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/1ea02a00/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/1ea02a00/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


[06/50] [abbrv] hadoop git commit: HADOOP-15612. Improve exception when tfile fails to load LzoCodec. (gera)

Posted by su...@apache.org.
HADOOP-15612. Improve exception when tfile fails to load LzoCodec. (gera)


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

Branch: refs/heads/YARN-3409
Commit: 6bec03cfc8bdcf6aa3df9c22231ab959ba31f2f5
Parents: ea2c6c8
Author: Gera Shegalov <ge...@apache.org>
Authored: Tue Jul 17 00:05:39 2018 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Tue Jul 24 14:32:30 2018 -0700

----------------------------------------------------------------------
 .../hadoop/io/file/tfile/Compression.java       | 31 +++++++++++-------
 .../hadoop/io/file/tfile/TestCompression.java   | 34 +++++++++++++++++++-
 2 files changed, 53 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6bec03cf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java
index fa85ed7..c4347e0 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Compression.java
@@ -5,9 +5,9 @@
  * 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
@@ -24,6 +24,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.ArrayList;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.compress.CodecPool;
 import org.apache.hadoop.io.compress.CompressionCodec;
@@ -78,25 +79,33 @@ public final class Compression {
   public enum Algorithm {
     LZO(TFile.COMPRESSION_LZO) {
       private transient boolean checked = false;
+      private transient ClassNotFoundException cnf;
+      private transient boolean reinitCodecInTests;
       private static final String defaultClazz =
           "org.apache.hadoop.io.compress.LzoCodec";
+      private transient String clazz;
       private transient CompressionCodec codec = null;
 
+      private String getLzoCodecClass() {
+        String extClazzConf = conf.get(CONF_LZO_CLASS);
+        String extClazz = (extClazzConf != null) ?
+            extClazzConf : System.getProperty(CONF_LZO_CLASS);
+        return (extClazz != null) ? extClazz : defaultClazz;
+      }
+
       @Override
       public synchronized boolean isSupported() {
-        if (!checked) {
+        if (!checked || reinitCodecInTests) {
           checked = true;
-          String extClazzConf = conf.get(CONF_LZO_CLASS);
-          String extClazz = (extClazzConf != null) ?
-              extClazzConf : System.getProperty(CONF_LZO_CLASS);
-          String clazz = (extClazz != null) ? extClazz : defaultClazz;
+          reinitCodecInTests = conf.getBoolean("test.reload.lzo.codec", false);
+          clazz = getLzoCodecClass();
           try {
             LOG.info("Trying to load Lzo codec class: " + clazz);
             codec =
                 (CompressionCodec) ReflectionUtils.newInstance(Class
                     .forName(clazz), conf);
           } catch (ClassNotFoundException e) {
-            // that is okay
+            cnf = e;
           }
         }
         return codec != null;
@@ -105,9 +114,9 @@ public final class Compression {
       @Override
       CompressionCodec getCodec() throws IOException {
         if (!isSupported()) {
-          throw new IOException(
-              "LZO codec class not specified. Did you forget to set property "
-                  + CONF_LZO_CLASS + "?");
+          throw new IOException(String.format(
+              "LZO codec %s=%s could not be loaded", CONF_LZO_CLASS, clazz),
+                  cnf);
         }
 
         return codec;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6bec03cf/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java
index ff6c72a..b1bf077 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java
@@ -17,14 +17,28 @@
  */
 package org.apache.hadoop.io.file.tfile;
 
-import org.junit.Test;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.*;
 
 import java.io.IOException;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class TestCompression {
 
+  @BeforeClass
+  public static void resetConfigBeforeAll() {
+    Compression.Algorithm.LZO.conf.setBoolean("test.reload.lzo.codec", true);
+  }
+
+  @AfterClass
+  public static void resetConfigAfterAll() {
+    Compression.Algorithm.LZO.conf.setBoolean("test.reload.lzo.codec", false);
+  }
+
   /**
    * Regression test for HADOOP-11418.
    * Verify we can set a LZO codec different from default LZO codec.
@@ -38,4 +52,22 @@ public class TestCompression {
     assertEquals(defaultCodec,
         Compression.Algorithm.LZO.getCodec().getClass().getName());
   }
+
+
+  @Test
+  public void testMisconfiguredLZOCodec() throws Exception {
+    // Dummy codec
+    String defaultCodec = "org.apache.hadoop.io.compress.InvalidLzoCodec";
+    Compression.Algorithm.conf.set(
+        Compression.Algorithm.CONF_LZO_CLASS, defaultCodec);
+    IOException ioEx = LambdaTestUtils.intercept(
+        IOException.class,
+        defaultCodec,
+        () -> Compression.Algorithm.LZO.getCodec());
+
+    if (!(ioEx.getCause() instanceof ClassNotFoundException)) {
+      throw ioEx;
+    }
+  }
+
 }


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


[44/50] [abbrv] 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/8ea6d0ee
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8ea6d0ee
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8ea6d0ee

Branch: refs/heads/YARN-3409
Commit: 8ea6d0ee4f2ea6508f81b86af085206e09e161e8
Parents: 9e1fb74
Author: Naganarasimha <na...@apache.org>
Authored: Tue Apr 10 07:28:53 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:44:38 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   |  33 ++--
 .../resourcemanager/TestClientRMService.java    | 126 +++++++++++++
 .../DefaultClientRequestInterceptor.java        |  17 ++
 .../clientrm/FederationClientInterceptor.java   |  17 ++
 .../router/clientrm/RouterClientRMService.java  |  19 ++
 .../PassThroughClientRequestInterceptor.java    |  17 ++
 29 files changed, 1361 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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 b43ff5a..a315836 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
@@ -3468,7 +3468,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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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 92a65ad..5ed3090 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
@@ -259,6 +259,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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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/8ea6d0ee/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 f14f17b..1320aae 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;
@@ -895,6 +899,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/8ea6d0ee/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/8ea6d0ee/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..510cbaf 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
@@ -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/8ea6d0ee/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 d66a866..585b484 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;
@@ -1996,6 +2003,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


[22/50] [abbrv] hadoop git commit: HDFS-13622. mkdir should print the parent directory in the error message when parent directories do not exist. Contributed by Shweta.

Posted by su...@apache.org.
HDFS-13622. mkdir should print the parent directory in the error message when parent directories do not exist. Contributed by Shweta.


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

Branch: refs/heads/YARN-3409
Commit: be150a17b15d15f5de6d4839d5e805e8d6c57850
Parents: a192295
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Jul 26 10:23:30 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Jul 26 10:24:32 2018 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/fs/shell/Mkdir.java    | 13 ++++++++-----
 .../test/java/org/apache/hadoop/hdfs/TestDFSShell.java |  8 ++++++++
 .../hadoop-hdfs/src/test/resources/testHDFSConf.xml    |  4 ++--
 3 files changed, 18 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/be150a17/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Mkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Mkdir.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Mkdir.java
index 9f39da2..5828b0b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Mkdir.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Mkdir.java
@@ -68,11 +68,14 @@ class Mkdir extends FsCommand {
 
   @Override
   protected void processNonexistentPath(PathData item) throws IOException {
-    // check if parent exists. this is complicated because getParent(a/b/c/) returns a/b/c, but
-    // we want a/b
-    if (!createParents &&
-        !item.fs.exists(new Path(item.path.toString()).getParent())) {
-      throw new PathNotFoundException(item.toString());
+    if (!createParents) {
+      // check if parent exists. this is complicated because getParent(a/b/c/) returns a/b/c, but
+      // we want a/b
+      final Path itemPath = new Path(item.path.toString());
+      final Path itemParentPath = itemPath.getParent();
+      if (!item.fs.exists(itemParentPath)) {
+        throw new PathNotFoundException(itemParentPath.toString());
+      }
     }
     if (!item.fs.mkdirs(item.path)) {
       throw new PathIOException(item.toString());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/be150a17/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
index b19bdea..1d2042e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
@@ -721,6 +721,14 @@ public class TestDFSShell {
       assertTrue(" -mkdir returned this is a file ",
           (returned.lastIndexOf("not a directory") != -1));
       out.reset();
+      argv[0] = "-mkdir";
+      argv[1] = "/testParent/testChild";
+      ret = ToolRunner.run(shell, argv);
+      returned = out.toString();
+      assertEquals(" -mkdir returned 1", 1, ret);
+      assertTrue(" -mkdir returned there is No file or directory but has testChild in the path",
+          (returned.lastIndexOf("testChild") == -1));
+      out.reset();
       argv = new String[3];
       argv[0] = "-mv";
       argv[1] = "/testfile";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/be150a17/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
index a13c441..4ab093b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
@@ -6183,11 +6183,11 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>mkdir: `dir0/dir1': No such file or directory</expected-output>
+          <expected-output>.*mkdir:.*dir0': No such file or directory$</expected-output>
         </comparator>
       </comparators>
     </test>
-    
+
     <test> <!-- TESTED -->
       <description>mkdir: Test recreate of existing directory fails</description>
       <test-commands>


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


[15/50] [abbrv] hadoop git commit: HDDS-288. Fix bugs in OpenContainerBlockMap. Contributed by Tsz Wo Nicholas Sze.

Posted by su...@apache.org.
HDDS-288. Fix bugs in OpenContainerBlockMap. Contributed by Tsz Wo Nicholas Sze.


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

Branch: refs/heads/YARN-3409
Commit: 3c4fbc635e8ed81cfbec00793a3767bb47f6d176
Parents: 3d3158c
Author: Nanda kumar <na...@apache.org>
Authored: Wed Jul 25 20:27:03 2018 +0530
Committer: Nanda kumar <na...@apache.org>
Committed: Wed Jul 25 20:27:03 2018 +0530

----------------------------------------------------------------------
 .../common/impl/OpenContainerBlockMap.java      | 139 +++++++------------
 .../container/keyvalue/KeyValueHandler.java     |  10 +-
 .../common/impl/TestCloseContainerHandler.java  |  30 ++--
 3 files changed, 73 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c4fbc63/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
index ab5f861..6a93c9d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
@@ -21,22 +21,52 @@ package org.apache.hadoop.ozone.container.common.impl;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.stream.Collectors;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
 
 /**
+ * Map: containerId -> (localId -> KeyData).
+ * The outer container map does not entail locking for a better performance.
+ * The inner {@link KeyDataMap} is synchronized.
+ *
  * This class will maintain list of open keys per container when closeContainer
  * command comes, it should autocommit all open keys of a open container before
  * marking the container as closed.
  */
 public class OpenContainerBlockMap {
+  /**
+   * Map: localId -> KeyData.
+   *
+   * In order to support {@link #getAll()}, the update operations are synchronized.
+   */
+  static class KeyDataMap {
+    private final ConcurrentMap<Long, KeyData> blocks = new ConcurrentHashMap<>();
+
+    KeyData get(long localId) {
+      return blocks.get(localId);
+    }
+
+    synchronized int removeAndGetSize(long localId) {
+      blocks.remove(localId);
+      return blocks.size();
+    }
+
+    synchronized KeyData computeIfAbsent(long localId, Function<Long, KeyData> f) {
+      return blocks.computeIfAbsent(localId, f);
+    }
+
+    synchronized List<KeyData> getAll() {
+      return new ArrayList<>(blocks.values());
+    }
+  }
 
   /**
    * TODO : We may construct the openBlockMap by reading the Block Layout
@@ -46,89 +76,36 @@ public class OpenContainerBlockMap {
    *
    * For now, we will track all open blocks of a container in the blockMap.
    */
-  private final ConcurrentHashMap<Long, HashMap<Long, KeyData>>
-      openContainerBlockMap;
+  private final ConcurrentMap<Long, KeyDataMap> containers = new ConcurrentHashMap<>();
 
   /**
-   * Constructs OpenContainerBlockMap.
-   */
-  public OpenContainerBlockMap() {
-     openContainerBlockMap = new ConcurrentHashMap<>();
-  }
-  /**
    * Removes the Container matching with specified containerId.
    * @param containerId containerId
    */
   public void removeContainer(long containerId) {
     Preconditions
         .checkState(containerId >= 0, "Container Id cannot be negative.");
-    openContainerBlockMap.computeIfPresent(containerId, (k, v) -> null);
-  }
-
-  /**
-   * updates the chunkInfoList in case chunk is added or deleted
-   * @param blockID id of the block.
-   * @param info - Chunk Info
-   * @param remove if true, deletes the chunkInfo list otherwise appends to the
-   *               chunkInfo List
-   * @throws IOException
-   */
-  public synchronized void updateOpenKeyMap(BlockID blockID,
-      ContainerProtos.ChunkInfo info, boolean remove) throws IOException {
-    if (remove) {
-      deleteChunkFromMap(blockID, info);
-    } else {
-      addChunkToMap(blockID, info);
-    }
+    containers.remove(containerId);
   }
 
-  private KeyData getKeyData(ContainerProtos.ChunkInfo info, BlockID blockID)
-      throws IOException {
-    KeyData keyData = new KeyData(blockID);
-    keyData.addMetadata("TYPE", "KEY");
-    keyData.addChunk(info);
-    return keyData;
-  }
-
-  private void addChunkToMap(BlockID blockID, ContainerProtos.ChunkInfo info)
-      throws IOException {
+  public void addChunk(BlockID blockID, ChunkInfo info) {
     Preconditions.checkNotNull(info);
-    long containerId = blockID.getContainerID();
-    long localID = blockID.getLocalID();
-
-    KeyData keyData = openContainerBlockMap.computeIfAbsent(containerId,
-        emptyMap -> new LinkedHashMap<Long, KeyData>())
-        .putIfAbsent(localID, getKeyData(info, blockID));
-    // KeyData != null means the block already exist
-    if (keyData != null) {
-      HashMap<Long, KeyData> keyDataSet =
-          openContainerBlockMap.get(containerId);
-      keyDataSet.putIfAbsent(blockID.getLocalID(), getKeyData(info, blockID));
-      keyDataSet.computeIfPresent(blockID.getLocalID(), (key, value) -> {
-        value.addChunk(info);
-        return value;
-      });
-    }
+    containers.computeIfAbsent(blockID.getContainerID(), id -> new KeyDataMap())
+        .computeIfAbsent(blockID.getLocalID(), id -> new KeyData(blockID))
+        .addChunk(info);
   }
 
   /**
-   * removes the chunks from the chunkInfo list for the given block.
+   * Removes the chunk from the chunkInfo list for the given block.
    * @param blockID id of the block
    * @param chunkInfo chunk info.
    */
-  private synchronized void deleteChunkFromMap(BlockID blockID,
-      ContainerProtos.ChunkInfo chunkInfo) {
+  public void removeChunk(BlockID blockID, ChunkInfo chunkInfo) {
     Preconditions.checkNotNull(chunkInfo);
     Preconditions.checkNotNull(blockID);
-    HashMap<Long, KeyData> keyDataMap =
-        openContainerBlockMap.get(blockID.getContainerID());
-    if (keyDataMap != null) {
-      long localId = blockID.getLocalID();
-      KeyData keyData = keyDataMap.get(localId);
-      if (keyData != null) {
-        keyData.removeChunk(chunkInfo);
-      }
-    }
+    Optional.ofNullable(containers.get(blockID.getContainerID()))
+        .map(blocks -> blocks.get(blockID.getLocalID()))
+        .ifPresent(keyData -> keyData.removeChunk(chunkInfo));
   }
 
   /**
@@ -137,31 +114,23 @@ public class OpenContainerBlockMap {
    * @return List of open Keys(blocks)
    */
   public List<KeyData> getOpenKeys(long containerId) {
-    HashMap<Long, KeyData> keyDataHashMap =
-        openContainerBlockMap.get(containerId);
-    return keyDataHashMap == null ? null :
-        keyDataHashMap.values().stream().collect(Collectors.toList());
+    return Optional.ofNullable(containers.get(containerId))
+        .map(KeyDataMap::getAll)
+        .orElseGet(Collections::emptyList);
   }
 
   /**
    * removes the block from the block map.
    * @param blockID
    */
-  public synchronized void removeFromKeyMap(BlockID blockID) {
+  public void removeFromKeyMap(BlockID blockID) {
     Preconditions.checkNotNull(blockID);
-    HashMap<Long, KeyData> keyDataMap =
-        openContainerBlockMap.get(blockID.getContainerID());
-    if (keyDataMap != null) {
-      keyDataMap.remove(blockID.getLocalID());
-      if (keyDataMap.size() == 0) {
-        removeContainer(blockID.getContainerID());
-      }
-    }
+    containers.computeIfPresent(blockID.getContainerID(), (containerId, blocks)
+        -> blocks.removeAndGetSize(blockID.getLocalID()) == 0? null: blocks);
   }
 
   @VisibleForTesting
-  public ConcurrentHashMap<Long,
-      HashMap<Long, KeyData>> getContainerOpenKeyMap() {
-    return openContainerBlockMap;
+  KeyDataMap getKeyDataMap(long containerId) {
+    return containers.get(containerId);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c4fbc63/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 4123dc8..b08e128 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -435,10 +435,8 @@ public class KeyValueHandler extends Handler {
     long containerId = kvContainer.getContainerData().getContainerID();
     List<KeyData> pendingKeys =
         this.openContainerBlockMap.getOpenKeys(containerId);
-    if (pendingKeys != null) {
-      for (KeyData keyData : pendingKeys) {
-        commitKey(keyData, kvContainer);
-      }
+    for(KeyData keyData : pendingKeys) {
+      commitKey(keyData, kvContainer);
     }
   }
 
@@ -598,7 +596,7 @@ public class KeyValueHandler extends Handler {
       Preconditions.checkNotNull(chunkInfo);
 
       chunkManager.deleteChunk(kvContainer, blockID, chunkInfo);
-      openContainerBlockMap.updateOpenKeyMap(blockID, chunkInfoProto, true);
+      openContainerBlockMap.removeChunk(blockID, chunkInfoProto);
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, request);
     } catch (IOException ex) {
@@ -648,7 +646,7 @@ public class KeyValueHandler extends Handler {
             .getChunkData().getLen());
         // the openContainerBlockMap should be updated only while writing data
         // not during COMMIT_STAGE of handling write chunk request.
-        openContainerBlockMap.updateOpenKeyMap(blockID, chunkInfoProto, false);
+        openContainerBlockMap.addChunk(blockID, chunkInfoProto);
       }
     } catch (StorageContainerException ex) {
       return ContainerUtils.logAndReturnError(LOG, ex, request);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c4fbc63/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
index 3ab593e..6d1c086 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestCloseContainerHandler.java
@@ -162,9 +162,9 @@ public class TestCloseContainerHandler {
     Pipeline pipeline = createSingleNodePipeline();
     List<ChunkInfo> chunkList = writeChunkBuilder(blockID, pipeline, 3);
     // the key should exist in the map
-    Assert.assertTrue(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID)
-            .containsKey(blockID.getLocalID()));
+    Assert.assertNotNull(
+        openContainerBlockMap.getKeyDataMap(testContainerID)
+            .get(blockID.getLocalID()));
     KeyData keyData = new KeyData(blockID);
     List<ContainerProtos.ChunkInfo> chunkProtoList = new LinkedList<>();
     for (ChunkInfo i : chunkList) {
@@ -184,7 +184,7 @@ public class TestCloseContainerHandler {
 
     //the open key should be removed from Map
     Assert.assertNull(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID));
+        openContainerBlockMap.getKeyDataMap(testContainerID));
   }
 
   @Test
@@ -196,11 +196,11 @@ public class TestCloseContainerHandler {
     Pipeline pipeline = createSingleNodePipeline();
     List<ChunkInfo> chunkList = writeChunkBuilder(blockID, pipeline, 3);
     // the key should exist in the map
+    Assert.assertNotNull(
+        openContainerBlockMap.getKeyDataMap(testContainerID)
+            .get(blockID.getLocalID()));
     Assert.assertTrue(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID)
-            .containsKey(blockID.getLocalID()));
-    Assert.assertTrue(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID)
+        openContainerBlockMap.getKeyDataMap(testContainerID)
             .get(blockID.getLocalID()).getChunks().size() == 3);
     ContainerProtos.DeleteChunkRequestProto.Builder deleteChunkProto =
         ContainerProtos.DeleteChunkRequestProto.newBuilder();
@@ -219,7 +219,7 @@ public class TestCloseContainerHandler {
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
     dispatcher.dispatch(request.build());
     Assert.assertTrue(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID)
+        openContainerBlockMap.getKeyDataMap(testContainerID)
             .get(blockID.getLocalID()).getChunks().size() == 2);
 
   }
@@ -234,12 +234,12 @@ public class TestCloseContainerHandler {
     List<ChunkInfo> chunkList = writeChunkBuilder(blockID, pipeline, 3);
 
     Container container = containerSet.getContainer(testContainerID);
-    KeyData keyData = openContainerBlockMap.getContainerOpenKeyMap().
-        get(testContainerID).get(blockID.getLocalID());
+    KeyData keyData = openContainerBlockMap.
+        getKeyDataMap(testContainerID).get(blockID.getLocalID());
     // the key should exist in the map
-    Assert.assertTrue(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID)
-            .containsKey(blockID.getLocalID()));
+    Assert.assertNotNull(
+        openContainerBlockMap.getKeyDataMap(testContainerID)
+            .get(blockID.getLocalID()));
     Assert.assertTrue(
         keyData.getChunks().size() == chunkList.size());
     ContainerProtos.CloseContainerRequestProto.Builder closeContainerProto =
@@ -253,7 +253,7 @@ public class TestCloseContainerHandler {
     request.setDatanodeUuid(pipeline.getLeader().getUuidString());
     dispatcher.dispatch(request.build());
     Assert.assertNull(
-        openContainerBlockMap.getContainerOpenKeyMap().get(testContainerID));
+        openContainerBlockMap.getKeyDataMap(testContainerID));
     // Make sure the key got committed
     Assert.assertNotNull(handler.getKeyManager().getKey(container, blockID));
   }


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


[04/50] [abbrv] hadoop git commit: YARN-8541. RM startup failure on recovery after user deletion. Contributed by Bibin A Chundatt.

Posted by su...@apache.org.
YARN-8541. RM startup failure on recovery after user deletion. 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/e673dd1d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e673dd1d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e673dd1d

Branch: refs/heads/YARN-3409
Commit: e673dd1d4d78b66e7b6705ec6dc3679d2347d704
Parents: cd0b9f1
Author: bibinchundatt <bi...@apache.org>
Authored: Tue Jul 24 18:36:49 2018 +0530
Committer: bibinchundatt <bi...@apache.org>
Committed: Tue Jul 24 23:26:59 2018 +0530

----------------------------------------------------------------------
 .../server/resourcemanager/RMAppManager.java    | 48 ++++++++++----------
 .../placement/PlacementManager.java             |  9 ----
 .../TestWorkPreservingRMRestart.java            | 48 ++++++++++++++++++++
 .../placement/TestPlacementManager.java         | 20 ++++----
 4 files changed, 80 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e673dd1d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.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/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index 3e64cfc..7011aaa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -364,17 +364,9 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       ApplicationSubmissionContext submissionContext, long submitTime,
       String user, boolean isRecovery, long startTime) throws YarnException {
 
-    ApplicationPlacementContext placementContext = null;
-    try {
-      placementContext = placeApplication(rmContext, submissionContext, user);
-    } catch (YarnException e) {
-      String msg =
-          "Failed to place application " + submissionContext.getApplicationId()
-              + " to queue and specified " + "queue is invalid : "
-              + submissionContext.getQueue();
-      LOG.error(msg, e);
-      throw e;
-    }
+    ApplicationPlacementContext placementContext =
+        placeApplication(rmContext.getQueuePlacementManager(),
+            submissionContext, user, isRecovery);
 
     // We only replace the queue when it's a new application
     if (!isRecovery) {
@@ -789,23 +781,31 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
   }
 
   @VisibleForTesting
-  ApplicationPlacementContext placeApplication(RMContext rmContext,
-      ApplicationSubmissionContext context, String user) throws YarnException {
+  ApplicationPlacementContext placeApplication(
+      PlacementManager placementManager, ApplicationSubmissionContext context,
+      String user, boolean isRecovery) throws YarnException {
     ApplicationPlacementContext placementContext = null;
-    PlacementManager placementManager = rmContext.getQueuePlacementManager();
-
     if (placementManager != null) {
-      placementContext = placementManager.placeApplication(context, user);
-    } else{
-      if ( context.getQueue() == null || context.getQueue().isEmpty()) {
-        final String msg = "Queue Placement Manager is not set. Cannot place "
-            + "application : " + context.getApplicationId() + " to queue and "
-            + "specified queue is invalid " + context.getQueue();
-        LOG.error(msg);
-        throw new YarnException(msg);
+      try {
+        placementContext = placementManager.placeApplication(context, user);
+      } catch (YarnException e) {
+        // Placement could also fail if the user doesn't exist in system
+        // skip if the user is not found during recovery.
+        if (isRecovery) {
+          LOG.warn("PlaceApplication failed,skipping on recovery of rm");
+          return placementContext;
+        }
+        throw e;
       }
     }
-
+    if (placementContext == null && (context.getQueue() == null) || context
+        .getQueue().isEmpty()) {
+      String msg = "Failed to place application " + context.getApplicationId()
+          + " to queue and specified " + "queue is invalid : " + context
+          .getQueue();
+      LOG.error(msg);
+      throw new YarnException(msg);
+    }
     return placementContext;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e673dd1d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.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/placement/PlacementManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.java
index 5fa7723..74cf7ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.java
@@ -70,15 +70,6 @@ public class PlacementManager {
         }
       }
 
-      // Failed to get where to place application
-      if (null == placement && null == asc.getQueue()) {
-        String msg = "Failed to place application " +
-            asc.getApplicationId() + " to queue and specified "
-            + "queue is invalid : " + asc.getQueue();
-        LOG.error(msg);
-        throw new YarnException(msg);
-      }
-
       return placement;
     } finally {
       readLock.unlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e673dd1d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.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/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
index 88c19a1..a821b0a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
@@ -39,8 +39,12 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart.TestSecurityMockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.placement
+    .ApplicationPlacementContext;
+import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationAttemptStateData;
@@ -105,6 +109,8 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler
 import static org.apache.hadoop.yarn.server.resourcemanager.webapp
     .RMWebServices.DEFAULT_QUEUE;
 import static org.junit.Assert.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -1555,6 +1561,48 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   }
 
   @Test(timeout = 30000)
+  public void testUnknownUserOnRecovery() throws Exception {
+
+    MockRM rm1 = new MockRM(conf);
+    rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
+    nm1.registerNode();
+
+    // create app and launch the UAM
+    RMApp app0 = rm1.submitApp(200, true);
+    MockAM am0 = MockRM.launchUAM(app0, rm1, nm1);
+    am0.registerAppAttempt();
+    rm1.killApp(app0.getApplicationId());
+    PlacementManager placementMgr = mock(PlacementManager.class);
+    doThrow(new YarnException("No groups for user")).when(placementMgr)
+        .placeApplication(any(ApplicationSubmissionContext.class),
+            any(String.class));
+    MockRM rm2 = new MockRM(conf, memStore) {
+      @Override
+      protected RMAppManager createRMAppManager() {
+        return new RMAppManager(this.rmContext, this.scheduler,
+            this.masterService, this.applicationACLsManager, conf) {
+          @Override
+          ApplicationPlacementContext placeApplication(
+              PlacementManager placementManager,
+              ApplicationSubmissionContext context, String user,
+              boolean isRecovery) throws YarnException {
+            return super
+                .placeApplication(placementMgr, context, user, isRecovery);
+          }
+        };
+      }
+    };
+    rm2.start();
+    RMApp recoveredApp =
+        rm2.getRMContext().getRMApps().get(app0.getApplicationId());
+    Assert.assertEquals(RMAppState.KILLED, recoveredApp.getState());
+  }
+
+  @Test(timeout = 30000)
   public void testDynamicAutoCreatedQueueRecoveryWithDefaultQueue()
       throws Exception {
     //if queue name is not specified, it should submit to 'default' queue

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e673dd1d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementManager.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/placement/TestPlacementManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementManager.java
index 13111be..db5cd60 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestPlacementManager.java
@@ -83,16 +83,11 @@ public class TestPlacementManager {
 
     ApplicationSubmissionContext asc = Records.newRecord(
         ApplicationSubmissionContext.class);
+    asc.setQueue(YarnConfiguration.DEFAULT_QUEUE_NAME);
     asc.setApplicationName(APP_NAME);
 
-    boolean caughtException = false;
-    try{
-      pm.placeApplication(asc, USER2);
-    } catch (Exception e) {
-      caughtException = true;
-    }
-    Assert.assertTrue(caughtException);
-
+    Assert.assertNull("Placement should be null",
+        pm.placeApplication(asc, USER2));
     QueueMappingEntity queueMappingEntity = new QueueMappingEntity(APP_NAME,
         USER1, PARENT_QUEUE);
 
@@ -100,12 +95,13 @@ public class TestPlacementManager {
         Arrays.asList(queueMappingEntity));
     queuePlacementRules.add(anRule);
     pm.updateRules(queuePlacementRules);
-    try{
-      pm.placeApplication(asc, USER2);
+    try {
+      ApplicationPlacementContext pc = pm.placeApplication(asc, USER2);
+      Assert.assertNotNull(pc);
     } catch (Exception e) {
-      caughtException = false;
+      e.printStackTrace();
+      Assert.fail("Exception not expected");
     }
-    Assert.assertFalse(caughtException);
   }
 
 }
\ No newline at end of file


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


[26/50] [abbrv] hadoop git commit: HADOOP-15593. Fixed NPE in UGI spawnAutoRenewalThreadForUserCreds. Contributed by Gabor Bota

Posted by su...@apache.org.
HADOOP-15593.  Fixed NPE in UGI spawnAutoRenewalThreadForUserCreds.
               Contributed by Gabor Bota


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

Branch: refs/heads/YARN-3409
Commit: 77721f39e26b630352a1f4087524a3fbd21ff06e
Parents: 40fad32
Author: Eric Yang <ey...@apache.org>
Authored: Thu Jul 26 18:35:36 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Thu Jul 26 18:35:36 2018 -0400

----------------------------------------------------------------------
 .../hadoop/security/UserGroupInformation.java   | 179 ++++++++++++-------
 .../security/TestUserGroupInformation.java      |  38 ++++
 2 files changed, 148 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/77721f39/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
index 29b9fea..6ce72edb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
@@ -40,6 +40,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Date;
 import java.util.EnumMap;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -851,81 +852,121 @@ public class UserGroupInformation {
     }
 
     //spawn thread only if we have kerb credentials
-    Thread t = new Thread(new Runnable() {
+    KerberosTicket tgt = getTGT();
+    if (tgt == null) {
+      return;
+    }
+    String cmd = conf.get("hadoop.kerberos.kinit.command", "kinit");
+    long nextRefresh = getRefreshTime(tgt);
+    Thread t =
+        new Thread(new AutoRenewalForUserCredsRunnable(tgt, cmd, nextRefresh));
+    t.setDaemon(true);
+    t.setName("TGT Renewer for " + getUserName());
+    t.start();
+  }
+
+  @VisibleForTesting
+  class AutoRenewalForUserCredsRunnable implements Runnable {
+    private KerberosTicket tgt;
+    private RetryPolicy rp;
+    private String kinitCmd;
+    private long nextRefresh;
+    private boolean runRenewalLoop = true;
+
+    AutoRenewalForUserCredsRunnable(KerberosTicket tgt, String kinitCmd,
+        long nextRefresh){
+      this.tgt = tgt;
+      this.kinitCmd = kinitCmd;
+      this.nextRefresh = nextRefresh;
+      this.rp = null;
+    }
+
+    public void setRunRenewalLoop(boolean runRenewalLoop) {
+      this.runRenewalLoop = runRenewalLoop;
+    }
 
-      @Override
-      public void run() {
-        String cmd = conf.get("hadoop.kerberos.kinit.command", "kinit");
-        KerberosTicket tgt = getTGT();
-        if (tgt == null) {
+    @Override
+    public void run() {
+      do {
+        try {
+          long now = Time.now();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Current time is " + now);
+            LOG.debug("Next refresh is " + nextRefresh);
+          }
+          if (now < nextRefresh) {
+            Thread.sleep(nextRefresh - now);
+          }
+          String output = Shell.execCommand(kinitCmd, "-R");
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Renewed ticket. kinit output: {}", output);
+          }
+          reloginFromTicketCache();
+          tgt = getTGT();
+          if (tgt == null) {
+            LOG.warn("No TGT after renewal. Aborting renew thread for " +
+                getUserName());
+            return;
+          }
+          nextRefresh = Math.max(getRefreshTime(tgt),
+              now + kerberosMinSecondsBeforeRelogin);
+          metrics.renewalFailures.set(0);
+          rp = null;
+        } catch (InterruptedException ie) {
+          LOG.warn("Terminating renewal thread");
           return;
-        }
-        long nextRefresh = getRefreshTime(tgt);
-        RetryPolicy rp = null;
-        while (true) {
+        } catch (IOException ie) {
+          metrics.renewalFailuresTotal.incr();
+          final long now = Time.now();
+
+          if (tgt.isDestroyed()) {
+            LOG.error("TGT is destroyed. Aborting renew thread for {}.",
+                getUserName());
+            return;
+          }
+
+          long tgtEndTime;
+          // As described in HADOOP-15593 we need to handle the case when
+          // tgt.getEndTime() throws NPE because of JDK issue JDK-8147772
+          // NPE is only possible if this issue is not fixed in the JDK
+          // currently used
           try {
-            long now = Time.now();
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Current time is " + now);
-              LOG.debug("Next refresh is " + nextRefresh);
-            }
-            if (now < nextRefresh) {
-              Thread.sleep(nextRefresh - now);
-            }
-            String output = Shell.execCommand(cmd, "-R");
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Renewed ticket. kinit output: {}", output);
-            }
-            reloginFromTicketCache();
-            tgt = getTGT();
-            if (tgt == null) {
-              LOG.warn("No TGT after renewal. Aborting renew thread for " +
-                  getUserName());
-              return;
-            }
-            nextRefresh = Math.max(getRefreshTime(tgt),
-              now + kerberosMinSecondsBeforeRelogin);
-            metrics.renewalFailures.set(0);
-            rp = null;
-          } catch (InterruptedException ie) {
-            LOG.warn("Terminating renewal thread");
+            tgtEndTime = tgt.getEndTime().getTime();
+          } catch (NullPointerException npe) {
+            LOG.error("NPE thrown while getting KerberosTicket endTime. "
+                + "Aborting renew thread for {}.", getUserName());
+            return;
+          }
+
+          LOG.warn("Exception encountered while running the renewal "
+                  + "command for {}. (TGT end time:{}, renewalFailures: {},"
+                  + "renewalFailuresTotal: {})", getUserName(), tgtEndTime,
+              metrics.renewalFailures.value(),
+              metrics.renewalFailuresTotal.value(), ie);
+          if (rp == null) {
+            // Use a dummy maxRetries to create the policy. The policy will
+            // only be used to get next retry time with exponential back-off.
+            // The final retry time will be later limited within the
+            // tgt endTime in getNextTgtRenewalTime.
+            rp = RetryPolicies.exponentialBackoffRetry(Long.SIZE - 2,
+                kerberosMinSecondsBeforeRelogin, TimeUnit.MILLISECONDS);
+          }
+          try {
+            nextRefresh = getNextTgtRenewalTime(tgtEndTime, now, rp);
+          } catch (Exception e) {
+            LOG.error("Exception when calculating next tgt renewal time", e);
+            return;
+          }
+          metrics.renewalFailures.incr();
+          // retry until close enough to tgt endTime.
+          if (now > nextRefresh) {
+            LOG.error("TGT is expired. Aborting renew thread for {}.",
+                getUserName());
             return;
-          } catch (IOException ie) {
-            metrics.renewalFailuresTotal.incr();
-            final long tgtEndTime = tgt.getEndTime().getTime();
-            LOG.warn("Exception encountered while running the renewal "
-                    + "command for {}. (TGT end time:{}, renewalFailures: {},"
-                    + "renewalFailuresTotal: {})", getUserName(), tgtEndTime,
-                metrics.renewalFailures, metrics.renewalFailuresTotal, ie);
-            final long now = Time.now();
-            if (rp == null) {
-              // Use a dummy maxRetries to create the policy. The policy will
-              // only be used to get next retry time with exponential back-off.
-              // The final retry time will be later limited within the
-              // tgt endTime in getNextTgtRenewalTime.
-              rp = RetryPolicies.exponentialBackoffRetry(Long.SIZE - 2,
-                  kerberosMinSecondsBeforeRelogin, TimeUnit.MILLISECONDS);
-            }
-            try {
-              nextRefresh = getNextTgtRenewalTime(tgtEndTime, now, rp);
-            } catch (Exception e) {
-              LOG.error("Exception when calculating next tgt renewal time", e);
-              return;
-            }
-            metrics.renewalFailures.incr();
-            // retry until close enough to tgt endTime.
-            if (now > nextRefresh) {
-              LOG.error("TGT is expired. Aborting renew thread for {}.",
-                  getUserName());
-              return;
-            }
           }
         }
-      }
-    });
-    t.setDaemon(true);
-    t.setName("TGT Renewer for " + getUserName());
-    t.start();
+      } while (runRenewalLoop);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77721f39/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
index 9477990..011e930 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
@@ -47,6 +47,7 @@ import org.slf4j.event.Level;
 
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.kerberos.KerberosTicket;
 import javax.security.auth.kerberos.KeyTab;
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.LoginContext;
@@ -61,6 +62,7 @@ import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.ConcurrentModificationException;
 import java.util.Date;
+import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.Set;
 import java.util.concurrent.Callable;
@@ -88,7 +90,10 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
 public class TestUserGroupInformation {
@@ -1211,4 +1216,37 @@ public class TestUserGroupInformation {
     barrier.await();
     assertSame(testUgi1.getSubject(), blockingLookup.get().getSubject());
   }
+
+  @Test
+  public void testKerberosTicketIsDestroyedChecked() throws Exception {
+    // Create UserGroupInformation
+    GenericTestUtils.setLogLevel(UserGroupInformation.LOG, Level.DEBUG);
+    Set<User> users = new HashSet<>();
+    users.add(new User("Foo"));
+    Subject subject =
+        new Subject(true, users, new HashSet<>(), new HashSet<>());
+    UserGroupInformation ugi = spy(new UserGroupInformation(subject));
+
+    // throw IOException in the middle of the autoRenewalForUserCreds
+    doThrow(new IOException()).when(ugi).reloginFromTicketCache();
+
+    // Create and destroy the KerberosTicket, so endTime will be null
+    Date d = new Date();
+    KerberosPrincipal kp = new KerberosPrincipal("Foo");
+    KerberosTicket tgt = spy(new KerberosTicket(new byte[]{}, kp, kp, new
+        byte[]{}, 0, null, d, d, d, d, null));
+    tgt.destroy();
+
+    // run AutoRenewalForUserCredsRunnable with this
+    UserGroupInformation.AutoRenewalForUserCredsRunnable userCredsRunnable =
+        ugi.new AutoRenewalForUserCredsRunnable(tgt,
+            Boolean.toString(Boolean.TRUE), 100);
+
+    // Set the runnable to not to run in a loop
+    userCredsRunnable.setRunRenewalLoop(false);
+    // there should be no exception when calling this
+    userCredsRunnable.run();
+    // isDestroyed should be called at least once
+    Mockito.verify(tgt, atLeastOnce()).isDestroyed();
+  }
 }


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


[40/50] [abbrv] 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/ae3b9099
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ae3b9099
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ae3b9099

Branch: refs/heads/YARN-3409
Commit: ae3b9099dec154a1b698971e4ceb02a5692b154c
Parents: 823000d
Author: bibinchundatt <bi...@apache.org>
Authored: Thu Apr 5 14:09:50 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:41:51 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/ae3b9099/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


[18/50] [abbrv] hadoop git commit: HDFS-11060. make DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED configurable. Contributed by Lantao Jin.

Posted by su...@apache.org.
HDFS-11060. make DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED configurable. Contributed by Lantao Jin.


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

Branch: refs/heads/YARN-3409
Commit: e95c5e9f62452ee848875ec2f8642eab4992cd23
Parents: 9485c9a
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Wed Jul 25 11:04:18 2018 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Wed Jul 25 11:04:18 2018 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java | 3 +++
 .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java    | 8 ++++++--
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml         | 9 +++++++++
 .../hdfs/server/namenode/TestListCorruptFileBlocks.java     | 2 +-
 4 files changed, 19 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e95c5e9f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index cc902b0..5a1266c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -241,6 +241,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_DEFAULT
       = 1;
 
+  public static final String  DFS_NAMENODE_MAX_CORRUPT_FILE_BLOCKS_RETURNED_KEY = "dfs.namenode.max-corrupt-file-blocks-returned";
+  public static final int     DFS_NAMENODE_MAX_CORRUPT_FILE_BLOCKS_RETURNED_DEFAULT = 100;
+
   public static final String  DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY =
       HdfsClientConfigKeys.DeprecatedKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY;
   public static final int     DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e95c5e9f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 66bc567..8c95f7d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -425,7 +425,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   public static final Log auditLog = LogFactory.getLog(
       FSNamesystem.class.getName() + ".audit");
 
-  static final int DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED = 100;
+  private final int maxCorruptFileBlocksReturn;
   static int BLOCK_DELETION_INCREMENT = 1000;
   private final boolean isPermissionEnabled;
   private final UserGroupInformation fsOwner;
@@ -831,6 +831,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFSConfigKeys.DFS_NAMENODE_FILE_CLOSE_NUM_COMMITTED_ALLOWED_KEY,
           DFSConfigKeys.DFS_NAMENODE_FILE_CLOSE_NUM_COMMITTED_ALLOWED_DEFAULT);
 
+      this.maxCorruptFileBlocksReturn = conf.getInt(
+          DFSConfigKeys.DFS_NAMENODE_MAX_CORRUPT_FILE_BLOCKS_RETURNED_KEY,
+          DFSConfigKeys.DFS_NAMENODE_MAX_CORRUPT_FILE_BLOCKS_RETURNED_DEFAULT);
+
       this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
       
       this.standbyShouldCheckpoint = conf.getBoolean(
@@ -5508,7 +5512,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           if (src.startsWith(path)){
             corruptFiles.add(new CorruptFileBlockInfo(src, blk));
             count++;
-            if (count >= DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED)
+            if (count >= maxCorruptFileBlocksReturn)
               break;
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e95c5e9f/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 384cedf..a10be27 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -616,6 +616,15 @@
 </property>
 
 <property>
+  <name>dfs.namenode.max-corrupt-file-blocks-returned</name>
+  <value>100</value>
+  <description>
+      The maximum number of corrupt file blocks listed by NameNode Web UI,
+      JMX and other client request.
+  </description>
+</property>
+
+<property>
   <name>dfs.blocksize</name>
   <value>134217728</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e95c5e9f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java
index 1f31bdc..e1c8ae3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java
@@ -452,7 +452,7 @@ public class TestListCorruptFileBlocks {
       cluster = new MiniDFSCluster.Builder(conf).build();
       FileSystem fs = cluster.getFileSystem();
       final int maxCorruptFileBlocks = 
-        FSNamesystem.DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED;
+        conf.getInt(DFSConfigKeys.DFS_NAMENODE_MAX_CORRUPT_FILE_BLOCKS_RETURNED_KEY, 100);
 
       // create 110 files with one block each
       DFSTestUtil util = new DFSTestUtil.Builder().setName("testMaxCorruptFiles").


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


[05/50] [abbrv] hadoop git commit: YARN-7133. Clean up lock-try order in fair scheduler. (Szilard Nemeth via Haibo Chen)

Posted by su...@apache.org.
YARN-7133. Clean up lock-try order in fair scheduler. (Szilard Nemeth via Haibo Chen)


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

Branch: refs/heads/YARN-3409
Commit: ea2c6c8c9a55813a19b3dbd0d29747d6a7739030
Parents: e673dd1
Author: Haibo Chen <ha...@apache.org>
Authored: Tue Jul 24 12:46:15 2018 -0700
Committer: Haibo Chen <ha...@apache.org>
Committed: Tue Jul 24 12:46:59 2018 -0700

----------------------------------------------------------------------
 .../scheduler/fair/FairScheduler.java           | 36 ++++++++++----------
 1 file changed, 18 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2c6c8c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.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/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index eb9f6af..20d1afe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -467,8 +467,8 @@ public class FairScheduler extends
       return;
     }
 
+    writeLock.lock();
     try {
-      writeLock.lock();
       RMApp rmApp = rmContext.getRMApps().get(applicationId);
       FSLeafQueue queue = assignToQueue(rmApp, queueName, user);
       if (queue == null) {
@@ -550,8 +550,8 @@ public class FairScheduler extends
       ApplicationAttemptId applicationAttemptId,
       boolean transferStateFromPreviousAttempt,
       boolean isAttemptRecovering) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       SchedulerApplication<FSAppAttempt> application = applications.get(
           applicationAttemptId.getApplicationId());
       String user = application.getUser();
@@ -653,8 +653,8 @@ public class FairScheduler extends
   private void removeApplicationAttempt(
       ApplicationAttemptId applicationAttemptId,
       RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       LOG.info("Application " + applicationAttemptId + " is done. finalState="
               + rmAppAttemptFinalState);
       FSAppAttempt attempt = getApplicationAttempt(applicationAttemptId);
@@ -720,8 +720,8 @@ public class FairScheduler extends
   protected void completedContainerInternal(
       RMContainer rmContainer, ContainerStatus containerStatus,
       RMContainerEventType event) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       Container container = rmContainer.getContainer();
 
       // Get the application for the finished container
@@ -768,8 +768,8 @@ public class FairScheduler extends
 
   private void addNode(List<NMContainerStatus> containerReports,
       RMNode node) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       FSSchedulerNode schedulerNode = new FSSchedulerNode(node,
           usePortForNodeName);
       nodeTracker.addNode(schedulerNode);
@@ -790,8 +790,8 @@ public class FairScheduler extends
   }
 
   private void removeNode(RMNode rmNode) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       NodeId nodeId = rmNode.getNodeID();
       FSSchedulerNode node = nodeTracker.getNode(nodeId);
       if (node == null) {
@@ -988,8 +988,8 @@ public class FairScheduler extends
 
   @Override
   protected void nodeUpdate(RMNode nm) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       long start = getClock().getTime();
       super.nodeUpdate(nm);
 
@@ -1089,8 +1089,8 @@ public class FairScheduler extends
 
   @VisibleForTesting
   void attemptScheduling(FSSchedulerNode node) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       if (rmContext.isWorkPreservingRecoveryEnabled() && !rmContext
           .isSchedulerReadyForAllocatingContainers()) {
         return;
@@ -1305,8 +1305,8 @@ public class FairScheduler extends
   private String resolveReservationQueueName(String queueName,
       ApplicationId applicationId, ReservationId reservationID,
       boolean isRecovering) {
+    readLock.lock();
     try {
-      readLock.lock();
       FSQueue queue = queueMgr.getQueue(queueName);
       if ((queue == null) || !allocConf.isReservable(queue.getQueueName())) {
         return queueName;
@@ -1372,8 +1372,8 @@ public class FairScheduler extends
 
   @SuppressWarnings("deprecation")
   private void initScheduler(Configuration conf) throws IOException {
+    writeLock.lock();
     try {
-      writeLock.lock();
       this.conf = new FairSchedulerConfiguration(conf);
       validateConf(this.conf);
       authorizer = YarnAuthorizationProvider.getInstance(conf);
@@ -1464,8 +1464,8 @@ public class FairScheduler extends
   }
 
   private void startSchedulerThreads() {
+    writeLock.lock();
     try {
-      writeLock.lock();
       Preconditions.checkNotNull(allocsLoader, "allocsLoader is null");
       if (continuousSchedulingEnabled) {
         Preconditions.checkNotNull(schedulingThread,
@@ -1499,8 +1499,8 @@ public class FairScheduler extends
   @SuppressWarnings("deprecation")
   @Override
   public void serviceStop() throws Exception {
+    writeLock.lock();
     try {
-      writeLock.lock();
       if (continuousSchedulingEnabled) {
         if (schedulingThread != null) {
           schedulingThread.interrupt();
@@ -1562,8 +1562,8 @@ public class FairScheduler extends
   @Override
   public boolean checkAccess(UserGroupInformation callerUGI,
       QueueACL acl, String queueName) {
+    readLock.lock();
     try {
-      readLock.lock();
       FSQueue queue = getQueueManager().getQueue(queueName);
       if (queue == null) {
         if (LOG.isDebugEnabled()) {
@@ -1691,8 +1691,8 @@ public class FairScheduler extends
   @Override
   public String moveApplication(ApplicationId appId,
       String queueName) throws YarnException {
+    writeLock.lock();
     try {
-      writeLock.lock();
       SchedulerApplication<FSAppAttempt> app = applications.get(appId);
       if (app == null) {
         throw new YarnException("App to be moved " + appId + " not found.");
@@ -1700,8 +1700,8 @@ public class FairScheduler extends
       FSAppAttempt attempt = (FSAppAttempt) app.getCurrentAppAttempt();
       // To serialize with FairScheduler#allocate, synchronize on app attempt
 
+      attempt.getWriteLock().lock();
       try {
-        attempt.getWriteLock().lock();
         FSLeafQueue oldQueue = (FSLeafQueue) app.getQueue();
         // Check if the attempt is already stopped: don't move stopped app
         // attempt. The attempt has already been removed from all queues.
@@ -1737,8 +1737,8 @@ public class FairScheduler extends
   @Override
   public void preValidateMoveApplication(ApplicationId appId, String newQueue)
       throws YarnException {
+    writeLock.lock();
     try {
-      writeLock.lock();
       SchedulerApplication<FSAppAttempt> app = applications.get(appId);
       if (app == null) {
         throw new YarnException("App to be moved " + appId + " not found.");
@@ -1747,8 +1747,8 @@ public class FairScheduler extends
       FSAppAttempt attempt = app.getCurrentAppAttempt();
       // To serialize with FairScheduler#allocate, synchronize on app attempt
 
+      attempt.getWriteLock().lock();
       try {
-        attempt.getWriteLock().lock();
         FSLeafQueue oldQueue = (FSLeafQueue) app.getQueue();
         String destQueueName = handleMoveToPlanQueue(newQueue);
         FSLeafQueue targetQueue = queueMgr.getLeafQueue(destQueueName, false);
@@ -1869,8 +1869,8 @@ public class FairScheduler extends
   @Override
   public void updateNodeResource(RMNode nm,
       ResourceOption resourceOption) {
+    writeLock.lock();
     try {
-      writeLock.lock();
       super.updateNodeResource(nm, resourceOption);
       updateRootQueueMetrics();
       queueMgr.getRootQueue().setSteadyFairShare(getClusterResource());


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


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

Branch: refs/heads/YARN-3409
Commit: acd7729af570ae2c59a547709d510bf20b0b1246
Parents: 649e014
Author: Naganarasimha <na...@apache.org>
Authored: Thu Jun 28 08:13:09 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 17:01:44 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 -
 .../nodelabels/NodeAttributesManagerImpl.java   |   2 +-
 .../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 +-
 .../clientrm/FederationClientInterceptor.java   |   6 +-
 25 files changed, 1053 insertions(+), 456 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/acd7729a/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/acd7729a/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/acd7729a/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/acd7729a/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/acd7729a/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/acd7729a/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/acd7729a/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/acd7729a/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/acd7729a/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/acd7729a/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/acd7729a/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 6b823b2..3623908 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;
@@ -1543,8 +1545,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);
@@ -1567,6 +1569,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();
@@ -1603,6 +1607,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();
@@ -1615,8 +1620,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);
@@ -1639,6 +1644,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();
@@ -2048,18 +2054,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++) {
@@ -2081,6 +2089,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/acd7729a/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


[42/50] [abbrv] 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/9e1fb743
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9e1fb743
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9e1fb743

Branch: refs/heads/YARN-3409
Commit: 9e1fb743fcdae30871a109c58df8c1a5326a907e
Parents: ae3b909
Author: Sunil G <su...@apache.org>
Authored: Fri Apr 6 07:09:27 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:42:38 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/9e1fb743/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 ebd789f..b43ff5a 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
@@ -3458,6 +3458,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/9e1fb743/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/9e1fb743/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/9e1fb743/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/9e1fb743/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/9e1fb743/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/9e1fb743/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/9e1fb743/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/9e1fb743/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/9e1fb743/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/9e1fb743/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/9e1fb743/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/9e1fb743/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 3f86266..22f77af 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
@@ -3949,4 +3949,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/9e1fb743/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/9e1fb743/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/9e1fb743/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/9e1fb743/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/9e1fb743/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


[25/50] [abbrv] hadoop git commit: YARN-8545. Return allocated resource to RM for failed container. Contributed by Chandni Singh

Posted by su...@apache.org.
YARN-8545.  Return allocated resource to RM for failed container.
            Contributed by Chandni Singh


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

Branch: refs/heads/YARN-3409
Commit: 40fad32824d2f8f960c779d78357e62103453da0
Parents: d70d845
Author: Eric Yang <ey...@apache.org>
Authored: Thu Jul 26 18:22:57 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Thu Jul 26 18:22:57 2018 -0400

----------------------------------------------------------------------
 .../hadoop/yarn/service/ServiceScheduler.java   |  3 +-
 .../yarn/service/component/Component.java       | 42 +++++++++++---------
 .../component/instance/ComponentInstance.java   | 21 +++++++---
 .../instance/ComponentInstanceEvent.java        |  2 +
 .../containerlaunch/ContainerLaunchService.java | 12 ++++--
 .../hadoop/yarn/service/MockServiceAM.java      | 34 +++++++++++++++-
 .../hadoop/yarn/service/TestServiceAM.java      | 35 ++++++++++++++++
 .../yarn/service/component/TestComponent.java   |  3 +-
 .../instance/TestComponentInstance.java         | 26 ++++++------
 9 files changed, 135 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
index d3e8e4f..cfaf356 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
@@ -687,7 +687,8 @@ public class ServiceScheduler extends CompositeService {
         }
         ComponentEvent event =
             new ComponentEvent(instance.getCompName(), CONTAINER_COMPLETED)
-                .setStatus(status).setInstance(instance);
+                .setStatus(status).setInstance(instance)
+                .setContainerId(containerId);
         dispatcher.getEventHandler().handle(event);
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
index a1ee796..aaa23da 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.service.component;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
@@ -518,10 +519,10 @@ public class Component implements EventHandler<ComponentEvent> {
   private static class ContainerCompletedTransition extends BaseTransition {
     @Override
     public void transition(Component component, ComponentEvent event) {
-
+      Preconditions.checkNotNull(event.getContainerId());
       component.updateMetrics(event.getStatus());
       component.dispatcher.getEventHandler().handle(
-          new ComponentInstanceEvent(event.getStatus().getContainerId(), STOP)
+          new ComponentInstanceEvent(event.getContainerId(), STOP)
               .setStatus(event.getStatus()));
 
       ComponentRestartPolicy restartPolicy =
@@ -784,28 +785,33 @@ public class Component implements EventHandler<ComponentEvent> {
   }
 
   private void updateMetrics(ContainerStatus status) {
-    switch (status.getExitStatus()) {
-    case SUCCESS:
-      componentMetrics.containersSucceeded.incr();
-      scheduler.getServiceMetrics().containersSucceeded.incr();
-      return;
-    case PREEMPTED:
-      componentMetrics.containersPreempted.incr();
-      scheduler.getServiceMetrics().containersPreempted.incr();
-      break;
-    case DISKS_FAILED:
-      componentMetrics.containersDiskFailure.incr();
-      scheduler.getServiceMetrics().containersDiskFailure.incr();
-      break;
-    default:
-      break;
+    //when a container preparation fails while building launch context, then
+    //the container status may not exist.
+    if (status != null) {
+      switch (status.getExitStatus()) {
+        case SUCCESS:
+          componentMetrics.containersSucceeded.incr();
+          scheduler.getServiceMetrics().containersSucceeded.incr();
+          return;
+        case PREEMPTED:
+          componentMetrics.containersPreempted.incr();
+          scheduler.getServiceMetrics().containersPreempted.incr();
+          break;
+        case DISKS_FAILED:
+          componentMetrics.containersDiskFailure.incr();
+          scheduler.getServiceMetrics().containersDiskFailure.incr();
+          break;
+        default:
+          break;
+      }
     }
 
     // containersFailed include preempted, disks_failed etc.
     componentMetrics.containersFailed.incr();
     scheduler.getServiceMetrics().containersFailed.incr();
 
-    if (Apps.shouldCountTowardsNodeBlacklisting(status.getExitStatus())) {
+    if (status != null && Apps.shouldCountTowardsNodeBlacklisting(
+        status.getExitStatus())) {
       String host = scheduler.getLiveInstances().get(status.getContainerId())
           .getNodeId().getHost();
       failureTracker.incNodeFailure(host);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
index 64f35d3..3499d92 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
@@ -76,6 +76,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
     Comparable<ComponentInstance> {
   private static final Logger LOG =
       LoggerFactory.getLogger(ComponentInstance.class);
+  private static final String FAILED_BEFORE_LAUNCH_DIAG =
+      "failed before launch";
 
   private  StateMachine<ComponentInstanceState, ComponentInstanceEventType,
       ComponentInstanceEvent> stateMachine;
@@ -241,7 +243,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
 
   @VisibleForTesting
   static void handleComponentInstanceRelaunch(
-      ComponentInstance compInstance, ComponentInstanceEvent event) {
+      ComponentInstance compInstance, ComponentInstanceEvent event,
+      boolean failureBeforeLaunch) {
     Component comp = compInstance.getComponent();
 
     // Do we need to relaunch the service?
@@ -257,8 +260,10 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
               + ": {} completed. Reinsert back to pending list and requested " +
               "a new container." + System.lineSeparator() +
               " exitStatus={}, diagnostics={}.",
-          event.getContainerId(), event.getStatus().getExitStatus(),
-          event.getStatus().getDiagnostics());
+          event.getContainerId(), failureBeforeLaunch ? null :
+              event.getStatus().getExitStatus(),
+          failureBeforeLaunch ? FAILED_BEFORE_LAUNCH_DIAG :
+              event.getStatus().getDiagnostics());
     } else {
       // When no relaunch, update component's #succeeded/#failed
       // instances.
@@ -297,8 +302,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
 
       Component comp = compInstance.component;
       String containerDiag =
-          compInstance.getCompInstanceId() + ": " + event.getStatus()
-              .getDiagnostics();
+          compInstance.getCompInstanceId() + ": " + (failedBeforeLaunching ?
+              FAILED_BEFORE_LAUNCH_DIAG : event.getStatus().getDiagnostics());
       compInstance.diagnostics.append(containerDiag + System.lineSeparator());
       compInstance.cancelContainerStatusRetriever();
       if (compInstance.getState().equals(ComponentInstanceState.UPGRADING)) {
@@ -312,6 +317,9 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
       boolean shouldFailService = false;
 
       final ServiceScheduler scheduler = comp.getScheduler();
+      scheduler.getAmRMClient().releaseAssignedContainer(
+          event.getContainerId());
+
       // Check if it exceeds the failure threshold, but only if health threshold
       // monitor is not enabled
       if (!comp.isHealthThresholdMonitorEnabled()
@@ -352,7 +360,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
 
       // According to component restart policy, handle container restart
       // or finish the service (if all components finished)
-      handleComponentInstanceRelaunch(compInstance, event);
+      handleComponentInstanceRelaunch(compInstance, event,
+          failedBeforeLaunching);
 
       if (shouldFailService) {
         scheduler.getTerminationHandler().terminate(-1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstanceEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstanceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstanceEvent.java
index 707b034..889da6e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstanceEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstanceEvent.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.service.component.instance;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.event.AbstractEvent;
@@ -32,6 +33,7 @@ public class ComponentInstanceEvent
   public ComponentInstanceEvent(ContainerId containerId,
       ComponentInstanceEventType componentInstanceEventType) {
     super(componentInstanceEventType);
+    Preconditions.checkNotNull(containerId);
     this.id = containerId;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
index 084c721..f674e0d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
@@ -22,8 +22,11 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.service.ServiceContext;
 import org.apache.hadoop.yarn.service.api.records.Artifact;
+import org.apache.hadoop.yarn.service.component.ComponentEvent;
+import org.apache.hadoop.yarn.service.component.ComponentEventType;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
 import org.apache.hadoop.yarn.service.provider.ProviderService;
 import org.apache.hadoop.yarn.service.provider.ProviderFactory;
@@ -116,9 +119,12 @@ public class ContainerLaunchService extends AbstractService{
                   launcher.completeContainerLaunch(), true);
         }
       } catch (Exception e) {
-        LOG.error(instance.getCompInstanceId()
-            + ": Failed to launch container. ", e);
-
+        LOG.error("{}: Failed to launch container.",
+            instance.getCompInstanceId(), e);
+        ComponentEvent event = new ComponentEvent(instance.getCompName(),
+            ComponentEventType.CONTAINER_COMPLETED)
+            .setInstance(instance).setContainerId(container.getId());
+        context.scheduler.getDispatcher().getEventHandler().handle(event);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java
index 4a75aef..729287c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java
@@ -68,6 +68,7 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeoutException;
 
@@ -99,6 +100,8 @@ public class MockServiceAM extends ServiceMaster {
   private Map<ContainerId, ContainerStatus> containerStatuses =
       new ConcurrentHashMap<>();
 
+  private Set<ContainerId> releasedContainers = ConcurrentHashMap.newKeySet();
+
   private Credentials amCreds;
 
   public MockServiceAM(Service service) {
@@ -223,6 +226,13 @@ public class MockServiceAM extends ServiceMaster {
             return response;
           }
 
+          @Override
+          public synchronized void releaseAssignedContainer(
+              ContainerId containerId) {
+            releasedContainers.add(containerId);
+            super.releaseAssignedContainer(containerId);
+          }
+
           @Override public void unregisterApplicationMaster(
               FinalApplicationStatus appStatus, String appMessage,
               String appTrackingUrl) {
@@ -288,7 +298,7 @@ public class MockServiceAM extends ServiceMaster {
   }
 
   /**
-   *
+   * Creates a mock container and container ID and feeds to the component.
    * @param service The service for the component
    * @param id The id for the container
    * @param compName The component to which the container is fed
@@ -297,6 +307,18 @@ public class MockServiceAM extends ServiceMaster {
   public Container feedContainerToComp(Service service, int id,
       String compName) {
     ContainerId containerId = createContainerId(id);
+    return feedContainerToComp(service, containerId, compName);
+  }
+
+  /**
+   * Feeds the container to the component.
+   * @param service The service for the component
+   * @param containerId container id
+   * @param compName The component to which the container is fed
+   * @return
+   */
+  public Container feedContainerToComp(Service service, ContainerId containerId,
+      String compName) {
     Container container = createContainer(containerId, compName);
     synchronized (feedContainers) {
       feedContainers.add(container);
@@ -423,4 +445,14 @@ public class MockServiceAM extends ServiceMaster {
     }
     return ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
   }
+
+  /**
+   * Waits for the container to get released
+   * @param containerId           ContainerId
+   */
+  public void waitForContainerToRelease(ContainerId containerId)
+      throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(() -> releasedContainers.contains(containerId),
+        1000, 9990000);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java
index e9478f0..21e93fa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.service;
 
+import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableMap;
 import org.apache.commons.io.FileUtils;
 import org.apache.curator.test.TestingCluster;
@@ -391,4 +392,38 @@ public class TestServiceAM extends ServiceTestUtils{
         .equals("newer.host"), 2000, 200000);
     am.stop();
   }
+
+  // Test to verify that the containers are released and the
+  // component instance is added to the pending queue when building the launch
+  // context fails.
+  @Test(timeout = 9990000)
+  public void testContainersReleasedWhenPreLaunchFails()
+      throws Exception {
+    ApplicationId applicationId = ApplicationId.newInstance(
+        System.currentTimeMillis(), 1);
+    Service exampleApp = new Service();
+    exampleApp.setId(applicationId.toString());
+    exampleApp.setVersion("v1");
+    exampleApp.setName("testContainersReleasedWhenPreLaunchFails");
+
+    Component compA = createComponent("compa", 1, "pwd");
+    Artifact artifact = new Artifact();
+    artifact.setType(Artifact.TypeEnum.TARBALL);
+    compA.artifact(artifact);
+    exampleApp.addComponent(compA);
+
+    MockServiceAM am = new MockServiceAM(exampleApp);
+    am.init(conf);
+    am.start();
+
+    ContainerId containerId = am.createContainerId(1);
+
+    // allocate a container
+    am.feedContainerToComp(exampleApp, containerId, "compa");
+    am.waitForContainerToRelease(containerId);
+
+    Assert.assertEquals(1,
+        am.getComponent("compa").getPendingInstances().size());
+    am.stop();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java
index d5fb941..2e17c7f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java
@@ -178,7 +178,8 @@ public class TestComponent {
           org.apache.hadoop.yarn.api.records.ContainerState.COMPLETE,
           "successful", 0);
       comp.handle(new ComponentEvent(comp.getName(),
-          ComponentEventType.CONTAINER_COMPLETED).setStatus(containerStatus));
+          ComponentEventType.CONTAINER_COMPLETED).setStatus(containerStatus)
+          .setContainerId(instanceContainer.getId()));
       componentInstance.handle(
           new ComponentInstanceEvent(componentInstance.getContainer().getId(),
               ComponentInstanceEventType.STOP).setStatus(containerStatus));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40fad328/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java
index 0e7816c..bb480ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java
@@ -245,7 +245,7 @@ public class TestComponentInstance {
         comp.getAllComponentInstances().iterator().next();
 
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
 
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
@@ -262,7 +262,7 @@ public class TestComponentInstance {
     componentInstance = comp.getAllComponentInstances().iterator().next();
     containerStatus.setExitStatus(1);
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(1)).reInsertPendingInstance(
@@ -286,7 +286,7 @@ public class TestComponentInstance {
     when(comp.getNumSucceededInstances()).thenReturn(new Long(1));
 
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
     verify(comp, times(1)).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(0)).reInsertPendingInstance(
@@ -304,7 +304,7 @@ public class TestComponentInstance {
 
     when(comp.getNumFailedInstances()).thenReturn(new Long(1));
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
 
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, times(1)).markAsFailed(any(ComponentInstance.class));
@@ -323,7 +323,7 @@ public class TestComponentInstance {
     componentInstance = comp.getAllComponentInstances().iterator().next();
     containerStatus.setExitStatus(1);
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(1)).reInsertPendingInstance(
@@ -340,7 +340,7 @@ public class TestComponentInstance {
     componentInstance = comp.getAllComponentInstances().iterator().next();
     containerStatus.setExitStatus(1);
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, times(1)).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(0)).reInsertPendingInstance(
@@ -363,7 +363,7 @@ public class TestComponentInstance {
     containerStatus.setExitStatus(1);
     ComponentInstance commponentInstance = iter.next();
     ComponentInstance.handleComponentInstanceRelaunch(commponentInstance,
-        componentInstanceEvent);
+        componentInstanceEvent, false);
 
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
@@ -404,7 +404,7 @@ public class TestComponentInstance {
       when(component2Instance.getComponent().getNumFailedInstances())
           .thenReturn(new Long(failed2Instances.size()));
       ComponentInstance.handleComponentInstanceRelaunch(component2Instance,
-          componentInstanceEvent);
+          componentInstanceEvent, false);
     }
 
     Map<String, ComponentInstance> failed1Instances = new HashMap<>();
@@ -418,7 +418,7 @@ public class TestComponentInstance {
       when(component1Instance.getComponent().getNumFailedInstances())
           .thenReturn(new Long(failed1Instances.size()));
       ComponentInstance.handleComponentInstanceRelaunch(component1Instance,
-          componentInstanceEvent);
+          componentInstanceEvent, false);
     }
 
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
@@ -458,7 +458,7 @@ public class TestComponentInstance {
       when(component2Instance.getComponent().getNumSucceededInstances())
           .thenReturn(new Long(succeeded2Instances.size()));
       ComponentInstance.handleComponentInstanceRelaunch(component2Instance,
-          componentInstanceEvent);
+          componentInstanceEvent, false);
     }
 
     Map<String, ComponentInstance> succeeded1Instances = new HashMap<>();
@@ -471,7 +471,7 @@ public class TestComponentInstance {
       when(component1Instance.getComponent().getNumSucceededInstances())
           .thenReturn(new Long(succeeded1Instances.size()));
       ComponentInstance.handleComponentInstanceRelaunch(component1Instance,
-          componentInstanceEvent);
+          componentInstanceEvent, false);
     }
 
     verify(comp, times(2)).markAsSucceeded(any(ComponentInstance.class));
@@ -500,7 +500,7 @@ public class TestComponentInstance {
 
     for (ComponentInstance component2Instance : component2Instances) {
       ComponentInstance.handleComponentInstanceRelaunch(component2Instance,
-          componentInstanceEvent);
+          componentInstanceEvent, false);
     }
 
     succeeded1Instances = new HashMap<>();
@@ -511,7 +511,7 @@ public class TestComponentInstance {
       when(component1Instance.getComponent().getSucceededInstances())
           .thenReturn(succeeded1Instances.values());
       ComponentInstance.handleComponentInstanceRelaunch(component1Instance,
-          componentInstanceEvent);
+          componentInstanceEvent, false);
     }
 
     verify(comp, times(2)).markAsSucceeded(any(ComponentInstance.class));


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


[38/50] [abbrv] 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/dae7437f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dae7437f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dae7437f

Branch: refs/heads/YARN-3409
Commit: dae7437f0fa1d4221134d820d267f96dbc3e6134
Parents: c7809fb
Author: Sunil G <su...@apache.org>
Authored: Thu Mar 29 17:12:08 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:40:34 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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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/dae7437f/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


[14/50] [abbrv] hadoop git commit: YARN-8577. Fix the broken anchor in SLS site-doc. Contributed by Weiwei Yang.

Posted by su...@apache.org.
YARN-8577. Fix the broken anchor in SLS site-doc. 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/3d3158ce
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3d3158ce
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3d3158ce

Branch: refs/heads/YARN-3409
Commit: 3d3158cea4580eb2e3b2af635c3a7d30f4dbb873
Parents: b507f83e
Author: bibinchundatt <bi...@apache.org>
Authored: Wed Jul 25 16:19:14 2018 +0530
Committer: bibinchundatt <bi...@apache.org>
Committed: Wed Jul 25 18:59:05 2018 +0530

----------------------------------------------------------------------
 .../hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md         | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d3158ce/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
index 9df4998..e487267 100644
--- a/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
+++ b/hadoop-tools/hadoop-sls/src/site/markdown/SchedulerLoadSimulator.md
@@ -27,7 +27,7 @@ YARN Scheduler Load Simulator (SLS)
     * [Metrics](#Metrics)
         * [Real-time Tracking](#Real-time_Tracking)
         * [Offline Analysis](#Offline_Analysis)
-    * [Synthetic Load Generator](#SynthGen)
+    * [Synthetic Load Generator](#Synthetic_Load_Generator)
     * [Appendix](#Appendix)
         * [Resources](#Resources)
         * [SLS JSON input file format](#SLS_JSON_input_file_format)


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


[46/50] [abbrv] 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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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 67e1f38..328910f 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,29 +19,28 @@
 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.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.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 +57,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 +76,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 +150,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 +211,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 +226,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 +263,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 +280,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 +318,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 +330,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 +341,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 +357,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 +403,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 +437,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 +486,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 +498,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 +545,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 +674,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/4770a85d/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/4770a85d/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/4770a85d/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 be816d8..d024f2d 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;
@@ -76,6 +66,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;
@@ -86,6 +80,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;
@@ -115,10 +111,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;
@@ -140,6 +141,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;
@@ -161,7 +163,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;
@@ -172,11 +173,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 {
 
@@ -2044,11 +2045,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();
   }
@@ -2067,17 +2069,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));
@@ -2095,35 +2097,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/4770a85d/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/4770a85d/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/4770a85d/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],


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


[02/50] [abbrv] hadoop git commit: YARN-7748. TestContainerResizing.testIncreaseContainerUnreservedWhenApplicationCompleted fails due to multiple container fail events. Contributed by Weiwei Yang.

Posted by su...@apache.org.
YARN-7748. TestContainerResizing.testIncreaseContainerUnreservedWhenApplicationCompleted fails due to multiple container fail events. 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/35ce6eb1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/35ce6eb1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/35ce6eb1

Branch: refs/heads/YARN-3409
Commit: 35ce6eb1f526ce3db7e015fb1761eee15604100c
Parents: 773d312
Author: Sunil G <su...@apache.org>
Authored: Tue Jul 24 22:20:06 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Tue Jul 24 22:20:17 2018 +0530

----------------------------------------------------------------------
 .../scheduler/capacity/TestContainerResizing.java | 18 +++++++++++++-----
 1 file changed, 13 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ce6eb1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.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/TestContainerResizing.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/TestContainerResizing.java
index eacbf6e..307d5ae 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/TestContainerResizing.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/TestContainerResizing.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
@@ -58,7 +59,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica
     .FiCaSchedulerNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.CandidateNodeSet;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -740,11 +740,14 @@ public class TestContainerResizing {
   @Test
   public void testIncreaseContainerUnreservedWhenApplicationCompleted()
       throws Exception {
+    // Disable relaunch app attempt on failure, in order to check
+    // resource usages for current app only.
+    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
     /**
      * Similar to testIncreaseContainerUnreservedWhenContainerCompleted, when
      * application finishes, reserved increase container should be cancelled
      */
-    MockRM rm1 = new MockRM() {
+    MockRM rm1 = new MockRM(conf) {
       @Override
       public RMNodeLabelsManager createNodeLabelManager() {
         return mgr;
@@ -807,9 +810,14 @@ public class TestContainerResizing {
     Assert.assertEquals(6 * GB,
         app.getAppAttemptResourceUsage().getReserved().getMemorySize());
 
-    // Kill the application
-    cs.handle(new AppAttemptRemovedSchedulerEvent(am1.getApplicationAttemptId(),
-        RMAppAttemptState.KILLED, false));
+    // Kill the application by killing the AM container
+    ContainerId amContainer =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+    cs.killContainer(cs.getRMContainer(amContainer));
+    rm1.waitForState(am1.getApplicationAttemptId(),
+        RMAppAttemptState.FAILED);
+    rm1.waitForState(am1.getApplicationAttemptId().getApplicationId(),
+        RMAppState.FAILED);
 
     /* Check statuses after reservation satisfied */
     // Increase request should be unreserved


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


[10/50] [abbrv] hadoop git commit: HDDS-203. Add getCommittedBlockLength API in datanode. Contributed by Shashikant Banerjee.

Posted by su...@apache.org.
HDDS-203. Add getCommittedBlockLength API in datanode. Contributed by Shashikant Banerjee.


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

Branch: refs/heads/YARN-3409
Commit: 955f795101c1a0ae176b4cebda3ffbe9850dd687
Parents: 81d5950
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Wed Jul 25 14:15:54 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Wed Jul 25 14:15:54 2018 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hdds/client/BlockID.java  |  20 +-
 .../scm/storage/ContainerProtocolCalls.java     |  28 +++
 .../ozone/container/common/helpers/KeyData.java |  27 ++-
 .../main/proto/DatanodeContainerProtocol.proto  |  17 +-
 .../container/common/impl/HddsDispatcher.java   |   2 +
 .../container/keyvalue/KeyValueHandler.java     |  34 ++++
 .../container/keyvalue/helpers/KeyUtils.java    |  20 ++
 .../container/keyvalue/impl/KeyManagerImpl.java |  27 +++
 .../keyvalue/interfaces/KeyManager.java         |   7 +
 .../ozone/scm/TestCommittedBlockLengthAPI.java  | 191 +++++++++++++++++++
 10 files changed, 368 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
index 62b12e3..74e90e9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
@@ -20,8 +20,10 @@ import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 
+import java.util.Objects;
+
 /**
- * BlockID of ozone (containerID + localID)
+ * BlockID of ozone (containerID  localID)
  */
 public class BlockID {
   private long containerID;
@@ -68,4 +70,20 @@ public class BlockID {
         blockID.getLocalID());
   }
 
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    BlockID blockID = (BlockID) o;
+    return containerID == blockID.containerID && localID == blockID.localID;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(containerID, localID);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
index f4f14ef..36cdfc9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
@@ -99,6 +99,34 @@ public final class ContainerProtocolCalls  {
   }
 
   /**
+   * Calls the container protocol to get the length of a committed block.
+   *
+   * @param xceiverClient client to perform call
+   * @param blockID blockId for the Block
+   * @param traceID container protocol call args
+   * @return container protocol getLastCommittedBlockLength response
+   * @throws IOException if there is an I/O error while performing the call
+   */
+  public static ContainerProtos.GetCommittedBlockLengthResponseProto
+  getCommittedBlockLength(
+      XceiverClientSpi xceiverClient, BlockID blockID, String traceID)
+      throws IOException {
+    ContainerProtos.GetCommittedBlockLengthRequestProto.Builder
+        getBlockLengthRequestBuilder =
+        ContainerProtos.GetCommittedBlockLengthRequestProto.newBuilder().
+            setBlockID(blockID.getDatanodeBlockIDProtobuf());
+    String id = xceiverClient.getPipeline().getLeader().getUuidString();
+    ContainerCommandRequestProto request =
+        ContainerCommandRequestProto.newBuilder()
+            .setCmdType(Type.GetCommittedBlockLength).setTraceID(traceID)
+            .setDatanodeUuid(id)
+            .setGetCommittedBlockLength(getBlockLengthRequestBuilder).build();
+    ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
+    validateContainerResponse(response);
+    return response.getGetCommittedBlockLength();
+  }
+
+  /**
    * Calls the container protocol to put a container key.
    *
    * @param xceiverClient client to perform call

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
index b63332f..1919ed9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
@@ -42,6 +42,11 @@ public class KeyData {
   private List<ContainerProtos.ChunkInfo> chunks;
 
   /**
+   * total size of the key.
+   */
+  private long size;
+
+  /**
    * Constructs a KeyData Object.
    *
    * @param blockID
@@ -49,6 +54,7 @@ public class KeyData {
   public KeyData(BlockID blockID) {
     this.blockID = blockID;
     this.metadata = new TreeMap<>();
+    this.size = 0;
   }
 
   /**
@@ -66,6 +72,9 @@ public class KeyData {
           data.getMetadata(x).getValue());
     }
     keyData.setChunks(data.getChunksList());
+    if (data.hasSize()) {
+      keyData.setSize(data.getSize());
+    }
     return keyData;
   }
 
@@ -84,6 +93,7 @@ public class KeyData {
       builder.addMetadata(keyValBuilder.setKey(entry.getKey())
           .setValue(entry.getValue()).build());
     }
+    builder.setSize(size);
     return builder.build();
   }
 
@@ -183,10 +193,25 @@ public class KeyData {
   }
 
   /**
+   * sets the total size of the block
+   * @param size size of the block
+   */
+  public void setSize(long size) {
+    this.size = size;
+  }
+
+  /**
    * Get the total size of chunks allocated for the key.
    * @return total size of the key.
    */
   public long getSize() {
-    return chunks.parallelStream().mapToLong(e->e.getLen()).sum();
+    return size;
+  }
+
+  /**
+   * computes the total size of chunks allocated for the key.
+   */
+  public void computeSize() {
+    setSize(chunks.parallelStream().mapToLong(e -> e.getLen()).sum());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
index ff1582e..a3c4467 100644
--- a/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
+++ b/hadoop-hdds/common/src/main/proto/DatanodeContainerProtocol.proto
@@ -99,7 +99,7 @@ enum Type {
   PutSmallFile = 15;
   GetSmallFile = 16;
   CloseContainer = 17;
-
+  GetCommittedBlockLength = 18;
 }
 
 
@@ -193,8 +193,8 @@ message ContainerCommandRequestProto {
   optional   PutSmallFileRequestProto putSmallFile = 16;
   optional   GetSmallFileRequestProto getSmallFile = 17;
   optional   CloseContainerRequestProto closeContainer = 18;
-
-  required   string datanodeUuid = 19;
+  optional   GetCommittedBlockLengthRequestProto getCommittedBlockLength = 19;
+  required   string datanodeUuid = 20;
 }
 
 message ContainerCommandResponseProto {
@@ -223,6 +223,7 @@ message ContainerCommandResponseProto {
   optional PutSmallFileResponseProto putSmallFile = 19;
   optional GetSmallFileResponseProto getSmallFile = 20;
   optional CloseContainerResponseProto closeContainer = 21;
+  optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 22;
 
 }
 
@@ -302,6 +303,7 @@ message KeyData {
   optional int64 flags = 2; // for future use.
   repeated KeyValue metadata = 3;
   repeated ChunkInfo chunks = 4;
+  optional int64 size = 5;
 }
 
 // Key Messages.
@@ -325,6 +327,15 @@ message  DeleteKeyRequestProto {
   required DatanodeBlockID blockID = 1;
 }
 
+message  GetCommittedBlockLengthRequestProto {
+  required DatanodeBlockID blockID = 1;
+}
+
+message  GetCommittedBlockLengthResponseProto {
+  required DatanodeBlockID blockID = 1;
+  required int64 blockLength = 2;
+}
+
 message   DeleteKeyResponseProto {
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
index bee8417..6d11abb 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java
@@ -181,6 +181,8 @@ public class HddsDispatcher implements ContainerDispatcher {
           .getContainerID();
     case GetSmallFile:
       return request.getGetSmallFile().getKey().getBlockID().getContainerID();
+    case GetCommittedBlockLength:
+      return request.getGetCommittedBlockLength().getBlockID().getContainerID();
     }
 
     throw new StorageContainerException(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index d3a1ca4..4123dc8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -198,6 +198,8 @@ public class KeyValueHandler extends Handler {
       return handlePutSmallFile(request, kvContainer);
     case GetSmallFile:
       return handleGetSmallFile(request, kvContainer);
+    case GetCommittedBlockLength:
+      return handleGetCommittedBlockLength(request, kvContainer);
     }
     return null;
   }
@@ -443,6 +445,8 @@ public class KeyValueHandler extends Handler {
   private void commitKey(KeyData keyData, KeyValueContainer kvContainer)
       throws IOException {
     Preconditions.checkNotNull(keyData);
+    //sets the total size of the key before committing
+    keyData.computeSize();
     keyManager.putKey(kvContainer, keyData);
     //update the open key Map in containerManager
     this.openContainerBlockMap.removeFromKeyMap(keyData.getBlockID());
@@ -479,6 +483,35 @@ public class KeyValueHandler extends Handler {
   }
 
   /**
+   * Handles GetCommittedBlockLength operation.
+   * Calls KeyManager to process the request.
+   */
+  ContainerCommandResponseProto handleGetCommittedBlockLength(
+      ContainerCommandRequestProto request, KeyValueContainer kvContainer) {
+    if (!request.hasGetCommittedBlockLength()) {
+      LOG.debug("Malformed Get Key request. trace ID: {}",
+          request.getTraceID());
+      return ContainerUtils.malformedRequest(request);
+    }
+
+    long blockLength;
+    try {
+      BlockID blockID = BlockID.getFromProtobuf(
+          request.getGetCommittedBlockLength().getBlockID());
+      blockLength = keyManager.getCommittedBlockLength(kvContainer, blockID);
+
+    } catch (StorageContainerException ex) {
+      return ContainerUtils.logAndReturnError(LOG, ex, request);
+    } catch (IOException ex) {
+      return ContainerUtils.logAndReturnError(LOG,
+          new StorageContainerException("GetCommittedBlockLength failed", ex,
+              IO_EXCEPTION), request);
+    }
+
+    return KeyUtils.getBlockLengthResponse(request, blockLength);
+  }
+
+  /**
    * Handle Delete Key operation. Calls KeyManager to process the request.
    */
   ContainerCommandResponseProto handleDeleteKey(
@@ -665,6 +698,7 @@ public class KeyValueHandler extends Handler {
       List<ContainerProtos.ChunkInfo> chunks = new LinkedList<>();
       chunks.add(chunkInfo.getProtoBufMessage());
       keyData.setChunks(chunks);
+      keyData.computeSize();
       keyManager.putKey(kvContainer, keyData);
       metrics.incContainerBytesStats(Type.PutSmallFile, data.length);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
index 5845fae..2be966d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyUtils.java
@@ -144,4 +144,24 @@ public final class KeyUtils {
     builder.setGetKey(getKey);
     return  builder.build();
   }
+
+  /**
+   * Returns successful getCommittedBlockLength Response.
+   * @param msg - Request.
+   * @return Response.
+   */
+  public static ContainerProtos.ContainerCommandResponseProto
+  getBlockLengthResponse(ContainerProtos.
+      ContainerCommandRequestProto msg, long blockLength) {
+    ContainerProtos.GetCommittedBlockLengthResponseProto.Builder
+        getCommittedBlockLengthResponseBuilder = ContainerProtos.
+        GetCommittedBlockLengthResponseProto.newBuilder();
+    getCommittedBlockLengthResponseBuilder.setBlockLength(blockLength);
+    getCommittedBlockLengthResponseBuilder
+        .setBlockID(msg.getGetCommittedBlockLength().getBlockID());
+    ContainerProtos.ContainerCommandResponseProto.Builder builder =
+        ContainerUtils.getSuccessResponseBuilder(msg);
+    builder.setGetCommittedBlockLength(getCommittedBlockLengthResponseBuilder);
+    return  builder.build();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
index 6a8897a..58bf1f8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/KeyManagerImpl.java
@@ -120,6 +120,32 @@ public class KeyManagerImpl implements KeyManager {
   }
 
   /**
+   * Returns the length of the committed block.
+   *
+   * @param container - Container from which key need to be get.
+   * @param blockID - BlockID of the key.
+   * @return length of the block.
+   * @throws IOException in case, the block key does not exist in db.
+   */
+  @Override
+  public long getCommittedBlockLength(Container container, BlockID blockID)
+      throws IOException {
+    KeyValueContainerData containerData = (KeyValueContainerData) container
+        .getContainerData();
+    MetadataStore db = KeyUtils.getDB(containerData, config);
+    // This is a post condition that acts as a hint to the user.
+    // Should never fail.
+    Preconditions.checkNotNull(db, "DB cannot be null here");
+    byte[] kData = db.get(Longs.toByteArray(blockID.getLocalID()));
+    if (kData == null) {
+      throw new StorageContainerException("Unable to find the key.",
+          NO_SUCH_KEY);
+    }
+    ContainerProtos.KeyData keyData = ContainerProtos.KeyData.parseFrom(kData);
+    return keyData.getSize();
+  }
+
+  /**
    * Deletes an existing Key.
    *
    * @param container - Container from which key need to be deleted.
@@ -164,6 +190,7 @@ public class KeyManagerImpl implements KeyManager {
    * @param count    - Number of keys to return.
    * @return List of Keys that match the criteria.
    */
+  @Override
   public List<KeyData> listKey(Container container, long startLocalID, int
       count) throws IOException {
     Preconditions.checkNotNull(container, "container cannot be null");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
index 7a5d48b..dad688e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
@@ -70,6 +70,13 @@ public interface KeyManager {
       IOException;
 
   /**
+   * Returns the last committed block length for the block.
+   * @param blockID blockId
+   */
+  long getCommittedBlockLength(Container container, BlockID blockID)
+      throws IOException;
+
+  /**
    * Shutdown ContainerManager.
    */
   void shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/955f7951/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java
new file mode 100644
index 0000000..7e8aa5f
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestCommittedBlockLengthAPI.java
@@ -0,0 +1,191 @@
+/**
+ * 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.ozone.scm;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.XceiverClientManager;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import org.apache.hadoop.hdds.scm.container.common.helpers.
+    ContainerWithPipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers.
+    StorageContainerException;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.
+    ContainerPlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.
+    SCMContainerPlacementCapacity;
+import org.apache.hadoop.hdds.scm.protocolPB.
+    StorageContainerLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+import java.util.UUID;
+
+/**
+ * Test Container calls.
+ */
+public class TestCommittedBlockLengthAPI {
+
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration ozoneConfig;
+  private static StorageContainerLocationProtocolClientSideTranslatorPB
+      storageContainerLocationClient;
+  private static XceiverClientManager xceiverClientManager;
+  private static String containerOwner = "OZONE";
+
+  @BeforeClass
+  public static void init() throws Exception {
+    ozoneConfig = new OzoneConfiguration();
+    ozoneConfig.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
+        SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class);
+    cluster =
+        MiniOzoneCluster.newBuilder(ozoneConfig).setNumDatanodes(1).build();
+    cluster.waitForClusterToBeReady();
+    storageContainerLocationClient =
+        cluster.getStorageContainerLocationClient();
+    xceiverClientManager = new XceiverClientManager(ozoneConfig);
+  }
+
+  @AfterClass
+  public static void shutdown() throws InterruptedException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+    IOUtils.cleanupWithLogger(null, storageContainerLocationClient);
+  }
+
+  @Test
+  public void tesGetCommittedBlockLength() throws Exception {
+    ContainerProtos.GetCommittedBlockLengthResponseProto response;
+    String traceID = UUID.randomUUID().toString();
+    ContainerWithPipeline container = storageContainerLocationClient
+        .allocateContainer(xceiverClientManager.getType(),
+            HddsProtos.ReplicationFactor.ONE, containerOwner);
+    long containerID = container.getContainerInfo().getContainerID();
+    Pipeline pipeline = container.getPipeline();
+    XceiverClientSpi client =
+        xceiverClientManager.acquireClient(pipeline, containerID);
+    //create the container
+    ContainerProtocolCalls.createContainer(client, containerID, traceID);
+
+    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
+    byte[] data =
+        RandomStringUtils.random(RandomUtils.nextInt(0, 1024)).getBytes();
+    ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
+        ContainerTestHelper
+            .getWriteChunkRequest(container.getPipeline(), blockID,
+                data.length);
+    client.sendCommand(writeChunkRequest);
+    try {
+      // since there is neither explicit putKey request made for the block,
+      // nor the container is closed, GetCommittedBlockLength request
+      // should fail here.
+      response = ContainerProtocolCalls
+          .getCommittedBlockLength(client, blockID, traceID);
+      Assert.fail("Expected exception not thrown");
+    } catch (StorageContainerException sce) {
+      Assert.assertTrue(sce.getMessage().contains("Unable to find the key"));
+    }
+    // Now, explicitly make a putKey request for the block.
+    ContainerProtos.ContainerCommandRequestProto putKeyRequest =
+        ContainerTestHelper
+            .getPutKeyRequest(pipeline, writeChunkRequest.getWriteChunk());
+    client.sendCommand(putKeyRequest);
+    response = ContainerProtocolCalls
+        .getCommittedBlockLength(client, blockID, traceID);
+    // make sure the block ids in the request and response are same.
+    Assert.assertTrue(
+        BlockID.getFromProtobuf(response.getBlockID()).equals(blockID));
+    Assert.assertTrue(response.getBlockLength() == data.length);
+    xceiverClientManager.releaseClient(client);
+  }
+
+  @Test
+  public void tesGetCommittedBlockLengthWithClosedContainer()
+      throws Exception {
+    String traceID = UUID.randomUUID().toString();
+    ContainerWithPipeline container = storageContainerLocationClient
+        .allocateContainer(xceiverClientManager.getType(),
+            HddsProtos.ReplicationFactor.ONE, containerOwner);
+    long containerID = container.getContainerInfo().getContainerID();
+    Pipeline pipeline = container.getPipeline();
+    XceiverClientSpi client =
+        xceiverClientManager.acquireClient(pipeline, containerID);
+    // create the container
+    ContainerProtocolCalls.createContainer(client, containerID, traceID);
+
+    byte[] data =
+        RandomStringUtils.random(RandomUtils.nextInt(0, 1024)).getBytes();
+    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
+    ContainerProtos.ContainerCommandRequestProto writeChunkRequest =
+        ContainerTestHelper
+            .getWriteChunkRequest(container.getPipeline(), blockID,
+                data.length);
+    client.sendCommand(writeChunkRequest);
+    // close the container
+    ContainerProtocolCalls.closeContainer(client, containerID, traceID);
+    ContainerProtos.GetCommittedBlockLengthResponseProto response =
+        ContainerProtocolCalls
+            .getCommittedBlockLength(client, blockID, traceID);
+    // make sure the block ids in the request and response are same.
+    // This will also ensure that closing the container committed the block
+    // on the Datanodes.
+    Assert.assertTrue(
+        BlockID.getFromProtobuf(response.getBlockID()).equals(blockID));
+    Assert.assertTrue(response.getBlockLength() == data.length);
+    xceiverClientManager.releaseClient(client);
+  }
+
+  @Test
+  public void tesGetCommittedBlockLengthForInvalidBlock() throws Exception {
+    String traceID = UUID.randomUUID().toString();
+    ContainerWithPipeline container = storageContainerLocationClient
+        .allocateContainer(xceiverClientManager.getType(),
+            HddsProtos.ReplicationFactor.ONE, containerOwner);
+    long containerID = container.getContainerInfo().getContainerID();
+    XceiverClientSpi client = xceiverClientManager
+        .acquireClient(container.getPipeline(), containerID);
+    ContainerProtocolCalls.createContainer(client, containerID, traceID);
+
+    BlockID blockID = ContainerTestHelper.getTestBlockID(containerID);
+    // move the container to closed state
+    ContainerProtocolCalls.closeContainer(client, containerID, traceID);
+    try {
+      // There is no block written inside the container. The request should
+      // fail.
+      ContainerProtocolCalls.getCommittedBlockLength(client, blockID, traceID);
+      Assert.fail("Expected exception not thrown");
+    } catch (StorageContainerException sce) {
+      Assert.assertTrue(sce.getMessage().contains("Unable to find the key"));
+    }
+    xceiverClientManager.releaseClient(client);
+  }
+}
\ No newline at end of file


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


[03/50] [abbrv] hadoop git commit: HDDS-282. Consolidate logging in scm/container-service. Contributed by Elek Marton.

Posted by su...@apache.org.
HDDS-282. Consolidate logging in scm/container-service. Contributed by Elek Marton.


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

Branch: refs/heads/YARN-3409
Commit: cd0b9f13805affcc91a2cba42b176bb9031378eb
Parents: 35ce6eb
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Jul 24 10:16:53 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Jul 24 10:17:03 2018 -0700

----------------------------------------------------------------------
 .../container/common/statemachine/EndpointStateMachine.java  | 4 ++--
 .../common/states/endpoint/RegisterEndpointTask.java         | 3 +--
 .../statemachine/background/BlockDeletingService.java        | 8 +++++---
 .../org/apache/hadoop/hdds/server/events/EventQueue.java     | 3 +--
 .../org/apache/hadoop/hdds/server/events/TypedEvent.java     | 6 ++++--
 5 files changed, 13 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd0b9f13/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
index 7e85923..fb32a05 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
@@ -203,11 +203,11 @@ public class EndpointStateMachine
     this.incMissed();
     if (this.getMissedCount() % getLogWarnInterval(conf) ==
         0) {
-      LOG.warn("Unable to communicate to SCM server at {}. We have not been " +
+      LOG.error("Unable to communicate to SCM server at {}. We have not been " +
               "able to communicate to this SCM server for past {} seconds.",
           this.getAddress().getHostString() + ":" + this.getAddress().getPort(),
           this.getMissedCount() * getScmHeartbeatInterval(
-              this.conf));
+              this.conf), ex);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd0b9f13/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
index b3d2b62..25af4a1 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
@@ -125,8 +125,7 @@ public final class RegisterEndpointTask implements
       rpcEndPoint.setState(nextState);
       rpcEndPoint.zeroMissedCount();
     } catch (IOException ex) {
-      rpcEndPoint.logIfNeeded(ex
-      );
+      rpcEndPoint.logIfNeeded(ex);
     } finally {
       rpcEndPoint.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd0b9f13/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
index 4a572ca..51eed7f 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/statemachine/background/BlockDeletingService.java
@@ -119,9 +119,11 @@ public class BlockDeletingService extends BackgroundService{
       // configured.
       containers = containerSet.chooseContainerForBlockDeletion(
           containerLimitPerInterval, containerDeletionPolicy);
-      LOG.info("Plan to choose {} containers for block deletion, "
-          + "actually returns {} valid containers.",
-          containerLimitPerInterval, containers.size());
+      if (containers.size() > 0) {
+        LOG.info("Plan to choose {} containers for block deletion, "
+                + "actually returns {} valid containers.",
+            containerLimitPerInterval, containers.size());
+      }
 
       for(ContainerData container : containers) {
         BlockDeletingTask containerTask =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd0b9f13/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java
index 7e29223..f93c54b 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java
@@ -155,8 +155,7 @@ public class EventQueue implements EventPublisher, AutoCloseable {
       }
 
     } else {
-      throw new IllegalArgumentException(
-          "No event handler registered for event " + event);
+      LOG.warn("No event handler registered for event " + event);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd0b9f13/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/TypedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/TypedEvent.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/TypedEvent.java
index 62e2419..27bba3a 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/TypedEvent.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/TypedEvent.java
@@ -50,7 +50,9 @@ public class TypedEvent<T> implements Event<T> {
 
   @Override
   public String toString() {
-    return "TypedEvent{" + "payloadType=" + payloadType + ", name='" + name
-        + '\'' + '}';
+    return "TypedEvent{" +
+        "payloadType=" + payloadType.getSimpleName() +
+        ", name='" + name + '\'' +
+        '}';
   }
 }


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


[08/50] [abbrv] hadoop git commit: HDFS-13761. Add toString Method to AclFeature Class. Contributed by Shweta.

Posted by su...@apache.org.
HDFS-13761. Add toString Method to AclFeature Class. Contributed by Shweta.


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

Branch: refs/heads/YARN-3409
Commit: 26864471c24bf389ab8fc913decc3d069404688b
Parents: 849c45d
Author: Xiao Chen <xi...@apache.org>
Authored: Tue Jul 24 21:42:00 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Tue Jul 24 21:42:47 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/26864471/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
index 97d4759..6d546f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
@@ -73,6 +73,11 @@ public class AclFeature implements INode.Feature, ReferenceCounter {
   }
 
   @Override
+  public String toString() {
+    return "AclFeature : " + Integer.toHexString(hashCode()) + " Size of entries : " + entries.length;
+  }
+
+  @Override
   public int hashCode() {
     return Arrays.hashCode(entries);
   }


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


[49/50] [abbrv] 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/acd7729a/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/acd7729a/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/acd7729a/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/acd7729a/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/acd7729a/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/acd7729a/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/acd7729a/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 6eb4589..bf9de15 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
@@ -34,7 +34,7 @@ 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.lang3.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acd7729a/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/acd7729a/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/acd7729a/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/acd7729a/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 38e40ef..d803bd3 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/acd7729a/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();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acd7729a/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 46f0e89..f17a60a 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
@@ -742,19 +742,19 @@ public class FederationClientInterceptor
   @Override
   public GetAttributesToNodesResponse getAttributesToNodes(
       GetAttributesToNodesRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
+    throw new NotImplementedException("Code is not implemented");
   }
 
   @Override
   public GetClusterNodeAttributesResponse getClusterNodeAttributes(
       GetClusterNodeAttributesRequest request)
       throws YarnException, IOException {
-    throw new NotImplementedException();
+    throw new NotImplementedException("Code is not implemented");
   }
 
   @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


[23/50] [abbrv] hadoop git commit: HDDS-277. PipelineStateMachine should handle closure of pipelines in SCM. Contributed by Mukul Kumar Singh.

Posted by su...@apache.org.
HDDS-277. PipelineStateMachine should handle closure of pipelines in SCM. Contributed by Mukul Kumar Singh.


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

Branch: refs/heads/YARN-3409
Commit: fd31cb6cfeef0c7e9bb0a054cb0f78853df8976f
Parents: be150a1
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Thu Jul 26 13:15:27 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Thu Jul 26 13:15:55 2018 -0700

----------------------------------------------------------------------
 .../container/common/helpers/ContainerInfo.java |   7 +-
 .../container/CloseContainerEventHandler.java   |  28 ++--
 .../hdds/scm/container/ContainerMapping.java    |  16 +-
 .../scm/container/ContainerStateManager.java    |  11 ++
 .../scm/container/states/ContainerStateMap.java |   2 +-
 .../hdds/scm/pipelines/Node2PipelineMap.java    |  33 ++--
 .../hdds/scm/pipelines/PipelineManager.java     |  31 ++--
 .../hdds/scm/pipelines/PipelineSelector.java    |  70 +++++++--
 .../scm/pipelines/ratis/RatisManagerImpl.java   |  14 +-
 .../standalone/StandaloneManagerImpl.java       |  13 +-
 .../scm/server/StorageContainerManager.java     |   2 +-
 .../hadoop/hdds/scm/block/TestBlockManager.java |   4 +-
 .../TestCloseContainerEventHandler.java         |  13 +-
 .../scm/container/TestContainerMapping.java     |   4 +-
 .../container/closer/TestContainerCloser.java   |   4 +-
 .../hdds/scm/node/TestContainerPlacement.java   |   3 +-
 .../hdds/scm/pipeline/TestPipelineClose.java    | 152 +++++++++++++++++++
 .../hadoop/ozone/scm/TestContainerSQLCli.java   |   4 +-
 18 files changed, 331 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
index 4074b21..b194c14 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
@@ -459,12 +459,13 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
 
   /**
    * Check if a container is in open state, this will check if the
-   * container is either open or allocated or creating. Any containers in
-   * these states is managed as an open container by SCM.
+   * container is either open, allocated, creating or creating.
+   * Any containers in these states is managed as an open container by SCM.
    */
   public boolean isContainerOpen() {
     return state == HddsProtos.LifeCycleState.ALLOCATED ||
         state == HddsProtos.LifeCycleState.CREATING ||
-        state == HddsProtos.LifeCycleState.OPEN;
+        state == HddsProtos.LifeCycleState.OPEN ||
+        state == HddsProtos.LifeCycleState.CLOSING;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
index 859e5d5..949eb13 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
@@ -21,7 +21,6 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
-import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.server.events.EventHandler;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
@@ -63,13 +62,13 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
           containerManager.getContainerWithPipeline(containerID.getId());
       info = containerWithPipeline.getContainerInfo();
       if (info == null) {
-        LOG.info("Failed to update the container state. Container with id : {} "
+        LOG.error("Failed to update the container state. Container with id : {} "
             + "does not exist", containerID.getId());
         return;
       }
     } catch (IOException e) {
-      LOG.info("Failed to update the container state. Container with id : {} "
-          + "does not exist", containerID.getId());
+      LOG.error("Failed to update the container state. Container with id : {} "
+          + "does not exist", containerID.getId(), e);
       return;
     }
 
@@ -85,11 +84,22 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
       try {
         // Finalize event will make sure the state of the container transitions
         // from OPEN to CLOSING in containerStateManager.
-        containerManager.getStateManager()
-            .updateContainerState(info, HddsProtos.LifeCycleEvent.FINALIZE);
-      } catch (SCMException ex) {
-        LOG.error("Failed to update the container state for container : {}"
-            + containerID);
+        containerManager.updateContainerState(containerID.getId(),
+            HddsProtos.LifeCycleEvent.FINALIZE);
+      } catch (IOException ex) {
+        LOG.error("Failed to update the container state to FINALIZE for"
+            + "container : {}" + containerID, ex);
+      }
+    } else if (info.getState() == HddsProtos.LifeCycleState.ALLOCATED) {
+      try {
+        // Create event will make sure the state of the container transitions
+        // from OPEN to CREATING in containerStateManager, this will move
+        // the container out of active allocation path.
+        containerManager.updateContainerState(containerID.getId(),
+            HddsProtos.LifeCycleEvent.CREATE);
+      } catch (IOException ex) {
+        LOG.error("Failed to update the container state to CREATE for"
+            + "container:{}" + containerID, ex);
       }
     } else {
       LOG.info("container with id : {} is in {} state and need not be closed.",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
index e17fe3d..d84551a 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.lease.Lease;
 import org.apache.hadoop.ozone.lease.LeaseException;
@@ -104,7 +105,7 @@ public class ContainerMapping implements Mapping {
   @SuppressWarnings("unchecked")
   public ContainerMapping(
       final Configuration conf, final NodeManager nodeManager, final int
-      cacheSizeMB) throws IOException {
+      cacheSizeMB, EventPublisher eventPublisher) throws IOException {
     this.nodeManager = nodeManager;
     this.cacheSize = cacheSizeMB;
     this.closer = new ContainerCloser(nodeManager, conf);
@@ -122,14 +123,15 @@ public class ContainerMapping implements Mapping {
 
     this.lock = new ReentrantLock();
 
-    this.pipelineSelector = new PipelineSelector(nodeManager, conf);
-
     // To be replaced with code getStorageSize once it is committed.
     size = conf.getLong(OZONE_SCM_CONTAINER_SIZE_GB,
         OZONE_SCM_CONTAINER_SIZE_DEFAULT) * 1024 * 1024 * 1024;
     this.containerStateManager =
         new ContainerStateManager(conf, this);
 
+    this.pipelineSelector = new PipelineSelector(nodeManager,
+        containerStateManager, conf, eventPublisher);
+
     this.containerCloseThreshold = conf.getFloat(
         ScmConfigKeys.OZONE_SCM_CONTAINER_CLOSE_THRESHOLD,
         ScmConfigKeys.OZONE_SCM_CONTAINER_CLOSE_THRESHOLD_DEFAULT);
@@ -372,6 +374,12 @@ public class ContainerMapping implements Mapping {
       // Like releasing the lease in case of BEGIN_CREATE.
       ContainerInfo updatedContainer = containerStateManager
           .updateContainerState(containerInfo, event);
+      if (!updatedContainer.isContainerOpen()) {
+        Pipeline pipeline = pipelineSelector
+            .getPipeline(containerInfo.getPipelineName(),
+                containerInfo.getReplicationType());
+        pipelineSelector.closePipelineIfNoOpenContainers(pipeline);
+      }
       containerStore.put(dbKey, updatedContainer.getProtobuf().toByteArray());
       return updatedContainer.getState();
     } catch (LeaseException e) {
@@ -446,7 +454,7 @@ public class ContainerMapping implements Mapping {
         .getPipeline(containerInfo.getPipelineName(),
             containerInfo.getReplicationType());
     if (pipeline == null) {
-      pipelineSelector
+      pipeline = pipelineSelector
           .getReplicationPipeline(containerInfo.getReplicationType(),
               containerInfo.getReplicationFactor());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
index b2431dc..f0ab213 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
@@ -463,6 +463,17 @@ public class ContainerStateManager implements Closeable {
   }
 
   /**
+   * Returns a set of open ContainerIDs that reside on a pipeline.
+   *
+   * @param pipeline Pipeline of the Containers.
+   * @return Set of containers that match the specific query parameters.
+   */
+  public NavigableSet<ContainerID> getMatchingContainerIDsByPipeline(String
+      pipeline) {
+    return containers.getOpenContainerIDsByPipeline(pipeline);
+  }
+
+  /**
    * Returns the containerInfo with pipeline for the given container id.
    * @param selector -- Pipeline selector class.
    * @param containerID id of the container

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java
index 46fe2ab..b358b7c 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java
@@ -346,7 +346,7 @@ public class ContainerStateMap {
     }
     // In case the container is set to closed state, it needs to be removed from
     // the pipeline Map.
-    if (newState == LifeCycleState.CLOSED) {
+    if (!info.isContainerOpen()) {
       openPipelineMap.remove(info.getPipelineName(), id);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java
index 2e89616..b860082 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java
@@ -102,18 +102,27 @@ public class Node2PipelineMap {
         Collections.unmodifiableSet(v));
   }
 
-/**
- * Adds a pipeline entry to a given dataNode in the map.
- * @param pipeline Pipeline to be added
- */
- public synchronized void addPipeline(Pipeline pipeline) throws SCMException {
-   for (DatanodeDetails details : pipeline.getDatanodes().values()) {
-     UUID dnId = details.getUuid();
-     dn2PipelineMap
-         .computeIfAbsent(dnId,k->Collections.synchronizedSet(new HashSet<>()))
-         .add(pipeline);
-   }
- }
+  /**
+   * Adds a pipeline entry to a given dataNode in the map.
+   * @param pipeline Pipeline to be added
+   */
+  public synchronized void addPipeline(Pipeline pipeline) {
+    for (DatanodeDetails details : pipeline.getDatanodes().values()) {
+      UUID dnId = details.getUuid();
+      dn2PipelineMap
+          .computeIfAbsent(dnId,
+              k -> Collections.synchronizedSet(new HashSet<>()))
+          .add(pipeline);
+    }
+  }
+
+  public synchronized void removePipeline(Pipeline pipeline) {
+    for (DatanodeDetails details : pipeline.getDatanodes().values()) {
+      UUID dnId = details.getUuid();
+      dn2PipelineMap.computeIfPresent(dnId,
+          (k, v) -> {v.remove(pipeline); return v;});
+    }
+  }
 
   public Map<UUID, Set<Pipeline>> getDn2PipelineMap() {
     return Collections.unmodifiableMap(dn2PipelineMap);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java
index 77d8211..266b1f3 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java
@@ -38,14 +38,14 @@ public abstract class PipelineManager {
   private static final Logger LOG =
       LoggerFactory.getLogger(PipelineManager.class);
   private final List<Pipeline> activePipelines;
-  private final Map<String, Pipeline> activePipelineMap;
+  private final Map<String, Pipeline> pipelineMap;
   private final AtomicInteger pipelineIndex;
   private final Node2PipelineMap node2PipelineMap;
 
   public PipelineManager(Node2PipelineMap map) {
     activePipelines = new LinkedList<>();
     pipelineIndex = new AtomicInteger(0);
-    activePipelineMap = new WeakHashMap<>();
+    pipelineMap = new WeakHashMap<>();
     node2PipelineMap = map;
   }
 
@@ -85,8 +85,8 @@ public abstract class PipelineManager {
     Pipeline pipeline = null;
 
     // 1. Check if pipeline already exists
-    if (activePipelineMap.containsKey(pipelineName)) {
-      pipeline = activePipelineMap.get(pipelineName);
+    if (pipelineMap.containsKey(pipelineName)) {
+      pipeline = pipelineMap.get(pipelineName);
       LOG.debug("Returning pipeline for pipelineName:{}", pipelineName);
       return pipeline;
     } else {
@@ -115,11 +115,6 @@ public abstract class PipelineManager {
    */
   public abstract void initializePipeline(Pipeline pipeline) throws IOException;
 
-  public void removePipeline(Pipeline pipeline) {
-    activePipelines.remove(pipeline);
-    activePipelineMap.remove(pipeline.getPipelineName());
-  }
-
   /**
    * Find a Pipeline that is operational.
    *
@@ -172,16 +167,28 @@ public abstract class PipelineManager {
               + "replicationType:{} replicationFactor:{}",
           pipeline.getPipelineName(), replicationType, replicationFactor);
       activePipelines.add(pipeline);
-      activePipelineMap.put(pipeline.getPipelineName(), pipeline);
+      pipelineMap.put(pipeline.getPipelineName(), pipeline);
       node2PipelineMap.addPipeline(pipeline);
     }
     return pipeline;
   }
 
   /**
-   * Close the  pipeline with the given clusterId.
+   * Remove the pipeline from active allocation
+   * @param pipeline pipeline to be finalized
+   */
+  public synchronized void finalizePipeline(Pipeline pipeline) {
+    activePipelines.remove(pipeline);
+  }
+
+  /**
+   *
+   * @param pipeline
    */
-  public abstract void closePipeline(String pipelineID) throws IOException;
+  public void closePipeline(Pipeline pipeline) {
+    pipelineMap.remove(pipeline.getPipelineName());
+    node2PipelineMap.removePipeline(pipeline);
+  }
 
   /**
    * list members in the pipeline .

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
index b1e1dd0..ebe39d0 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
@@ -19,11 +19,14 @@ package org.apache.hadoop.hdds.scm.pipelines;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.container.placement.algorithms
     .ContainerPlacementPolicy;
 import org.apache.hadoop.hdds.scm.container.placement.algorithms
     .SCMContainerPlacementRandom;
+import org.apache.hadoop.hdds.scm.events.SCMEvents;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.pipelines.ratis.RatisManagerImpl;
@@ -33,6 +36,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.statemachine
     .InvalidStateTransitionException;
@@ -48,6 +52,7 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.HashSet;
 import java.util.List;
+import java.util.NavigableSet;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
@@ -65,6 +70,8 @@ public class PipelineSelector {
   private final ContainerPlacementPolicy placementPolicy;
   private final NodeManager nodeManager;
   private final Configuration conf;
+  private final ContainerStateManager containerStateManager;
+  private final EventPublisher eventPublisher;
   private final RatisManagerImpl ratisManager;
   private final StandaloneManagerImpl standaloneManager;
   private final long containerSize;
@@ -79,9 +86,12 @@ public class PipelineSelector {
    * @param nodeManager - node manager
    * @param conf - Ozone Config
    */
-  public PipelineSelector(NodeManager nodeManager, Configuration conf) {
+  public PipelineSelector(NodeManager nodeManager,
+      ContainerStateManager containerStateManager, Configuration conf,
+      EventPublisher eventPublisher) {
     this.nodeManager = nodeManager;
     this.conf = conf;
+    this.eventPublisher = eventPublisher;
     this.placementPolicy = createContainerPlacementPolicy(nodeManager, conf);
     this.containerSize = OzoneConsts.GB * this.conf.getInt(
         ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
@@ -99,6 +109,7 @@ public class PipelineSelector {
         ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT,
         ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT_DEFAULT,
         TimeUnit.MILLISECONDS);
+    this.containerStateManager = containerStateManager;
     pipelineLeaseManager = new LeaseManager<>("PipelineCreation",
         pipelineCreationLeaseTimeout);
     pipelineLeaseManager.start();
@@ -306,15 +317,54 @@ public class PipelineSelector {
   }
 
   /**
-   * Close the  pipeline with the given clusterId.
+   * Finalize a given pipeline.
    */
+  public void finalizePipeline(Pipeline pipeline) throws IOException {
+    PipelineManager manager = getPipelineManager(pipeline.getType());
+    Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
+    LOG.debug("Finalizing pipeline. pipelineID: {}", pipeline.getPipelineName());
+    // Remove the pipeline from active allocation
+    manager.finalizePipeline(pipeline);
+    updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.FINALIZE);
+    closePipelineIfNoOpenContainers(pipeline);
+  }
 
-  public void closePipeline(ReplicationType replicationType, String
-      pipelineID) throws IOException {
-    PipelineManager manager = getPipelineManager(replicationType);
+  /**
+   * Close a given pipeline.
+   */
+  public void closePipelineIfNoOpenContainers(Pipeline pipeline) throws IOException {
+    if (pipeline.getLifeCycleState() != LifeCycleState.CLOSING) {
+      return;
+    }
+    NavigableSet<ContainerID> containerIDS = containerStateManager
+        .getMatchingContainerIDsByPipeline(pipeline.getPipelineName());
+    if (containerIDS.size() == 0) {
+      updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.CLOSE);
+      LOG.info("Closing pipeline. pipelineID: {}", pipeline.getPipelineName());
+    }
+  }
+
+  /**
+   * Close a given pipeline.
+   */
+  private void closePipeline(Pipeline pipeline) {
+    PipelineManager manager = getPipelineManager(pipeline.getType());
     Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
-    LOG.debug("Closing pipeline. pipelineID: {}", pipelineID);
-    manager.closePipeline(pipelineID);
+    LOG.debug("Closing pipeline. pipelineID: {}", pipeline.getPipelineName());
+    NavigableSet<ContainerID> containers =
+        containerStateManager
+            .getMatchingContainerIDsByPipeline(pipeline.getPipelineName());
+    Preconditions.checkArgument(containers.size() == 0);
+    manager.closePipeline(pipeline);
+  }
+
+  private void closeContainersByPipeline(Pipeline pipeline) {
+    NavigableSet<ContainerID> containers =
+        containerStateManager
+            .getMatchingContainerIDsByPipeline(pipeline.getPipelineName());
+    for (ContainerID id : containers) {
+      eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, id);
+    }
   }
 
   /**
@@ -352,7 +402,7 @@ public class PipelineSelector {
         node2PipelineMap.getPipelines(dnId);
     for (Pipeline pipeline : pipelineSet) {
       getPipelineManager(pipeline.getType())
-          .removePipeline(pipeline);
+          .closePipeline(pipeline);
     }
     node2PipelineMap.removeDatanode(dnId);
   }
@@ -398,12 +448,12 @@ public class PipelineSelector {
         break;
 
       case FINALIZE:
-        //TODO: cleanup pipeline by closing all the containers on the pipeline
+        closeContainersByPipeline(pipeline);
         break;
 
       case CLOSE:
       case TIMEOUT:
-        // TODO: Release the nodes here when pipelines are destroyed
+        closePipeline(pipeline);
         break;
       default:
         throw new SCMException("Unsupported pipeline LifeCycleEvent.",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java
index c726ef6..fdd0605 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/ratis/RatisManagerImpl.java
@@ -108,13 +108,15 @@ public class RatisManagerImpl extends PipelineManager {
   }
 
   /**
-   * Close the  pipeline with the given clusterId.
-   *
-   * @param pipelineID
+   * Close the pipeline.
    */
-  @Override
-  public void closePipeline(String pipelineID) throws IOException {
-
+  public void closePipeline(Pipeline pipeline) {
+    super.closePipeline(pipeline);
+    for (DatanodeDetails node : pipeline.getMachines()) {
+      // A node should always be the in ratis members list.
+      Preconditions.checkArgument(ratisMembers.remove(node));
+    }
+    //TODO: should the raft ring also be destroyed as well?
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java
index bb4951f..0506e59 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/standalone/StandaloneManagerImpl.java
@@ -101,13 +101,14 @@ public class StandaloneManagerImpl extends PipelineManager {
   }
 
   /**
-   * Close the  pipeline with the given clusterId.
-   *
-   * @param pipelineID
+   * Close the pipeline.
    */
-  @Override
-  public void closePipeline(String pipelineID) throws IOException {
-
+  public void closePipeline(Pipeline pipeline) {
+    super.closePipeline(pipeline);
+    for (DatanodeDetails node : pipeline.getMachines()) {
+      // A node should always be the in standalone members list.
+      Preconditions.checkArgument(standAloneMembers.remove(node));
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
index 165805f..be8fb43 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
@@ -192,7 +192,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
     scmNodeManager = new SCMNodeManager(
         conf, scmStorage.getClusterID(), this, eventQueue);
     scmContainerManager = new ContainerMapping(
-        conf, getScmNodeManager(), cacheSize);
+        conf, getScmNodeManager(), cacheSize, eventQueue);
     scmBlockManager = new BlockManagerImpl(
         conf, getScmNodeManager(), scmContainerManager, eventQueue);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
index 06e7420..7049029 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.scm.container.ContainerMapping;
 import org.apache.hadoop.hdds.scm.container.MockNodeManager;
 import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.events.EventQueue;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -73,7 +74,8 @@ public class TestBlockManager {
       throw new IOException("Unable to create test directory path");
     }
     nodeManager = new MockNodeManager(true, 10);
-    mapping = new ContainerMapping(conf, nodeManager, 128);
+    mapping =
+        new ContainerMapping(conf, nodeManager, 128, new EventQueue());
     blockManager = new BlockManagerImpl(conf, nodeManager, mapping, null);
     if(conf.getBoolean(ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY,
         ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT)){

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
index 0764b12..543cad3 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
@@ -66,7 +66,8 @@ public class TestCloseContainerEventHandler {
     configuration
         .set(OzoneConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
     nodeManager = new MockNodeManager(true, 10);
-    mapping = new ContainerMapping(configuration, nodeManager, 128);
+    mapping = new ContainerMapping(configuration, nodeManager, 128,
+        new EventQueue());
     eventQueue = new EventQueue();
     eventQueue.addHandler(CLOSE_CONTAINER,
         new CloseContainerEventHandler(mapping));
@@ -122,12 +123,7 @@ public class TestCloseContainerEventHandler {
     // state, so firing close container event should not queue CLOSE
     // command in the Datanode
     Assert.assertEquals(0, nodeManager.getCommandCount(datanode));
-    // Make sure the information is logged
-    Assert.assertTrue(logCapturer.getOutput().contains(
-        "container with id : " + id.getId()
-            + " is in ALLOCATED state and need not be closed"));
     //Execute these state transitions so that we can close the container.
-    mapping.updateContainerState(id.getId(), CREATE);
     mapping.updateContainerState(id.getId(), CREATED);
     eventQueue.fireEvent(CLOSE_CONTAINER,
         new ContainerID(
@@ -164,12 +160,7 @@ public class TestCloseContainerEventHandler {
       Assert.assertEquals(closeCount[i], nodeManager.getCommandCount(details));
       i++;
     }
-    // Make sure the information is logged
-    Assert.assertTrue(logCapturer.getOutput().contains(
-        "container with id : " + id.getId()
-            + " is in ALLOCATED state and need not be closed"));
     //Execute these state transitions so that we can close the container.
-    mapping.updateContainerState(id.getId(), CREATE);
     mapping.updateContainerState(id.getId(), CREATED);
     eventQueue.fireEvent(CLOSE_CONTAINER, id);
     eventQueue.processAll(1000);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
index 79ac9cf..6269514 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
+import org.apache.hadoop.hdds.server.events.EventQueue;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -84,7 +85,8 @@ public class TestContainerMapping {
       throw new IOException("Unable to create test directory path");
     }
     nodeManager = new MockNodeManager(true, 10);
-    mapping = new ContainerMapping(conf, nodeManager, 128);
+    mapping = new ContainerMapping(conf, nodeManager, 128,
+        new EventQueue());
     xceiverClientManager = new XceiverClientManager(conf);
     random = new Random();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
index cc25544..0c0f25d 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
+import org.apache.hadoop.hdds.server.events.EventQueue;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -79,7 +80,8 @@ public class TestContainerCloser {
     configuration.set(OzoneConfigKeys.OZONE_METADATA_DIRS,
         testDir.getAbsolutePath());
     nodeManager = new MockNodeManager(true, 10);
-    mapping = new ContainerMapping(configuration, nodeManager, 128);
+    mapping = new ContainerMapping(configuration, nodeManager, 128,
+        new EventQueue());
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
index 2fef620..52963c0 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
@@ -105,9 +105,10 @@ public class TestContainerPlacement {
 
   ContainerMapping createContainerManager(Configuration config,
       NodeManager scmNodeManager) throws IOException {
+    EventQueue eventQueue = new EventQueue();
     final int cacheSize = config.getInt(OZONE_SCM_DB_CACHE_SIZE_MB,
         OZONE_SCM_DB_CACHE_SIZE_DEFAULT);
-    return new ContainerMapping(config, scmNodeManager, cacheSize);
+    return new ContainerMapping(config, scmNodeManager, cacheSize, eventQueue);
 
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java
new file mode 100644
index 0000000..24e25ab
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java
@@ -0,0 +1,152 @@
+/*
+ * 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.hdds.scm.pipeline;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerMapping;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .ContainerWithPipeline;
+import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
+import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.apache.hadoop.ozone.HddsDatanodeService;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.NavigableSet;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
+    .ReplicationFactor.THREE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
+    .ReplicationType.RATIS;
+
+public class TestPipelineClose {
+
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf;
+  private static StorageContainerManager scm;
+  private static ContainerWithPipeline ratisContainer1;
+  private static ContainerWithPipeline ratisContainer2;
+  private static ContainerStateMap stateMap;
+  private static ContainerMapping mapping;
+  private static PipelineSelector pipelineSelector;
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   *
+   * @throws IOException
+   */
+  @BeforeClass
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(6).build();
+    cluster.waitForClusterToBeReady();
+    scm = cluster.getStorageContainerManager();
+    mapping = (ContainerMapping)scm.getScmContainerManager();
+    stateMap = mapping.getStateManager().getContainerStateMap();
+    ratisContainer1 = mapping.allocateContainer(RATIS, THREE, "testOwner");
+    ratisContainer2 = mapping.allocateContainer(RATIS, THREE, "testOwner");
+    pipelineSelector = mapping.getPipelineSelector();
+    // At this stage, there should be 2 pipeline one with 1 open container each.
+    // Try closing the both the pipelines, one with a closed container and
+    // the other with an open container.
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterClass
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+
+  @Test
+  public void testPipelineCloseWithClosedContainer() throws IOException {
+    NavigableSet<ContainerID> set = stateMap.getOpenContainerIDsByPipeline(
+        ratisContainer1.getPipeline().getPipelineName());
+
+    long cId = ratisContainer1.getContainerInfo().getContainerID();
+    Assert.assertEquals(1, set.size());
+    Assert.assertEquals(cId, set.first().getId());
+
+    // Now close the container and it should not show up while fetching
+    // containers by pipeline
+    mapping
+        .updateContainerState(cId, HddsProtos.LifeCycleEvent.CREATE);
+    mapping
+        .updateContainerState(cId, HddsProtos.LifeCycleEvent.CREATED);
+    mapping
+        .updateContainerState(cId, HddsProtos.LifeCycleEvent.FINALIZE);
+    mapping
+        .updateContainerState(cId, HddsProtos.LifeCycleEvent.CLOSE);
+
+    NavigableSet<ContainerID> setClosed = stateMap.getOpenContainerIDsByPipeline(
+        ratisContainer1.getPipeline().getPipelineName());
+    Assert.assertEquals(0, setClosed.size());
+
+    pipelineSelector.finalizePipeline(ratisContainer1.getPipeline());
+    Pipeline pipeline1 = pipelineSelector
+        .getPipeline(ratisContainer1.getPipeline().getPipelineName(),
+            ratisContainer1.getContainerInfo().getReplicationType());
+    Assert.assertNull(pipeline1);
+    Assert.assertEquals(ratisContainer1.getPipeline().getLifeCycleState(),
+        HddsProtos.LifeCycleState.CLOSED);
+    for (DatanodeDetails dn : ratisContainer1.getPipeline().getMachines()) {
+      // Assert that the pipeline has been removed from Node2PipelineMap as well
+      Assert.assertEquals(pipelineSelector.getNode2PipelineMap()
+          .getPipelines(dn.getUuid()).size(), 0);
+    }
+  }
+
+  @Test
+  public void testPipelineCloseWithOpenContainer() throws IOException,
+      TimeoutException, InterruptedException {
+    NavigableSet<ContainerID> setOpen = stateMap.getOpenContainerIDsByPipeline(
+        ratisContainer2.getPipeline().getPipelineName());
+    Assert.assertEquals(1, setOpen.size());
+
+    long cId2 = ratisContainer2.getContainerInfo().getContainerID();
+    mapping
+        .updateContainerState(cId2, HddsProtos.LifeCycleEvent.CREATE);
+    mapping
+        .updateContainerState(cId2, HddsProtos.LifeCycleEvent.CREATED);
+    pipelineSelector.finalizePipeline(ratisContainer2.getPipeline());
+    Assert.assertEquals(ratisContainer2.getPipeline().getLifeCycleState(),
+        HddsProtos.LifeCycleState.CLOSING);
+    Pipeline pipeline2 = pipelineSelector
+        .getPipeline(ratisContainer2.getPipeline().getPipelineName(),
+            ratisContainer2.getContainerInfo().getReplicationType());
+    Assert.assertEquals(pipeline2.getLifeCycleState(),
+        HddsProtos.LifeCycleState.CLOSING);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd31cb6c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java
index a878627..65bd036 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSQLCli.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.ozone.scm;
 
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.server.events.EventQueue;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -115,7 +116,8 @@ public class TestContainerSQLCli {
     cluster.getStorageContainerManager().stop();
 
     nodeManager = cluster.getStorageContainerManager().getScmNodeManager();
-    mapping = new ContainerMapping(conf, nodeManager, 128);
+    mapping = new ContainerMapping(conf, nodeManager, 128,
+        new EventQueue());
     blockManager = new BlockManagerImpl(conf, nodeManager, mapping, null);
 
     // blockManager.allocateBlock() will create containers if there is none


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


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

Posted by su...@apache.org.
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/843dac46
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/843dac46
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/843dac46

Branch: refs/heads/YARN-3409
Commit: 843dac46ff0d125941e994e3df7039e6e9e1e9c2
Parents: bc66343
Author: Weiwei Yang <ww...@apache.org>
Authored: Wed Jan 31 20:28:41 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:38:12 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/843dac46/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/843dac46/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/843dac46/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/843dac46/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


[28/50] [abbrv] 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/110ef21e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/110ef21e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/110ef21e

Branch: refs/heads/YARN-3409
Commit: 110ef21e1d7b4f9bfe47541eeba12be040dca2c9
Parents: 8d3c068
Author: Naganarasimha <na...@apache.org>
Authored: Sun Jan 21 00:53:02 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:38:11 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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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 9b4d91d..f14f17b 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
@@ -163,6 +163,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;
 
@@ -180,15 +182,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;
 
@@ -486,7 +488,7 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     if (request.getApplicationSubmissionContext().getUnmanagedAM()
         || request.getApplicationSubmissionContext()
             .getKeepContainersAcrossApplicationAttempts()) {
-      keepContainerOnUams.add(appId);
+	keepContainerOnUams.add(appId);
     }
     return SubmitApplicationResponse.newInstance();
   }
@@ -504,7 +506,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);
@@ -892,4 +893,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/110ef21e/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/110ef21e/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/110ef21e/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/110ef21e/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


[19/50] [abbrv] hadoop git commit: YARN-8330. Improved publishing ALLOCATED events to ATS. Contributed by Suma Shivaprasad

Posted by su...@apache.org.
YARN-8330.  Improved publishing ALLOCATED events to ATS.
            Contributed by Suma Shivaprasad


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

Branch: refs/heads/YARN-3409
Commit: f93ecf5c1e0b3db27424963814fc01ec43eb76e0
Parents: e95c5e9
Author: Eric Yang <ey...@apache.org>
Authored: Wed Jul 25 18:49:30 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Wed Jul 25 18:49:30 2018 -0400

----------------------------------------------------------------------
 .../rmcontainer/RMContainerImpl.java            | 64 +++++++++++---------
 .../rmcontainer/TestRMContainerImpl.java        | 11 +++-
 2 files changed, 43 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f93ecf5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.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/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index efac666..945e7cb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -244,23 +244,13 @@ public class RMContainerImpl implements RMContainer {
     this.readLock = lock.readLock();
     this.writeLock = lock.writeLock();
 
-    saveNonAMContainerMetaInfo = rmContext.getYarnConfiguration().getBoolean(
-       YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
-       YarnConfiguration
-                 .DEFAULT_APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO);
+    saveNonAMContainerMetaInfo =
+        shouldPublishNonAMContainerEventstoATS(rmContext);
 
     if (container.getId() != null) {
       rmContext.getRMApplicationHistoryWriter().containerStarted(this);
     }
 
-    // If saveNonAMContainerMetaInfo is true, store system metrics for all
-    // containers. If false, and if this container is marked as the AM, metrics
-    // will still be published for this container, but that calculation happens
-    // later.
-    if (saveNonAMContainerMetaInfo && null != container.getId()) {
-      rmContext.getSystemMetricsPublisher().containerCreated(
-          this, this.creationTime);
-    }
     if (this.container != null) {
       this.allocationTags = this.container.getAllocationTags();
     }
@@ -590,8 +580,12 @@ public class RMContainerImpl implements RMContainer {
           container.getNodeId(), container.getContainerId(),
           container.getAllocationTags());
 
-      container.eventHandler.handle(new RMAppAttemptEvent(
-          container.appAttemptId, RMAppAttemptEventType.CONTAINER_ALLOCATED));
+      container.eventHandler.handle(
+          new RMAppAttemptEvent(container.appAttemptId,
+              RMAppAttemptEventType.CONTAINER_ALLOCATED));
+
+      publishNonAMContainerEventstoATS(container);
+
     }
   }
 
@@ -610,9 +604,11 @@ public class RMContainerImpl implements RMContainer {
       // Tell the app
       container.eventHandler.handle(new RMAppRunningOnNodeEvent(container
           .getApplicationAttemptId().getApplicationId(), container.nodeId));
+
+      publishNonAMContainerEventstoATS(container);
     }
   }
-  
+
   private static final class ContainerAcquiredWhileRunningTransition extends
       BaseTransition {
 
@@ -718,17 +714,12 @@ public class RMContainerImpl implements RMContainer {
         container);
 
       boolean saveNonAMContainerMetaInfo =
-          container.rmContext.getYarnConfiguration().getBoolean(
-              YarnConfiguration
-                .APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
-              YarnConfiguration
-                .DEFAULT_APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO);
+          shouldPublishNonAMContainerEventstoATS(container.rmContext);
 
       if (saveNonAMContainerMetaInfo || container.isAMContainer()) {
         container.rmContext.getSystemMetricsPublisher().containerFinished(
             container, container.finishTime);
       }
-
     }
 
     private static void updateAttemptMetrics(RMContainerImpl container) {
@@ -754,6 +745,29 @@ public class RMContainerImpl implements RMContainer {
     }
   }
 
+  private static boolean shouldPublishNonAMContainerEventstoATS(
+      RMContext rmContext) {
+    return rmContext.getYarnConfiguration().getBoolean(
+        YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
+        YarnConfiguration
+            .DEFAULT_APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO);
+  }
+
+  private static void publishNonAMContainerEventstoATS(
+      RMContainerImpl rmContainer) {
+    boolean saveNonAMContainerMetaInfo = shouldPublishNonAMContainerEventstoATS(
+        rmContainer.rmContext);
+
+    // If saveNonAMContainerMetaInfo is true, store system metrics for all
+    // containers. If false, and if this container is marked as the AM, metrics
+    // will still be published for this container, but that calculation happens
+    // later.
+    if (saveNonAMContainerMetaInfo && null != rmContainer.container.getId()) {
+      rmContainer.rmContext.getSystemMetricsPublisher().containerCreated(
+          rmContainer, rmContainer.creationTime);
+    }
+  }
+
   private static final class KillTransition extends FinishedTransition {
 
     @Override
@@ -884,13 +898,5 @@ public class RMContainerImpl implements RMContainer {
     if (containerId != null) {
       rmContext.getRMApplicationHistoryWriter().containerStarted(this);
     }
-    // If saveNonAMContainerMetaInfo is true, store system metrics for all
-    // containers. If false, and if this container is marked as the AM, metrics
-    // will still be published for this container, but that calculation happens
-    // later.
-    if (saveNonAMContainerMetaInfo && null != container.getId()) {
-      rmContext.getSystemMetricsPublisher().containerCreated(
-          this, this.creationTime);
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f93ecf5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.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/rmcontainer/TestRMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
index 1115e8c..bb6591b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
@@ -135,7 +135,6 @@ public class TestRMContainerImpl {
     assertEquals(priority,
         rmContainer.getAllocatedSchedulerKey().getPriority());
     verify(writer).containerStarted(any(RMContainer.class));
-    verify(publisher).containerCreated(any(RMContainer.class), anyLong());
 
     rmContainer.handle(new RMContainerEvent(containerId,
         RMContainerEventType.START));
@@ -150,6 +149,8 @@ public class TestRMContainerImpl {
         RMContainerEventType.LAUNCHED));
     drainDispatcher.await();
     assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+    verify(publisher, times(2)).containerCreated(any(RMContainer.class),
+        anyLong());
     assertEquals("http://host:3465/node/containerlogs/container_1_0001_01_000001/user",
         rmContainer.getLogURL());
 
@@ -240,22 +241,25 @@ public class TestRMContainerImpl {
     assertEquals(priority,
         rmContainer.getAllocatedSchedulerKey().getPriority());
     verify(writer).containerStarted(any(RMContainer.class));
-    verify(publisher).containerCreated(any(RMContainer.class), anyLong());
 
     rmContainer.handle(new RMContainerEvent(containerId,
         RMContainerEventType.START));
     drainDispatcher.await();
     assertEquals(RMContainerState.ALLOCATED, rmContainer.getState());
+    verify(publisher).containerCreated(any(RMContainer.class), anyLong());
 
     rmContainer.handle(new RMContainerEvent(containerId,
         RMContainerEventType.ACQUIRED));
     drainDispatcher.await();
     assertEquals(RMContainerState.ACQUIRED, rmContainer.getState());
+    verify(publisher, times(2)).containerCreated(any(RMContainer.class),
+        anyLong());
 
     rmContainer.handle(new RMContainerEvent(containerId,
         RMContainerEventType.LAUNCHED));
     drainDispatcher.await();
     assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+
     assertEquals("http://host:3465/node/containerlogs/container_1_0001_01_000001/user",
         rmContainer.getLogURL());
 
@@ -340,7 +344,8 @@ public class TestRMContainerImpl {
     // RMContainer should be publishing system metrics for all containers.
     // Since there is 1 AM container and 1 non-AM container, there should be 2
     // container created events and 2 container finished events.
-    verify(publisher, times(2)).containerCreated(any(RMContainer.class), anyLong());
+    verify(publisher, times(4)).containerCreated(any(RMContainer.class),
+        anyLong());
     verify(publisher, times(2)).containerFinished(any(RMContainer.class), anyLong());
   }
 


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


[39/50] [abbrv] 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/1950aee2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1950aee2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1950aee2

Branch: refs/heads/YARN-3409
Commit: 1950aee2d3a8c560adacdf6de278fd582c7ab7d2
Parents: 293ad9e
Author: Sunil G <su...@apache.org>
Authored: Sun Apr 1 18:07:07 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:41:50 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/1950aee2/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/1950aee2/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/1950aee2/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/1950aee2/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/1950aee2/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/1950aee2/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/1950aee2/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/1950aee2/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/1950aee2/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 9041132..38e40ef 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


[35/50] [abbrv] 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/e6b84252
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e6b84252
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e6b84252

Branch: refs/heads/YARN-3409
Commit: e6b84252588c9f4da73f6dead23dc60991596c05
Parents: d3a8187
Author: Naganarasimha <na...@apache.org>
Authored: Tue Feb 27 18:46:16 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:40:34 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/e6b84252/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/e6b84252/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/e6b84252/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


[29/50] [abbrv] 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/bc66343f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bc66343f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bc66343f

Branch: refs/heads/YARN-3409
Commit: bc66343f1de31ed0b4ceaebcba9a5957dc5e0575
Parents: 110ef21
Author: Naganarasimha <na...@apache.org>
Authored: Tue Jan 23 07:18:20 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:38:11 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/bc66343f/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/bc66343f/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/bc66343f/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/bc66343f/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


[24/50] [abbrv] hadoop git commit: HDDS-291. Initialize hadoop metrics system in standalone hdds datanodes. Contributed by Elek Marton.

Posted by su...@apache.org.
HDDS-291. Initialize hadoop metrics system in standalone hdds datanodes. Contributed by Elek Marton.


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

Branch: refs/heads/YARN-3409
Commit: d70d84570575574b7e3ad0f00baf54f1dde76d97
Parents: fd31cb6
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Thu Jul 26 13:17:37 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Thu Jul 26 13:17:37 2018 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java | 2 ++
 .../ozone/container/common/statemachine/SCMConnectionManager.java  | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d70d8457/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
index ddeec87..f359e72 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hdds.HddsUtils;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.statemachine
     .DatanodeStateMachine;
@@ -241,6 +242,7 @@ public class HddsDatanodeService implements ServicePlugin {
         System.exit(1);
       }
       StringUtils.startupShutdownMessage(HddsDatanodeService.class, args, LOG);
+      DefaultMetricsSystem.initialize("HddsDatanode");
       HddsDatanodeService hddsDatanodeService =
           createHddsDatanodeService(conf);
       hddsDatanodeService.start(null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d70d8457/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
index 19722f0..85fb580 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/SCMConnectionManager.java
@@ -67,7 +67,7 @@ public class SCMConnectionManager
     this.rpcTimeout = timeOut.intValue();
     this.scmMachines = new HashMap<>();
     this.conf = conf;
-    jmxBean = MBeans.register("OzoneDataNode",
+    jmxBean = MBeans.register("HddsDatanode",
         "SCMConnectionManager",
         this);
   }


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


[33/50] [abbrv] 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/fbfbd34d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fbfbd34d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fbfbd34d

Branch: refs/heads/YARN-3409
Commit: fbfbd34d0bdb53d390364e8d771c76321212e77a
Parents: 0771f5b
Author: Sunil G <su...@apache.org>
Authored: Fri Feb 23 08:01:58 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:40:34 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/fbfbd34d/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/fbfbd34d/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/fbfbd34d/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/fbfbd34d/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/fbfbd34d/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/fbfbd34d/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/fbfbd34d/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/fbfbd34d/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/fbfbd34d/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/fbfbd34d/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/fbfbd34d/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 8421eea..2615beb 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/fbfbd34d/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 66065e3..2fe98d8 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;
@@ -99,6 +100,7 @@ public class RMActiveServiceContext {
   private ApplicationMasterService applicationMasterService;
 
   private RMNodeLabelsManager nodeLabelManager;
+  private NodeAttributesManager nodeAttributesManager;
   private RMDelegatedNodeLabelsUpdater rmDelegatedNodeLabelsUpdater;
   private long epoch;
   private Clock systemClock = SystemClock.getInstance();
@@ -404,6 +406,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/fbfbd34d/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 eb91a31..2e2eadf 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;
@@ -132,6 +133,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/fbfbd34d/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 84e0f6f..46930ca 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;
@@ -504,6 +505,11 @@ public class RMContextImpl implements RMContext {
   }
 
   @Override
+  public void setNodeAttributesManager(NodeAttributesManager mgr) {
+    activeServiceContext.setNodeAttributesManager(mgr);
+  }
+
+  @Override
   public AllocationTagsManager getAllocationTagsManager() {
     return activeServiceContext.getAllocationTagsManager();
   }
@@ -620,4 +626,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/fbfbd34d/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 0b7e87c..2a869bd 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
@@ -61,15 +61,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;
@@ -507,6 +509,11 @@ public class ResourceManager extends CompositeService implements Recoverable {
     return new RMNodeLabelsManager();
   }
 
+  protected NodeAttributesManager createNodeAttributesManager()
+      throws InstantiationException, IllegalAccessException {
+    return new NodeAttributesManagerImpl();
+  }
+
   protected AllocationTagsManager createAllocationTagsManager() {
     return new AllocationTagsManager(this.rmContext);
   }
@@ -642,6 +649,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
       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/fbfbd34d/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/fbfbd34d/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/fbfbd34d/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


[16/50] [abbrv] hadoop git commit: HADOOP-15395. DefaultImpersonationProvider fails to parse proxy user config if username has . in it. Contributed by Ajay Kumar.

Posted by su...@apache.org.
HADOOP-15395. DefaultImpersonationProvider fails to parse proxy user config if username has . in it. Contributed by Ajay Kumar.


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

Branch: refs/heads/YARN-3409
Commit: 5f0b924360b345f491c2d6693882f1069c7f3508
Parents: 3c4fbc6
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Wed Jul 25 21:09:11 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Wed Jul 25 21:09:11 2018 +0530

----------------------------------------------------------------------
 .../authorize/DefaultImpersonationProvider.java |   4 +-
 .../TestDefaultImpersonationProvider.java       | 100 +++++++++++++++++++
 2 files changed, 102 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f0b9243/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/DefaultImpersonationProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/DefaultImpersonationProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/DefaultImpersonationProvider.java
index 26cd7ab..b766d5c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/DefaultImpersonationProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/DefaultImpersonationProvider.java
@@ -75,9 +75,9 @@ public class DefaultImpersonationProvider implements ImpersonationProvider {
     //   $configPrefix.[ANY].hosts
     //
     String prefixRegEx = configPrefix.replace(".", "\\.");
-    String usersGroupsRegEx = prefixRegEx + "[^.]*(" +
+    String usersGroupsRegEx = prefixRegEx + "[\\S]*(" +
         Pattern.quote(CONF_USERS) + "|" + Pattern.quote(CONF_GROUPS) + ")";
-    String hostsRegEx = prefixRegEx + "[^.]*" + Pattern.quote(CONF_HOSTS);
+    String hostsRegEx = prefixRegEx + "[\\S]*" + Pattern.quote(CONF_HOSTS);
 
   // get list of users and groups per proxyuser
     Map<String,String> allMatchKeys = 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f0b9243/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestDefaultImpersonationProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestDefaultImpersonationProvider.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestDefaultImpersonationProvider.java
new file mode 100644
index 0000000..ef86697
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestDefaultImpersonationProvider.java
@@ -0,0 +1,100 @@
+/**
+ * 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.security.authorize;
+
+import static org.mockito.Mockito.when;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.mockito.Mockito;
+
+/**
+ * Test class for @DefaultImpersonationProvider
+ */
+public class TestDefaultImpersonationProvider {
+
+  private String proxyUser;
+  private String user;
+  private DefaultImpersonationProvider provider;
+  private UserGroupInformation userGroupInformation = Mockito
+      .mock(UserGroupInformation.class);
+  private UserGroupInformation realUserUGI = Mockito
+      .mock(UserGroupInformation.class);
+  private Configuration conf;
+  @Rule
+  public Timeout globalTimeout = new Timeout(10000);
+
+  @Before
+  public void setup() {
+    conf = new Configuration();
+    provider = new DefaultImpersonationProvider();
+
+    // Setup 3 proxy users
+    conf.set("hadoop.proxyuser.fakeuser.groups", "*");
+    conf.set("hadoop.proxyuser.fakeuser.hosts", "*");
+    conf.set("hadoop.proxyuser.test.user.groups", "*");
+    conf.set("hadoop.proxyuser.test.user.hosts", "*");
+    conf.set("hadoop.proxyuser.test user2.groups", "*");
+    conf.set("hadoop.proxyuser.test user2.hosts", "*");
+    provider.setConf(conf);
+    provider.init(ProxyUsers.CONF_HADOOP_PROXYUSER);
+  }
+
+  @Test
+  public void testAuthorizationSuccess() throws AuthorizationException {
+    proxyUser = "fakeuser";
+    user = "dummyUser";
+    when(realUserUGI.getShortUserName()).thenReturn(proxyUser);
+    when(userGroupInformation.getRealUser()).thenReturn(realUserUGI);
+    provider.authorize(userGroupInformation, "2.2.2.2");
+
+    user = "somerandomuser";
+    proxyUser = "test.user";
+    when(realUserUGI.getShortUserName()).thenReturn(proxyUser);
+    when(userGroupInformation.getRealUser()).thenReturn(realUserUGI);
+    provider.authorize(userGroupInformation, "2.2.2.2");
+  }
+
+  @Test
+  public void testAuthorizationFailure() throws Exception {
+    user = "dummyUser";
+    proxyUser = "test user2";
+    when(realUserUGI.getShortUserName()).thenReturn(proxyUser);
+    when(realUserUGI.getUserName()).thenReturn(proxyUser);
+    when(userGroupInformation.getUserName()).thenReturn(user);
+    when(userGroupInformation.getRealUser()).thenReturn(realUserUGI);
+    LambdaTestUtils.intercept(AuthorizationException.class, "User: "
+        + proxyUser + " is not allowed to impersonate " + user, () ->
+        provider.authorize(userGroupInformation, "2.2.2.2"));
+  }
+
+  @After
+  public void clear() {
+    provider = null;
+    conf = null;
+    userGroupInformation = null;
+    realUserUGI = null;
+  }
+
+}


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


[17/50] [abbrv] hadoop git commit: YARN-4606. CapacityScheduler: applications could get starved because computation of #activeUsers considers pending apps. Contributed by Manikandan R

Posted by su...@apache.org.
YARN-4606. CapacityScheduler: applications could get starved because computation of #activeUsers considers pending apps. Contributed by Manikandan R


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

Branch: refs/heads/YARN-3409
Commit: 9485c9aee6e9bb935c3e6ae4da81d70b621781de
Parents: 5f0b924
Author: Eric E Payne <er...@oath.com>
Authored: Wed Jul 25 16:22:04 2018 +0000
Committer: Eric E Payne <er...@oath.com>
Committed: Wed Jul 25 16:22:04 2018 +0000

----------------------------------------------------------------------
 .../scheduler/capacity/UsersManager.java        |  27 +++-
 .../capacity/TestCapacityScheduler.java         | 128 +++++++++++++++++++
 .../capacity/TestContainerAllocation.java       |  43 +++++++
 3 files changed, 197 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9485c9ae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.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/UsersManager.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/UsersManager.java
index 747a488..83ee6c0 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/UsersManager.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/UsersManager.java
@@ -85,6 +85,7 @@ public class UsersManager implements AbstractUsersManager {
 
   private final QueueMetrics metrics;
   private AtomicInteger activeUsers = new AtomicInteger(0);
+  private AtomicInteger activeUsersWithOnlyPendingApps = new AtomicInteger(0);
   private Map<String, Set<ApplicationId>> usersApplications =
       new HashMap<String, Set<ApplicationId>>();
 
@@ -671,9 +672,23 @@ public class UsersManager implements AbstractUsersManager {
     // update in local storage
     userLimitPerSchedulingMode.put(schedulingMode, computedUserLimit);
 
+    computeNumActiveUsersWithOnlyPendingApps();
+
     return userLimitPerSchedulingMode;
   }
 
+  // This method is called within the lock.
+  private void computeNumActiveUsersWithOnlyPendingApps() {
+    int numPendingUsers = 0;
+    for (User user : users.values()) {
+      if ((user.getPendingApplications() > 0)
+          && (user.getActiveApplications() <= 0)) {
+        numPendingUsers++;
+      }
+    }
+    activeUsersWithOnlyPendingApps = new AtomicInteger(numPendingUsers);
+  }
+
   private Resource computeUserLimit(String userName, Resource clusterResource,
       String nodePartition, SchedulingMode schedulingMode, boolean activeUser) {
     Resource partitionResource = labelManager.getResourceByLabel(nodePartition,
@@ -839,6 +854,11 @@ public class UsersManager implements AbstractUsersManager {
     try {
       this.writeLock.lock();
 
+      User userDesc = getUser(user);
+      if (userDesc != null && userDesc.getActiveApplications() <= 0) {
+        return;
+      }
+
       Set<ApplicationId> userApps = usersApplications.get(user);
       if (userApps == null) {
         userApps = new HashSet<ApplicationId>();
@@ -893,7 +913,7 @@ public class UsersManager implements AbstractUsersManager {
 
   @Override
   public int getNumActiveUsers() {
-    return activeUsers.get();
+    return activeUsers.get() + activeUsersWithOnlyPendingApps.get();
   }
 
   float sumActiveUsersTimesWeights() {
@@ -1090,4 +1110,9 @@ public class UsersManager implements AbstractUsersManager {
       this.writeLock.unlock();
     }
   }
+
+  @VisibleForTesting
+  public int getNumActiveUsersWithOnlyPendingApps() {
+    return activeUsersWithOnlyPendingApps.get();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9485c9ae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.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/TestCapacityScheduler.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/TestCapacityScheduler.java
index 79cdcfe..8d948b5 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/TestCapacityScheduler.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/TestCapacityScheduler.java
@@ -4978,4 +4978,132 @@ public class TestCapacityScheduler extends CapacitySchedulerTestBase {
     Assert.assertEquals(AllocationState.QUEUE_SKIPPED,
         ContainerAllocation.QUEUE_SKIPPED.getAllocationState());
   }
+
+  @Test
+  public void testMoveAppWithActiveUsersWithOnlyPendingApps() throws Exception {
+
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+      ResourceScheduler.class);
+
+    CapacitySchedulerConfiguration newConf =
+        new CapacitySchedulerConfiguration(conf);
+
+    // Define top-level queues
+    newConf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] { "a", "b" });
+
+    newConf.setCapacity(A, 50);
+    newConf.setCapacity(B, 50);
+
+    // Define 2nd-level queues
+    newConf.setQueues(A, new String[] { "a1" });
+    newConf.setCapacity(A1, 100);
+    newConf.setUserLimitFactor(A1, 2.0f);
+    newConf.setMaximumAMResourcePercentPerPartition(A1, "", 0.1f);
+
+    newConf.setQueues(B, new String[] { "b1" });
+    newConf.setCapacity(B1, 100);
+    newConf.setUserLimitFactor(B1, 2.0f);
+
+    LOG.info("Setup top-level queues a and b");
+
+    MockRM rm = new MockRM(newConf);
+    rm.start();
+
+    CapacityScheduler scheduler =
+        (CapacityScheduler) rm.getResourceScheduler();
+
+    MockNM nm1 = rm.registerNode("h1:1234", 16 * GB);
+
+    // submit an app
+    RMApp app = rm.submitApp(GB, "test-move-1", "u1", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app, rm, nm1);
+
+    ApplicationAttemptId appAttemptId =
+        rm.getApplicationReport(app.getApplicationId())
+            .getCurrentApplicationAttemptId();
+
+    RMApp app2 = rm.submitApp(1 * GB, "app", "u2", null, "a1");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm1);
+
+    RMApp app3 = rm.submitApp(1 * GB, "app", "u3", null, "a1");
+
+    RMApp app4 = rm.submitApp(1 * GB, "app", "u4", null, "a1");
+
+    // Each application asks 50 * 1GB containers
+    am1.allocate("*", 1 * GB, 50, null);
+    am2.allocate("*", 1 * GB, 50, null);
+
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    // check preconditions
+    List<ApplicationAttemptId> appsInA1 = scheduler.getAppsInQueue("a1");
+    assertEquals(4, appsInA1.size());
+    String queue =
+        scheduler.getApplicationAttempt(appsInA1.get(0)).getQueue()
+            .getQueueName();
+    Assert.assertEquals("a1", queue);
+
+    List<ApplicationAttemptId> appsInA = scheduler.getAppsInQueue("a");
+    assertTrue(appsInA.contains(appAttemptId));
+    assertEquals(4, appsInA.size());
+
+    List<ApplicationAttemptId> appsInRoot = scheduler.getAppsInQueue("root");
+    assertTrue(appsInRoot.contains(appAttemptId));
+    assertEquals(4, appsInRoot.size());
+
+    List<ApplicationAttemptId> appsInB1 = scheduler.getAppsInQueue("b1");
+    assertTrue(appsInB1.isEmpty());
+
+    List<ApplicationAttemptId> appsInB = scheduler.getAppsInQueue("b");
+    assertTrue(appsInB.isEmpty());
+
+    UsersManager um =
+        (UsersManager) scheduler.getQueue("a1").getAbstractUsersManager();
+
+    assertEquals(4, um.getNumActiveUsers());
+    assertEquals(2, um.getNumActiveUsersWithOnlyPendingApps());
+
+    // now move the app
+    scheduler.moveAllApps("a1", "b1");
+
+    //Triggering this event so that user limit computation can
+    //happen again
+    for (int i = 0; i < 10; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      Thread.sleep(500);
+    }
+
+    // check postconditions
+    appsInB1 = scheduler.getAppsInQueue("b1");
+
+    assertEquals(4, appsInB1.size());
+    queue =
+        scheduler.getApplicationAttempt(appsInB1.get(0)).getQueue()
+            .getQueueName();
+    Assert.assertEquals("b1", queue);
+
+    appsInB = scheduler.getAppsInQueue("b");
+    assertTrue(appsInB.contains(appAttemptId));
+    assertEquals(4, appsInB.size());
+
+    appsInRoot = scheduler.getAppsInQueue("root");
+    assertTrue(appsInRoot.contains(appAttemptId));
+    assertEquals(4, appsInRoot.size());
+
+    List<ApplicationAttemptId> oldAppsInA1 = scheduler.getAppsInQueue("a1");
+    assertEquals(0, oldAppsInA1.size());
+
+    UsersManager um_b1 =
+        (UsersManager) scheduler.getQueue("b1").getAbstractUsersManager();
+
+    assertEquals(2, um_b1.getNumActiveUsers());
+    assertEquals(2, um_b1.getNumActiveUsersWithOnlyPendingApps());
+
+    appsInB1 = scheduler.getAppsInQueue("b1");
+    assertEquals(4, appsInB1.size());
+    rm.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9485c9ae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.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/TestContainerAllocation.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/TestContainerAllocation.java
index 25e535a..b9bfc2a 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/TestContainerAllocation.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/TestContainerAllocation.java
@@ -941,4 +941,47 @@ public class TestContainerAllocation {
 
     rm1.close();
   }
+
+  @Test
+  public void testActiveUsersWithOnlyPendingApps() throws Exception {
+
+    CapacitySchedulerConfiguration newConf =
+        new CapacitySchedulerConfiguration(conf);
+    newConf.setMaximumAMResourcePercentPerPartition(
+        CapacitySchedulerConfiguration.ROOT + ".default", "", 0.2f);
+    MockRM rm1 = new MockRM(newConf);
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB);
+
+    RMApp app1 = rm1.submitApp(1 * GB, "app", "u1", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    RMApp app2 = rm1.submitApp(1 * GB, "app", "u2", null, "default");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+
+    RMApp app3 = rm1.submitApp(1 * GB, "app", "u3", null, "default");
+
+    RMApp app4 = rm1.submitApp(1 * GB, "app", "u4", null, "default");
+
+    // Each application asks 50 * 1GB containers
+    am1.allocate("*", 1 * GB, 50, null);
+    am2.allocate("*", 1 * GB, 50, null);
+
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    for (int i = 0; i < 10; i++) {
+      cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+      Thread.sleep(1000);
+    }
+    LeafQueue lq = (LeafQueue) cs.getQueue("default");
+    UsersManager um = (UsersManager) lq.getAbstractUsersManager();
+
+    Assert.assertEquals(4, um.getNumActiveUsers());
+    Assert.assertEquals(2, um.getNumActiveUsersWithOnlyPendingApps());
+    Assert.assertEquals(2, lq.getMetrics().getAppsPending());
+    rm1.close();
+  }
 }


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


[20/50] [abbrv] hadoop git commit: HADOOP-15611. Log more details for FairCallQueue. Contributed by Ryan Wu.

Posted by su...@apache.org.
HADOOP-15611. Log more details for FairCallQueue. Contributed by Ryan Wu.


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

Branch: refs/heads/YARN-3409
Commit: 9089790cabb4771198be0fe64c1317a3ff1c80f1
Parents: f93ecf5
Author: Yiqun Lin <yq...@apache.org>
Authored: Thu Jul 26 18:08:28 2018 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Thu Jul 26 18:08:28 2018 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java   | 8 ++++++++
 .../org/apache/hadoop/ipc/WeightedRoundRobinMultiplexer.java | 3 +++
 2 files changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9089790c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
index f12ecb6..8bb0ce4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
@@ -391,6 +391,7 @@ public class DecayRpcScheduler implements RpcScheduler,
    * counts current.
    */
   private void decayCurrentCounts() {
+    LOG.debug("Start to decay current counts.");
     try {
       long totalDecayedCount = 0;
       long totalRawCount = 0;
@@ -410,7 +411,12 @@ public class DecayRpcScheduler implements RpcScheduler,
         totalDecayedCount += nextValue;
         decayedCount.set(nextValue);
 
+        LOG.debug("Decaying counts for the user: {}, " +
+            "its decayedCount: {}, rawCount: {}", entry.getKey(),
+            nextValue, rawCount.get());
         if (nextValue == 0) {
+          LOG.debug("The decayed count for the user {} is zero " +
+              "and being cleaned.", entry.getKey());
           // We will clean up unused keys here. An interesting optimization
           // might be to have an upper bound on keyspace in callCounts and only
           // clean once we pass it.
@@ -422,6 +428,8 @@ public class DecayRpcScheduler implements RpcScheduler,
       totalDecayedCallCount.set(totalDecayedCount);
       totalRawCallCount.set(totalRawCount);
 
+      LOG.debug("After decaying the stored counts, totalDecayedCount: {}, " +
+          "totalRawCallCount: {}.", totalDecayedCount, totalRawCount);
       // Now refresh the cache of scheduling decisions
       recomputeScheduleCache();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9089790c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WeightedRoundRobinMultiplexer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WeightedRoundRobinMultiplexer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WeightedRoundRobinMultiplexer.java
index d308725..096cc1a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WeightedRoundRobinMultiplexer.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WeightedRoundRobinMultiplexer.java
@@ -109,6 +109,9 @@ public class WeightedRoundRobinMultiplexer implements RpcMultiplexer {
     // Finally, reset requestsLeft. This will enable moveToNextQueue to be
     // called again, for the new currentQueueIndex
     this.requestsLeft.set(this.queueWeights[nextIdx]);
+    LOG.debug("Moving to next queue from queue index {} to index {}, " +
+        "number of requests left for current queue: {}.",
+        thisIdx, nextIdx, requestsLeft);
   }
 
   /**


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


[27/50] [abbrv] hadoop git commit: YARN-8429. Improve diagnostic message when artifact is not set properly. Contributed by Gour Saha

Posted by su...@apache.org.
YARN-8429. Improve diagnostic message when artifact is not set properly.
           Contributed by Gour Saha


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

Branch: refs/heads/YARN-3409
Commit: 8d3c068e59fdddd18e3f8260713fee83c458aa1d
Parents: 77721f3
Author: Eric Yang <ey...@apache.org>
Authored: Thu Jul 26 20:02:13 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Thu Jul 26 20:02:13 2018 -0400

----------------------------------------------------------------------
 .../exceptions/RestApiErrorMessages.java        |  6 +-
 .../provider/AbstractClientProvider.java        | 14 ++---
 .../defaultImpl/DefaultClientProvider.java      | 22 ++++---
 .../provider/docker/DockerClientProvider.java   | 15 ++---
 .../provider/tarball/TarballClientProvider.java | 27 ++++----
 .../yarn/service/utils/ServiceApiUtil.java      |  4 +-
 .../hadoop/yarn/service/TestServiceApiUtil.java |  9 ++-
 .../providers/TestAbstractClientProvider.java   | 29 ++++-----
 .../providers/TestDefaultClientProvider.java    | 66 ++++++++++++++++++++
 9 files changed, 138 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
index 5b3c72c..f10d884 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/exceptions/RestApiErrorMessages.java
@@ -50,6 +50,10 @@ public interface RestApiErrorMessages {
       "Artifact id (like docker image name) is either empty or not provided";
   String ERROR_ARTIFACT_ID_FOR_COMP_INVALID =
       ERROR_ARTIFACT_ID_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_ARTIFACT_PATH_FOR_COMP_INVALID = "For component %s with %s "
+      + "artifact, path does not exist: %s";
+  String ERROR_CONFIGFILE_DEST_FILE_FOR_COMP_NOT_ABSOLUTE = "For component %s "
+      + "with %s artifact, dest_file must be a relative path: %s";
 
   String ERROR_RESOURCE_INVALID = "Resource is not provided";
   String ERROR_RESOURCE_FOR_COMP_INVALID =
@@ -89,7 +93,7 @@ public interface RestApiErrorMessages {
   String ERROR_ABSENT_NUM_OF_INSTANCE =
       "Num of instances should appear either globally or per component";
   String ERROR_ABSENT_LAUNCH_COMMAND =
-      "Launch_command is required when type is not DOCKER";
+      "launch_command is required when type is not DOCKER";
 
   String ERROR_QUICKLINKS_FOR_COMP_INVALID = "Quicklinks specified at"
       + " component level, needs corresponding values set at service level";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java
index 672c435..ae79619 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractClientProvider.java
@@ -68,18 +68,18 @@ public abstract class AbstractClientProvider {
    * Validate the artifact.
    * @param artifact
    */
-  public abstract void validateArtifact(Artifact artifact, FileSystem
-      fileSystem) throws IOException;
+  public abstract void validateArtifact(Artifact artifact, String compName,
+      FileSystem fileSystem) throws IOException;
 
-  protected abstract void validateConfigFile(ConfigFile configFile, FileSystem
-      fileSystem) throws IOException;
+  protected abstract void validateConfigFile(ConfigFile configFile,
+      String compName, FileSystem fileSystem) throws IOException;
 
   /**
    * Validate the config files.
    * @param configFiles config file list
    * @param fs file system
    */
-  public void validateConfigFiles(List<ConfigFile> configFiles,
+  public void validateConfigFiles(List<ConfigFile> configFiles, String compName,
       FileSystem fs) throws IOException {
     Set<String> destFileSet = new HashSet<>();
 
@@ -128,7 +128,7 @@ public abstract class AbstractClientProvider {
       }
 
       if (StringUtils.isEmpty(file.getDestFile())) {
-        throw new IllegalArgumentException("Dest_file is empty.");
+        throw new IllegalArgumentException("dest_file is empty.");
       }
 
       if (destFileSet.contains(file.getDestFile())) {
@@ -144,7 +144,7 @@ public abstract class AbstractClientProvider {
       }
 
       // provider-specific validation
-      validateConfigFile(file, fs);
+      validateConfigFile(file, compName, fs);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java
index 0920a9c..999a8dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/defaultImpl/DefaultClientProvider.java
@@ -17,13 +17,16 @@
  */
 package org.apache.hadoop.yarn.service.provider.defaultImpl;
 
+import java.io.IOException;
+import java.nio.file.Paths;
+
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
 import org.apache.hadoop.yarn.service.api.records.Artifact;
 import org.apache.hadoop.yarn.service.api.records.ConfigFile;
+import org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages;
+import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
 
-import java.io.IOException;
-import java.nio.file.Paths;
+import com.google.common.annotations.VisibleForTesting;
 
 public class DefaultClientProvider extends AbstractClientProvider {
 
@@ -31,16 +34,19 @@ public class DefaultClientProvider extends AbstractClientProvider {
   }
 
   @Override
-  public void validateArtifact(Artifact artifact, FileSystem fileSystem) {
+  public void validateArtifact(Artifact artifact, String compName,
+      FileSystem fileSystem) {
   }
 
   @Override
-  protected void validateConfigFile(ConfigFile configFile, FileSystem
-      fileSystem) throws IOException {
+  @VisibleForTesting
+  public void validateConfigFile(ConfigFile configFile, String compName,
+      FileSystem fileSystem) throws IOException {
     // validate dest_file is not absolute
     if (Paths.get(configFile.getDestFile()).isAbsolute()) {
-      throw new IllegalArgumentException(
-          "Dest_file must not be absolute path: " + configFile.getDestFile());
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_CONFIGFILE_DEST_FILE_FOR_COMP_NOT_ABSOLUTE,
+          compName, "no", configFile.getDestFile()));
     }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java
index f91742e..901d779 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/docker/DockerClientProvider.java
@@ -35,19 +35,20 @@ public class DockerClientProvider extends AbstractClientProvider
   }
 
   @Override
-  public void validateArtifact(Artifact artifact, FileSystem fileSystem) {
+  public void validateArtifact(Artifact artifact, String compName,
+      FileSystem fileSystem) {
     if (artifact == null) {
-      throw new IllegalArgumentException(
-          RestApiErrorMessages.ERROR_ARTIFACT_INVALID);
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_ARTIFACT_FOR_COMP_INVALID, compName));
     }
     if (StringUtils.isEmpty(artifact.getId())) {
-      throw new IllegalArgumentException(
-          RestApiErrorMessages.ERROR_ARTIFACT_ID_INVALID);
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName));
     }
   }
 
   @Override
-  protected void validateConfigFile(ConfigFile configFile, FileSystem
-      fileSystem) throws IOException {
+  protected void validateConfigFile(ConfigFile configFile, String compName,
+      FileSystem fileSystem) throws IOException {
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java
index 3b890fd..b801e0c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/tarball/TarballClientProvider.java
@@ -36,30 +36,33 @@ public class TarballClientProvider extends AbstractClientProvider
   }
 
   @Override
-  public void validateArtifact(Artifact artifact, FileSystem fs)
-      throws IOException {
+  public void validateArtifact(Artifact artifact, String compName,
+      FileSystem fs) throws IOException {
     if (artifact == null) {
-      throw new IllegalArgumentException(
-          RestApiErrorMessages.ERROR_ARTIFACT_INVALID);
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_ARTIFACT_FOR_COMP_INVALID, compName));
     }
     if (StringUtils.isEmpty(artifact.getId())) {
-      throw new IllegalArgumentException(
-          RestApiErrorMessages.ERROR_ARTIFACT_ID_INVALID);
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName));
     }
     Path p = new Path(artifact.getId());
     if (!fs.exists(p)) {
-      throw new IllegalArgumentException( "Artifact tarball does not exist "
-          + artifact.getId());
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_ARTIFACT_PATH_FOR_COMP_INVALID, compName,
+          Artifact.TypeEnum.TARBALL.name(), artifact.getId()));
     }
   }
 
   @Override
-  protected void validateConfigFile(ConfigFile configFile, FileSystem
-      fileSystem) throws IOException {
+  protected void validateConfigFile(ConfigFile configFile, String compName,
+      FileSystem fileSystem) throws IOException {
     // validate dest_file is not absolute
     if (Paths.get(configFile.getDestFile()).isAbsolute()) {
-      throw new IllegalArgumentException(
-          "Dest_file must not be absolute path: " + configFile.getDestFile());
+      throw new IllegalArgumentException(String.format(
+          RestApiErrorMessages.ERROR_CONFIGFILE_DEST_FILE_FOR_COMP_NOT_ABSOLUTE,
+          compName, Artifact.TypeEnum.TARBALL.name(),
+          configFile.getDestFile()));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
index 447250f..bebf52c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
@@ -282,7 +282,7 @@ public class ServiceApiUtil {
 
     AbstractClientProvider compClientProvider = ProviderFactory
         .getClientProvider(comp.getArtifact());
-    compClientProvider.validateArtifact(comp.getArtifact(), fs);
+    compClientProvider.validateArtifact(comp.getArtifact(), comp.getName(), fs);
 
     if (comp.getLaunchCommand() == null && (comp.getArtifact() == null || comp
         .getArtifact().getType() != Artifact.TypeEnum.DOCKER)) {
@@ -299,7 +299,7 @@ public class ServiceApiUtil {
               + ": " + comp.getNumberOfContainers(), comp.getName()));
     }
     compClientProvider.validateConfigFiles(comp.getConfiguration()
-        .getFiles(), fs);
+        .getFiles(), comp.getName(), fs);
 
     MonitorUtils.getProbe(comp.getReadinessCheck());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
index ae031d4..47b2803 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
@@ -227,14 +227,16 @@ public class TestServiceApiUtil {
     // no artifact id fails with default type
     Artifact artifact = new Artifact();
     app.setArtifact(artifact);
-    Component comp = ServiceTestUtils.createComponent("comp1");
+    String compName = "comp1";
+    Component comp = ServiceTestUtils.createComponent(compName);
 
     app.setComponents(Collections.singletonList(comp));
     try {
       ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
       Assert.fail(EXCEPTION_PREFIX + "service with no artifact id");
     } catch (IllegalArgumentException e) {
-      assertEquals(ERROR_ARTIFACT_ID_INVALID, e.getMessage());
+      assertEquals(String.format(ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName),
+          e.getMessage());
     }
 
     // no artifact id fails with SERVICE type
@@ -252,7 +254,8 @@ public class TestServiceApiUtil {
       ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
       Assert.fail(EXCEPTION_PREFIX + "service with no artifact id");
     } catch (IllegalArgumentException e) {
-      assertEquals(ERROR_ARTIFACT_ID_INVALID, e.getMessage());
+      assertEquals(String.format(ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName),
+          e.getMessage());
     }
 
     // everything valid here

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java
index 1d6be91..ae62608 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestAbstractClientProvider.java
@@ -45,12 +45,12 @@ public class TestAbstractClientProvider {
 
   private static class ClientProvider extends AbstractClientProvider {
     @Override
-    public void validateArtifact(Artifact artifact, FileSystem fileSystem)
-        throws IOException {
+    public void validateArtifact(Artifact artifact, String compName,
+        FileSystem fileSystem) throws IOException {
     }
 
     @Override
-    protected void validateConfigFile(ConfigFile configFile,
+    protected void validateConfigFile(ConfigFile configFile, String compName,
         FileSystem fileSystem) throws IOException {
     }
   }
@@ -62,33 +62,34 @@ public class TestAbstractClientProvider {
     FileStatus mockFileStatus = mock(FileStatus.class);
     when(mockFs.exists(anyObject())).thenReturn(true);
 
+    String compName = "sleeper";
     ConfigFile configFile = new ConfigFile();
     List<ConfigFile> configFiles = new ArrayList<>();
     configFiles.add(configFile);
 
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "null file type");
     } catch (IllegalArgumentException e) {
     }
 
     configFile.setType(ConfigFile.TypeEnum.TEMPLATE);
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "empty src_file for type template");
     } catch (IllegalArgumentException e) {
     }
 
     configFile.setSrcFile("srcfile");
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "empty dest file");
     } catch (IllegalArgumentException e) {
     }
 
     configFile.setDestFile("destfile");
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
     } catch (IllegalArgumentException e) {
       Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
     }
@@ -99,21 +100,21 @@ public class TestAbstractClientProvider {
     configFile.setDestFile("path/destfile2");
     configFiles.add(configFile);
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "dest file with multiple path elements");
     } catch (IllegalArgumentException e) {
     }
 
     configFile.setDestFile("/path/destfile2");
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
     } catch (IllegalArgumentException e) {
       Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
     }
 
     configFile.setDestFile("destfile");
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "duplicate dest file");
     } catch (IllegalArgumentException e) {
     }
@@ -125,14 +126,14 @@ public class TestAbstractClientProvider {
     configFile.setDestFile("path/destfile3");
     configFiles.add(configFile);
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "dest file with multiple path elements");
     } catch (IllegalArgumentException e) {
     }
 
     configFile.setDestFile("/path/destfile3");
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "src file should be specified");
     } catch (IllegalArgumentException e) {
     }
@@ -140,7 +141,7 @@ public class TestAbstractClientProvider {
     //should succeed
     configFile.setSrcFile("srcFile");
     configFile.setDestFile("destfile3");
-    clientProvider.validateConfigFiles(configFiles, mockFs);
+    clientProvider.validateConfigFiles(configFiles, compName, mockFs);
 
     when(mockFileStatus.isDirectory()).thenReturn(true);
     when(mockFs.getFileStatus(new Path("srcFile")))
@@ -154,7 +155,7 @@ public class TestAbstractClientProvider {
     configFiles.add(configFile);
 
     try {
-      clientProvider.validateConfigFiles(configFiles, mockFs);
+      clientProvider.validateConfigFiles(configFiles, compName, mockFs);
       Assert.fail(EXCEPTION_PREFIX + "src file is a directory");
     } catch (IllegalArgumentException e) {
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d3c068e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestDefaultClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestDefaultClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestDefaultClientProvider.java
new file mode 100644
index 0000000..366515c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/providers/TestDefaultClientProvider.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.service.providers;
+
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.yarn.service.api.records.ConfigFile;
+import org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages;
+import org.apache.hadoop.yarn.service.provider.defaultImpl.DefaultClientProvider;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestDefaultClientProvider {
+  private static final String EXCEPTION_PREFIX = "Should have thrown "
+      + "exception: ";
+  private static final String NO_EXCEPTION_PREFIX = "Should not have thrown "
+      + "exception: ";
+
+  @Test
+  public void testConfigFile() throws IOException {
+    DefaultClientProvider defaultClientProvider = new DefaultClientProvider();
+    FileSystem mockFs = mock(FileSystem.class);
+    when(mockFs.exists(anyObject())).thenReturn(true);
+
+    String compName = "sleeper";
+    ConfigFile configFile = new ConfigFile();
+    configFile.setDestFile("/var/tmp/a.txt");
+
+    try {
+      defaultClientProvider.validateConfigFile(configFile, compName, mockFs);
+      Assert.fail(EXCEPTION_PREFIX + " dest_file must be relative");
+    } catch (IllegalArgumentException e) {
+      String actualMsg = String.format(
+          RestApiErrorMessages.ERROR_CONFIGFILE_DEST_FILE_FOR_COMP_NOT_ABSOLUTE,
+          compName, "no", configFile.getDestFile());
+      Assert.assertEquals(actualMsg, e.getLocalizedMessage());
+    }
+
+    configFile.setDestFile("../a.txt");
+    try {
+      defaultClientProvider.validateConfigFile(configFile, compName, mockFs);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getLocalizedMessage());
+    }
+  }
+}


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


[30/50] [abbrv] 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/8017ef75
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8017ef75
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8017ef75

Branch: refs/heads/YARN-3409
Commit: 8017ef75b94570ae09c4930d70d5cd1110fd6ddb
Parents: 843dac4
Author: bibinchundatt <bi...@apache.org>
Authored: Fri Feb 2 10:31:00 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:38:12 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/8017ef75/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/8017ef75/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/8017ef75/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/8017ef75/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


[47/50] [abbrv] 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/4770a85d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4770a85d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4770a85d

Branch: refs/heads/YARN-3409
Commit: 4770a85da48b0aa17bb6b517f46e7b2f08095eb5
Parents: 1ea02a0
Author: bibinchundatt <bi...@apache.org>
Authored: Sun May 13 09:35:11 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:45:20 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   | 115 ++++++++-------
 .../nodelabels/NodeLabelsUtils.java             |  19 +++
 .../webapp/dao/NodeAttributeInfo.java           |   4 +-
 .../resourcemanager/TestClientRMService.java    | 110 ++++++++------
 .../TestResourceTrackerService.java             |   4 +-
 .../TestFileSystemNodeAttributeStore.java       |   6 +-
 .../nodelabels/TestNodeAttributesManager.java   |   9 +-
 35 files changed, 1118 insertions(+), 357 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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 d3f4081..4fdfee9 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
@@ -263,11 +263,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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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/4770a85d/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


[12/50] [abbrv] hadoop git commit: HDDS-285. Create a generic Metadata Iterator. Contributed by Bharat Viswanadham.

Posted by su...@apache.org.
HDDS-285. Create a generic Metadata Iterator. Contributed by Bharat Viswanadham.


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

Branch: refs/heads/YARN-3409
Commit: 43db0cb518375eb767401fa525ea6b5026ed9a8a
Parents: 5be9f4a
Author: Nanda kumar <na...@apache.org>
Authored: Wed Jul 25 18:11:35 2018 +0530
Committer: Nanda kumar <na...@apache.org>
Committed: Wed Jul 25 18:13:49 2018 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/utils/LevelDBStore.java   |   5 +
 .../hadoop/utils/LevelDBStoreIterator.java      |  64 ++++++
 .../apache/hadoop/utils/MetaStoreIterator.java  |  39 ++++
 .../org/apache/hadoop/utils/MetadataStore.java  |  55 +++++
 .../org/apache/hadoop/utils/RocksDBStore.java   |   5 +
 .../hadoop/utils/RocksDBStoreIterator.java      |  66 ++++++
 .../apache/hadoop/utils/TestMetadataStore.java  | 206 ++++++++++++-------
 7 files changed, 366 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java
index 13b9180..ed116a3 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStore.java
@@ -379,4 +379,9 @@ public class LevelDBStore implements MetadataStore {
     }
     return result;
   }
+
+  @Override
+  public MetaStoreIterator<KeyValue> iterator() {
+    return new LevelDBStoreIterator(db.iterator());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStoreIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStoreIterator.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStoreIterator.java
new file mode 100644
index 0000000..7b62f7a
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/LevelDBStoreIterator.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.utils;
+
+import org.iq80.leveldb.DBIterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.utils.MetadataStore.KeyValue;
+
+
+/**
+ * LevelDB store iterator.
+ */
+public class LevelDBStoreIterator implements MetaStoreIterator<KeyValue> {
+
+
+  private DBIterator levelDBIterator;
+
+  public LevelDBStoreIterator(DBIterator iterator) {
+    this.levelDBIterator = iterator;
+    levelDBIterator.seekToFirst();
+  }
+
+  @Override
+  public boolean hasNext() {
+    return levelDBIterator.hasNext();
+  }
+
+  @Override
+  public KeyValue next() {
+    if(levelDBIterator.hasNext()) {
+      Map.Entry<byte[], byte[]> entry = levelDBIterator.next();
+      return KeyValue.create(entry.getKey(), entry.getValue());
+    }
+    throw new NoSuchElementException("LevelDB Store has no more elements");
+  }
+
+  @Override
+  public void seekToFirst() {
+    levelDBIterator.seekToFirst();
+  }
+
+  @Override
+  public void seekToLast() {
+    levelDBIterator.seekToLast();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetaStoreIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetaStoreIterator.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetaStoreIterator.java
new file mode 100644
index 0000000..758d194
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetaStoreIterator.java
@@ -0,0 +1,39 @@
+/*
+ * 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.utils;
+
+import java.util.Iterator;
+
+/**
+ * Iterator for MetaDataStore DB.
+ * @param <T>
+ */
+interface MetaStoreIterator<T> extends Iterator<T> {
+
+  /**
+   * seek to first entry.
+   */
+  void seekToFirst();
+
+  /**
+   * seek to last entry.
+   */
+  void seekToLast();
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStore.java
index b90b08f..7d3bc6b 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStore.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataStore.java
@@ -169,4 +169,59 @@ public interface MetadataStore extends Closeable{
    */
   void iterate(byte[] from, EntryConsumer consumer)
       throws IOException;
+
+  /**
+   * Returns the iterator for this metadata store.
+   * @return MetaStoreIterator
+   */
+  MetaStoreIterator<KeyValue> iterator();
+
+  /**
+   * Class used to represent the key and value pair of a db entry.
+   */
+  class KeyValue {
+
+    private final byte[] key;
+    private final byte[] value;
+
+    /**
+     * KeyValue Constructor, used to represent a key and value of a db entry.
+     * @param key
+     * @param value
+     */
+    private KeyValue(byte[] key, byte[] value) {
+      this.key = key;
+      this.value = value;
+    }
+
+    /**
+     * Return key.
+     * @return byte[]
+     */
+    public byte[] getKey() {
+      byte[] result = new byte[key.length];
+      System.arraycopy(key, 0, result, 0, key.length);
+      return result;
+    }
+
+    /**
+     * Return value.
+     * @return byte[]
+     */
+    public byte[] getValue() {
+      byte[] result = new byte[value.length];
+      System.arraycopy(value, 0, result, 0, value.length);
+      return result;
+    }
+
+    /**
+     * Create a KeyValue pair.
+     * @param key
+     * @param value
+     * @return KeyValue object.
+     */
+    public static KeyValue create(byte[] key, byte[] value) {
+      return new KeyValue(key, value);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java
index 0dfca20..f5f070d 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java
@@ -380,4 +380,9 @@ public class RocksDBStore implements MetadataStore {
     return statMBeanName;
   }
 
+  @Override
+  public MetaStoreIterator<KeyValue> iterator() {
+    return new RocksDBStoreIterator(db.newIterator());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStoreIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStoreIterator.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStoreIterator.java
new file mode 100644
index 0000000..6e9b695
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStoreIterator.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.utils;
+
+import org.rocksdb.RocksIterator;
+
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.utils.MetadataStore.KeyValue;
+
+/**
+ * RocksDB store iterator.
+ */
+public class RocksDBStoreIterator implements MetaStoreIterator<KeyValue> {
+
+  private RocksIterator rocksDBIterator;
+
+  public RocksDBStoreIterator(RocksIterator iterator) {
+    this.rocksDBIterator = iterator;
+    rocksDBIterator.seekToFirst();
+  }
+
+  @Override
+  public boolean hasNext() {
+    return rocksDBIterator.isValid();
+  }
+
+  @Override
+  public KeyValue next() {
+    if (rocksDBIterator.isValid()) {
+      KeyValue value = KeyValue.create(rocksDBIterator.key(), rocksDBIterator
+          .value());
+      rocksDBIterator.next();
+      return value;
+    }
+    throw new NoSuchElementException("RocksDB Store has no more elements");
+  }
+
+  @Override
+  public void seekToFirst() {
+    rocksDBIterator.seekToFirst();
+  }
+
+  @Override
+  public void seekToLast() {
+    rocksDBIterator.seekToLast();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43db0cb5/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
index d697bbf..1bce022 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
@@ -28,10 +28,10 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.utils.MetadataStore.KeyValue;
 import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
 import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -48,10 +48,17 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.NoSuchElementException;
 import java.util.UUID;
+
 import java.util.concurrent.atomic.AtomicInteger;
 
-import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import static org.junit.runners.Parameterized.Parameters;
 
 /**
@@ -110,6 +117,58 @@ public class TestMetadataStore {
   }
 
   @Test
+  public void testIterator() throws Exception {
+    Configuration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
+    File dbDir = GenericTestUtils.getRandomizedTestDir();
+    MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
+        .setConf(conf)
+        .setCreateIfMissing(true)
+        .setDbFile(dbDir)
+        .build();
+
+    //As database is empty, check whether iterator is working as expected or
+    // not.
+    MetaStoreIterator<KeyValue> metaStoreIterator = dbStore.iterator();
+    assertFalse(metaStoreIterator.hasNext());
+    try {
+      metaStoreIterator.next();
+      fail("testIterator failed");
+    } catch (NoSuchElementException ex) {
+      GenericTestUtils.assertExceptionContains("Store has no more elements",
+          ex);
+    }
+
+    for (int i = 0; i < 10; i++) {
+      store.put(getBytes("a" + i), getBytes("a-value" + i));
+    }
+
+    metaStoreIterator = dbStore.iterator();
+
+    int i = 0;
+    while (metaStoreIterator.hasNext()) {
+      KeyValue val = metaStoreIterator.next();
+      assertEquals("a" + i, getString(val.getKey()));
+      assertEquals("a-value" + i, getString(val.getValue()));
+      i++;
+    }
+
+    // As we have iterated all the keys in database, hasNext should return
+    // false and next() should throw NoSuchElement exception.
+
+    assertFalse(metaStoreIterator.hasNext());
+    try {
+      metaStoreIterator.next();
+      fail("testIterator failed");
+    } catch (NoSuchElementException ex) {
+      GenericTestUtils.assertExceptionContains("Store has no more elements",
+          ex);
+    }
+    FileUtils.deleteDirectory(dbDir);
+
+  }
+
+  @Test
   public void testMetaStoreConfigDifferentFromType() throws IOException {
 
     Configuration conf = new OzoneConfiguration();
@@ -183,17 +242,17 @@ public class TestMetadataStore {
   public void testGetDelete() throws IOException {
     for (int i=0; i<10; i++) {
       byte[] va = store.get(getBytes("a" + i));
-      Assert.assertEquals("a-value" + i, getString(va));
+      assertEquals("a-value" + i, getString(va));
 
       byte[] vb = store.get(getBytes("b" + i));
-      Assert.assertEquals("b-value" + i, getString(vb));
+      assertEquals("b-value" + i, getString(vb));
     }
 
     String keyToDel = "del-" + UUID.randomUUID().toString();
     store.put(getBytes(keyToDel), getBytes(keyToDel));
-    Assert.assertEquals(keyToDel, getString(store.get(getBytes(keyToDel))));
+    assertEquals(keyToDel, getString(store.get(getBytes(keyToDel))));
     store.delete(getBytes(keyToDel));
-    Assert.assertEquals(null, store.get(getBytes(keyToDel)));
+    assertEquals(null, store.get(getBytes(keyToDel)));
   }
 
   @Test
@@ -228,8 +287,8 @@ public class TestMetadataStore {
       k = getString(current.getKey());
       v = getString(current.getValue());
     }
-    Assert.assertEquals(peekKey, k);
-    Assert.assertEquals(v, getExpectedValue(peekKey));
+    assertEquals(peekKey, k);
+    assertEquals(v, getExpectedValue(peekKey));
 
     // Look for prev
     k = null;
@@ -240,8 +299,8 @@ public class TestMetadataStore {
       k = getString(prev.getKey());
       v = getString(prev.getValue());
     }
-    Assert.assertEquals(prevKey, k);
-    Assert.assertEquals(v, getExpectedValue(prevKey));
+    assertEquals(prevKey, k);
+    assertEquals(v, getExpectedValue(prevKey));
 
     // Look for next
     k = null;
@@ -252,8 +311,8 @@ public class TestMetadataStore {
       k = getString(next.getKey());
       v = getString(next.getValue());
     }
-    Assert.assertEquals(nextKey, k);
-    Assert.assertEquals(v, getExpectedValue(nextKey));
+    assertEquals(nextKey, k);
+    assertEquals(v, getExpectedValue(nextKey));
   }
 
   @Test
@@ -271,9 +330,9 @@ public class TestMetadataStore {
       return true;
     });
 
-    Assert.assertFalse(result.isEmpty());
+    assertFalse(result.isEmpty());
     for (int i=0; i<result.size(); i++) {
-      Assert.assertEquals("b-value" + (i+1), result.get(i));
+      assertEquals("b-value" + (i+1), result.get(i));
     }
 
     // iterate from a non exist key
@@ -282,7 +341,7 @@ public class TestMetadataStore {
       result.add(getString(v));
       return true;
     });
-    Assert.assertTrue(result.isEmpty());
+    assertTrue(result.isEmpty());
 
     // iterate from the beginning
     result.clear();
@@ -290,7 +349,7 @@ public class TestMetadataStore {
       result.add(getString(v));
       return true;
     });
-    Assert.assertEquals(20, result.size());
+    assertEquals(20, result.size());
   }
 
   @Test
@@ -299,66 +358,66 @@ public class TestMetadataStore {
 
     // Set empty startKey will return values from beginning.
     result = store.getRangeKVs(null, 5);
-    Assert.assertEquals(5, result.size());
-    Assert.assertEquals("a-value2", getString(result.get(2).getValue()));
+    assertEquals(5, result.size());
+    assertEquals("a-value2", getString(result.get(2).getValue()));
 
     // Empty list if startKey doesn't exist.
     result = store.getRangeKVs(getBytes("a12"), 5);
-    Assert.assertEquals(0, result.size());
+    assertEquals(0, result.size());
 
     // Returns max available entries after a valid startKey.
     result = store.getRangeKVs(getBytes("b0"), MAX_GETRANGE_LENGTH);
-    Assert.assertEquals(10, result.size());
-    Assert.assertEquals("b0", getString(result.get(0).getKey()));
-    Assert.assertEquals("b-value0", getString(result.get(0).getValue()));
+    assertEquals(10, result.size());
+    assertEquals("b0", getString(result.get(0).getKey()));
+    assertEquals("b-value0", getString(result.get(0).getValue()));
     result = store.getRangeKVs(getBytes("b0"), 5);
-    Assert.assertEquals(5, result.size());
+    assertEquals(5, result.size());
 
     // Both startKey and count are honored.
     result = store.getRangeKVs(getBytes("a9"), 2);
-    Assert.assertEquals(2, result.size());
-    Assert.assertEquals("a9", getString(result.get(0).getKey()));
-    Assert.assertEquals("a-value9", getString(result.get(0).getValue()));
-    Assert.assertEquals("b0", getString(result.get(1).getKey()));
-    Assert.assertEquals("b-value0", getString(result.get(1).getValue()));
+    assertEquals(2, result.size());
+    assertEquals("a9", getString(result.get(0).getKey()));
+    assertEquals("a-value9", getString(result.get(0).getValue()));
+    assertEquals("b0", getString(result.get(1).getKey()));
+    assertEquals("b-value0", getString(result.get(1).getValue()));
 
     // Filter keys by prefix.
     // It should returns all "b*" entries.
     MetadataKeyFilter filter1 = new KeyPrefixFilter().addFilter("b");
     result = store.getRangeKVs(null, 100, filter1);
-    Assert.assertEquals(10, result.size());
-    Assert.assertTrue(result.stream().allMatch(entry ->
+    assertEquals(10, result.size());
+    assertTrue(result.stream().allMatch(entry ->
         new String(entry.getKey()).startsWith("b")
     ));
-    Assert.assertEquals(20, filter1.getKeysScannedNum());
-    Assert.assertEquals(10, filter1.getKeysHintedNum());
+    assertEquals(20, filter1.getKeysScannedNum());
+    assertEquals(10, filter1.getKeysHintedNum());
     result = store.getRangeKVs(null, 3, filter1);
-    Assert.assertEquals(3, result.size());
+    assertEquals(3, result.size());
     result = store.getRangeKVs(getBytes("b3"), 1, filter1);
-    Assert.assertEquals("b-value3", getString(result.get(0).getValue()));
+    assertEquals("b-value3", getString(result.get(0).getValue()));
 
     // Define a customized filter that filters keys by suffix.
     // Returns all "*2" entries.
     MetadataKeyFilter filter2 = (preKey, currentKey, nextKey)
         -> getString(currentKey).endsWith("2");
     result = store.getRangeKVs(null, MAX_GETRANGE_LENGTH, filter2);
-    Assert.assertEquals(2, result.size());
-    Assert.assertEquals("a2", getString(result.get(0).getKey()));
-    Assert.assertEquals("b2", getString(result.get(1).getKey()));
+    assertEquals(2, result.size());
+    assertEquals("a2", getString(result.get(0).getKey()));
+    assertEquals("b2", getString(result.get(1).getKey()));
     result = store.getRangeKVs(null, 1, filter2);
-    Assert.assertEquals(1, result.size());
-    Assert.assertEquals("a2", getString(result.get(0).getKey()));
+    assertEquals(1, result.size());
+    assertEquals("a2", getString(result.get(0).getKey()));
 
     // Apply multiple filters.
     result = store.getRangeKVs(null, MAX_GETRANGE_LENGTH, filter1, filter2);
-    Assert.assertEquals(1, result.size());
-    Assert.assertEquals("b2", getString(result.get(0).getKey()));
-    Assert.assertEquals("b-value2", getString(result.get(0).getValue()));
+    assertEquals(1, result.size());
+    assertEquals("b2", getString(result.get(0).getKey()));
+    assertEquals("b-value2", getString(result.get(0).getValue()));
 
     // If filter is null, no effect.
     result = store.getRangeKVs(null, 1, null);
-    Assert.assertEquals(1, result.size());
-    Assert.assertEquals("a0", getString(result.get(0).getKey()));
+    assertEquals(1, result.size());
+    assertEquals("a0", getString(result.get(0).getKey()));
   }
 
   @Test
@@ -368,16 +427,16 @@ public class TestMetadataStore {
     // Suppose to return a2 and b2
     List<Map.Entry<byte[], byte[]>> result =
         store.getRangeKVs(null, MAX_GETRANGE_LENGTH, suffixFilter);
-    Assert.assertEquals(2, result.size());
-    Assert.assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
-    Assert.assertEquals("b2", DFSUtil.bytes2String(result.get(1).getKey()));
+    assertEquals(2, result.size());
+    assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
+    assertEquals("b2", DFSUtil.bytes2String(result.get(1).getKey()));
 
     // Suppose to return just a2, because when it iterates to a3,
     // the filter no long matches and it should stop from there.
     result = store.getSequentialRangeKVs(null,
         MAX_GETRANGE_LENGTH, suffixFilter);
-    Assert.assertEquals(1, result.size());
-    Assert.assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
+    assertEquals(1, result.size());
+    assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
   }
 
   @Test
@@ -385,10 +444,10 @@ public class TestMetadataStore {
     List<Map.Entry<byte[], byte[]>> result = null;
 
     result = store.getRangeKVs(null, 0);
-    Assert.assertEquals(0, result.size());
+    assertEquals(0, result.size());
 
     result = store.getRangeKVs(null, 1);
-    Assert.assertEquals(1, result.size());
+    assertEquals(1, result.size());
 
     // Count less than zero is invalid.
     expectedException.expect(IllegalArgumentException.class);
@@ -401,7 +460,7 @@ public class TestMetadataStore {
     // If startKey is invalid, the returned list should be empty.
     List<Map.Entry<byte[], byte[]>> kvs =
         store.getRangeKVs(getBytes("unknownKey"), MAX_GETRANGE_LENGTH);
-    Assert.assertEquals(kvs.size(), 0);
+    assertEquals(kvs.size(), 0);
   }
 
   @Test
@@ -421,13 +480,13 @@ public class TestMetadataStore {
     dbStore.put(getBytes("key1"), getBytes("value1"));
     dbStore.put(getBytes("key2"), getBytes("value2"));
 
-    Assert.assertFalse(dbStore.isEmpty());
-    Assert.assertTrue(dbDir.exists());
-    Assert.assertTrue(dbDir.listFiles().length > 0);
+    assertFalse(dbStore.isEmpty());
+    assertTrue(dbDir.exists());
+    assertTrue(dbDir.listFiles().length > 0);
 
     dbStore.destroy();
 
-    Assert.assertFalse(dbDir.exists());
+    assertFalse(dbDir.exists());
   }
 
   @Test
@@ -469,7 +528,7 @@ public class TestMetadataStore {
       return it.hasNext() && it.next().equals(getString(key));
     });
 
-    Assert.assertEquals(8, count.get());
+    assertEquals(8, count.get());
   }
 
   @Test
@@ -482,52 +541,51 @@ public class TestMetadataStore {
     } catch (IllegalArgumentException e) {
       exception = e;
     }
-    Assert.assertTrue(
-        exception.getMessage().contains("KeyPrefix: b already rejected"));
+    assertTrue(exception.getMessage().contains("KeyPrefix: b already " +
+        "rejected"));
 
     try {
       new KeyPrefixFilter().addFilter("b0").addFilter("b", true);
     } catch (IllegalArgumentException e) {
       exception = e;
     }
-    Assert.assertTrue(
-        exception.getMessage().contains("KeyPrefix: b already accepted"));
+    assertTrue(exception.getMessage().contains("KeyPrefix: b already " +
+        "accepted"));
 
     try {
       new KeyPrefixFilter().addFilter("b", true).addFilter("b0");
     } catch (IllegalArgumentException e) {
       exception = e;
     }
-    Assert.assertTrue(
-        exception.getMessage().contains("KeyPrefix: b0 already rejected"));
+    assertTrue(exception.getMessage().contains("KeyPrefix: b0 already " +
+        "rejected"));
 
     try {
       new KeyPrefixFilter().addFilter("b").addFilter("b0", true);
     } catch (IllegalArgumentException e) {
       exception = e;
     }
-    Assert.assertTrue(
-        exception.getMessage().contains("KeyPrefix: b0 already accepted"));
+    assertTrue(exception.getMessage().contains("KeyPrefix: b0 already " +
+        "accepted"));
 
     MetadataKeyFilter filter1 = new KeyPrefixFilter(true)
             .addFilter("a0")
             .addFilter("a1")
             .addFilter("b", true);
     result = store.getRangeKVs(null, 100, filter1);
-    Assert.assertEquals(2, result.size());
-    Assert.assertTrue(result.stream()
-        .anyMatch(entry -> new String(entry.getKey()).startsWith("a0"))
-        && result.stream()
-        .anyMatch(entry -> new String(entry.getKey()).startsWith("a1")));
+    assertEquals(2, result.size());
+    assertTrue(result.stream().anyMatch(entry -> new String(entry.getKey())
+        .startsWith("a0")) && result.stream().anyMatch(entry -> new String(
+            entry.getKey()).startsWith("a1")));
 
     filter1 = new KeyPrefixFilter(true).addFilter("b", true);
     result = store.getRangeKVs(null, 100, filter1);
-    Assert.assertEquals(0, result.size());
+    assertEquals(0, result.size());
 
     filter1 = new KeyPrefixFilter().addFilter("b", true);
     result = store.getRangeKVs(null, 100, filter1);
-    Assert.assertEquals(10, result.size());
-    Assert.assertTrue(result.stream()
-        .allMatch(entry -> new String(entry.getKey()).startsWith("a")));
+    assertEquals(10, result.size());
+    assertTrue(result.stream().allMatch(entry -> new String(entry.getKey())
+        .startsWith("a")));
   }
 }


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


[13/50] [abbrv] hadoop git commit: HDDS-266. Integrate checksum into .container file. Contributed by Hanisha Koneru.

Posted by su...@apache.org.
HDDS-266. Integrate checksum into .container file. Contributed by Hanisha Koneru.


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

Branch: refs/heads/YARN-3409
Commit: b507f83e15b47163724d550dfeb41627f26fd551
Parents: 43db0cb
Author: Nanda kumar <na...@apache.org>
Authored: Wed Jul 25 18:55:26 2018 +0530
Committer: Nanda kumar <na...@apache.org>
Committed: Wed Jul 25 18:55:26 2018 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/ozone/OzoneConsts.java    |   2 +-
 .../common/helpers/ContainerUtils.java          |  59 ++++++++-
 .../container/common/impl/ContainerData.java    |  66 +++++++++-
 .../common/impl/ContainerDataYaml.java          |  98 +++++++++-----
 .../container/keyvalue/KeyValueContainer.java   | 121 +++++------------
 .../keyvalue/KeyValueContainerData.java         |  30 ++---
 .../helpers/KeyValueContainerLocationUtil.java  |  32 ++---
 .../keyvalue/helpers/KeyValueContainerUtil.java | 130 +++----------------
 .../container/ozoneimpl/ContainerReader.java    |  92 +++++++------
 .../common/impl/TestContainerDataYaml.java      | 107 +++++++++++----
 .../keyvalue/TestKeyValueContainer.java         |  21 ++-
 .../test/resources/additionalfields.container   |   3 +-
 .../test/resources/incorrect.checksum.container |  11 ++
 .../src/test/resources/incorrect.container      |   3 +-
 hadoop-hdds/pom.xml                             |   1 +
 .../common/impl/TestContainerPersistence.java   |   8 +-
 .../container/ozoneimpl/TestOzoneContainer.java |   7 +
 17 files changed, 430 insertions(+), 361 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 25b68e0..f912f02 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -98,7 +98,6 @@ public final class OzoneConsts {
   public static final String OM_DB_NAME = "om.db";
 
   public static final String STORAGE_DIR_CHUNKS = "chunks";
-  public static final String CONTAINER_FILE_CHECKSUM_EXTENSION = ".chksm";
 
   /**
    * Supports Bucket Versioning.
@@ -190,4 +189,5 @@ public final class OzoneConsts {
   public static final String METADATA_PATH = "metadataPath";
   public static final String CHUNKS_PATH = "chunksPath";
   public static final String CONTAINER_DB_TYPE = "containerDBType";
+  public static final String CHECKSUM = "checksum";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
index 1d5dfc5..77a891a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
@@ -19,9 +19,11 @@
 package org.apache.hadoop.ozone.container.common.helpers;
 
 import com.google.common.base.Preconditions;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
@@ -31,8 +33,8 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.ozone.OzoneConsts;
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,8 +45,17 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import org.yaml.snakeyaml.Yaml;
 
 import static org.apache.commons.io.FilenameUtils.removeExtension;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CONTAINER_CHECKSUM_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.NO_SUCH_ALGORITHM;
+import static org.apache.hadoop.ozone.container.common.impl.ContainerData
+    .CHARSET_ENCODING;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
+
 
 /**
  * A set of helper functions to create proper responses.
@@ -245,4 +256,48 @@ public final class ContainerUtils {
           + path.getAbsolutePath(), e);
     }
   }
+
+  /**
+   * Verify that the checksum stored in containerData is equal to the
+   * computed checksum.
+   * @param containerData
+   * @throws IOException
+   */
+  public static void verifyChecksum(ContainerData containerData)
+      throws IOException {
+    String storedChecksum = containerData.getChecksum();
+
+    Yaml yaml = ContainerDataYaml.getYamlForContainerType(
+        containerData.getContainerType());
+    containerData.computeAndSetChecksum(yaml);
+    String computedChecksum = containerData.getChecksum();
+
+    if (storedChecksum == null || !storedChecksum.equals(computedChecksum)) {
+      throw new StorageContainerException("Container checksum error for " +
+          "ContainerID: " + containerData.getContainerID() + ". " +
+          "\nStored Checksum: " + storedChecksum +
+          "\nExpected Checksum: " + computedChecksum,
+          CONTAINER_CHECKSUM_ERROR);
+    }
+  }
+
+  /**
+   * Return the SHA-256 chesksum of the containerData.
+   * @param containerDataYamlStr ContainerData as a Yaml String
+   * @return Checksum of the container data
+   * @throws StorageContainerException
+   */
+  public static String getChecksum(String containerDataYamlStr)
+      throws StorageContainerException {
+    MessageDigest sha;
+    try {
+      sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
+      sha.update(containerDataYamlStr.getBytes(CHARSET_ENCODING));
+      return DigestUtils.sha256Hex(sha.digest());
+    } catch (NoSuchAlgorithmException e) {
+      throw new StorageContainerException("Unable to create Message Digest, " +
+          "usually this is a java configuration issue.", NO_SUCH_ALGORITHM);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
index a7e2b55..5803628 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -18,22 +18,33 @@
 package org.apache.hadoop.ozone.container.common.impl;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.nio.charset.Charset;
 import java.util.List;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
     ContainerType;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
     ContainerLifeCycleState;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 
-import java.io.IOException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import org.yaml.snakeyaml.Yaml;
 
 import static java.lang.Math.max;
+import static org.apache.hadoop.ozone.OzoneConsts.CHECKSUM;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ID;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_TYPE;
+import static org.apache.hadoop.ozone.OzoneConsts.LAYOUTVERSION;
+import static org.apache.hadoop.ozone.OzoneConsts.MAX_SIZE_GB;
+import static org.apache.hadoop.ozone.OzoneConsts.METADATA;
+import static org.apache.hadoop.ozone.OzoneConsts.STATE;
 
 /**
  * ContainerData is the in-memory representation of container metadata and is
@@ -72,6 +83,23 @@ public abstract class ContainerData {
 
   private long deleteTransactionId;
 
+  private String checksum;
+  public static final Charset CHARSET_ENCODING = Charset.forName("UTF-8");
+  private static final String DUMMY_CHECKSUM = new String(new byte[64],
+      CHARSET_ENCODING);
+
+  // Common Fields need to be stored in .container file.
+  protected static final List<String> YAML_FIELDS =
+      Collections.unmodifiableList(Lists.newArrayList(
+      CONTAINER_TYPE,
+      CONTAINER_ID,
+      LAYOUTVERSION,
+      STATE,
+      METADATA,
+      MAX_SIZE_GB,
+      CHECKSUM));
+
+
   /**
    * Number of pending deletion blocks in container.
    */
@@ -113,6 +141,7 @@ public abstract class ContainerData {
     this.maxSizeGB = size;
     this.numPendingDeletionBlocks = new AtomicInteger(0);
     this.deleteTransactionId = 0;
+    setChecksumTo0ByteArray();
   }
 
   /**
@@ -400,6 +429,41 @@ public abstract class ContainerData {
     return this.numPendingDeletionBlocks.get();
   }
 
+  public void setChecksumTo0ByteArray() {
+    this.checksum = DUMMY_CHECKSUM;
+  }
+
+  public void setChecksum(String checkSum) {
+    this.checksum = checkSum;
+  }
+
+  public String getChecksum() {
+    return this.checksum;
+  }
+
+  /**
+   * Compute the checksum for ContainerData using the specified Yaml (based
+   * on ContainerType) and set the checksum.
+   *
+   * Checksum of ContainerData is calculated by setting the
+   * {@link ContainerData#checksum} field to a 64-byte array with all 0's -
+   * {@link ContainerData#DUMMY_CHECKSUM}. After the checksum is calculated,
+   * the checksum field is updated with this value.
+   *
+   * @param yaml Yaml for ContainerType to get the ContainerData as Yaml String
+   * @throws IOException
+   */
+  public void computeAndSetChecksum(Yaml yaml) throws IOException {
+    // Set checksum to dummy value - 0 byte array, to calculate the checksum
+    // of rest of the data.
+    setChecksumTo0ByteArray();
+
+    // Dump yaml data into a string to compute its checksum
+    String containerDataYamlStr = yaml.dump(this);
+
+    this.checksum = ContainerUtils.getChecksum(containerDataYamlStr);
+  }
+
   /**
    * Returns a ProtoBuf Message from ContainerData.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
index 90af24f..aed75d3 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
@@ -20,10 +20,14 @@ package org.apache.hadoop.ozone.container.common.impl;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerType;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Yaml;
 
 import java.beans.IntrospectionException;
@@ -59,9 +63,13 @@ import static org.apache.hadoop.ozone.container.keyvalue
 
 public final class ContainerDataYaml {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContainerDataYaml.class);
+
   private ContainerDataYaml() {
 
   }
+
   /**
    * Creates a .container file in yaml format.
    *
@@ -69,38 +77,29 @@ public final class ContainerDataYaml {
    * @param containerData
    * @throws IOException
    */
-  public static void createContainerFile(ContainerProtos.ContainerType
-                                             containerType, File containerFile,
-                                         ContainerData containerData) throws
-      IOException {
-
-    Preconditions.checkNotNull(containerFile, "yamlFile cannot be null");
-    Preconditions.checkNotNull(containerData, "containerData cannot be null");
-    Preconditions.checkNotNull(containerType, "containerType cannot be null");
-
-    PropertyUtils propertyUtils = new PropertyUtils();
-    propertyUtils.setBeanAccess(BeanAccess.FIELD);
-    propertyUtils.setAllowReadOnlyProperties(true);
-
-    switch(containerType) {
-    case KeyValueContainer:
-      Representer representer = new ContainerDataRepresenter();
-      representer.setPropertyUtils(propertyUtils);
-      representer.addClassTag(KeyValueContainerData.class,
-          KeyValueContainerData.KEYVALUE_YAML_TAG);
-
-      Constructor keyValueDataConstructor = new ContainerDataConstructor();
+  public static void createContainerFile(ContainerType containerType,
+      ContainerData containerData, File containerFile) throws IOException {
+    Writer writer = null;
+    try {
+      // Create Yaml for given container type
+      Yaml yaml = getYamlForContainerType(containerType);
+      // Compute Checksum and update ContainerData
+      containerData.computeAndSetChecksum(yaml);
 
-      Yaml yaml = new Yaml(keyValueDataConstructor, representer);
-      Writer writer = new OutputStreamWriter(new FileOutputStream(
+      // Write the ContainerData with checksum to Yaml file.
+      writer = new OutputStreamWriter(new FileOutputStream(
           containerFile), "UTF-8");
       yaml.dump(containerData, writer);
-      writer.close();
-      break;
-    default:
-      throw new StorageContainerException("Unrecognized container Type " +
-          "format " + containerType, ContainerProtos.Result
-          .UNKNOWN_CONTAINER_TYPE);
+
+    } finally {
+      try {
+        if (writer != null) {
+          writer.close();
+        }
+      } catch (IOException ex) {
+        LOG.warn("Error occurred during closing the writer. ContainerID: " +
+            containerData.getContainerID());
+      }
     }
   }
 
@@ -141,6 +140,39 @@ public final class ContainerDataYaml {
   }
 
   /**
+   * Given a ContainerType this method returns a Yaml representation of
+   * the container properties.
+   *
+   * @param containerType type of container
+   * @return Yamal representation of container properties
+   *
+   * @throws StorageContainerException if the type is unrecognized
+   */
+  public static Yaml getYamlForContainerType(ContainerType containerType)
+      throws StorageContainerException {
+    PropertyUtils propertyUtils = new PropertyUtils();
+    propertyUtils.setBeanAccess(BeanAccess.FIELD);
+    propertyUtils.setAllowReadOnlyProperties(true);
+
+    switch (containerType) {
+    case KeyValueContainer:
+      Representer representer = new ContainerDataRepresenter();
+      representer.setPropertyUtils(propertyUtils);
+      representer.addClassTag(
+          KeyValueContainerData.class,
+          KeyValueContainerData.KEYVALUE_YAML_TAG);
+
+      Constructor keyValueDataConstructor = new ContainerDataConstructor();
+
+      return new Yaml(keyValueDataConstructor, representer);
+    default:
+      throw new StorageContainerException("Unrecognized container Type " +
+          "format " + containerType, ContainerProtos.Result
+          .UNKNOWN_CONTAINER_TYPE);
+    }
+  }
+
+  /**
    * Representer class to define which fields need to be stored in yaml file.
    */
   private static class ContainerDataRepresenter extends Representer {
@@ -192,8 +224,9 @@ public final class ContainerDataYaml {
         int maxSize = (int) size;
 
         //When a new field is added, it needs to be added here.
-        KeyValueContainerData kvData = new KeyValueContainerData((long) nodes
-            .get(OzoneConsts.CONTAINER_ID), lv, maxSize);
+        KeyValueContainerData kvData = new KeyValueContainerData(
+            (long) nodes.get(OzoneConsts.CONTAINER_ID), lv, maxSize);
+
         kvData.setContainerDBType((String)nodes.get(
             OzoneConsts.CONTAINER_DB_TYPE));
         kvData.setMetadataPath((String) nodes.get(
@@ -201,6 +234,7 @@ public final class ContainerDataYaml {
         kvData.setChunksPath((String) nodes.get(OzoneConsts.CHUNKS_PATH));
         Map<String, String> meta = (Map) nodes.get(OzoneConsts.METADATA);
         kvData.setMetadata(meta);
+        kvData.setChecksum((String) nodes.get(OzoneConsts.CHECKSUM));
         String state = (String) nodes.get(OzoneConsts.STATE);
         switch (state) {
         case "OPEN":
@@ -215,7 +249,7 @@ public final class ContainerDataYaml {
         default:
           throw new IllegalStateException("Unexpected " +
               "ContainerLifeCycleState " + state + " for the containerId " +
-              (long) nodes.get(OzoneConsts.CONTAINER_ID));
+              nodes.get(OzoneConsts.CONTAINER_ID));
         }
         return kvData;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index f381e24..14f731a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -24,12 +24,12 @@ import java.nio.file.StandardCopyOption;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerLifeCycleState;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerType;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -49,10 +49,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
 
 import java.util.Map;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -113,26 +110,24 @@ public class KeyValueContainer implements Container {
           .getVolumesList(), maxSize);
       String hddsVolumeDir = containerVolume.getHddsRootDir().toString();
 
-      long containerId = containerData.getContainerID();
-      String containerName = Long.toString(containerId);
+      long containerID = containerData.getContainerID();
 
       containerMetaDataPath = KeyValueContainerLocationUtil
-          .getContainerMetaDataPath(hddsVolumeDir, scmId, containerId);
+          .getContainerMetaDataPath(hddsVolumeDir, scmId, containerID);
       File chunksPath = KeyValueContainerLocationUtil.getChunksLocationPath(
-          hddsVolumeDir, scmId, containerId);
+          hddsVolumeDir, scmId, containerID);
+
       File containerFile = KeyValueContainerLocationUtil.getContainerFile(
-          containerMetaDataPath, containerName);
-      File containerCheckSumFile = KeyValueContainerLocationUtil
-          .getContainerCheckSumFile(containerMetaDataPath, containerName);
+          containerMetaDataPath, containerID);
       File dbFile = KeyValueContainerLocationUtil.getContainerDBFile(
-          containerMetaDataPath, containerName);
+          containerMetaDataPath, containerID);
 
       // Check if it is new Container.
       ContainerUtils.verifyIsNewContainer(containerMetaDataPath);
 
       //Create Metadata path chunks path and metadata db
       KeyValueContainerUtil.createContainerMetaData(containerMetaDataPath,
-          chunksPath, dbFile, containerName, config);
+          chunksPath, dbFile, config);
 
       String impl = config.getTrimmed(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL,
           OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT);
@@ -144,9 +139,8 @@ public class KeyValueContainer implements Container {
       containerData.setDbFile(dbFile);
       containerData.setVolume(containerVolume);
 
-      // Create .container file and .chksm file
-      writeToContainerFile(containerFile, containerCheckSumFile, true);
-
+      // Create .container file
+      writeToContainerFile(containerFile, true);
 
     } catch (StorageContainerException ex) {
       if (containerMetaDataPath != null && containerMetaDataPath.getParentFile()
@@ -176,97 +170,64 @@ public class KeyValueContainer implements Container {
    * Creates .container file and checksum file.
    *
    * @param containerFile
-   * @param checksumFile
    * @param isCreate true if we are creating a new container file and false if
    *                we are updating an existing container file.
    * @throws StorageContainerException
    */
-  private void writeToContainerFile(File containerFile, File
-      checksumFile, boolean isCreate)
+  private void writeToContainerFile(File containerFile, boolean isCreate)
       throws StorageContainerException {
     File tempContainerFile = null;
-    File tempChecksumFile = null;
-    FileOutputStream containerCheckSumStream = null;
-    Writer writer = null;
     long containerId = containerData.getContainerID();
     try {
       tempContainerFile = createTempFile(containerFile);
-      tempChecksumFile = createTempFile(checksumFile);
-      ContainerDataYaml.createContainerFile(ContainerProtos.ContainerType
-              .KeyValueContainer, tempContainerFile, containerData);
-
-      //Compute Checksum for container file
-      String checksum = KeyValueContainerUtil.computeCheckSum(containerId,
-          tempContainerFile);
-      containerCheckSumStream = new FileOutputStream(tempChecksumFile);
-      writer = new OutputStreamWriter(containerCheckSumStream, "UTF-8");
-      writer.write(checksum);
-      writer.flush();
+      ContainerDataYaml.createContainerFile(
+          ContainerType.KeyValueContainer, containerData, tempContainerFile);
 
       if (isCreate) {
         // When creating a new container, .container file should not exist
         // already.
         NativeIO.renameTo(tempContainerFile, containerFile);
-        NativeIO.renameTo(tempChecksumFile, checksumFile);
       } else {
         // When updating a container, the .container file should exist. If
         // not, the container is in an inconsistent state.
         Files.move(tempContainerFile.toPath(), containerFile.toPath(),
             StandardCopyOption.REPLACE_EXISTING);
-        Files.move(tempChecksumFile.toPath(), checksumFile.toPath(),
-            StandardCopyOption.REPLACE_EXISTING);
       }
 
     } catch (IOException ex) {
       throw new StorageContainerException("Error during creation of " +
-          "required files(.container, .chksm) for container. ContainerID: "
-          + containerId, ex, CONTAINER_FILES_CREATE_ERROR);
+          ".container file. ContainerID: " + containerId, ex,
+          CONTAINER_FILES_CREATE_ERROR);
     } finally {
-      IOUtils.closeStream(containerCheckSumStream);
       if (tempContainerFile != null && tempContainerFile.exists()) {
         if (!tempContainerFile.delete()) {
           LOG.warn("Unable to delete container temporary file: {}.",
               tempContainerFile.getAbsolutePath());
         }
       }
-      if (tempChecksumFile != null && tempChecksumFile.exists()) {
-        if (!tempChecksumFile.delete()) {
-          LOG.warn("Unable to delete container temporary checksum file: {}.",
-              tempContainerFile.getAbsolutePath());
-        }
-      }
-      try {
-        if (writer != null) {
-          writer.close();
-        }
-      } catch (IOException ex) {
-        LOG.warn("Error occurred during closing the writer.  Container " +
-            "Name:" + containerId);
-      }
-
     }
   }
 
 
-  private void updateContainerFile(File containerFile, File
-      checksumFile) throws StorageContainerException {
+  private void updateContainerFile(File containerFile)
+      throws StorageContainerException {
 
     long containerId = containerData.getContainerID();
 
-    if (containerFile.exists() && checksumFile.exists()) {
-      try {
-        writeToContainerFile(containerFile, checksumFile, false);
-      } catch (IOException e) {
-        //TODO : Container update failure is not handled currently. Might
-        // lead to loss of .container file. When Update container feature
-        // support is added, this failure should also be handled.
-        throw new StorageContainerException("Container update failed. " +
-            "ContainerID: " + containerId, CONTAINER_FILES_CREATE_ERROR);
-      }
-    } else {
+    if (!containerFile.exists()) {
       throw new StorageContainerException("Container is an Inconsistent " +
-          "state, missing required files(.container, .chksm). ContainerID: " +
-          containerId, INVALID_CONTAINER_STATE);
+          "state, missing .container file. ContainerID: " + containerId,
+          INVALID_CONTAINER_STATE);
+    }
+
+    try {
+      writeToContainerFile(containerFile, false);
+    } catch (IOException e) {
+      //TODO : Container update failure is not handled currently. Might
+      // lead to loss of .container file. When Update container feature
+      // support is added, this failure should also be handled.
+      throw new StorageContainerException("Container update failed. " +
+          "ContainerID: " + containerId, CONTAINER_FILES_CREATE_ERROR);
     }
   }
 
@@ -305,10 +266,9 @@ public class KeyValueContainer implements Container {
       }
       containerData.closeContainer();
       File containerFile = getContainerFile();
-      File containerCheckSumFile = getContainerCheckSumFile();
 
       // update the new container data to .container File
-      updateContainerFile(containerFile, containerCheckSumFile);
+      updateContainerFile(containerFile);
 
     } catch (StorageContainerException ex) {
       throw ex;
@@ -340,8 +300,8 @@ public class KeyValueContainer implements Container {
   }
 
   @Override
-  public ContainerProtos.ContainerType getContainerType() {
-    return ContainerProtos.ContainerType.KeyValueContainer;
+  public ContainerType getContainerType() {
+    return ContainerType.KeyValueContainer;
   }
 
   @Override
@@ -369,10 +329,10 @@ public class KeyValueContainer implements Container {
       for (Map.Entry<String, String> entry : metadata.entrySet()) {
         containerData.addMetadata(entry.getKey(), entry.getValue());
       }
+
       File containerFile = getContainerFile();
-      File containerCheckSumFile = getContainerCheckSumFile();
       // update the new container data to .container File
-      updateContainerFile(containerFile, containerCheckSumFile);
+      updateContainerFile(containerFile);
     } catch (StorageContainerException  ex) {
       // TODO:
       // On error, reset the metadata.
@@ -461,15 +421,6 @@ public class KeyValueContainer implements Container {
   }
 
   /**
-   * Returns container checksum file.
-   * @return container checksum file
-   */
-  private File getContainerCheckSumFile() {
-    return new File(containerData.getMetadataPath(), containerData
-        .getContainerID() + OzoneConsts.CONTAINER_FILE_CHECKSUM_EXTENSION);
-  }
-
-  /**
    * Creates a temporary file.
    * @param file
    * @return

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index 3e3cc77..34035c8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.container.keyvalue;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
+import java.util.Collections;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -34,13 +35,7 @@ import java.util.Map;
 
 import static org.apache.hadoop.ozone.OzoneConsts.CHUNKS_PATH;
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_TYPE;
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ID;
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_TYPE;
-import static org.apache.hadoop.ozone.OzoneConsts.LAYOUTVERSION;
-import static org.apache.hadoop.ozone.OzoneConsts.MAX_SIZE_GB;
-import static org.apache.hadoop.ozone.OzoneConsts.METADATA;
 import static org.apache.hadoop.ozone.OzoneConsts.METADATA_PATH;
-import static org.apache.hadoop.ozone.OzoneConsts.STATE;
 
 /**
  * This class represents the KeyValueContainer metadata, which is the
@@ -53,17 +48,7 @@ public class KeyValueContainerData extends ContainerData {
   public static final Tag KEYVALUE_YAML_TAG = new Tag("KeyValueContainerData");
 
   // Fields need to be stored in .container file.
-  private static final List<String> YAML_FIELDS =
-      Lists.newArrayList(
-          CONTAINER_TYPE,
-          CONTAINER_ID,
-          LAYOUTVERSION,
-          STATE,
-          METADATA,
-          METADATA_PATH,
-          CHUNKS_PATH,
-          CONTAINER_DB_TYPE,
-          MAX_SIZE_GB);
+  private static final List<String> KV_YAML_FIELDS;
 
   // Path to Container metadata Level DB/RocksDB Store and .container file.
   private String metadataPath;
@@ -76,6 +61,15 @@ public class KeyValueContainerData extends ContainerData {
 
   private File dbFile = null;
 
+  static {
+    // Initialize YAML fields
+    KV_YAML_FIELDS = Lists.newArrayList();
+    KV_YAML_FIELDS.addAll(YAML_FIELDS);
+    KV_YAML_FIELDS.add(METADATA_PATH);
+    KV_YAML_FIELDS.add(CHUNKS_PATH);
+    KV_YAML_FIELDS.add(CONTAINER_DB_TYPE);
+  }
+
   /**
    * Constructs KeyValueContainerData object.
    * @param id - ContainerId
@@ -210,7 +204,7 @@ public class KeyValueContainerData extends ContainerData {
   }
 
   public static List<String> getYamlFields() {
-    return YAML_FIELDS;
+    return Collections.unmodifiableList(KV_YAML_FIELDS);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
index 868b9f4..02a8e73 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerLocationUtil.java
@@ -101,42 +101,26 @@ public final class KeyValueContainerLocationUtil {
   /**
    * Returns containerFile.
    * @param containerMetaDataPath
-   * @param containerName
+   * @param containerID
    * @return .container File name
    */
-  public static File getContainerFile(File containerMetaDataPath, String
-      containerName) {
+  public static File getContainerFile(File containerMetaDataPath,
+      long containerID) {
     Preconditions.checkNotNull(containerMetaDataPath);
-    Preconditions.checkNotNull(containerName);
-    return new File(containerMetaDataPath, containerName +
+    return new File(containerMetaDataPath, containerID +
         OzoneConsts.CONTAINER_EXTENSION);
   }
 
   /**
    * Return containerDB File.
    * @param containerMetaDataPath
-   * @param containerName
+   * @param containerID
    * @return containerDB File name
    */
-  public static File getContainerDBFile(File containerMetaDataPath, String
-      containerName) {
+  public static File getContainerDBFile(File containerMetaDataPath,
+      long containerID) {
     Preconditions.checkNotNull(containerMetaDataPath);
-    Preconditions.checkNotNull(containerName);
-    return new File(containerMetaDataPath, containerName + OzoneConsts
+    return new File(containerMetaDataPath, containerID + OzoneConsts
         .DN_CONTAINER_DB);
   }
-
-  /**
-   * Returns container checksum file.
-   * @param containerMetaDataPath
-   * @param containerName
-   * @return container checksum file
-   */
-  public static File getContainerCheckSumFile(File containerMetaDataPath,
-                                              String containerName) {
-    Preconditions.checkNotNull(containerMetaDataPath);
-    Preconditions.checkNotNull(containerName);
-    return new File(containerMetaDataPath, containerName + OzoneConsts
-        .CONTAINER_FILE_CHECKSUM_EXTENSION);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
index 1856111..2352cf6 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
@@ -18,23 +18,16 @@
 package org.apache.hadoop.ozone.container.keyvalue.helpers;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandResponseProto;
-import org.apache.hadoop.hdds.scm.container.common.helpers
-    .StorageContainerException;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
-import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.utils.MetadataKeyFilters;
 import org.apache.hadoop.utils.MetadataStore;
@@ -43,18 +36,12 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
-import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.*;
-
 /**
  * Class which defines utility methods for KeyValueContainer.
  */
@@ -77,10 +64,8 @@ public final class KeyValueContainerUtil {
    * @throws IOException
    */
   public static void createContainerMetaData(File containerMetaDataPath, File
-      chunksPath, File dbFile, String containerName, Configuration conf) throws
-      IOException {
+      chunksPath, File dbFile, Configuration conf) throws IOException {
     Preconditions.checkNotNull(containerMetaDataPath);
-    Preconditions.checkNotNull(containerName);
     Preconditions.checkNotNull(conf);
 
     if (!containerMetaDataPath.mkdirs()) {
@@ -166,106 +151,31 @@ public final class KeyValueContainerUtil {
   }
 
   /**
-   * Compute checksum of the .container file.
-   * @param containerId
-   * @param containerFile
-   * @throws StorageContainerException
-   */
-  public static String computeCheckSum(long containerId, File
-      containerFile) throws StorageContainerException {
-    Preconditions.checkNotNull(containerFile, "containerFile cannot be null");
-    MessageDigest sha;
-    FileInputStream containerFileStream = null;
-    try {
-      sha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
-    } catch (NoSuchAlgorithmException e) {
-      throw new StorageContainerException("Unable to create Message Digest, " +
-          "usually this is a java configuration issue.", NO_SUCH_ALGORITHM);
-    }
-    try {
-      containerFileStream = new FileInputStream(containerFile);
-      byte[] byteArray = new byte[1024];
-      int bytesCount = 0;
-      while ((bytesCount = containerFileStream.read(byteArray)) != -1) {
-        sha.update(byteArray, 0, bytesCount);
-      }
-      String checksum = DigestUtils.sha256Hex(sha.digest());
-      return checksum;
-    } catch (IOException ex) {
-      throw new StorageContainerException("Error during computing checksum: " +
-          "for container " + containerId, ex, CONTAINER_CHECKSUM_ERROR);
-    } finally {
-      IOUtils.closeStream(containerFileStream);
-    }
-  }
-
-  /**
-   * Verify checksum of the container.
-   * @param containerId
-   * @param checksumFile
-   * @param checksum
-   * @throws StorageContainerException
-   */
-  public static void verifyCheckSum(long containerId, File checksumFile,
-                                    String checksum)
-      throws StorageContainerException {
-    try {
-      Preconditions.checkNotNull(checksum);
-      Preconditions.checkNotNull(checksumFile);
-      Path path = Paths.get(checksumFile.getAbsolutePath());
-      List<String> fileCheckSum = Files.readAllLines(path);
-      Preconditions.checkState(fileCheckSum.size() == 1, "checksum " +
-          "should be 32 byte string");
-      if (!checksum.equals(fileCheckSum.get(0))) {
-        LOG.error("Checksum mismatch for the container {}", containerId);
-        throw new StorageContainerException("Checksum mismatch for " +
-            "the container " + containerId, CHECKSUM_MISMATCH);
-      }
-    } catch (StorageContainerException ex) {
-      throw ex;
-    } catch (IOException ex) {
-      LOG.error("Error during verify checksum for container {}", containerId);
-      throw new StorageContainerException("Error during verify checksum" +
-          " for container " + containerId, IO_EXCEPTION);
-    }
-  }
-
-  /**
    * Parse KeyValueContainerData and verify checksum.
-   * @param containerData
-   * @param containerFile
-   * @param checksumFile
-   * @param dbFile
+   * @param kvContainerData
    * @param config
    * @throws IOException
    */
-  public static void parseKeyValueContainerData(
-      KeyValueContainerData containerData, File containerFile, File
-      checksumFile, File dbFile, OzoneConfiguration config) throws IOException {
+  public static void parseKVContainerData(KeyValueContainerData kvContainerData,
+      OzoneConfiguration config) throws IOException {
 
-    Preconditions.checkNotNull(containerData, "containerData cannot be null");
-    Preconditions.checkNotNull(containerFile, "containerFile cannot be null");
-    Preconditions.checkNotNull(checksumFile, "checksumFile cannot be null");
-    Preconditions.checkNotNull(dbFile, "dbFile cannot be null");
-    Preconditions.checkNotNull(config, "ozone config cannot be null");
-
-    long containerId = containerData.getContainerID();
-    String containerName = String.valueOf(containerId);
-    File metadataPath = new File(containerData.getMetadataPath());
-
-    Preconditions.checkNotNull(containerName, "container Name cannot be " +
-        "null");
-    Preconditions.checkNotNull(metadataPath, "metadata path cannot be " +
-        "null");
+    long containerID = kvContainerData.getContainerID();
+    File metadataPath = new File(kvContainerData.getMetadataPath());
 
     // Verify Checksum
-    String checksum = KeyValueContainerUtil.computeCheckSum(
-        containerData.getContainerID(), containerFile);
-    KeyValueContainerUtil.verifyCheckSum(containerId, checksumFile, checksum);
-
-    containerData.setDbFile(dbFile);
+    ContainerUtils.verifyChecksum(kvContainerData);
+
+    File dbFile = KeyValueContainerLocationUtil.getContainerDBFile(
+        metadataPath, containerID);
+    if (!dbFile.exists()) {
+      LOG.error("Container DB file is missing for ContainerID {}. " +
+          "Skipping loading of this container.", containerID);
+      // Don't further process this container, as it is missing db file.
+      return;
+    }
+    kvContainerData.setDbFile(dbFile);
 
-    MetadataStore metadata = KeyUtils.getDB(containerData, config);
+    MetadataStore metadata = KeyUtils.getDB(kvContainerData, config);
     long bytesUsed = 0;
     List<Map.Entry<byte[], byte[]>> liveKeys = metadata
         .getRangeKVs(null, Integer.MAX_VALUE,
@@ -279,8 +189,8 @@ public final class KeyValueContainerUtil {
         return 0L;
       }
     }).sum();
-    containerData.setBytesUsed(bytesUsed);
-    containerData.setKeyCount(liveKeys.size());
+    kvContainerData.setBytesUsed(bytesUsed);
+    kvContainerData.setKeyCount(liveKeys.size());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
index c1595b2..dc33f2e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
@@ -20,6 +20,9 @@ package org.apache.hadoop.ozone.container.ozoneimpl;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.Storage;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
@@ -38,7 +41,6 @@ import java.io.File;
 import java.io.FileFilter;
 import java.io.IOException;
 
-
 /**
  * Class used to read .container files from Volume and build container map.
  *
@@ -46,15 +48,19 @@ import java.io.IOException;
  *
  * ../hdds/VERSION
  * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<containerID>/metadata/<<containerID>>.container
- * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<containerID>/metadata/<<containerID>>.checksum
- * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<containerID>/metadata/<<containerID>>.db
  * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<containerID>/<<dataPath>>
  *
+ * Some ContainerTypes will have extra metadata other than the .container
+ * file. For example, KeyValueContainer will have a .db file. This .db file
+ * will also be stored in the metadata folder along with the .container file.
+ *
+ * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<KVcontainerID>/metadata/<<KVcontainerID>>.db
+ *
  * Note that the <<dataPath>> is dependent on the ContainerType.
  * For KeyValueContainers, the data is stored in a "chunks" folder. As such,
  * the <<dataPath>> layout for KeyValueContainers is
  *
- * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<containerID>/chunks/<<chunksFile>>
+ * ../hdds/<<scmUuid>>/current/<<containerDir>>/<<KVcontainerID>/chunks/<<chunksFile>>
  *
  */
 public class ContainerReader implements Runnable {
@@ -124,22 +130,19 @@ public class ContainerReader implements Runnable {
               for (File containerDir : containerDirs) {
                 File metadataPath = new File(containerDir + File.separator +
                     OzoneConsts.CONTAINER_META_PATH);
-                String containerName = containerDir.getName();
+                long containerID = Long.parseLong(containerDir.getName());
                 if (metadataPath.exists()) {
                   File containerFile = KeyValueContainerLocationUtil
-                      .getContainerFile(metadataPath, containerName);
-                  File checksumFile = KeyValueContainerLocationUtil
-                      .getContainerCheckSumFile(metadataPath, containerName);
-                  if (containerFile.exists() && checksumFile.exists()) {
-                    verifyContainerFile(containerName, containerFile,
-                        checksumFile);
+                      .getContainerFile(metadataPath, containerID);
+                  if (containerFile.exists()) {
+                    verifyContainerFile(containerID, containerFile);
                   } else {
-                    LOG.error("Missing container metadata files for " +
-                        "Container: {}", containerName);
+                    LOG.error("Missing .container file for ContainerID: {}",
+                        containerID);
                   }
                 } else {
                   LOG.error("Missing container metadata directory for " +
-                      "Container: {}", containerName);
+                      "ContainerID: {}", containerID);
                 }
               }
             }
@@ -149,39 +152,46 @@ public class ContainerReader implements Runnable {
     }
   }
 
-  private void verifyContainerFile(String containerName, File containerFile,
-                                   File checksumFile) {
+  private void verifyContainerFile(long containerID, File containerFile) {
     try {
-      ContainerData containerData =  ContainerDataYaml.readContainerFile(
+      ContainerData containerData = ContainerDataYaml.readContainerFile(
           containerFile);
+      if (containerID != containerData.getContainerID()) {
+        LOG.error("Invalid ContainerID in file {}. " +
+            "Skipping loading of this container.", containerFile);
+        return;
+      }
+      verifyContainerData(containerData);
+    } catch (IOException ex) {
+      LOG.error("Failed to parse ContainerFile for ContainerID: {}",
+          containerID, ex);
+    }
+  }
 
-      switch (containerData.getContainerType()) {
-      case KeyValueContainer:
-        KeyValueContainerData keyValueContainerData = (KeyValueContainerData)
+  public void verifyContainerData(ContainerData containerData)
+      throws IOException {
+    switch (containerData.getContainerType()) {
+    case KeyValueContainer:
+      if (containerData instanceof KeyValueContainerData) {
+        KeyValueContainerData kvContainerData = (KeyValueContainerData)
             containerData;
         containerData.setVolume(hddsVolume);
-        File dbFile = KeyValueContainerLocationUtil
-            .getContainerDBFile(new File(containerFile.getParent()),
-                containerName);
-        if (!dbFile.exists()) {
-          LOG.error("Container DB file is missing for Container {}, skipping " +
-                  "this", containerName);
-          // Don't further process this container, as it is missing db file.
-          return;
-        }
-        KeyValueContainerUtil.parseKeyValueContainerData(keyValueContainerData,
-            containerFile, checksumFile, dbFile, config);
-        KeyValueContainer keyValueContainer = new KeyValueContainer(
-            keyValueContainerData, config);
-        containerSet.addContainer(keyValueContainer);
-        break;
-      default:
-        LOG.error("Unrecognized ContainerType {} format during verify " +
-            "ContainerFile", containerData.getContainerType());
+
+        KeyValueContainerUtil.parseKVContainerData(kvContainerData, config);
+        KeyValueContainer kvContainer = new KeyValueContainer(
+            kvContainerData, config);
+        containerSet.addContainer(kvContainer);
+      } else {
+        throw new StorageContainerException("Container File is corrupted. " +
+            "ContainerType is KeyValueContainer but cast to " +
+            "KeyValueContainerData failed. ",
+            ContainerProtos.Result.CONTAINER_METADATA_ERROR);
       }
-    } catch (IOException ex) {
-      LOG.error("Error during reading container file {}", containerFile);
+      break;
+    default:
+      throw new StorageContainerException("Unrecognized ContainerType " +
+          containerData.getContainerType(),
+          ContainerProtos.Result.UNKNOWN_CONTAINER_TYPE);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
index d734271..fd51db3 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDataYaml.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.container.common.impl;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
@@ -37,39 +38,58 @@ import static org.junit.Assert.fail;
  */
 public class TestContainerDataYaml {
 
+  private static long testContainerID = 1234;
+
+  private static String testRoot = new FileSystemTestHelper().getTestRootDir();
+
   private static final int MAXSIZE = 5;
-  @Test
-  public void testCreateContainerFile() throws IOException {
-    String path = new FileSystemTestHelper().getTestRootDir();
-    String containerPath = "1.container";
 
-    File filePath = new File(new FileSystemTestHelper().getTestRootDir());
-    filePath.mkdirs();
+  /**
+   * Creates a .container file. cleanup() should be called at the end of the
+   * test when container file is created.
+   */
+  private File createContainerFile(long containerID) throws IOException {
+    new File(testRoot).mkdirs();
+
+    String containerPath = containerID + ".container";
 
     KeyValueContainerData keyValueContainerData = new KeyValueContainerData(
-        Long.MAX_VALUE, MAXSIZE);
+        containerID, MAXSIZE);
     keyValueContainerData.setContainerDBType("RocksDB");
-    keyValueContainerData.setMetadataPath(path);
-    keyValueContainerData.setChunksPath(path);
+    keyValueContainerData.setMetadataPath(testRoot);
+    keyValueContainerData.setChunksPath(testRoot);
 
-    File containerFile = new File(filePath, containerPath);
+    File containerFile = new File(testRoot, containerPath);
 
     // Create .container file with ContainerData
     ContainerDataYaml.createContainerFile(ContainerProtos.ContainerType
-            .KeyValueContainer, containerFile, keyValueContainerData);
+        .KeyValueContainer, keyValueContainerData, containerFile);
 
     //Check .container file exists or not.
     assertTrue(containerFile.exists());
 
+    return containerFile;
+  }
+
+  private void cleanup() {
+    FileUtil.fullyDelete(new File(testRoot));
+  }
+
+  @Test
+  public void testCreateContainerFile() throws IOException {
+    long containerID = testContainerID++;
+
+    File containerFile = createContainerFile(containerID);
+
     // Read from .container file, and verify data.
     KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
         .readContainerFile(containerFile);
-    assertEquals(Long.MAX_VALUE, kvData.getContainerID());
+    assertEquals(containerID, kvData.getContainerID());
     assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
         .getContainerType());
     assertEquals("RocksDB", kvData.getContainerDBType());
-    assertEquals(path, kvData.getMetadataPath());
-    assertEquals(path, kvData.getChunksPath());
+    assertEquals(containerFile.getParent(), kvData.getMetadataPath());
+    assertEquals(containerFile.getParent(), kvData.getChunksPath());
     assertEquals(ContainerProtos.ContainerLifeCycleState.OPEN, kvData
         .getState());
     assertEquals(1, kvData.getLayOutVersion());
@@ -82,22 +102,20 @@ public class TestContainerDataYaml {
     kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSED);
 
 
-    // Update .container file with new ContainerData.
-    containerFile = new File(filePath, containerPath);
     ContainerDataYaml.createContainerFile(ContainerProtos.ContainerType
-            .KeyValueContainer, containerFile, kvData);
+            .KeyValueContainer, kvData, containerFile);
 
     // Reading newly updated data from .container file
     kvData =  (KeyValueContainerData) ContainerDataYaml.readContainerFile(
         containerFile);
 
     // verify data.
-    assertEquals(Long.MAX_VALUE, kvData.getContainerID());
+    assertEquals(containerID, kvData.getContainerID());
     assertEquals(ContainerProtos.ContainerType.KeyValueContainer, kvData
         .getContainerType());
     assertEquals("RocksDB", kvData.getContainerDBType());
-    assertEquals(path, kvData.getMetadataPath());
-    assertEquals(path, kvData.getChunksPath());
+    assertEquals(containerFile.getParent(), kvData.getMetadataPath());
+    assertEquals(containerFile.getParent(), kvData.getChunksPath());
     assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, kvData
         .getState());
     assertEquals(1, kvData.getLayOutVersion());
@@ -105,19 +123,15 @@ public class TestContainerDataYaml {
     assertEquals("hdfs", kvData.getMetadata().get("VOLUME"));
     assertEquals("ozone", kvData.getMetadata().get("OWNER"));
     assertEquals(MAXSIZE, kvData.getMaxSizeGB());
-
-    FileUtil.fullyDelete(filePath);
-
-
   }
 
   @Test
   public void testIncorrectContainerFile() throws IOException{
     try {
-      String path = "incorrect.container";
+      String containerFile = "incorrect.container";
       //Get file from resources folder
       ClassLoader classLoader = getClass().getClassLoader();
-      File file = new File(classLoader.getResource(path).getFile());
+      File file = new File(classLoader.getResource(containerFile).getFile());
       KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
           .readContainerFile(file);
       fail("testIncorrectContainerFile failed");
@@ -137,12 +151,13 @@ public class TestContainerDataYaml {
     // created or not.
 
     try {
-      String path = "additionalfields.container";
+      String containerFile = "additionalfields.container";
       //Get file from resources folder
       ClassLoader classLoader = getClass().getClassLoader();
-      File file = new File(classLoader.getResource(path).getFile());
+      File file = new File(classLoader.getResource(containerFile).getFile());
       KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
           .readContainerFile(file);
+      ContainerUtils.verifyChecksum(kvData);
 
       //Checking the Container file data is consistent or not
       assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED, kvData
@@ -159,9 +174,45 @@ public class TestContainerDataYaml {
       assertEquals(2, kvData.getMetadata().size());
 
     } catch (Exception ex) {
+      ex.printStackTrace();
       fail("testCheckBackWardCompatabilityOfContainerFile failed");
     }
   }
 
+  /**
+   * Test to verify {@link ContainerUtils#verifyChecksum(ContainerData)}.
+   */
+  @Test
+  public void testChecksumInContainerFile() throws IOException {
+    long containerID = testContainerID++;
 
+    File containerFile = createContainerFile(containerID);
+
+    // Read from .container file, and verify data.
+    KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
+        .readContainerFile(containerFile);
+    ContainerUtils.verifyChecksum(kvData);
+
+    cleanup();
+  }
+
+  /**
+   * Test to verify incorrect checksum is detected.
+   */
+  @Test
+  public void testIncorrectChecksum() {
+    try {
+      String containerFile = "incorrect.checksum.container";
+      //Get file from resources folder
+      ClassLoader classLoader = getClass().getClassLoader();
+      File file = new File(classLoader.getResource(containerFile).getFile());
+      KeyValueContainerData kvData = (KeyValueContainerData) ContainerDataYaml
+          .readContainerFile(file);
+      ContainerUtils.verifyChecksum(kvData);
+      fail("testIncorrectChecksum failed");
+    } catch (Exception ex) {
+      GenericTestUtils.assertExceptionContains("Container checksum error for " +
+          "ContainerID:", ex);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
index 4f00507..2bf41e5 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
@@ -69,8 +69,7 @@ public class TestKeyValueContainer {
   private String scmId = UUID.randomUUID().toString();
   private VolumeSet volumeSet;
   private RoundRobinVolumeChoosingPolicy volumeChoosingPolicy;
-  private long containerId = 1L;
-  private String containerName = String.valueOf(containerId);
+  private long containerID = 1L;
   private KeyValueContainerData keyValueContainerData;
   private KeyValueContainer keyValueContainer;
 
@@ -111,16 +110,12 @@ public class TestKeyValueContainer {
     assertTrue(chunksPath != null);
     File containerMetaDataLoc = new File(containerMetaDataPath);
 
-    //Check whether container file, check sum file and container db file exists
-    // or not.
+    //Check whether container file and container db file exists or not.
     assertTrue(KeyValueContainerLocationUtil.getContainerFile(
-        containerMetaDataLoc, containerName).exists(), ".Container File does" +
+        containerMetaDataLoc, containerID).exists(), ".Container File does" +
         " not exist");
-    assertTrue(KeyValueContainerLocationUtil.getContainerCheckSumFile(
-        containerMetaDataLoc, containerName).exists(), "Container check sum " +
-        "File does" + " not exist");
     assertTrue(KeyValueContainerLocationUtil.getContainerDBFile(
-        containerMetaDataLoc, containerName).exists(), "Container DB does " +
+        containerMetaDataLoc, containerID).exists(), "Container DB does " +
         "not exist");
   }
 
@@ -172,10 +167,10 @@ public class TestKeyValueContainer {
 
     assertFalse("Container File still exists",
         KeyValueContainerLocationUtil.getContainerFile(containerMetaDataLoc,
-            containerName).exists());
+            containerID).exists());
     assertFalse("Container DB file still exists",
         KeyValueContainerLocationUtil.getContainerDBFile(containerMetaDataLoc,
-            containerName).exists());
+            containerID).exists());
   }
 
 
@@ -195,7 +190,7 @@ public class TestKeyValueContainer {
         .getMetadataPath();
     File containerMetaDataLoc = new File(containerMetaDataPath);
     File containerFile = KeyValueContainerLocationUtil.getContainerFile(
-        containerMetaDataLoc, containerName);
+        containerMetaDataLoc, containerID);
 
     keyValueContainerData = (KeyValueContainerData) ContainerDataYaml
         .readContainerFile(containerFile);
@@ -236,7 +231,7 @@ public class TestKeyValueContainer {
         .getMetadataPath();
     File containerMetaDataLoc = new File(containerMetaDataPath);
     File containerFile = KeyValueContainerLocationUtil.getContainerFile(
-        containerMetaDataLoc, containerName);
+        containerMetaDataLoc, containerID);
 
     keyValueContainerData = (KeyValueContainerData) ContainerDataYaml
         .readContainerFile(containerFile);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/test/resources/additionalfields.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/additionalfields.container b/hadoop-hdds/container-service/src/test/resources/additionalfields.container
index f437a95..73cf5f3 100644
--- a/hadoop-hdds/container-service/src/test/resources/additionalfields.container
+++ b/hadoop-hdds/container-service/src/test/resources/additionalfields.container
@@ -8,4 +8,5 @@ layOutVersion: 1
 maxSizeGB: 5
 metadata: {OWNER: ozone, VOLUME: hdfs}
 state: CLOSED
-aclEnabled: true
\ No newline at end of file
+aclEnabled: true
+checksum: 1bbff32aeaa8fadc0b80c5c1e0597036e96acd8ae4bddbed188a2162762251a2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/test/resources/incorrect.checksum.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/incorrect.checksum.container b/hadoop-hdds/container-service/src/test/resources/incorrect.checksum.container
new file mode 100644
index 0000000..feeeadc
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/resources/incorrect.checksum.container
@@ -0,0 +1,11 @@
+!<KeyValueContainerData>
+containerDBType: RocksDB
+chunksPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
+containerID: 9223372036854775807
+containerType: KeyValueContainer
+metadataPath: /hdds/current/aed-fg4-hji-jkl/containerdir0/1
+layOutVersion: 1
+maxSizeGB: 5
+metadata: {OWNER: ozone, VOLUME: hdfs}
+state: OPEN
+checksum: 08bc9d390f9183aeed3cf33c789e2a07310bba60f3cf55941caccc939db8670f
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/container-service/src/test/resources/incorrect.container
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/resources/incorrect.container b/hadoop-hdds/container-service/src/test/resources/incorrect.container
index 38a8857..8aeb30c 100644
--- a/hadoop-hdds/container-service/src/test/resources/incorrect.container
+++ b/hadoop-hdds/container-service/src/test/resources/incorrect.container
@@ -7,4 +7,5 @@ metadataPath: /hdds/current/aed-fg4-hji-jkl/containerDir0/1
 layOutVersion: 1
 maxSizeGB: 5
 metadata: {OWNER: ozone, VOLUME: hdfs}
-state: INVALID
\ No newline at end of file
+state: INVALID
+checksum: 08bc9d390f9183aeed3cf33c789e2a07310bba60f3cf55941caccc939db8670f
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-hdds/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/pom.xml b/hadoop-hdds/pom.xml
index 09fac33..f655c2f 100644
--- a/hadoop-hdds/pom.xml
+++ b/hadoop-hdds/pom.xml
@@ -104,6 +104,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/resources/webapps/static/nvd3-1.8.5.min.js.map</exclude>
             <exclude>src/test/resources/incorrect.container</exclude>
             <exclude>src/test/resources/additionalfields.container</exclude>
+            <exclude>src/test/resources/incorrect.checksum.container</exclude>
           </excludes>
         </configuration>
       </plugin>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
index d29937e..c2e1645 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
@@ -716,7 +716,7 @@ public class TestContainerPersistence {
 
     File orgContainerFile = KeyValueContainerLocationUtil.getContainerFile(
         new File(container.getContainerData().getMetadataPath()),
-        String.valueOf(testContainerID));
+        testContainerID);
     Assert.assertTrue(orgContainerFile.exists());
 
     Map<String, String> newMetadata = Maps.newHashMap();
@@ -740,7 +740,7 @@ public class TestContainerPersistence {
     // Verify container data on disk
     File newContainerFile = KeyValueContainerLocationUtil.getContainerFile(
         new File(actualNewData.getMetadataPath()),
-        String.valueOf(testContainerID));
+        testContainerID);
     Assert.assertTrue("Container file should exist.",
         newContainerFile.exists());
     Assert.assertEquals("Container file should be in same location.",
@@ -780,8 +780,8 @@ public class TestContainerPersistence {
 
     // Update a non-existing container
     exception.expect(StorageContainerException.class);
-    exception.expectMessage("Container is an Inconsistent state, missing " +
-        "required files(.container, .chksm).");
+    exception.expectMessage("Container is an Inconsistent " +
+        "state, missing .container file.");
     Container nonExistentContainer = new KeyValueContainer(
         new KeyValueContainerData(RandomUtils.nextLong(),
             ContainerTestHelper.CONTAINER_MAX_SIZE_GB), conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b507f83e/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index 1522283..d271ed3 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -33,11 +33,14 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 import org.junit.rules.Timeout;
 
 import java.util.*;
 import java.util.concurrent.CompletableFuture;
 
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
+
 /**
  * Tests ozone containers.
  */
@@ -48,6 +51,9 @@ public class TestOzoneContainer {
   @Rule
   public Timeout testTimeout = new Timeout(300000);
 
+  @Rule
+  public TemporaryFolder tempFolder = new TemporaryFolder();
+
   @Test
   public void testCreateOzoneContainer() throws Exception {
     long containerID = ContainerTestHelper.getTestContainerID();
@@ -60,6 +66,7 @@ public class TestOzoneContainer {
       // We don't start Ozone Container via data node, we will do it
       // independently in our test path.
       Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline();
+      conf.set(HDDS_DATANODE_DIR_KEY, tempFolder.getRoot().getPath());
       conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, pipeline.getLeader()
               .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
       conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false);


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


[36/50] [abbrv] 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/c7809fbd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c7809fbd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c7809fbd

Branch: refs/heads/YARN-3409
Commit: c7809fbd593fe78779a1b8889f8c83e9b09e4dda
Parents: e6b8425
Author: Naganarasimha <na...@apache.org>
Authored: Mon Mar 12 08:05:53 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:40:34 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/c7809fbd/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 e6e9198..ebd789f 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
@@ -3525,9 +3525,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.";
@@ -3559,6 +3562,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/c7809fbd/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/c7809fbd/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/c7809fbd/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 ec955b2..3f86266 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
@@ -2896,6 +2896,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
@@ -2933,6 +2947,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/c7809fbd/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 c8234bd..69644b3 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
@@ -64,6 +64,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;
@@ -121,6 +124,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;
@@ -160,14 +164,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)
@@ -180,10 +215,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:
@@ -405,16 +440,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/c7809fbd/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/c7809fbd/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 2615beb..517c6b3 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/c7809fbd/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/c7809fbd/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/c7809fbd/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/c7809fbd/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/c7809fbd/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/c7809fbd/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/c7809fbd/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/c7809fbd/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


[09/50] [abbrv] hadoop git commit: HADOOP-15609. Retry KMS calls when SSLHandshakeException occurs. Contributed by Kitti Nanasi.

Posted by su...@apache.org.
HADOOP-15609. Retry KMS calls when SSLHandshakeException occurs. Contributed by Kitti Nanasi.


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

Branch: refs/heads/YARN-3409
Commit: 81d59506e539673edde12e19c0df5c2edd9d02ad
Parents: 2686447
Author: Xiao Chen <xi...@apache.org>
Authored: Tue Jul 24 21:45:14 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Tue Jul 24 21:45:43 2018 -0700

----------------------------------------------------------------------
 .../key/kms/LoadBalancingKMSClientProvider.java | 17 ++++-
 .../kms/TestLoadBalancingKMSClientProvider.java | 79 ++++++++++++++++++++
 2 files changed, 92 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d59506/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
index 1ac3fd3..23cdc50 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.crypto.key.kms;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
+import java.net.ConnectException;
 import java.security.GeneralSecurityException;
 import java.security.NoSuchAlgorithmException;
 import java.util.Arrays;
@@ -27,6 +28,8 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import javax.net.ssl.SSLHandshakeException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
@@ -115,7 +118,6 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
     if (providers.length == 0) {
       throw new IOException("No providers configured !");
     }
-    IOException ex = null;
     int numFailovers = 0;
     for (int i = 0;; i++, numFailovers++) {
       KMSClientProvider provider = providers[(currPos + i) % providers.length];
@@ -130,8 +132,15 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
       } catch (IOException ioe) {
         LOG.warn("KMS provider at [{}] threw an IOException: ",
             provider.getKMSUrl(), ioe);
-        ex = ioe;
-
+        // SSLHandshakeException can occur here because of lost connection
+        // with the KMS server, creating a ConnectException from it,
+        // so that the FailoverOnNetworkExceptionRetry policy will retry
+        if (ioe instanceof SSLHandshakeException) {
+          Exception cause = ioe;
+          ioe = new ConnectException("SSLHandshakeException: "
+              + cause.getMessage());
+          ioe.initCause(cause);
+        }
         RetryAction action = null;
         try {
           action = retryPolicy.shouldRetry(ioe, 0, numFailovers, false);
@@ -153,7 +162,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
                   CommonConfigurationKeysPublic.
                   KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, providers.length),
               providers.length);
-          throw ex;
+          throw ioe;
         }
         if (((numFailovers + 1) % providers.length) == 0) {
           // Sleep only after we try all the providers for every cycle.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d59506/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
index bd68dca..4e7aed9 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.crypto.key.kms;
 
 import static org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -26,12 +27,15 @@ import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.verify;
 
 import java.io.IOException;
+import java.net.ConnectException;
 import java.net.NoRouteToHostException;
 import java.net.URI;
 import java.net.UnknownHostException;
 import java.security.GeneralSecurityException;
 import java.security.NoSuchAlgorithmException;
 
+import javax.net.ssl.SSLHandshakeException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProvider.Options;
@@ -44,13 +48,18 @@ import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.junit.After;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 import org.mockito.Mockito;
 
 import com.google.common.collect.Sets;
 
 public class TestLoadBalancingKMSClientProvider {
 
+  @Rule
+  public Timeout testTimeout = new Timeout(30 * 1000);
+
   @BeforeClass
   public static void setup() throws IOException {
     SecurityUtil.setTokenServiceUseIp(false);
@@ -638,4 +647,74 @@ public class TestLoadBalancingKMSClientProvider {
     verify(p2, Mockito.times(1)).createKey(Mockito.eq("test3"),
             Mockito.any(Options.class));
   }
+
+  /**
+   * Tests the operation succeeds second time after SSLHandshakeException.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesWithSSLHandshakeExceptionSucceedsSecondTime()
+      throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 3);
+    final String keyName = "test";
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new SSLHandshakeException("p1"))
+        .thenReturn(new KMSClientProvider.KMSKeyVersion(keyName, "v1",
+            new byte[0]));
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new ConnectException("p2"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2}, 0, conf);
+
+    kp.createKey(keyName, new Options(conf));
+    verify(p1, Mockito.times(2)).createKey(Mockito.eq(keyName),
+        Mockito.any(Options.class));
+    verify(p2, Mockito.times(1)).createKey(Mockito.eq(keyName),
+        Mockito.any(Options.class));
+  }
+
+  /**
+   * Tests the operation fails at every attempt after SSLHandshakeException.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesWithSSLHandshakeExceptionFailsAtEveryAttempt()
+      throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 2);
+    final String keyName = "test";
+    final String exceptionMessage = "p1 exception message";
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    Exception originalSslEx = new SSLHandshakeException(exceptionMessage);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(originalSslEx);
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new ConnectException("p2 exception message"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2}, 0, conf);
+
+    Exception interceptedEx = intercept(ConnectException.class,
+        "SSLHandshakeException: " + exceptionMessage,
+        ()-> kp.createKey(keyName, new Options(conf)));
+    assertEquals(originalSslEx, interceptedEx.getCause());
+
+    verify(p1, Mockito.times(2)).createKey(Mockito.eq(keyName),
+        Mockito.any(Options.class));
+    verify(p2, Mockito.times(1)).createKey(Mockito.eq(keyName),
+        Mockito.any(Options.class));
+  }
 }
\ No newline at end of file


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


[34/50] [abbrv] 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/d3a81876
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d3a81876
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d3a81876

Branch: refs/heads/YARN-3409
Commit: d3a81876c4e6ed70d28f1004a2d1fc34f13df4a9
Parents: fbfbd34
Author: Sunil G <su...@apache.org>
Authored: Tue Feb 27 08:15:42 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:40:34 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/d3a81876/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/d3a81876/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/d3a81876/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/d3a81876/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


[37/50] [abbrv] 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/293ad9ef
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/293ad9ef
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/293ad9ef

Branch: refs/heads/YARN-3409
Commit: 293ad9efcf38e96dda2984bfb43d7f5209a00695
Parents: dae7437
Author: Sunil G <su...@apache.org>
Authored: Sat Mar 31 19:53:06 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Jul 27 16:40:34 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/293ad9ef/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/293ad9ef/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/293ad9ef/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