You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by su...@apache.org on 2018/08/25 15:49:18 UTC

[01/50] [abbrv] hadoop git commit: YARN-8242. YARN NM: OOM error while reading back the state store on recovery. Contributed by Pradeep Ambati and Kanwaljeet Sachdev [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-3409 7ad66e806 -> 0f0b55f7f (forced update)


YARN-8242. YARN NM: OOM error while reading back the state store on recovery. Contributed by Pradeep Ambati and Kanwaljeet Sachdev


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

Branch: refs/heads/YARN-3409
Commit: 65e7469712be6cf393e29ef73cc94727eec81227
Parents: 01ff817
Author: Jason Lowe <jl...@apache.org>
Authored: Mon Aug 20 10:14:40 2018 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Mon Aug 20 10:14:40 2018 -0500

----------------------------------------------------------------------
 .../server/nodemanager/DeletionService.java     |  25 +-
 .../containermanager/ContainerManagerImpl.java  |  26 +-
 .../localizer/ResourceLocalizationService.java  |  56 +--
 .../recovery/NMLeveldbStateStoreService.java    | 412 ++++++++++++-------
 .../recovery/NMNullStateStoreService.java       |   2 +-
 .../recovery/NMStateStoreService.java           |  55 +--
 .../nodemanager/recovery/RecoveryIterator.java  |  41 ++
 .../security/NMContainerTokenSecretManager.java |  27 +-
 .../security/NMTokenSecretManagerInNM.java      |  15 +-
 .../recovery/NMMemoryStateStoreService.java     |  82 +++-
 .../TestNMLeveldbStateStoreService.java         | 216 +++++++---
 11 files changed, 647 insertions(+), 310 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.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/DeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
index ae81dc1..e665c5a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
@@ -19,13 +19,14 @@
 package org.apache.hadoop.yarn.server.nodemanager;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
+
+import org.apache.hadoop.yarn.server.nodemanager.recovery.RecoveryIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
@@ -96,16 +97,20 @@ public class DeletionService extends AbstractService {
 
   private void recover(NMStateStoreService.RecoveredDeletionServiceState state)
       throws IOException {
-    List<DeletionServiceDeleteTaskProto> taskProtos = state.getTasks();
     Map<Integer, DeletionTaskRecoveryInfo> idToInfoMap =
-        new HashMap<>(taskProtos.size());
-    Set<Integer> successorTasks = new HashSet<>();
-    for (DeletionServiceDeleteTaskProto proto : taskProtos) {
-      DeletionTaskRecoveryInfo info =
-          NMProtoUtils.convertProtoToDeletionTaskRecoveryInfo(proto, this);
-      idToInfoMap.put(info.getTask().getTaskId(), info);
-      nextTaskId.set(Math.max(nextTaskId.get(), info.getTask().getTaskId()));
-      successorTasks.addAll(info.getSuccessorTaskIds());
+        new HashMap<Integer, DeletionTaskRecoveryInfo>();
+    Set<Integer> successorTasks = new HashSet<Integer>();
+
+    try (RecoveryIterator<DeletionServiceDeleteTaskProto> it =
+             state.getIterator()) {
+      while (it.hasNext()) {
+        DeletionServiceDeleteTaskProto proto = it.next();
+        DeletionTaskRecoveryInfo info =
+            NMProtoUtils.convertProtoToDeletionTaskRecoveryInfo(proto, this);
+        idToInfoMap.put(info.getTask().getTaskId(), info);
+        nextTaskId.set(Math.max(nextTaskId.get(), info.getTask().getTaskId()));
+        successorTasks.addAll(info.getSuccessorTaskIds());
+      }
     }
 
     // restore the task dependencies and schedule the deletion tasks that

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.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/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 8b35258..b89e2dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -23,6 +23,7 @@ import com.google.protobuf.ByteString;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.UpdateContainerTokenEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerTokenUpdatedEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerSchedulerEvent;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.RecoveryIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -356,19 +357,26 @@ public class ContainerManagerImpl extends CompositeService implements
           stateStore.loadLocalizationState());
 
       RecoveredApplicationsState appsState = stateStore.loadApplicationsState();
-      for (ContainerManagerApplicationProto proto :
-           appsState.getApplications()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Recovering application with state: " + proto.toString());
+      try (RecoveryIterator<ContainerManagerApplicationProto> rasIterator =
+               appsState.getIterator()) {
+        while (rasIterator.hasNext()) {
+          ContainerManagerApplicationProto proto = rasIterator.next();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Recovering application with state: " + proto.toString());
+          }
+          recoverApplication(proto);
         }
-        recoverApplication(proto);
       }
 
-      for (RecoveredContainerState rcs : stateStore.loadContainersState()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Recovering container with state: " + rcs);
+      try (RecoveryIterator<RecoveredContainerState> rcsIterator =
+               stateStore.getContainerStateIterator()) {
+        while (rcsIterator.hasNext()) {
+          RecoveredContainerState rcs = rcsIterator.next();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Recovering container with state: " + rcs);
+          }
+          recoverContainer(rcs);
         }
-        recoverContainer(rcs);
       }
 
       // Recovery AMRMProxy state after apps and containers are recovered

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.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/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index 3834ece..2892d1f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer;
 
 import static org.apache.hadoop.fs.CreateFlag.CREATE;
 import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
+
+import org.apache.hadoop.yarn.server.nodemanager.recovery.RecoveryIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -295,42 +297,46 @@ public class ResourceLocalizationService extends CompositeService
 
   //Recover localized resources after an NM restart
   public void recoverLocalizedResources(RecoveredLocalizationState state)
-      throws URISyntaxException {
+      throws URISyntaxException, IOException {
     LocalResourceTrackerState trackerState = state.getPublicTrackerState();
     recoverTrackerResources(publicRsrc, trackerState);
 
-    for (Map.Entry<String, RecoveredUserResources> userEntry :
-         state.getUserResources().entrySet()) {
-      String user = userEntry.getKey();
-      RecoveredUserResources userResources = userEntry.getValue();
-      trackerState = userResources.getPrivateTrackerState();
-      if (!trackerState.isEmpty()) {
-        LocalResourcesTracker tracker = new LocalResourcesTrackerImpl(user,
-            null, dispatcher, true, super.getConfig(), stateStore, dirsHandler);
-        LocalResourcesTracker oldTracker = privateRsrc.putIfAbsent(user,
-            tracker);
-        if (oldTracker != null) {
-          tracker = oldTracker;
-        }
-        recoverTrackerResources(tracker, trackerState);
-      }
-
-      for (Map.Entry<ApplicationId, LocalResourceTrackerState> appEntry :
-           userResources.getAppTrackerStates().entrySet()) {
-        trackerState = appEntry.getValue();
+    try (RecoveryIterator<Map.Entry<String, RecoveredUserResources>> it
+             = state.getIterator()) {
+      while (it.hasNext()) {
+        Map.Entry<String, RecoveredUserResources> userEntry = it.next();
+        String user = userEntry.getKey();
+        RecoveredUserResources userResources = userEntry.getValue();
+        trackerState = userResources.getPrivateTrackerState();
         if (!trackerState.isEmpty()) {
-          ApplicationId appId = appEntry.getKey();
-          String appIdStr = appId.toString();
           LocalResourcesTracker tracker = new LocalResourcesTrackerImpl(user,
-              appId, dispatcher, false, super.getConfig(), stateStore,
+              null, dispatcher, true, super.getConfig(), stateStore,
               dirsHandler);
-          LocalResourcesTracker oldTracker = appRsrc.putIfAbsent(appIdStr,
+          LocalResourcesTracker oldTracker = privateRsrc.putIfAbsent(user,
               tracker);
           if (oldTracker != null) {
             tracker = oldTracker;
           }
           recoverTrackerResources(tracker, trackerState);
         }
+
+        for (Map.Entry<ApplicationId, LocalResourceTrackerState> appEntry :
+            userResources.getAppTrackerStates().entrySet()) {
+          trackerState = appEntry.getValue();
+          if (!trackerState.isEmpty()) {
+            ApplicationId appId = appEntry.getKey();
+            String appIdStr = appId.toString();
+            LocalResourcesTracker tracker = new LocalResourcesTrackerImpl(user,
+                appId, dispatcher, false, super.getConfig(), stateStore,
+                dirsHandler);
+            LocalResourcesTracker oldTracker = appRsrc.putIfAbsent(appIdStr,
+                tracker);
+            if (oldTracker != null) {
+              tracker = oldTracker;
+            }
+            recoverTrackerResources(tracker, trackerState);
+          }
+        }
       }
     }
   }
@@ -556,7 +562,7 @@ public class ResourceLocalizationService extends CompositeService
       rsrcCleanup.getResources();
     for (Map.Entry<LocalResourceVisibility, Collection<LocalResourceRequest>> e :
          rsrcs.entrySet()) {
-      LocalResourcesTracker tracker = getLocalResourcesTracker(e.getKey(), c.getUser(), 
+      LocalResourcesTracker tracker = getLocalResourcesTracker(e.getKey(), c.getUser(),
           c.getContainerId().getApplicationAttemptId()
           .getApplicationId());
       for (LocalResourceRequest req : e.getValue()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.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/recovery/NMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
index 67f642d..5d4253d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
@@ -66,6 +66,7 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -73,6 +74,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.NoSuchElementException;
 import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
@@ -225,68 +227,119 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     return isHealthy;
   }
 
-  @Override
-  public List<RecoveredContainerState> loadContainersState()
+  // LeveldbIterator starting at startkey
+  private LeveldbIterator getLevelDBIterator(String startKey)
       throws IOException {
-    ArrayList<RecoveredContainerState> containers =
-        new ArrayList<RecoveredContainerState>();
-    ArrayList<ContainerId> containersToRemove =
-              new ArrayList<ContainerId>();
-    LeveldbIterator iter = null;
     try {
-      iter = new LeveldbIterator(db);
-      iter.seek(bytes(CONTAINERS_KEY_PREFIX));
+      LeveldbIterator it = new LeveldbIterator(db);
+      it.seek(bytes(startKey));
+      return it;
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
 
-      while (iter.hasNext()) {
-        Entry<byte[], byte[]> entry = iter.peekNext();
+  // Base Recovery Iterator
+  private abstract class BaseRecoveryIterator<T> implements
+      RecoveryIterator<T> {
+    LeveldbIterator it;
+    T nextItem;
+
+    BaseRecoveryIterator(String dbKey) throws IOException {
+      this.it = getLevelDBIterator(dbKey);
+      this.nextItem = null;
+    }
+
+    protected abstract T getNextItem(LeveldbIterator it) throws IOException;
+
+    @Override
+    public boolean hasNext() throws IOException {
+      if (nextItem == null) {
+        nextItem = getNextItem(it);
+      }
+      return (nextItem != null);
+    }
+
+    @Override
+    public T next() throws IOException, NoSuchElementException {
+      T tmp = nextItem;
+      if (tmp != null) {
+        nextItem = null;
+        return tmp;
+      } else {
+        tmp = getNextItem(it);
+        if (tmp == null) {
+          throw new NoSuchElementException();
+        }
+        return tmp;
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (it != null) {
+        it.close();
+      }
+    }
+  }
+
+  //  Container Recovery Iterator
+  private class ContainerStateIterator extends
+      BaseRecoveryIterator<RecoveredContainerState> {
+    ContainerStateIterator() throws IOException {
+      super(CONTAINERS_KEY_PREFIX);
+    }
+
+    @Override
+    protected RecoveredContainerState getNextItem(LeveldbIterator it)
+        throws IOException {
+      return getNextRecoveredContainer(it);
+    }
+  }
+
+  private RecoveredContainerState getNextRecoveredContainer(LeveldbIterator it)
+      throws IOException {
+    RecoveredContainerState rcs = null;
+    try {
+      while (it.hasNext()) {
+        Entry<byte[], byte[]> entry = it.peekNext();
         String key = asString(entry.getKey());
         if (!key.startsWith(CONTAINERS_KEY_PREFIX)) {
-          break;
+          return null;
         }
 
         int idEndPos = key.indexOf('/', CONTAINERS_KEY_PREFIX.length());
         if (idEndPos < 0) {
           throw new IOException("Unable to determine container in key: " + key);
         }
-        ContainerId containerId = ContainerId.fromString(
-            key.substring(CONTAINERS_KEY_PREFIX.length(), idEndPos));
-        String keyPrefix = key.substring(0, idEndPos+1);
-        RecoveredContainerState rcs = loadContainerState(containerId,
-            iter, keyPrefix);
-        // Don't load container without StartContainerRequest
+        String keyPrefix = key.substring(0, idEndPos + 1);
+        rcs = loadContainerState(it, keyPrefix);
         if (rcs.startRequest != null) {
-          containers.add(rcs);
+          break;
         } else {
-          containersToRemove.add(containerId);
+          removeContainer(rcs.getContainerId());
+          rcs = null;
         }
       }
     } catch (DBException e) {
       throw new IOException(e);
-    } finally {
-      if (iter != null) {
-        iter.close();
-      }
     }
+    return rcs;
+  }
 
-    // remove container without StartContainerRequest
-    for (ContainerId containerId : containersToRemove) {
-      LOG.warn("Remove container " + containerId +
-          " with incomplete records");
-      try {
-        removeContainer(containerId);
-        // TODO: kill and cleanup the leaked container
-      } catch (IOException e) {
-        LOG.error("Unable to remove container " + containerId +
-            " in store", e);
-      }
-    }
 
-    return containers;
+  @Override
+  public RecoveryIterator<RecoveredContainerState> getContainerStateIterator()
+      throws IOException {
+    return new ContainerStateIterator();
   }
 
-  private RecoveredContainerState loadContainerState(ContainerId containerId,
-      LeveldbIterator iter, String keyPrefix) throws IOException {
-    RecoveredContainerState rcs = new RecoveredContainerState();
+  private RecoveredContainerState loadContainerState(LeveldbIterator iter,
+       String keyPrefix) throws IOException {
+    ContainerId containerId = ContainerId.fromString(
+        keyPrefix.substring(CONTAINERS_KEY_PREFIX.length(),
+            keyPrefix.length()-1));
+    RecoveredContainerState rcs = new RecoveredContainerState(containerId);
     rcs.status = RecoveredContainerStatus.REQUESTED;
     while (iter.hasNext()) {
       Entry<byte[],byte[]> entry = iter.peekNext();
@@ -680,35 +733,45 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
   }
 
 
-  @Override
-  public RecoveredApplicationsState loadApplicationsState()
-      throws IOException {
-    RecoveredApplicationsState state = new RecoveredApplicationsState();
-    state.applications = new ArrayList<ContainerManagerApplicationProto>();
-    String keyPrefix = APPLICATIONS_KEY_PREFIX;
-    LeveldbIterator iter = null;
+  // Application Recovery Iterator
+  private class ApplicationStateIterator extends
+      BaseRecoveryIterator<ContainerManagerApplicationProto> {
+    ApplicationStateIterator() throws IOException {
+      super(APPLICATIONS_KEY_PREFIX);
+    }
+
+    @Override
+    protected ContainerManagerApplicationProto getNextItem(LeveldbIterator it)
+        throws IOException {
+      return getNextRecoveredApplication(it);
+    }
+  }
+
+  private ContainerManagerApplicationProto getNextRecoveredApplication(
+      LeveldbIterator it) throws IOException {
+    ContainerManagerApplicationProto applicationProto = null;
     try {
-      iter = new LeveldbIterator(db);
-      iter.seek(bytes(keyPrefix));
-      while (iter.hasNext()) {
-        Entry<byte[], byte[]> entry = iter.next();
+      if (it.hasNext()) {
+        Entry<byte[], byte[]> entry = it.next();
         String key = asString(entry.getKey());
-        if (!key.startsWith(keyPrefix)) {
-          break;
+        if (!key.startsWith(APPLICATIONS_KEY_PREFIX)) {
+          return null;
         }
-        state.applications.add(
-            ContainerManagerApplicationProto.parseFrom(entry.getValue()));
+        applicationProto = ContainerManagerApplicationProto.parseFrom(
+            entry.getValue());
       }
     } catch (DBException e) {
       throw new IOException(e);
-    } finally {
-      if (iter != null) {
-        iter.close();
-      }
     }
+    return applicationProto;
+  }
 
+  @Override
+  public RecoveredApplicationsState loadApplicationsState()
+      throws IOException {
+    RecoveredApplicationsState state = new RecoveredApplicationsState();
+    state.it = new ApplicationStateIterator();
     cleanupDeprecatedFinishedApps();
-
     return state;
   }
 
@@ -752,24 +815,29 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
   }
 
 
-  @Override
-  public RecoveredLocalizationState loadLocalizationState()
-      throws IOException {
-    RecoveredLocalizationState state = new RecoveredLocalizationState();
+  // User Resource Recovery Iterator.
+  private class UserResourcesIterator extends
+      BaseRecoveryIterator<Entry<String, RecoveredUserResources>> {
+    UserResourcesIterator() throws IOException {
+      super(LOCALIZATION_PRIVATE_KEY_PREFIX);
+    }
 
-    LeveldbIterator iter = null;
-    try {
-      iter = new LeveldbIterator(db);
-      iter.seek(bytes(LOCALIZATION_PUBLIC_KEY_PREFIX));
-      state.publicTrackerState = loadResourceTrackerState(iter,
-          LOCALIZATION_PUBLIC_KEY_PREFIX);
+    @Override
+    protected Entry<String, RecoveredUserResources> getNextItem(
+        LeveldbIterator it) throws IOException {
+      return getNextRecoveredPrivateLocalizationEntry(it);
+    }
+  }
 
-      iter.seek(bytes(LOCALIZATION_PRIVATE_KEY_PREFIX));
-      while (iter.hasNext()) {
-        Entry<byte[],byte[]> entry = iter.peekNext();
+  private Entry<String, RecoveredUserResources> getNextRecoveredPrivateLocalizationEntry(
+      LeveldbIterator it) throws IOException {
+    Entry<String, RecoveredUserResources> localEntry = null;
+    try {
+      if (it.hasNext()) {
+        Entry<byte[], byte[]> entry = it.peekNext();
         String key = asString(entry.getKey());
         if (!key.startsWith(LOCALIZATION_PRIVATE_KEY_PREFIX)) {
-          break;
+          return null;
         }
 
         int userEndPos = key.indexOf('/',
@@ -780,17 +848,24 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         }
         String user = key.substring(
             LOCALIZATION_PRIVATE_KEY_PREFIX.length(), userEndPos);
-        state.userResources.put(user, loadUserLocalizedResources(iter,
-            key.substring(0, userEndPos+1)));
+        RecoveredUserResources val = loadUserLocalizedResources(it,
+            key.substring(0, userEndPos+1));
+        localEntry = new AbstractMap.SimpleEntry<>(user, val);
       }
     } catch (DBException e) {
       throw new IOException(e);
-    } finally {
-      if (iter != null) {
-        iter.close();
-      }
     }
+    return localEntry;
+  }
 
+  @Override
+  public RecoveredLocalizationState loadLocalizationState()
+      throws IOException {
+    RecoveredLocalizationState state = new RecoveredLocalizationState();
+    LeveldbIterator it = getLevelDBIterator(LOCALIZATION_PUBLIC_KEY_PREFIX);
+    state.publicTrackerState = loadResourceTrackerState(it,
+        LOCALIZATION_PUBLIC_KEY_PREFIX);
+    state.it = new UserResourcesIterator();
     return state;
   }
 
@@ -800,7 +875,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     final String startedPrefix = keyPrefix + LOCALIZATION_STARTED_SUFFIX;
     LocalResourceTrackerState state = new LocalResourceTrackerState();
     while (iter.hasNext()) {
-      Entry<byte[],byte[]> entry = iter.peekNext();
+      Entry<byte[], byte[]> entry = iter.peekNext();
       String key = asString(entry.getKey());
       if (!key.startsWith(keyPrefix)) {
         break;
@@ -981,32 +1056,44 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         + LOCALIZATION_APPCACHE_SUFFIX + appId + "/";
   }
 
+  // Deletion State Recovery Iterator.
+  private class DeletionStateIterator extends
+      BaseRecoveryIterator<DeletionServiceDeleteTaskProto> {
+    DeletionStateIterator() throws IOException {
+      super(DELETION_TASK_KEY_PREFIX);
+    }
 
-  @Override
-  public RecoveredDeletionServiceState loadDeletionServiceState()
-      throws IOException {
-    RecoveredDeletionServiceState state = new RecoveredDeletionServiceState();
-    state.tasks = new ArrayList<DeletionServiceDeleteTaskProto>();
-    LeveldbIterator iter = null;
+    @Override
+    protected DeletionServiceDeleteTaskProto getNextItem(LeveldbIterator it)
+        throws IOException {
+      return getNextRecoveredDeletionService(it);
+    }
+  }
+
+  private DeletionServiceDeleteTaskProto getNextRecoveredDeletionService(
+      LeveldbIterator it) throws IOException {
+    DeletionServiceDeleteTaskProto deleteProto = null;
     try {
-      iter = new LeveldbIterator(db);
-      iter.seek(bytes(DELETION_TASK_KEY_PREFIX));
-      while (iter.hasNext()) {
-        Entry<byte[], byte[]> entry = iter.next();
+      if (it.hasNext()) {
+        Entry<byte[], byte[]> entry = it.next();
         String key = asString(entry.getKey());
         if (!key.startsWith(DELETION_TASK_KEY_PREFIX)) {
-          break;
+          return null;
         }
-        state.tasks.add(
-            DeletionServiceDeleteTaskProto.parseFrom(entry.getValue()));
+        deleteProto = DeletionServiceDeleteTaskProto.parseFrom(
+            entry.getValue());
       }
     } catch (DBException e) {
       throw new IOException(e);
-    } finally {
-      if (iter != null) {
-        iter.close();
-      }
     }
+    return deleteProto;
+  }
+
+  @Override
+  public RecoveredDeletionServiceState loadDeletionServiceState()
+      throws IOException {
+    RecoveredDeletionServiceState state = new RecoveredDeletionServiceState();
+    state.it = new DeletionStateIterator();
     return state;
   }
 
@@ -1033,29 +1120,44 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     }
   }
 
+  private MasterKey getMasterKey(String dbKey) throws IOException {
+    try{
+      byte[] data = db.get(bytes(dbKey));
+      if (data == null || data.length == 0) {
+        return null;
+      }
+      return parseMasterKey(data);
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
 
-  @Override
-  public RecoveredNMTokensState loadNMTokensState() throws IOException {
-    RecoveredNMTokensState state = new RecoveredNMTokensState();
-    state.applicationMasterKeys =
-        new HashMap<ApplicationAttemptId, MasterKey>();
-    LeveldbIterator iter = null;
+  // Recover NMTokens Iterator
+  private class NMTokensStateIterator extends
+      BaseRecoveryIterator<Entry<ApplicationAttemptId, MasterKey>> {
+    NMTokensStateIterator() throws IOException {
+      super(NM_TOKENS_KEY_PREFIX);
+    }
+
+    @Override
+    protected Entry<ApplicationAttemptId, MasterKey> getNextItem(
+        LeveldbIterator it) throws IOException {
+      return getNextMasterKeyEntry(it);
+    }
+  }
+
+  private Entry<ApplicationAttemptId, MasterKey> getNextMasterKeyEntry(
+      LeveldbIterator it) throws IOException {
+    Entry<ApplicationAttemptId, MasterKey> masterKeyentry = null;
     try {
-      iter = new LeveldbIterator(db);
-      iter.seek(bytes(NM_TOKENS_KEY_PREFIX));
-      while (iter.hasNext()) {
-        Entry<byte[], byte[]> entry = iter.next();
+      while (it.hasNext()) {
+        Entry<byte[], byte[]> entry = it.next();
         String fullKey = asString(entry.getKey());
         if (!fullKey.startsWith(NM_TOKENS_KEY_PREFIX)) {
           break;
         }
         String key = fullKey.substring(NM_TOKENS_KEY_PREFIX.length());
-        if (key.equals(CURRENT_MASTER_KEY_SUFFIX)) {
-          state.currentMasterKey = parseMasterKey(entry.getValue());
-        } else if (key.equals(PREV_MASTER_KEY_SUFFIX)) {
-          state.previousMasterKey = parseMasterKey(entry.getValue());
-        } else if (key.startsWith(
-            ApplicationAttemptId.appAttemptIdStrPrefix)) {
+        if (key.startsWith(ApplicationAttemptId.appAttemptIdStrPrefix)) {
           ApplicationAttemptId attempt;
           try {
             attempt = ApplicationAttemptId.fromString(key);
@@ -1063,17 +1165,25 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
             throw new IOException("Bad application master key state for "
                 + fullKey, e);
           }
-          state.applicationMasterKeys.put(attempt,
+          masterKeyentry = new AbstractMap.SimpleEntry<>(attempt,
               parseMasterKey(entry.getValue()));
+          break;
         }
       }
     } catch (DBException e) {
       throw new IOException(e);
-    } finally {
-      if (iter != null) {
-        iter.close();
-      }
     }
+    return masterKeyentry;
+  }
+
+  @Override
+  public RecoveredNMTokensState loadNMTokensState() throws IOException {
+    RecoveredNMTokensState state = new RecoveredNMTokensState();
+    state.currentMasterKey = getMasterKey(NM_TOKENS_KEY_PREFIX
+                                          + CURRENT_MASTER_KEY_SUFFIX);
+    state.previousMasterKey = getMasterKey(NM_TOKENS_KEY_PREFIX
+                                            + PREV_MASTER_KEY_SUFFIX);
+    state.it = new NMTokensStateIterator();
     return state;
   }
 
@@ -1122,45 +1232,45 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     }
   }
 
+  // Recover ContainersToken Iterator.
+  private class ContainerTokensStateIterator extends
+      BaseRecoveryIterator<Entry<ContainerId, Long>> {
+    ContainerTokensStateIterator() throws IOException {
+      super(CONTAINER_TOKENS_KEY_PREFIX);
+    }
 
-  @Override
-  public RecoveredContainerTokensState loadContainerTokensState()
+    @Override
+    protected Entry<ContainerId, Long> getNextItem(LeveldbIterator it)
+        throws IOException {
+      return getNextContainerToken(it);
+    }
+  }
+
+  private Entry<ContainerId, Long> getNextContainerToken(LeveldbIterator it)
       throws IOException {
-    RecoveredContainerTokensState state = new RecoveredContainerTokensState();
-    state.activeTokens = new HashMap<ContainerId, Long>();
-    LeveldbIterator iter = null;
+    Entry<ContainerId, Long> containerTokenEntry = null;
     try {
-      iter = new LeveldbIterator(db);
-      iter.seek(bytes(CONTAINER_TOKENS_KEY_PREFIX));
-      final int containerTokensKeyPrefixLength =
-          CONTAINER_TOKENS_KEY_PREFIX.length();
-      while (iter.hasNext()) {
-        Entry<byte[], byte[]> entry = iter.next();
+      while (it.hasNext()) {
+        Entry<byte[], byte[]> entry = it.next();
         String fullKey = asString(entry.getKey());
         if (!fullKey.startsWith(CONTAINER_TOKENS_KEY_PREFIX)) {
           break;
         }
-        String key = fullKey.substring(containerTokensKeyPrefixLength);
-        if (key.equals(CURRENT_MASTER_KEY_SUFFIX)) {
-          state.currentMasterKey = parseMasterKey(entry.getValue());
-        } else if (key.equals(PREV_MASTER_KEY_SUFFIX)) {
-          state.previousMasterKey = parseMasterKey(entry.getValue());
-        } else if (key.startsWith(ConverterUtils.CONTAINER_PREFIX)) {
-          loadContainerToken(state, fullKey, key, entry.getValue());
+        String key = fullKey.substring(CONTAINER_TOKENS_KEY_PREFIX.length());
+        if (key.startsWith(ConverterUtils.CONTAINER_PREFIX)) {
+          containerTokenEntry = loadContainerToken(fullKey, key,
+              entry.getValue());
+          break;
         }
       }
     } catch (DBException e) {
       throw new IOException(e);
-    } finally {
-      if (iter != null) {
-        iter.close();
-      }
     }
-    return state;
+    return containerTokenEntry;
   }
 
-  private static void loadContainerToken(RecoveredContainerTokensState state,
-      String key, String containerIdStr, byte[] value) throws IOException {
+  private static Entry<ContainerId, Long> loadContainerToken(String key,
+      String containerIdStr, byte[] value) throws IOException {
     ContainerId containerId;
     Long expTime;
     try {
@@ -1169,7 +1279,19 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     } catch (IllegalArgumentException e) {
       throw new IOException("Bad container token state for " + key, e);
     }
-    state.activeTokens.put(containerId, expTime);
+    return new AbstractMap.SimpleEntry<>(containerId, expTime);
+  }
+
+  @Override
+  public RecoveredContainerTokensState loadContainerTokensState()
+      throws IOException {
+    RecoveredContainerTokensState state = new RecoveredContainerTokensState();
+    state.currentMasterKey = getMasterKey(CONTAINER_TOKENS_KEY_PREFIX
+        + CURRENT_MASTER_KEY_SUFFIX);
+    state.previousMasterKey = getMasterKey(CONTAINER_TOKENS_KEY_PREFIX
+        + PREV_MASTER_KEY_SUFFIX);
+    state.it = new ContainerTokensStateIterator();
+    return state;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.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/recovery/NMNullStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
index dfad9cf..3ae00f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
@@ -65,7 +65,7 @@ public class NMNullStateStoreService extends NMStateStoreService {
   }
 
   @Override
-  public List<RecoveredContainerState> loadContainersState()
+  public RecoveryIterator<RecoveredContainerState> getContainerStateIterator()
       throws IOException {
     throw new UnsupportedOperationException(
         "Recovery not supported by this state store");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.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/recovery/NMStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
index 70decdb..35caec9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -67,12 +68,11 @@ public abstract class NMStateStoreService extends AbstractService {
   }
 
   public static class RecoveredApplicationsState {
-    List<ContainerManagerApplicationProto> applications;
+    RecoveryIterator<ContainerManagerApplicationProto> it = null;
 
-    public List<ContainerManagerApplicationProto> getApplications() {
-      return applications;
+    public RecoveryIterator<ContainerManagerApplicationProto> getIterator() {
+      return it;
     }
-
   }
 
   /**
@@ -106,6 +106,15 @@ public abstract class NMStateStoreService extends AbstractService {
         RecoveredContainerType.RECOVER;
     private long startTime;
     private ResourceMappings resMappings = new ResourceMappings();
+    private final ContainerId containerId;
+
+    RecoveredContainerState(ContainerId containerId){
+      this.containerId = containerId;
+    }
+
+    public ContainerId getContainerId() {
+      return containerId;
+    }
 
     public RecoveredContainerStatus getStatus() {
       return status;
@@ -248,30 +257,33 @@ public abstract class NMStateStoreService extends AbstractService {
   public static class RecoveredLocalizationState {
     LocalResourceTrackerState publicTrackerState =
         new LocalResourceTrackerState();
-    Map<String, RecoveredUserResources> userResources =
-        new HashMap<String, RecoveredUserResources>();
+    RecoveryIterator<Entry<String, RecoveredUserResources>> it = null;
 
     public LocalResourceTrackerState getPublicTrackerState() {
       return publicTrackerState;
     }
 
-    public Map<String, RecoveredUserResources> getUserResources() {
-      return userResources;
+    public RecoveryIterator<Entry<String, RecoveredUserResources>> getIterator() {
+      return it;
     }
   }
 
   public static class RecoveredDeletionServiceState {
-    List<DeletionServiceDeleteTaskProto> tasks;
+    RecoveryIterator<DeletionServiceDeleteTaskProto> it = null;
 
-    public List<DeletionServiceDeleteTaskProto> getTasks() {
-      return tasks;
+    public RecoveryIterator<DeletionServiceDeleteTaskProto> getIterator(){
+      return it;
     }
   }
 
   public static class RecoveredNMTokensState {
     MasterKey currentMasterKey;
     MasterKey previousMasterKey;
-    Map<ApplicationAttemptId, MasterKey> applicationMasterKeys;
+    RecoveryIterator<Entry<ApplicationAttemptId, MasterKey>> it = null;
+
+    public RecoveryIterator<Entry<ApplicationAttemptId, MasterKey>> getIterator() {
+      return it;
+    }
 
     public MasterKey getCurrentMasterKey() {
       return currentMasterKey;
@@ -281,15 +293,16 @@ public abstract class NMStateStoreService extends AbstractService {
       return previousMasterKey;
     }
 
-    public Map<ApplicationAttemptId, MasterKey> getApplicationMasterKeys() {
-      return applicationMasterKeys;
-    }
   }
 
   public static class RecoveredContainerTokensState {
     MasterKey currentMasterKey;
     MasterKey previousMasterKey;
-    Map<ContainerId, Long> activeTokens;
+    RecoveryIterator<Entry<ContainerId, Long>> it = null;
+
+    public RecoveryIterator<Entry<ContainerId, Long>> getIterator() {
+      return it;
+    }
 
     public MasterKey getCurrentMasterKey() {
       return currentMasterKey;
@@ -299,9 +312,6 @@ public abstract class NMStateStoreService extends AbstractService {
       return previousMasterKey;
     }
 
-    public Map<ContainerId, Long> getActiveTokens() {
-      return activeTokens;
-    }
   }
 
   public static class RecoveredLogDeleterState {
@@ -400,11 +410,10 @@ public abstract class NMStateStoreService extends AbstractService {
 
 
   /**
-   * Load the state of containers
-   * @return recovered state for containers
-   * @throws IOException
+   * get the Recovered Container State Iterator
+   * @return recovery iterator
    */
-  public abstract List<RecoveredContainerState> loadContainersState()
+  public abstract RecoveryIterator<RecoveredContainerState> getContainerStateIterator()
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/RecoveryIterator.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/recovery/RecoveryIterator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/RecoveryIterator.java
new file mode 100644
index 0000000..0bb262a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/RecoveryIterator.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.recovery;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+/**
+ * A wrapper for a Iterator to translate the raw RuntimeExceptions that
+ * can be thrown into IOException.
+ */
+public interface RecoveryIterator<T> extends Closeable {
+
+  /**
+   * Returns true if the iteration has more elements.
+   */
+  boolean hasNext() throws IOException;
+
+  /**
+   * Returns the next element in the iteration.
+   */
+  T next() throws IOException, NoSuchElementException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMContainerTokenSecretManager.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/security/NMContainerTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMContainerTokenSecretManager.java
index 256f649..b3df69b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMContainerTokenSecretManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMContainerTokenSecretManager.java
@@ -24,6 +24,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.TreeMap;
+
+import org.apache.hadoop.yarn.server.nodemanager.recovery.RecoveryIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -90,17 +92,20 @@ public class NMContainerTokenSecretManager extends
       super.serialNo = super.currentMasterKey.getMasterKey().getKeyId() + 1;
     }
 
-    for (Entry<ContainerId, Long> entry : state.getActiveTokens().entrySet()) {
-      ContainerId containerId = entry.getKey();
-      Long expTime = entry.getValue();
-      List<ContainerId> containerList =
-          recentlyStartedContainerTracker.get(expTime);
-      if (containerList == null) {
-        containerList = new ArrayList<ContainerId>();
-        recentlyStartedContainerTracker.put(expTime, containerList);
-      }
-      if (!containerList.contains(containerId)) {
-        containerList.add(containerId);
+    try (RecoveryIterator<Entry<ContainerId, Long>> it = state.getIterator()) {
+      while (it.hasNext()) {
+        Entry<ContainerId, Long> entry = it.next();
+        ContainerId containerId = entry.getKey();
+        Long expTime = entry.getValue();
+        List<ContainerId> containerList =
+            recentlyStartedContainerTracker.get(expTime);
+        if (containerList == null) {
+          containerList = new ArrayList<ContainerId>();
+          recentlyStartedContainerTracker.put(expTime, containerList);
+        }
+        if (!containerList.contains(containerId)) {
+          containerList.add(containerId);
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMTokenSecretManagerInNM.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/security/NMTokenSecretManagerInNM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMTokenSecretManagerInNM.java
index 0956e77..f895791 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMTokenSecretManagerInNM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMTokenSecretManagerInNM.java
@@ -23,6 +23,8 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+
+import org.apache.hadoop.yarn.server.nodemanager.recovery.RecoveryIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -87,11 +89,14 @@ public class NMTokenSecretManagerInNM extends BaseNMTokenSecretManager {
       super.serialNo = super.currentMasterKey.getMasterKey().getKeyId() + 1;
     }
 
-    for (Map.Entry<ApplicationAttemptId, MasterKey> entry :
-         state.getApplicationMasterKeys().entrySet()) {
-      key = entry.getValue();
-      oldMasterKeys.put(entry.getKey(),
-          new MasterKeyData(key, createSecretKey(key.getBytes().array())));
+    try (RecoveryIterator<Map.Entry<ApplicationAttemptId, MasterKey>> it =
+             state.getIterator()) {
+      while (it.hasNext()) {
+        Map.Entry<ApplicationAttemptId, MasterKey> entry = it.next();
+        key = entry.getValue();
+        oldMasterKeys.put(entry.getKey(),
+            new MasterKeyData(key, createSecretKey(key.getBytes().array())));
+      }
     }
 
     // reconstruct app to app attempts map

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.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/recovery/NMMemoryStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
index c5428d1..9658ecd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
@@ -23,6 +23,7 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -56,6 +57,8 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   private Map<Integer, DeletionServiceDeleteTaskProto> deleteTasks;
   private RecoveredNMTokensState nmTokenState;
   private RecoveredContainerTokensState containerTokenState;
+  private Map<ApplicationAttemptId, MasterKey> applicationMasterKeys;
+  private Map<ContainerId, Long> activeTokens;
   private Map<ApplicationId, LogDeleterProto> logDeleterState;
   private RecoveredAMRMProxyState amrmProxyState;
 
@@ -68,10 +71,9 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
     apps = new HashMap<ApplicationId, ContainerManagerApplicationProto>();
     containerStates = new HashMap<ContainerId, RecoveredContainerState>();
     nmTokenState = new RecoveredNMTokensState();
-    nmTokenState.applicationMasterKeys =
-        new HashMap<ApplicationAttemptId, MasterKey>();
+    applicationMasterKeys = new HashMap<ApplicationAttemptId, MasterKey>();
     containerTokenState = new RecoveredContainerTokensState();
-    containerTokenState.activeTokens = new HashMap<ContainerId, Long>();
+    activeTokens = new HashMap<ContainerId, Long>();
     trackerStates = new HashMap<TrackerKey, TrackerState>();
     deleteTasks = new HashMap<Integer, DeletionServiceDeleteTaskProto>();
     logDeleterState = new HashMap<ApplicationId, LogDeleterProto>();
@@ -86,13 +88,39 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   protected void closeStorage() {
   }
 
+  // Recovery Iterator Implementation.
+  private class NMMemoryRecoveryIterator<T> implements RecoveryIterator<T> {
+
+    private Iterator<T> it;
+
+    NMMemoryRecoveryIterator(Iterator<T> it){
+      this.it = it;
+    }
+
+    @Override
+    public boolean hasNext() {
+      return it.hasNext();
+    }
+
+    @Override
+    public T next() throws IOException {
+      return it.next();
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+  }
 
   @Override
   public synchronized RecoveredApplicationsState loadApplicationsState()
       throws IOException {
     RecoveredApplicationsState state = new RecoveredApplicationsState();
-    state.applications = new ArrayList<ContainerManagerApplicationProto>(
-        apps.values());
+    List<ContainerManagerApplicationProto> containerList =
+        new ArrayList<ContainerManagerApplicationProto>(apps.values());
+    state.it = new NMMemoryRecoveryIterator<ContainerManagerApplicationProto>(
+        containerList.iterator());
     return state;
   }
 
@@ -111,13 +139,13 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   }
 
   @Override
-  public synchronized List<RecoveredContainerState> loadContainersState()
+  public RecoveryIterator<RecoveredContainerState> getContainerStateIterator()
       throws IOException {
     // return a copy so caller can't modify our state
     List<RecoveredContainerState> result =
         new ArrayList<RecoveredContainerState>(containerStates.size());
     for (RecoveredContainerState rcs : containerStates.values()) {
-      RecoveredContainerState rcsCopy = new RecoveredContainerState();
+      RecoveredContainerState rcsCopy = new RecoveredContainerState(rcs.getContainerId());
       rcsCopy.status = rcs.status;
       rcsCopy.exitCode = rcs.exitCode;
       rcsCopy.killed = rcs.killed;
@@ -131,13 +159,14 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
       rcsCopy.setResourceMappings(rcs.getResourceMappings());
       result.add(rcsCopy);
     }
-    return result;
+    return new NMMemoryRecoveryIterator<RecoveredContainerState>(
+        result.iterator());
   }
 
   @Override
   public synchronized void storeContainer(ContainerId containerId,
       int version, long startTime, StartContainerRequest startRequest) {
-    RecoveredContainerState rcs = new RecoveredContainerState();
+    RecoveredContainerState rcs = new RecoveredContainerState(containerId);
     rcs.startRequest = startRequest;
     rcs.version = version;
     try {
@@ -284,6 +313,8 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   @Override
   public synchronized RecoveredLocalizationState loadLocalizationState() {
     RecoveredLocalizationState result = new RecoveredLocalizationState();
+    Map<String, RecoveredUserResources> userResources =
+        new HashMap<String, RecoveredUserResources>();
     for (Map.Entry<TrackerKey, TrackerState> e : trackerStates.entrySet()) {
       TrackerKey tk = e.getKey();
       TrackerState ts = e.getValue();
@@ -294,10 +325,10 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
       if (tk.user == null) {
         result.publicTrackerState = loadTrackerState(ts);
       } else {
-        RecoveredUserResources rur = result.userResources.get(tk.user);
+        RecoveredUserResources rur = userResources.get(tk.user);
         if (rur == null) {
           rur = new RecoveredUserResources();
-          result.userResources.put(tk.user, rur);
+          userResources.put(tk.user, rur);
         }
         if (tk.appId == null) {
           rur.privateTrackerState = loadTrackerState(ts);
@@ -306,6 +337,8 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
         }
       }
     }
+    result.it = new NMMemoryRecoveryIterator<Map.Entry<String, RecoveredUserResources>>(
+        userResources.entrySet().iterator());
     return result;
   }
 
@@ -341,8 +374,10 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
       throws IOException {
     RecoveredDeletionServiceState result =
         new RecoveredDeletionServiceState();
-    result.tasks = new ArrayList<DeletionServiceDeleteTaskProto>(
-        deleteTasks.values());
+    List<DeletionServiceDeleteTaskProto> deleteTaskProtos =
+        new ArrayList<DeletionServiceDeleteTaskProto>(deleteTasks.values());
+    result.it = new NMMemoryRecoveryIterator<DeletionServiceDeleteTaskProto>(
+        deleteTaskProtos.iterator());
     return result;
   }
 
@@ -365,9 +400,10 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
     RecoveredNMTokensState result = new RecoveredNMTokensState();
     result.currentMasterKey = nmTokenState.currentMasterKey;
     result.previousMasterKey = nmTokenState.previousMasterKey;
-    result.applicationMasterKeys =
-        new HashMap<ApplicationAttemptId, MasterKey>(
-            nmTokenState.applicationMasterKeys);
+    Map<ApplicationAttemptId, MasterKey> masterKeysMap =
+        new HashMap<ApplicationAttemptId, MasterKey>(applicationMasterKeys);
+    result.it = new NMMemoryRecoveryIterator<Map.Entry<ApplicationAttemptId, MasterKey>>(
+        masterKeysMap.entrySet().iterator());
     return result;
   }
 
@@ -389,14 +425,14 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   public synchronized void storeNMTokenApplicationMasterKey(
       ApplicationAttemptId attempt, MasterKey key) throws IOException {
     MasterKeyPBImpl keypb = (MasterKeyPBImpl) key;
-    nmTokenState.applicationMasterKeys.put(attempt,
+    applicationMasterKeys.put(attempt,
         new MasterKeyPBImpl(keypb.getProto()));
   }
 
   @Override
   public synchronized void removeNMTokenApplicationMasterKey(
       ApplicationAttemptId attempt) throws IOException {
-    nmTokenState.applicationMasterKeys.remove(attempt);
+    applicationMasterKeys.remove(attempt);
   }
 
 
@@ -408,8 +444,10 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
         new RecoveredContainerTokensState();
     result.currentMasterKey = containerTokenState.currentMasterKey;
     result.previousMasterKey = containerTokenState.previousMasterKey;
-    result.activeTokens =
-        new HashMap<ContainerId, Long>(containerTokenState.activeTokens);
+    Map<ContainerId, Long> containersTokenMap =
+        new HashMap<ContainerId, Long>(activeTokens);
+    result.it = new NMMemoryRecoveryIterator<Map.Entry<ContainerId, Long>>(
+        containersTokenMap.entrySet().iterator());
     return result;
   }
 
@@ -432,13 +470,13 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   @Override
   public synchronized void storeContainerToken(ContainerId containerId,
       Long expirationTime) throws IOException {
-    containerTokenState.activeTokens.put(containerId, expirationTime);
+    activeTokens.put(containerId, expirationTime);
   }
 
   @Override
   public synchronized void removeContainerToken(ContainerId containerId)
       throws IOException {
-    containerTokenState.activeTokens.remove(containerId);
+    activeTokens.remove(containerId);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.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/recovery/TestNMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
index 8a8cfa2..fcbbc52 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
@@ -125,6 +125,73 @@ public class TestNMLeveldbStateStoreService {
     FileUtil.fullyDelete(TMP_DIR);
   }
 
+  private List<RecoveredContainerState> loadContainersState(
+      RecoveryIterator<RecoveredContainerState> it) throws IOException {
+    List<RecoveredContainerState> containers =
+        new ArrayList<RecoveredContainerState>();
+    while (it.hasNext()) {
+      RecoveredContainerState rcs = it.next();
+      containers.add(rcs);
+    }
+    return containers;
+  }
+
+  private List<ContainerManagerApplicationProto> loadApplicationProtos(
+      RecoveryIterator<ContainerManagerApplicationProto> it)
+      throws IOException {
+    List<ContainerManagerApplicationProto> applicationProtos =
+        new ArrayList<ContainerManagerApplicationProto>();
+    while (it.hasNext()) {
+      applicationProtos.add(it.next());
+    }
+    return applicationProtos;
+  }
+
+  private List<DeletionServiceDeleteTaskProto> loadDeletionTaskProtos(
+      RecoveryIterator<DeletionServiceDeleteTaskProto> it) throws IOException {
+    List<DeletionServiceDeleteTaskProto> deleteTaskProtos =
+        new ArrayList<DeletionServiceDeleteTaskProto>();
+    while (it.hasNext()) {
+      deleteTaskProtos.add(it.next());
+    }
+    return deleteTaskProtos;
+  }
+
+  private Map<String, RecoveredUserResources> loadUserResources(
+      RecoveryIterator<Map.Entry<String, RecoveredUserResources>> it)
+      throws IOException {
+    Map<String, RecoveredUserResources> userResources =
+        new HashMap<String, RecoveredUserResources>();
+    while (it.hasNext()) {
+      Map.Entry<String, RecoveredUserResources> entry = it.next();
+      userResources.put(entry.getKey(), entry.getValue());
+    }
+    return userResources;
+  }
+
+  private Map<ApplicationAttemptId, MasterKey> loadNMTokens(
+      RecoveryIterator<Map.Entry<ApplicationAttemptId, MasterKey>> it)
+      throws IOException {
+    Map<ApplicationAttemptId, MasterKey> nmTokens =
+        new HashMap<ApplicationAttemptId, MasterKey>();
+    while (it.hasNext()) {
+      Map.Entry<ApplicationAttemptId, MasterKey> entry = it.next();
+      nmTokens.put(entry.getKey(), entry.getValue());
+    }
+    return nmTokens;
+  }
+
+  private Map<ContainerId, Long> loadContainerTokens(
+      RecoveryIterator<Map.Entry<ContainerId, Long>> it) throws IOException {
+    Map<ContainerId, Long> containerTokens =
+        new HashMap<ContainerId, Long>();
+    while (it.hasNext()) {
+      Map.Entry<ContainerId, Long> entry = it.next();
+      containerTokens.put(entry.getKey(), entry.getValue());
+    }
+    return containerTokens;
+  }
+
   private void restartStateStore() throws IOException {
     // need to close so leveldb releases database lock
     if (stateStore != null) {
@@ -142,7 +209,7 @@ public class TestNMLeveldbStateStoreService {
     assertNotNull(pubts);
     assertTrue(pubts.getLocalizedResources().isEmpty());
     assertTrue(pubts.getInProgressResources().isEmpty());
-    assertTrue(state.getUserResources().isEmpty());
+    assertTrue(loadUserResources(state.getIterator()).isEmpty());
   }
 
   @Test
@@ -183,7 +250,7 @@ public class TestNMLeveldbStateStoreService {
       restartStateStore();
       Assert.fail("Incompatible version, should expect fail here.");
     } catch (ServiceStateException e) {
-      Assert.assertTrue("Exception message mismatch", 
+      Assert.assertTrue("Exception message mismatch",
         e.getMessage().contains("Incompatible version for NM state:"));
     }
   }
@@ -192,7 +259,9 @@ public class TestNMLeveldbStateStoreService {
   public void testApplicationStorage() throws IOException {
     // test empty when no state
     RecoveredApplicationsState state = stateStore.loadApplicationsState();
-    assertTrue(state.getApplications().isEmpty());
+    List<ContainerManagerApplicationProto> apps =
+        loadApplicationProtos(state.getIterator());
+    assertTrue(apps.isEmpty());
 
     // store an application and verify recovered
     final ApplicationId appId1 = ApplicationId.newInstance(1234, 1);
@@ -204,8 +273,9 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeApplication(appId1, appProto1);
     restartStateStore();
     state = stateStore.loadApplicationsState();
-    assertEquals(1, state.getApplications().size());
-    assertEquals(appProto1, state.getApplications().get(0));
+    apps = loadApplicationProtos(state.getIterator());
+    assertEquals(1, apps.size());
+    assertEquals(appProto1, apps.get(0));
 
     // add a new app
     final ApplicationId appId2 = ApplicationId.newInstance(1234, 2);
@@ -216,23 +286,25 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeApplication(appId2, appProto2);
     restartStateStore();
     state = stateStore.loadApplicationsState();
-    assertEquals(2, state.getApplications().size());
-    assertTrue(state.getApplications().contains(appProto1));
-    assertTrue(state.getApplications().contains(appProto2));
+    apps = loadApplicationProtos(state.getIterator());
+    assertEquals(2, apps.size());
+    assertTrue(apps.contains(appProto1));
+    assertTrue(apps.contains(appProto2));
 
     // test removing an application
     stateStore.removeApplication(appId2);
     restartStateStore();
     state = stateStore.loadApplicationsState();
-    assertEquals(1, state.getApplications().size());
-    assertEquals(appProto1, state.getApplications().get(0));
+    apps = loadApplicationProtos(state.getIterator());
+    assertEquals(1, apps.size());
+    assertEquals(appProto1, apps.get(0));
   }
 
   @Test
   public void testContainerStorage() throws IOException {
     // test empty when no state
     List<RecoveredContainerState> recoveredContainers =
-        stateStore.loadContainersState();
+        loadContainersState(stateStore.getContainerStateIterator());
     assertTrue(recoveredContainers.isEmpty());
 
     // create a container request
@@ -254,7 +326,8 @@ public class TestNMLeveldbStateStoreService {
         stateStore.getContainerVersionKey(containerId.toString()))));
 
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     RecoveredContainerState rcs = recoveredContainers.get(0);
     assertEquals(0, rcs.getVersion());
@@ -269,14 +342,16 @@ public class TestNMLeveldbStateStoreService {
     // store a new container record without StartContainerRequest
     ContainerId containerId1 = ContainerId.newContainerId(appAttemptId, 6);
     stateStore.storeContainerLaunched(containerId1);
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     // check whether the new container record is discarded
     assertEquals(1, recoveredContainers.size());
 
     // queue the container, and verify recovered
     stateStore.storeContainerQueued(containerId);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(RecoveredContainerStatus.QUEUED, rcs.getStatus());
@@ -292,7 +367,8 @@ public class TestNMLeveldbStateStoreService {
     diags.append("some diags for container");
     stateStore.storeContainerDiagnostics(containerId, diags);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(RecoveredContainerStatus.LAUNCHED, rcs.getStatus());
@@ -305,7 +381,8 @@ public class TestNMLeveldbStateStoreService {
     // pause the container, and verify recovered
     stateStore.storeContainerPaused(containerId);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(RecoveredContainerStatus.PAUSED, rcs.getStatus());
@@ -316,7 +393,8 @@ public class TestNMLeveldbStateStoreService {
     // Resume the container
     stateStore.removeContainerPaused(containerId);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
 
     // increase the container size, and verify recovered
@@ -328,7 +406,8 @@ public class TestNMLeveldbStateStoreService {
     stateStore
         .storeContainerUpdateToken(containerId, updateTokenIdentifier);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(0, rcs.getVersion());
@@ -342,7 +421,8 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerDiagnostics(containerId, diags);
     stateStore.storeContainerKilled(containerId);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(RecoveredContainerStatus.LAUNCHED, rcs.getStatus());
@@ -358,7 +438,8 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerDiagnostics(containerId, diags);
     stateStore.storeContainerCompleted(containerId, 21);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(RecoveredContainerStatus.COMPLETED, rcs.getStatus());
@@ -371,7 +452,8 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerWorkDir(containerId, "/test/workdir");
     stateStore.storeContainerLogDir(containerId, "/test/logdir");
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(6, rcs.getRemainingRetryAttempts());
@@ -382,12 +464,13 @@ public class TestNMLeveldbStateStoreService {
     // remove the container and verify not recovered
     stateStore.removeContainer(containerId);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertTrue(recoveredContainers.isEmpty());
     // recover again to check remove clears all containers
     restartStateStore();
     NMStateStoreService nmStoreSpy = spy(stateStore);
-    nmStoreSpy.loadContainersState();
+    loadContainersState(nmStoreSpy.getContainerStateIterator());
     verify(nmStoreSpy,times(0)).removeContainer(any(ContainerId.class));
   }
 
@@ -399,7 +482,8 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerRestartTimes(containerId,
         finishTimeForRetryAttempts);
     restartStateStore();
-    RecoveredContainerState rcs = stateStore.loadContainersState().get(0);
+    RecoveredContainerState rcs =
+        loadContainersState(stateStore.getContainerStateIterator()).get(0);
     List<Long> recoveredRestartTimes = rcs.getRestartTimes();
     assertEquals(1462700529039L, (long)recoveredRestartTimes.get(0));
     assertEquals(1462700529050L, (long)recoveredRestartTimes.get(1));
@@ -481,7 +565,7 @@ public class TestNMLeveldbStateStoreService {
     assertTrue(pubts.getLocalizedResources().isEmpty());
     assertTrue(pubts.getInProgressResources().isEmpty());
     Map<String, RecoveredUserResources> userResources =
-        state.getUserResources();
+        loadUserResources(state.getIterator());
     assertEquals(1, userResources.size());
     RecoveredUserResources rur = userResources.get(user);
     LocalResourceTrackerState privts = rur.getPrivateTrackerState();
@@ -535,7 +619,7 @@ public class TestNMLeveldbStateStoreService {
         pubts.getInProgressResources().get(pubRsrcProto1));
     assertEquals(pubRsrcLocalPath2,
         pubts.getInProgressResources().get(pubRsrcProto2));
-    userResources = state.getUserResources();
+    userResources = loadUserResources(state.getIterator());
     assertEquals(1, userResources.size());
     rur = userResources.get(user);
     privts = rur.getPrivateTrackerState();
@@ -584,7 +668,7 @@ public class TestNMLeveldbStateStoreService {
     assertTrue(pubts.getLocalizedResources().isEmpty());
     assertTrue(pubts.getInProgressResources().isEmpty());
     Map<String, RecoveredUserResources> userResources =
-        state.getUserResources();
+        loadUserResources(state.getIterator());
     assertEquals(1, userResources.size());
     RecoveredUserResources rur = userResources.get(user);
     LocalResourceTrackerState privts = rur.getPrivateTrackerState();
@@ -654,7 +738,7 @@ public class TestNMLeveldbStateStoreService {
     assertEquals(1, pubts.getInProgressResources().size());
     assertEquals(pubRsrcLocalPath2,
         pubts.getInProgressResources().get(pubRsrcProto2));
-    userResources = state.getUserResources();
+    userResources = loadUserResources(state.getIterator());
     assertEquals(1, userResources.size());
     rur = userResources.get(user);
     privts = rur.getPrivateTrackerState();
@@ -762,7 +846,7 @@ public class TestNMLeveldbStateStoreService {
     assertEquals(pubLocalizedProto1,
         pubts.getLocalizedResources().iterator().next());
     Map<String, RecoveredUserResources> userResources =
-        state.getUserResources();
+        loadUserResources(state.getIterator());
     assertTrue(userResources.isEmpty());
   }
 
@@ -771,7 +855,9 @@ public class TestNMLeveldbStateStoreService {
     // test empty when no state
     RecoveredDeletionServiceState state =
         stateStore.loadDeletionServiceState();
-    assertTrue(state.getTasks().isEmpty());
+    List<DeletionServiceDeleteTaskProto> deleteTaskProtos =
+        loadDeletionTaskProtos(state.getIterator());
+    assertTrue(deleteTaskProtos.isEmpty());
 
     // store a deletion task and verify recovered
     DeletionServiceDeleteTaskProto proto =
@@ -788,8 +874,9 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeDeletionTask(proto.getId(), proto);
     restartStateStore();
     state = stateStore.loadDeletionServiceState();
-    assertEquals(1, state.getTasks().size());
-    assertEquals(proto, state.getTasks().get(0));
+    deleteTaskProtos = loadDeletionTaskProtos(state.getIterator());
+    assertEquals(1, deleteTaskProtos.size());
+    assertEquals(proto, deleteTaskProtos.get(0));
 
     // store another deletion task
     DeletionServiceDeleteTaskProto proto2 =
@@ -802,31 +889,36 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeDeletionTask(proto2.getId(), proto2);
     restartStateStore();
     state = stateStore.loadDeletionServiceState();
-    assertEquals(2, state.getTasks().size());
-    assertTrue(state.getTasks().contains(proto));
-    assertTrue(state.getTasks().contains(proto2));
+    deleteTaskProtos = loadDeletionTaskProtos(state.getIterator());
+    assertEquals(2, deleteTaskProtos.size());
+    assertTrue(deleteTaskProtos.contains(proto));
+    assertTrue(deleteTaskProtos.contains(proto2));
+
 
     // delete a task and verify gone after recovery
     stateStore.removeDeletionTask(proto2.getId());
     restartStateStore();
-    state = stateStore.loadDeletionServiceState();
-    assertEquals(1, state.getTasks().size());
-    assertEquals(proto, state.getTasks().get(0));
+    state =  stateStore.loadDeletionServiceState();
+    deleteTaskProtos = loadDeletionTaskProtos(state.getIterator());
+    assertEquals(1, deleteTaskProtos.size());
+    assertEquals(proto, deleteTaskProtos.get(0));
 
     // delete the last task and verify none left
     stateStore.removeDeletionTask(proto.getId());
     restartStateStore();
     state = stateStore.loadDeletionServiceState();
-    assertTrue(state.getTasks().isEmpty());
-  }
+    deleteTaskProtos = loadDeletionTaskProtos(state.getIterator());
+    assertTrue(deleteTaskProtos.isEmpty());  }
 
   @Test
   public void testNMTokenStorage() throws IOException {
     // test empty when no state
     RecoveredNMTokensState state = stateStore.loadNMTokensState();
+    Map<ApplicationAttemptId, MasterKey> loadedAppKeys =
+        loadNMTokens(state.getIterator());
     assertNull(state.getCurrentMasterKey());
     assertNull(state.getPreviousMasterKey());
-    assertTrue(state.getApplicationMasterKeys().isEmpty());
+    assertTrue(loadedAppKeys.isEmpty());
 
     // store a master key and verify recovered
     NMTokenSecretManagerForTest secretMgr = new NMTokenSecretManagerForTest();
@@ -834,18 +926,20 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeNMTokenCurrentMasterKey(currentKey);
     restartStateStore();
     state = stateStore.loadNMTokensState();
+    loadedAppKeys = loadNMTokens(state.getIterator());
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertNull(state.getPreviousMasterKey());
-    assertTrue(state.getApplicationMasterKeys().isEmpty());
+    assertTrue(loadedAppKeys.isEmpty());
 
     // store a previous key and verify recovered
     MasterKey prevKey = secretMgr.generateKey();
     stateStore.storeNMTokenPreviousMasterKey(prevKey);
     restartStateStore();
     state = stateStore.loadNMTokensState();
+    loadedAppKeys = loadNMTokens(state.getIterator());
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertEquals(prevKey, state.getPreviousMasterKey());
-    assertTrue(state.getApplicationMasterKeys().isEmpty());
+    assertTrue(loadedAppKeys.isEmpty());
 
     // store a few application keys and verify recovered
     ApplicationAttemptId attempt1 = ApplicationAttemptId.newInstance(
@@ -858,10 +952,9 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeNMTokenApplicationMasterKey(attempt2, attemptKey2);
     restartStateStore();
     state = stateStore.loadNMTokensState();
+    loadedAppKeys = loadNMTokens(state.getIterator());
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertEquals(prevKey, state.getPreviousMasterKey());
-    Map<ApplicationAttemptId, MasterKey> loadedAppKeys =
-        state.getApplicationMasterKeys();
     assertEquals(2, loadedAppKeys.size());
     assertEquals(attemptKey1, loadedAppKeys.get(attempt1));
     assertEquals(attemptKey2, loadedAppKeys.get(attempt2));
@@ -880,9 +973,9 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeNMTokenCurrentMasterKey(currentKey);
     restartStateStore();
     state = stateStore.loadNMTokensState();
+    loadedAppKeys = loadNMTokens(state.getIterator());
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertEquals(prevKey, state.getPreviousMasterKey());
-    loadedAppKeys = state.getApplicationMasterKeys();
     assertEquals(2, loadedAppKeys.size());
     assertNull(loadedAppKeys.get(attempt1));
     assertEquals(attemptKey2, loadedAppKeys.get(attempt2));
@@ -894,9 +987,10 @@ public class TestNMLeveldbStateStoreService {
     // test empty when no state
     RecoveredContainerTokensState state =
         stateStore.loadContainerTokensState();
+    Map<ContainerId, Long> loadedActiveTokens = loadContainerTokens(state.it);
     assertNull(state.getCurrentMasterKey());
     assertNull(state.getPreviousMasterKey());
-    assertTrue(state.getActiveTokens().isEmpty());
+    assertTrue(loadedActiveTokens.isEmpty());
 
     // store a master key and verify recovered
     ContainerTokenKeyGeneratorForTest keygen =
@@ -905,18 +999,20 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerTokenCurrentMasterKey(currentKey);
     restartStateStore();
     state = stateStore.loadContainerTokensState();
+    loadedActiveTokens = loadContainerTokens(state.it);
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertNull(state.getPreviousMasterKey());
-    assertTrue(state.getActiveTokens().isEmpty());
+    assertTrue(loadedActiveTokens.isEmpty());
 
     // store a previous key and verify recovered
     MasterKey prevKey = keygen.generateKey();
     stateStore.storeContainerTokenPreviousMasterKey(prevKey);
     restartStateStore();
     state = stateStore.loadContainerTokensState();
+    loadedActiveTokens = loadContainerTokens(state.it);
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertEquals(prevKey, state.getPreviousMasterKey());
-    assertTrue(state.getActiveTokens().isEmpty());
+    assertTrue(loadedActiveTokens.isEmpty());
 
     // store a few container tokens and verify recovered
     ContainerId cid1 = BuilderUtils.newContainerId(1, 1, 1, 1);
@@ -927,10 +1023,9 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerToken(cid2, expTime2);
     restartStateStore();
     state = stateStore.loadContainerTokensState();
+    loadedActiveTokens = loadContainerTokens(state.it);
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertEquals(prevKey, state.getPreviousMasterKey());
-    Map<ContainerId, Long> loadedActiveTokens =
-        state.getActiveTokens();
     assertEquals(2, loadedActiveTokens.size());
     assertEquals(expTime1, loadedActiveTokens.get(cid1));
     assertEquals(expTime2, loadedActiveTokens.get(cid2));
@@ -948,9 +1043,9 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerTokenCurrentMasterKey(currentKey);
     restartStateStore();
     state = stateStore.loadContainerTokensState();
+    loadedActiveTokens = loadContainerTokens(state.it);
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertEquals(prevKey, state.getPreviousMasterKey());
-    loadedActiveTokens = state.getActiveTokens();
     assertEquals(2, loadedActiveTokens.size());
     assertNull(loadedActiveTokens.get(cid1));
     assertEquals(expTime2, loadedActiveTokens.get(cid2));
@@ -1029,8 +1124,8 @@ public class TestNMLeveldbStateStoreService {
   @Test
   public void testUnexpectedKeyDoesntThrowException() throws IOException {
     // test empty when no state
-    List<RecoveredContainerState> recoveredContainers = stateStore
-        .loadContainersState();
+    List<RecoveredContainerState> recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertTrue(recoveredContainers.isEmpty());
 
     ApplicationId appId = ApplicationId.newInstance(1234, 3);
@@ -1045,7 +1140,8 @@ public class TestNMLeveldbStateStoreService {
     + containerId.toString() + "/invalidKey1234").getBytes();
     stateStore.getDB().put(invalidKey, new byte[1]);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     RecoveredContainerState rcs = recoveredContainers.get(0);
     assertEquals(RecoveredContainerStatus.REQUESTED, rcs.getStatus());
@@ -1162,8 +1258,8 @@ public class TestNMLeveldbStateStoreService {
   @Test
   public void testStateStoreForResourceMapping() throws IOException {
     // test empty when no state
-    List<RecoveredContainerState> recoveredContainers = stateStore
-        .loadContainersState();
+    List<RecoveredContainerState> recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertTrue(recoveredContainers.isEmpty());
 
     ApplicationId appId = ApplicationId.newInstance(1234, 3);
@@ -1190,7 +1286,8 @@ public class TestNMLeveldbStateStoreService {
 
     // add a invalid key
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     RecoveredContainerState rcs = recoveredContainers.get(0);
     List<Serializable> res = rcs.getResourceMappings()
@@ -1253,7 +1350,8 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerRestartTimes(containerId,
         restartTimes);
     restartStateStore();
-    RecoveredContainerState rcs = stateStore.loadContainersState().get(0);
+    RecoveredContainerState rcs =
+        loadContainersState(stateStore.getContainerStateIterator()).get(0);
     List<Long> recoveredRestartTimes = rcs.getRestartTimes();
     assertTrue(recoveredRestartTimes.isEmpty());
   }


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

Branch: refs/heads/YARN-3409
Commit: 2c0a6c8c2b962217c259b027f32b2e1365d597e5
Parents: d4440fd
Author: Sunil G <su...@apache.org>
Authored: Sat Mar 31 19:53:06 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:57 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/2c0a6c8c/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/2c0a6c8c/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/2c0a6c8c/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


[42/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
index b6c0cc1..7c822b4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
@@ -292,8 +292,8 @@ public class MockNodes {
     }
 
     @Override
-    public Map<String, Set<NodeAttribute>> getAllNodeAttributes() {
-      return null;
+    public Set<NodeAttribute> getAllNodeAttributes() {
+      return Collections.emptySet();
     }
 
     @Override

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


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

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


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

Branch: refs/heads/YARN-3409
Commit: 0f0b55f7f3ce72538de8adb29dc095d82ac18add
Parents: 339fd5f
Author: Naganarasimha <na...@apache.org>
Authored: Sun Jul 29 13:52:52 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:57 2018 +0530

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f0b55f7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestNodeLabelUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestNodeLabelUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestNodeLabelUtil.java
new file mode 100644
index 0000000..d43da4f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestNodeLabelUtil.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.nodelabels;
+
+import static org.junit.Assert.fail;
+import org.junit.Test;
+
+public class TestNodeLabelUtil {
+
+  @Test
+  public void testAttributeValueAddition() {
+    String[] values =
+        new String[] {"1_8", "1.8", "ABZ", "ABZ", "az", "a-z","a_z", "123456789"};
+    for (String val : values) {
+      try {
+        NodeLabelUtil.checkAndThrowAttributeValue(val);
+      } catch (Exception e) {
+        fail("Valid values for NodeAttributeValue :" + val);
+      }
+    }
+
+    String[] invalidVals = new String[] {"_18","1,8","1/5",".15","1\\5"};
+    for (String val : invalidVals) {
+      try {
+        NodeLabelUtil.checkAndThrowAttributeValue(val);
+        fail("Valid values for NodeAttributeValue :" + val);
+      } catch (Exception e) {
+        // IGNORE
+      }
+    }
+  }
+}


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


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

Branch: refs/heads/YARN-3409
Commit: e20f6b6e80eea4f60257f0ead38b34c12b6db6f5
Parents: 7c30fa0
Author: Sunil G <su...@apache.org>
Authored: Fri Feb 23 08:01:58 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:56 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/e20f6b6e/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/e20f6b6e/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/e20f6b6e/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/e20f6b6e/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/e20f6b6e/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/e20f6b6e/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/e20f6b6e/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/e20f6b6e/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/e20f6b6e/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/e20f6b6e/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/e20f6b6e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index 33096bd..7be9ef7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -34,11 +34,6 @@ import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
 
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePlugin;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePluginManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataInputByteBuffer;
@@ -64,6 +59,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.impl.pb.RecordFactoryPBImpl;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.nodelabels.NodeLabelUtil;
 import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
 import org.apache.hadoop.yarn.server.api.ServerRMProxy;
@@ -76,23 +72,27 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResp
 import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
 import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
-import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
+import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePlugin;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePluginManager;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider;
 import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
 import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
-import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -1012,7 +1012,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       StringBuilder errorMsg = new StringBuilder("");
       while (iterator.hasNext()) {
         try {
-          CommonNodeLabelsManager
+          NodeLabelUtil
               .checkAndThrowLabelName(iterator.next().getName());
         } catch (IOException e) {
           errorMsg.append(e.getMessage());

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

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

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

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

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


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

Branch: refs/heads/YARN-3409
Commit: 9a3bf23ce19e54b025e449e92de96e5b34bb832e
Parents: 009cec0
Author: Weiwei Yang <ww...@apache.org>
Authored: Wed Jan 31 20:28:41 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:56 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/9a3bf23c/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/9a3bf23c/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/9a3bf23c/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/9a3bf23c/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


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

Branch: refs/heads/YARN-3409
Commit: 2b3d54bcd0c59319f2a4b747faf11a3dd9018ef0
Parents: c190002
Author: bibinchundatt <bi...@apache.org>
Authored: Sun May 13 09:35:11 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:57 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/2b3d54bc/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/2b3d54bc/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/2b3d54bc/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/2b3d54bc/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/2b3d54bc/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/2b3d54bc/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/2b3d54bc/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/2b3d54bc/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/2b3d54bc/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/2b3d54bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index 439780b..248f775 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -264,11 +264,11 @@ message GetClusterNodeAttributesRequestProto {
 }
 
 message GetClusterNodeAttributesResponseProto {
-  repeated NodeAttributeProto nodeAttributes = 1;
+  repeated NodeAttributeInfoProto nodeAttributes = 1;
 }
 
 message GetAttributesToNodesRequestProto {
-  repeated NodeAttributeProto nodeAttributes = 1;
+  repeated NodeAttributeKeyProto nodeAttributes = 1;
 }
 
 message GetAttributesToNodesResponseProto {

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


[09/50] [abbrv] hadoop git commit: YARN-7494. Add muti-node lookup mechanism and pluggable nodes sorting policies to optimize placement decision. Contributed by Sunil Govindan.

Posted by su...@apache.org.
YARN-7494. Add muti-node lookup mechanism and pluggable nodes sorting policies to optimize placement decision. Contributed by Sunil Govindan.


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

Branch: refs/heads/YARN-3409
Commit: 9c3fc3ef2865164aa5f121793ac914cfeb21a181
Parents: 54d0bf8
Author: Weiwei Yang <ww...@apache.org>
Authored: Tue Aug 21 22:42:23 2018 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Tue Aug 21 22:42:28 2018 +0800

----------------------------------------------------------------------
 .../resourcemanager/RMActiveServiceContext.java |  16 ++
 .../yarn/server/resourcemanager/RMContext.java  |   8 +-
 .../server/resourcemanager/RMContextImpl.java   |  14 +-
 .../server/resourcemanager/ResourceManager.java |  12 ++
 .../scheduler/AppSchedulingInfo.java            |  11 +-
 .../scheduler/ClusterNodeTracker.java           |  61 +++++++
 .../scheduler/activities/ActivitiesLogger.java  |  32 ++--
 .../scheduler/activities/ActivitiesManager.java |   8 +-
 .../scheduler/capacity/AbstractCSQueue.java     |  16 +-
 .../scheduler/capacity/CSQueue.java             |   6 +
 .../scheduler/capacity/CapacityScheduler.java   |  77 ++++++++-
 .../CapacitySchedulerConfiguration.java         | 116 +++++++++++++
 .../scheduler/capacity/LeafQueue.java           |  49 +++---
 .../scheduler/capacity/ParentQueue.java         |   4 +-
 .../allocator/RegularContainerAllocator.java    |  35 ++--
 .../common/ApplicationSchedulingConfig.java     |   4 +
 .../scheduler/common/fica/FiCaSchedulerApp.java |  23 +++
 .../LocalityAppPlacementAllocator.java          |  34 +++-
 .../placement/MultiNodeLookupPolicy.java        |  67 ++++++++
 .../placement/MultiNodePolicySpec.java          |  56 +++++++
 .../scheduler/placement/MultiNodeSorter.java    | 167 +++++++++++++++++++
 .../placement/MultiNodeSortingManager.java      | 139 +++++++++++++++
 .../ResourceUsageMultiNodeLookupPolicy.java     |  79 +++++++++
 .../reservation/ReservationSystemTestUtil.java  |   3 +
 .../scheduler/TestAppSchedulingInfo.java        |   3 +-
 .../capacity/CapacitySchedulerTestBase.java     |  13 ++
 .../capacity/TestCapacityScheduler.java         |  15 --
 .../TestCapacitySchedulerMultiNodes.java        | 166 ++++++++++++++++++
 .../TestCapacitySchedulerNodeLabelUpdate.java   |  70 ++++++++
 29 files changed, 1211 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/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..8fb0de6 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
@@ -43,9 +43,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.monitor.RMAppLifetime
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.PlacementConstraintManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.QueueLimitCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
@@ -113,6 +115,7 @@ public class RMActiveServiceContext {
   private AllocationTagsManager allocationTagsManager;
   private PlacementConstraintManager placementConstraintManager;
   private ResourceProfilesManager resourceProfilesManager;
+  private MultiNodeSortingManager<SchedulerNode> multiNodeSortingManager;
 
   public RMActiveServiceContext() {
     queuePlacementManager = new PlacementManager();
@@ -443,6 +446,19 @@ public class RMActiveServiceContext {
 
   @Private
   @Unstable
+  public MultiNodeSortingManager<SchedulerNode> getMultiNodeSortingManager() {
+    return multiNodeSortingManager;
+  }
+
+  @Private
+  @Unstable
+  public void setMultiNodeSortingManager(
+      MultiNodeSortingManager<SchedulerNode> multiNodeSortingManager) {
+    this.multiNodeSortingManager = multiNodeSortingManager;
+  }
+
+  @Private
+  @Unstable
   public void setSchedulerRecoveryStartAndWaitTime(long waitTime) {
     this.schedulerRecoveryStartTime = systemClock.getTime();
     this.schedulerRecoveryWaitTime = waitTime;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/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..a30ff76 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
@@ -42,10 +42,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.monitor.RMAppLifetime
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.PlacementConstraintManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.QueueLimitCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
@@ -177,4 +178,9 @@ public interface RMContext extends ApplicationMasterServiceContext {
 
   void setPlacementConstraintManager(
       PlacementConstraintManager placementConstraintManager);
+
+  MultiNodeSortingManager<SchedulerNode> getMultiNodeSortingManager();
+
+  void setMultiNodeSortingManager(
+      MultiNodeSortingManager<SchedulerNode> multiNodeSortingManager);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/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..cb1d56f 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
@@ -48,10 +48,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.monitor.RMAppLifetime
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.PlacementConstraintManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.QueueLimitCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
@@ -538,6 +539,17 @@ public class RMContextImpl implements RMContext {
         delegatedNodeLabelsUpdater);
   }
 
+  @Override
+  public MultiNodeSortingManager<SchedulerNode> getMultiNodeSortingManager() {
+    return activeServiceContext.getMultiNodeSortingManager();
+  }
+
+  @Override
+  public void setMultiNodeSortingManager(
+      MultiNodeSortingManager<SchedulerNode> multiNodeSortingManager) {
+    activeServiceContext.setMultiNodeSortingManager(multiNodeSortingManager);
+  }
+
   public void setSchedulerRecoveryStartAndWaitTime(long waitTime) {
     activeServiceContext.setSchedulerRecoveryStartAndWaitTime(waitTime);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/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 d459f0e..bdda871 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
@@ -96,11 +96,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.MemoryPlacementConstraintManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.PlacementConstraintManagerService;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
 import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
@@ -546,6 +548,10 @@ public class ResourceManager extends CompositeService
     return new FederationStateStoreService(rmContext);
   }
 
+  protected MultiNodeSortingManager<SchedulerNode> createMultiNodeSortingManager() {
+    return new MultiNodeSortingManager<SchedulerNode>();
+  }
+
   protected SystemMetricsPublisher createSystemMetricsPublisher() {
     List<SystemMetricsPublisher> publishers =
         new ArrayList<SystemMetricsPublisher>();
@@ -665,6 +671,12 @@ public class ResourceManager extends CompositeService
       resourceProfilesManager.init(conf);
       rmContext.setResourceProfilesManager(resourceProfilesManager);
 
+      MultiNodeSortingManager<SchedulerNode> multiNodeSortingManager =
+          createMultiNodeSortingManager();
+      multiNodeSortingManager.setRMContext(rmContext);
+      addService(multiNodeSortingManager);
+      rmContext.setMultiNodeSortingManager(multiNodeSortingManager);
+
       RMDelegatedNodeLabelsUpdater delegatedNodeLabelsUpdater =
           createRMDelegatedNodeLabelsUpdater();
       if (delegatedNodeLabelsUpdater != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.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/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index d63d2b82..ca7d9ce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -93,7 +93,7 @@ public class AppSchedulingInfo {
   private final ReentrantReadWriteLock.WriteLock writeLock;
 
   public final ContainerUpdateContext updateContext;
-  public final Map<String, String> applicationSchedulingEnvs = new HashMap<>();
+  private final Map<String, String> applicationSchedulingEnvs = new HashMap<>();
   private final RMContext rmContext;
 
   public AppSchedulingInfo(ApplicationAttemptId appAttemptId, String user,
@@ -782,4 +782,13 @@ public class AppSchedulingInfo {
       this.readLock.unlock();
     }
   }
+
+  /**
+   * Get scheduling envs configured for this application.
+   *
+   * @return a map of applicationSchedulingEnvs
+   */
+  public Map<String, String> getApplicationSchedulingEnvs() {
+    return applicationSchedulingEnvs;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.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/ClusterNodeTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.java
index 66d8810..8c7e447 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.java
@@ -37,6 +37,7 @@ import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -57,6 +58,7 @@ public class ClusterNodeTracker<N extends SchedulerNode> {
   private HashMap<NodeId, N> nodes = new HashMap<>();
   private Map<String, N> nodeNameToNodeMap = new HashMap<>();
   private Map<String, List<N>> nodesPerRack = new HashMap<>();
+  private Map<String, List<N>> nodesPerLabel = new HashMap<>();
 
   private Resource clusterCapacity = Resources.createResource(0, 0);
   private volatile Resource staleClusterCapacity =
@@ -80,6 +82,16 @@ public class ClusterNodeTracker<N extends SchedulerNode> {
       nodes.put(node.getNodeID(), node);
       nodeNameToNodeMap.put(node.getNodeName(), node);
 
+      List<N> nodesPerLabels = nodesPerLabel.get(node.getPartition());
+
+      if (nodesPerLabels == null) {
+        nodesPerLabels = new ArrayList<N>();
+      }
+      nodesPerLabels.add(node);
+
+      // Update new set of nodes for given partition.
+      nodesPerLabel.put(node.getPartition(), nodesPerLabels);
+
       // Update nodes per rack as well
       String rackName = node.getRackName();
       List<N> nodesList = nodesPerRack.get(rackName);
@@ -174,6 +186,16 @@ public class ClusterNodeTracker<N extends SchedulerNode> {
         }
       }
 
+      List<N> nodesPerPartition = nodesPerLabel.get(node.getPartition());
+      nodesPerPartition.remove(node);
+
+      // Update new set of nodes for given partition.
+      if (nodesPerPartition.isEmpty()) {
+        nodesPerLabel.remove(node.getPartition());
+      } else {
+        nodesPerLabel.put(node.getPartition(), nodesPerPartition);
+      }
+
       // Update cluster capacity
       Resources.subtractFrom(clusterCapacity, node.getTotalResource());
       staleClusterCapacity = Resources.clone(clusterCapacity);
@@ -420,4 +442,43 @@ public class ClusterNodeTracker<N extends SchedulerNode> {
     }
     return retNodes;
   }
+
+  /**
+   * update cached nodes per partition on a node label change event.
+   * @param partition nodeLabel
+   * @param nodeIds List of Node IDs
+   */
+  public void updateNodesPerPartition(String partition, Set<NodeId> nodeIds) {
+    writeLock.lock();
+    try {
+      // Clear all entries.
+      nodesPerLabel.remove(partition);
+
+      List<N> nodesPerPartition = new ArrayList<N>();
+      for (NodeId nodeId : nodeIds) {
+        N n = getNode(nodeId);
+        if (n != null) {
+          nodesPerPartition.add(n);
+        }
+      }
+
+      // Update new set of nodes for given partition.
+      nodesPerLabel.put(partition, nodesPerPartition);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public List<N> getNodesPerPartition(String partition) {
+    List<N> nodesPerPartition = null;
+    readLock.lock();
+    try {
+      if (nodesPerLabel.containsKey(partition)) {
+        nodesPerPartition = new ArrayList<N>(nodesPerLabel.get(partition));
+      }
+    } finally {
+      readLock.unlock();
+    }
+    return nodesPerPartition;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesLogger.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/activities/ActivitiesLogger.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesLogger.java
index 0c351b6..8a3ffce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesLogger.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesLogger.java
@@ -21,13 +21,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 
 /**
  * Utility for logging scheduler activities
@@ -63,7 +63,7 @@ public class ActivitiesLogger {
         SchedulerApplicationAttempt application, Priority priority,
         String diagnostic) {
       String type = "app";
-      if (activitiesManager == null) {
+      if (node == null || activitiesManager == null) {
         return;
       }
       if (activitiesManager.shouldRecordThisNode(node.getNodeID())) {
@@ -84,18 +84,18 @@ public class ActivitiesLogger {
         ActivitiesManager activitiesManager, SchedulerNode node,
         SchedulerApplicationAttempt application, Priority priority,
         String diagnostic, ActivityState appState) {
-      if (activitiesManager == null) {
+      if (node == null || activitiesManager == null) {
         return;
       }
       if (activitiesManager.shouldRecordThisNode(node.getNodeID())) {
         String type = "container";
         // Add application-container activity into specific node allocation.
-        activitiesManager.addSchedulingActivityForNode(node.getNodeID(),
+        activitiesManager.addSchedulingActivityForNode(node,
             application.getApplicationId().toString(), null,
             priority.toString(), ActivityState.SKIPPED, diagnostic, type);
         type = "app";
         // Add queue-application activity into specific node allocation.
-        activitiesManager.addSchedulingActivityForNode(node.getNodeID(),
+        activitiesManager.addSchedulingActivityForNode(node,
             application.getQueueName(),
             application.getApplicationId().toString(),
             application.getPriority().toString(), ActivityState.SKIPPED,
@@ -121,20 +121,20 @@ public class ActivitiesLogger {
         ActivitiesManager activitiesManager, SchedulerNode node,
         SchedulerApplicationAttempt application, RMContainer updatedContainer,
         ActivityState activityState) {
-      if (activitiesManager == null) {
+      if (node == null || activitiesManager == null) {
         return;
       }
       if (activitiesManager.shouldRecordThisNode(node.getNodeID())) {
         String type = "container";
         // Add application-container activity into specific node allocation.
-        activitiesManager.addSchedulingActivityForNode(node.getNodeID(),
+        activitiesManager.addSchedulingActivityForNode(node,
             application.getApplicationId().toString(),
             updatedContainer.getContainer().toString(),
             updatedContainer.getContainer().getPriority().toString(),
             activityState, ActivityDiagnosticConstant.EMPTY, type);
         type = "app";
         // Add queue-application activity into specific node allocation.
-        activitiesManager.addSchedulingActivityForNode(node.getNodeID(),
+        activitiesManager.addSchedulingActivityForNode(node,
             application.getQueueName(),
             application.getApplicationId().toString(),
             application.getPriority().toString(), ActivityState.ACCEPTED,
@@ -157,13 +157,15 @@ public class ActivitiesLogger {
      * update.
      */
     public static void startAppAllocationRecording(
-        ActivitiesManager activitiesManager, NodeId nodeId, long currentTime,
+        ActivitiesManager activitiesManager, FiCaSchedulerNode node,
+        long currentTime,
         SchedulerApplicationAttempt application) {
-      if (activitiesManager == null) {
+      if (node == null || activitiesManager == null) {
         return;
       }
-      activitiesManager.startAppAllocationRecording(nodeId, currentTime,
-          application);
+      activitiesManager
+          .startAppAllocationRecording(node.getNodeID(), currentTime,
+              application);
     }
 
     /*
@@ -208,7 +210,7 @@ public class ActivitiesLogger {
     public static void recordQueueActivity(ActivitiesManager activitiesManager,
         SchedulerNode node, String parentQueueName, String queueName,
         ActivityState state, String diagnostic) {
-      if (activitiesManager == null) {
+      if (node == null || activitiesManager == null) {
         return;
       }
       if (activitiesManager.shouldRecordThisNode(node.getNodeID())) {
@@ -240,7 +242,7 @@ public class ActivitiesLogger {
     public static void finishAllocatedNodeAllocation(
         ActivitiesManager activitiesManager, SchedulerNode node,
         ContainerId containerId, AllocationState containerState) {
-      if (activitiesManager == null) {
+      if (node == null || activitiesManager == null) {
         return;
       }
       if (activitiesManager.shouldRecordThisNode(node.getNodeID())) {
@@ -277,7 +279,7 @@ public class ActivitiesLogger {
       SchedulerNode node, String parentName, String childName,
       Priority priority, ActivityState state, String diagnostic, String type) {
 
-    activitiesManager.addSchedulingActivityForNode(node.getNodeID(), parentName,
+    activitiesManager.addSchedulingActivityForNode(node, parentName,
         childName, priority != null ? priority.toString() : null, state,
         diagnostic, type);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.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/activities/ActivitiesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java
index 8498c40..5d96b17 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
 import org.apache.hadoop.yarn.util.SystemClock;
@@ -197,11 +198,12 @@ public class ActivitiesManager extends AbstractService {
   }
 
   // Add queue, application or container activity into specific node allocation.
-  void addSchedulingActivityForNode(NodeId nodeID, String parentName,
+  void addSchedulingActivityForNode(SchedulerNode node, String parentName,
       String childName, String priority, ActivityState state, String diagnostic,
       String type) {
-    if (shouldRecordThisNode(nodeID)) {
-      NodeAllocation nodeAllocation = getCurrentNodeAllocation(nodeID);
+    if (shouldRecordThisNode(node.getNodeID())) {
+      NodeAllocation nodeAllocation = getCurrentNodeAllocation(
+          node.getNodeID());
       nodeAllocation.addAllocationActivity(parentName, childName, priority,
           state, diagnostic, type);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.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/AbstractCSQueue.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/AbstractCSQueue.java
index 9c3e98f..2c9f9a3 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/AbstractCSQueue.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/AbstractCSQueue.java
@@ -92,7 +92,8 @@ public abstract class AbstractCSQueue implements CSQueue {
   Set<String> resourceTypes;
   final RMNodeLabelsManager labelManager;
   String defaultLabelExpression;
-  
+  private String multiNodeSortingPolicyName = null;
+
   Map<AccessType, AccessControlList> acls = 
       new HashMap<AccessType, AccessControlList>();
   volatile boolean reservationsContinueLooking;
@@ -414,6 +415,10 @@ public abstract class AbstractCSQueue implements CSQueue {
       this.priority = configuration.getQueuePriority(
           getQueuePath());
 
+      // Update multi-node sorting algorithm for scheduling as configured.
+      setMultiNodeSortingPolicyName(
+          configuration.getMultiNodesSortingAlgorithmPolicy(getQueuePath()));
+
       this.userWeights = getUserWeightsFromHierarchy(configuration);
     } finally {
       writeLock.unlock();
@@ -1259,4 +1264,13 @@ public abstract class AbstractCSQueue implements CSQueue {
       this.writeLock.unlock();
     }
   }
+
+  @Override
+  public String getMultiNodeSortingPolicyName() {
+    return this.multiNodeSortingPolicyName;
+  }
+
+  public void setMultiNodeSortingPolicyName(String policyName) {
+    this.multiNodeSortingPolicyName = policyName;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.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/CSQueue.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/CSQueue.java
index 3963dc0..c0c280e 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/CSQueue.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/CSQueue.java
@@ -430,4 +430,10 @@ public interface CSQueue extends SchedulerQueue<CSQueue> {
    * @return effective max queue capacity
    */
   Resource getEffectiveMaxCapacityDown(String label, Resource factor);
+
+  /**
+   * Get Multi Node scheduling policy name.
+   * @return policy name
+   */
+  String getMultiNodeSortingPolicyName();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index 0b7fe92..dec1301 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -251,6 +252,7 @@ public class CapacityScheduler extends
   private ResourceCommitterService resourceCommitterService;
   private RMNodeLabelsManager labelManager;
   private AppPriorityACLsManager appPriorityACLManager;
+  private boolean multiNodePlacementEnabled;
 
   private static boolean printedVerboseLoggingForAsyncScheduling = false;
 
@@ -391,12 +393,23 @@ public class CapacityScheduler extends
       // Setup how many containers we can allocate for each round
       offswitchPerHeartbeatLimit = this.conf.getOffSwitchPerHeartbeatLimit();
 
+      // Register CS specific multi-node policies to common MultiNodeManager
+      // which will add to a MultiNodeSorter which gives a pre-sorted list of
+      // nodes to scheduler's allocation.
+      multiNodePlacementEnabled = this.conf.getMultiNodePlacementEnabled();
+      if(rmContext.getMultiNodeSortingManager() != null) {
+        rmContext.getMultiNodeSortingManager().registerMultiNodePolicyNames(
+            multiNodePlacementEnabled,
+            this.conf.getMultiNodePlacementPolicies());
+      }
+
       LOG.info("Initialized CapacityScheduler with " + "calculator="
           + getResourceCalculator().getClass() + ", " + "minimumAllocation=<"
           + getMinimumResourceCapability() + ">, " + "maximumAllocation=<"
           + getMaximumResourceCapability() + ">, " + "asynchronousScheduling="
           + scheduleAsynchronously + ", " + "asyncScheduleInterval="
-          + asyncScheduleInterval + "ms");
+          + asyncScheduleInterval + "ms" + ",multiNodePlacementEnabled="
+          + multiNodePlacementEnabled);
     } finally {
       writeLock.unlock();
     }
@@ -1373,18 +1386,23 @@ public class CapacityScheduler extends
         assignment.getAssignmentInformation().getAllocationDetails();
     List<AssignmentInformation.AssignmentDetails> reservations =
         assignment.getAssignmentInformation().getReservationDetails();
+    // Get nodeId from allocated container if incoming argument is null.
+    NodeId updatedNodeid = (nodeId == null)
+        ? allocations.get(allocations.size() - 1).rmContainer.getNodeId()
+        : nodeId;
+
     if (!allocations.isEmpty()) {
       ContainerId allocatedContainerId =
           allocations.get(allocations.size() - 1).containerId;
       String allocatedQueue = allocations.get(allocations.size() - 1).queue;
-      schedulerHealth.updateAllocation(now, nodeId, allocatedContainerId,
+      schedulerHealth.updateAllocation(now, updatedNodeid, allocatedContainerId,
         allocatedQueue);
     }
     if (!reservations.isEmpty()) {
       ContainerId reservedContainerId =
           reservations.get(reservations.size() - 1).containerId;
       String reservedQueue = reservations.get(reservations.size() - 1).queue;
-      schedulerHealth.updateReservation(now, nodeId, reservedContainerId,
+      schedulerHealth.updateReservation(now, updatedNodeid, reservedContainerId,
         reservedQueue);
     }
     schedulerHealth.updateSchedulerReservationCounts(assignment
@@ -1421,6 +1439,23 @@ public class CapacityScheduler extends
             || assignedContainers < maxAssignPerHeartbeat);
   }
 
+  private CandidateNodeSet<FiCaSchedulerNode> getCandidateNodeSet(
+      FiCaSchedulerNode node) {
+    CandidateNodeSet<FiCaSchedulerNode> candidates = null;
+    candidates = new SimpleCandidateNodeSet<>(node);
+    if (multiNodePlacementEnabled) {
+      Map<NodeId, FiCaSchedulerNode> nodesByPartition = new HashMap<>();
+      List<FiCaSchedulerNode> nodes = nodeTracker
+          .getNodesPerPartition(node.getPartition());
+      if (nodes != null && !nodes.isEmpty()) {
+        nodes.forEach(n -> nodesByPartition.put(n.getNodeID(), n));
+        candidates = new SimpleCandidateNodeSet<FiCaSchedulerNode>(
+            nodesByPartition, node.getPartition());
+      }
+    }
+    return candidates;
+  }
+
   /**
    * We need to make sure when doing allocation, Node should be existed
    * And we will construct a {@link CandidateNodeSet} before proceeding
@@ -1432,8 +1467,8 @@ public class CapacityScheduler extends
       int offswitchCount = 0;
       int assignedContainers = 0;
 
-      CandidateNodeSet<FiCaSchedulerNode> candidates =
-          new SimpleCandidateNodeSet<>(node);
+      CandidateNodeSet<FiCaSchedulerNode> candidates = getCandidateNodeSet(
+          node);
       CSAssignment assignment = allocateContainersToNode(candidates,
           withNodeHeartbeat);
       // Only check if we can allocate more container on the same node when
@@ -1599,10 +1634,13 @@ public class CapacityScheduler extends
 
     if (Resources.greaterThan(calculator, getClusterResource(),
         assignment.getResource(), Resources.none())) {
+      FiCaSchedulerNode node = CandidateNodeSetUtils.getSingleNode(candidates);
+      NodeId nodeId = null;
+      if (node != null) {
+        nodeId = node.getNodeID();
+      }
       if (withNodeHeartbeat) {
-        updateSchedulerHealth(lastNodeUpdateTime,
-            CandidateNodeSetUtils.getSingleNode(candidates).getNodeID(),
-            assignment);
+        updateSchedulerHealth(lastNodeUpdateTime, nodeId, assignment);
       }
       return assignment;
     }
@@ -1681,7 +1719,7 @@ public class CapacityScheduler extends
     // We have two different logics to handle allocation on single node / multi
     // nodes.
     CSAssignment assignment;
-    if (null != node) {
+    if (!multiNodePlacementEnabled) {
       assignment = allocateContainerOnSingleNode(candidates,
           node, withNodeHeartbeat);
     } else{
@@ -1869,12 +1907,21 @@ public class CapacityScheduler extends
       NodeLabelsUpdateSchedulerEvent labelUpdateEvent) {
     try {
       writeLock.lock();
+      Set<String> updateLabels = new HashSet<String>();
       for (Entry<NodeId, Set<String>> entry : labelUpdateEvent
           .getUpdatedNodeToLabels().entrySet()) {
         NodeId id = entry.getKey();
         Set<String> labels = entry.getValue();
+        FiCaSchedulerNode node = nodeTracker.getNode(id);
+
+        if (node != null) {
+          // Update old partition to list.
+          updateLabels.add(node.getPartition());
+        }
         updateLabelsOnNode(id, labels);
+        updateLabels.addAll(labels);
       }
+      refreshLabelToNodeCache(updateLabels);
       Resource clusterResource = getClusterResource();
       getRootQueue().updateClusterResource(clusterResource,
           new ResourceLimits(clusterResource));
@@ -1883,6 +1930,18 @@ public class CapacityScheduler extends
     }
   }
 
+  private void refreshLabelToNodeCache(Set<String> updateLabels) {
+    Map<String, Set<NodeId>> labelMapping = labelManager
+        .getLabelsToNodes(updateLabels);
+    for (String label : updateLabels) {
+      Set<NodeId> nodes = labelMapping.get(label);
+      if (nodes == null) {
+        continue;
+      }
+      nodeTracker.updateNodesPerPartition(label, nodes);
+    }
+  }
+
   private void addNode(RMNode nodeManager) {
     try {
       writeLock.lock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
index e8de096..b937ae7 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
@@ -45,6 +45,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AppPriorityACLConfigurationParser.AppPriorityACLKeyType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy.PriorityUtilizationQueueOrderingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy.QueueOrderingPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeLookupPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodePolicySpec;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FairOrderingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
@@ -2129,4 +2131,118 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
       break;
     }
   }
+
+  @Private public static final String MULTI_NODE_SORTING_POLICIES =
+      PREFIX + "multi-node-sorting.policy.names";
+
+  @Private public static final String MULTI_NODE_SORTING_POLICY_NAME =
+      PREFIX + "multi-node-sorting.policy";
+
+  /**
+   * resource usage based node sorting algorithm.
+   */
+  public static final String DEFAULT_NODE_SORTING_POLICY = "default";
+  public static final String DEFAULT_NODE_SORTING_POLICY_CLASSNAME
+      = "org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.ResourceUsageMultiNodeLookupPolicy";
+  public static final long DEFAULT_MULTI_NODE_SORTING_INTERVAL = 1000L;
+
+  @Private
+  public static final String MULTI_NODE_PLACEMENT_ENABLED = PREFIX
+      + "multi-node-placement-enabled";
+
+  @Private
+  public static final boolean DEFAULT_MULTI_NODE_PLACEMENT_ENABLED = false;
+
+  public String getMultiNodesSortingAlgorithmPolicy(
+      String queue) {
+
+    String policyName = get(
+        getQueuePrefix(queue) + "multi-node-sorting.policy");
+
+    if (policyName == null) {
+      policyName = get(MULTI_NODE_SORTING_POLICY_NAME);
+    }
+
+    // If node sorting policy is not configured in queue and in cluster level,
+    // it is been assumed that this queue is not enabled with multi-node lookup.
+    if (policyName == null || policyName.isEmpty()) {
+      return null;
+    }
+
+    String policyClassName = get(MULTI_NODE_SORTING_POLICY_NAME + DOT
+        + policyName.trim() + DOT + "class");
+
+    if (policyClassName == null || policyClassName.isEmpty()) {
+      throw new YarnRuntimeException(
+          policyName.trim() + " Class is not configured or not an instance of "
+              + MultiNodeLookupPolicy.class.getCanonicalName());
+    }
+
+    return normalizePolicyName(policyClassName.trim());
+  }
+
+  public boolean getMultiNodePlacementEnabled() {
+    return getBoolean(MULTI_NODE_PLACEMENT_ENABLED,
+        DEFAULT_MULTI_NODE_PLACEMENT_ENABLED);
+  }
+
+  public Set<MultiNodePolicySpec> getMultiNodePlacementPolicies() {
+    String[] policies = getTrimmedStrings(MULTI_NODE_SORTING_POLICIES);
+
+    // In other cases, split the accessibleLabelStr by ","
+    Set<MultiNodePolicySpec> set = new HashSet<MultiNodePolicySpec>();
+    for (String str : policies) {
+      if (!str.trim().isEmpty()) {
+        String policyClassName = get(
+            MULTI_NODE_SORTING_POLICY_NAME + DOT + str.trim() + DOT + "class");
+        if (str.trim().equals(DEFAULT_NODE_SORTING_POLICY)) {
+          policyClassName = get(
+              MULTI_NODE_SORTING_POLICY_NAME + DOT + str.trim() + DOT + "class",
+              DEFAULT_NODE_SORTING_POLICY_CLASSNAME);
+        }
+
+        // This check is needed as default class name is loaded only for
+        // DEFAULT_NODE_SORTING_POLICY.
+        if (policyClassName == null) {
+          throw new YarnRuntimeException(
+              str.trim() + " Class is not configured or not an instance of "
+                  + MultiNodeLookupPolicy.class.getCanonicalName());
+        }
+        policyClassName = normalizePolicyName(policyClassName.trim());
+        long policySortingInterval = getLong(
+            MULTI_NODE_SORTING_POLICY_NAME + DOT + str.trim()
+                + DOT + "sorting-interval.ms",
+            DEFAULT_MULTI_NODE_SORTING_INTERVAL);
+        if (policySortingInterval < 0) {
+          throw new YarnRuntimeException(
+              str.trim()
+                  + " multi-node policy is configured with invalid"
+                  + " sorting-interval:" + policySortingInterval);
+        }
+        set.add(
+            new MultiNodePolicySpec(policyClassName, policySortingInterval));
+      }
+    }
+
+    return Collections.unmodifiableSet(set);
+  }
+
+  private String normalizePolicyName(String policyName) {
+
+    // Ensure that custom node sorting algorithm class is valid.
+    try {
+      Class<?> nodeSortingPolicyClazz = getClassByName(policyName);
+      if (MultiNodeLookupPolicy.class
+          .isAssignableFrom(nodeSortingPolicyClazz)) {
+        return policyName;
+      } else {
+        throw new YarnRuntimeException(
+            "Class: " + policyName + " not instance of "
+                + MultiNodeLookupPolicy.class.getCanonicalName());
+      }
+    } catch (ClassNotFoundException e) {
+      throw new YarnRuntimeException(
+          "Could not instantiate " + "NodesSortingPolicy: " + policyName, e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.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/LeafQueue.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/LeafQueue.java
index 366bad0..ffe862f 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/LeafQueue.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/LeafQueue.java
@@ -53,10 +53,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsMana
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
-
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.*;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger;
@@ -1036,23 +1032,24 @@ public class LeafQueue extends AbstractCSQueue {
   private CSAssignment allocateFromReservedContainer(Resource clusterResource,
       CandidateNodeSet<FiCaSchedulerNode> candidates,
       ResourceLimits currentResourceLimits, SchedulingMode schedulingMode) {
-    FiCaSchedulerNode node = CandidateNodeSetUtils.getSingleNode(candidates);
-    if (null == node) {
-      return null;
-    }
-
-    RMContainer reservedContainer = node.getReservedContainer();
-    if (reservedContainer != null) {
-      FiCaSchedulerApp application = getApplication(
-          reservedContainer.getApplicationAttemptId());
-
-      if (null != application) {
-        ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager,
-            node.getNodeID(), SystemClock.getInstance().getTime(), application);
-        CSAssignment assignment = application.assignContainers(clusterResource,
-            candidates, currentResourceLimits, schedulingMode,
-            reservedContainer);
-        return assignment;
+    // Considering multi-node scheduling, its better to iterate through
+    // all candidates and stop once we get atleast one good node to allocate
+    // where reservation was made earlier. In normal case, there is only one
+    // node and hence there wont be any impact after this change.
+    for (FiCaSchedulerNode node : candidates.getAllNodes().values()) {
+      RMContainer reservedContainer = node.getReservedContainer();
+      if (reservedContainer != null) {
+        FiCaSchedulerApp application = getApplication(
+            reservedContainer.getApplicationAttemptId());
+
+        if (null != application) {
+          ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager,
+              node, SystemClock.getInstance().getTime(), application);
+          CSAssignment assignment = application.assignContainers(
+              clusterResource, candidates, currentResourceLimits,
+              schedulingMode, reservedContainer);
+          return assignment;
+        }
       }
     }
 
@@ -1114,13 +1111,14 @@ public class LeafQueue extends AbstractCSQueue {
       FiCaSchedulerApp application = assignmentIterator.next();
 
       ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager,
-          node.getNodeID(), SystemClock.getInstance().getTime(), application);
+          node, SystemClock.getInstance().getTime(), application);
 
       // Check queue max-capacity limit
       Resource appReserved = application.getCurrentReservation();
       if (needAssignToQueueCheck) {
-        if (!super.canAssignToThisQueue(clusterResource, node.getPartition(),
-            currentResourceLimits, appReserved, schedulingMode)) {
+        if (!super.canAssignToThisQueue(clusterResource,
+            candidates.getPartition(), currentResourceLimits, appReserved,
+            schedulingMode)) {
           ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue(
               activitiesManager, node, application, application.getPriority(),
               ActivityDiagnosticConstant.QUEUE_MAX_CAPACITY_LIMIT);
@@ -1155,7 +1153,8 @@ public class LeafQueue extends AbstractCSQueue {
         userAssignable = false;
       } else {
         userAssignable = canAssignToUser(clusterResource, application.getUser(),
-            userLimit, application, node.getPartition(), currentResourceLimits);
+            userLimit, application, candidates.getPartition(),
+            currentResourceLimits);
         if (!userAssignable && Resources.fitsIn(cul.reservation, appReserved)) {
           cul.canAssign = false;
           cul.reservation = appReserved;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.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/ParentQueue.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/ParentQueue.java
index 2363b88..80549ca 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/ParentQueue.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/ParentQueue.java
@@ -553,8 +553,8 @@ public class ParentQueue extends AbstractCSQueue {
 
       ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
           getParentName(), getQueueName(), ActivityState.REJECTED,
-          ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION + node
-              .getPartition());
+          ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION
+              + candidates.getPartition());
       if (rootQueue) {
         ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager,
             node);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.java
index a843002..3e337ef 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/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.java
@@ -96,11 +96,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
    * headroom, etc.
    */
   private ContainerAllocation preCheckForNodeCandidateSet(
-      Resource clusterResource, CandidateNodeSet<FiCaSchedulerNode> candidates,
+      Resource clusterResource, FiCaSchedulerNode node,
       SchedulingMode schedulingMode, ResourceLimits resourceLimits,
       SchedulerRequestKey schedulerKey) {
     Priority priority = schedulerKey.getPriority();
-    FiCaSchedulerNode node = CandidateNodeSetUtils.getSingleNode(candidates);
 
     PendingAsk offswitchPendingAsk = application.getPendingAsk(schedulerKey,
         ResourceRequest.ANY);
@@ -164,7 +163,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     }
 
     if (!checkHeadroom(clusterResource, resourceLimits, required,
-        candidates.getPartition())) {
+        node.getPartition())) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("cannot allocate required resource=" + required
             + " because of headroom");
@@ -801,20 +800,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     // Do checks before determining which node to allocate
     // Directly return if this check fails.
     ContainerAllocation result;
-    if (reservedContainer == null) {
-      result = preCheckForNodeCandidateSet(clusterResource, candidates,
-          schedulingMode, resourceLimits, schedulerKey);
-      if (null != result) {
-        return result;
-      }
-    } else {
-      // pre-check when allocating reserved container
-      if (application.getOutstandingAsksCount(schedulerKey) == 0) {
-        // Release
-        return new ContainerAllocation(reservedContainer, null,
-            AllocationState.QUEUE_SKIPPED);
-      }
-    }
 
     AppPlacementAllocator<FiCaSchedulerNode> schedulingPS =
         application.getAppSchedulingInfo().getAppPlacementAllocator(
@@ -833,6 +818,22 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     while (iter.hasNext()) {
       FiCaSchedulerNode node = iter.next();
 
+      if (reservedContainer == null) {
+        result = preCheckForNodeCandidateSet(clusterResource, node,
+            schedulingMode, resourceLimits, schedulerKey);
+        if (null != result) {
+          continue;
+        }
+      } else {
+        // pre-check when allocating reserved container
+        if (application.getOutstandingAsksCount(schedulerKey) == 0) {
+          // Release
+          result = new ContainerAllocation(reservedContainer, null,
+              AllocationState.QUEUE_SKIPPED);
+          continue;
+        }
+      }
+
       result = tryAllocateOnNode(clusterResource, node, schedulingMode,
           resourceLimits, schedulerKey, reservedContainer);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ApplicationSchedulingConfig.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/ApplicationSchedulingConfig.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/ApplicationSchedulingConfig.java
index 1bd3743..06f74de 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/ApplicationSchedulingConfig.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/ApplicationSchedulingConfig.java
@@ -32,4 +32,8 @@ public class ApplicationSchedulingConfig {
   @InterfaceAudience.Private
   public static final Class<? extends AppPlacementAllocator>
       DEFAULT_APPLICATION_PLACEMENT_TYPE_CLASS = LocalityAppPlacementAllocator.class;
+
+  @InterfaceAudience.Private
+  public static final String ENV_MULTI_NODE_SORTING_POLICY_CLASS =
+      "MULTI_NODE_SORTING_POLICY_CLASS";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/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 6a5af81..4bfdae9 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
@@ -75,6 +75,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCap
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.AbstractContainerAllocator;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.ContainerAllocator;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ApplicationSchedulingConfig;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
@@ -170,10 +171,32 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
       rc = scheduler.getResourceCalculator();
     }
 
+    // Update multi-node sorting algorithm to scheduler envs
+    updateMultiNodeSortingPolicy(rmApp);
+
     containerAllocator = new ContainerAllocator(this, rc, rmContext,
         activitiesManager);
   }
 
+  private void updateMultiNodeSortingPolicy(RMApp rmApp) {
+    if (rmApp == null) {
+      return;
+    }
+
+    String queueName = null;
+    if (scheduler instanceof CapacityScheduler) {
+      queueName = getCSLeafQueue().getMultiNodeSortingPolicyName();
+    }
+
+    if (!appSchedulingInfo.getApplicationSchedulingEnvs().containsKey(
+        ApplicationSchedulingConfig.ENV_MULTI_NODE_SORTING_POLICY_CLASS)
+        && queueName != null) {
+      appSchedulingInfo.getApplicationSchedulingEnvs().put(
+          ApplicationSchedulingConfig.ENV_MULTI_NODE_SORTING_POLICY_CLASS,
+          queueName);
+    }
+  }
+
   public boolean containerCompleted(RMContainer rmContainer,
       ContainerStatus containerStatus, RMContainerEventType event,
       String partition) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
index f1df343..9d30e90 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
@@ -24,11 +24,14 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ApplicationSchedulingConfig;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
@@ -55,6 +58,8 @@ public class LocalityAppPlacementAllocator <N extends SchedulerNode>
       new ConcurrentHashMap<>();
   private volatile String primaryRequestedPartition =
       RMNodeLabelsManager.NO_LABEL;
+  private MultiNodeSortingManager<N> multiNodeSortingManager = null;
+  private String multiNodeSortPolicyName;
 
   private final ReentrantReadWriteLock.ReadLock readLock;
   private final ReentrantReadWriteLock.WriteLock writeLock;
@@ -65,6 +70,26 @@ public class LocalityAppPlacementAllocator <N extends SchedulerNode>
     writeLock = lock.writeLock();
   }
 
+  @SuppressWarnings("unchecked")
+  @Override
+  public void initialize(AppSchedulingInfo appSchedulingInfo,
+      SchedulerRequestKey schedulerRequestKey, RMContext rmContext) {
+    super.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+    multiNodeSortPolicyName = appSchedulingInfo
+        .getApplicationSchedulingEnvs().get(
+            ApplicationSchedulingConfig.ENV_MULTI_NODE_SORTING_POLICY_CLASS);
+    multiNodeSortingManager = (MultiNodeSortingManager<N>) rmContext
+        .getMultiNodeSortingManager();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          "nodeLookupPolicy used for " + appSchedulingInfo
+              .getApplicationId()
+              + " is " + ((multiNodeSortPolicyName != null) ?
+              multiNodeSortPolicyName :
+              ""));
+    }
+  }
+
   @Override
   @SuppressWarnings("unchecked")
   public Iterator<N> getPreferredNodeIterator(
@@ -74,11 +99,16 @@ public class LocalityAppPlacementAllocator <N extends SchedulerNode>
     // in.
 
     N singleNode = CandidateNodeSetUtils.getSingleNode(candidateNodeSet);
-    if (null != singleNode) {
+    if (singleNode != null) {
       return IteratorUtils.singletonIterator(singleNode);
     }
 
-    return IteratorUtils.emptyIterator();
+    // singleNode will be null if Multi-node placement lookup is enabled, and
+    // hence could consider sorting policies.
+    return multiNodeSortingManager.getMultiNodeSortIterator(
+        candidateNodeSet.getAllNodes().values(),
+        candidateNodeSet.getPartition(),
+        multiNodeSortPolicyName);
   }
 
   private boolean hasRequestLabelChanged(ResourceRequest requestOne,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeLookupPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeLookupPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeLookupPolicy.java
new file mode 100644
index 0000000..662e34d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeLookupPolicy.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Set;
+
+/**
+ * <p>
+ * This class has the following functionality.
+ *
+ * <p>
+ * Provide an interface for MultiNodeLookupPolicy so that different placement
+ * allocator can choose nodes based on need.
+ * </p>
+ */
+public interface MultiNodeLookupPolicy<N extends SchedulerNode> {
+  /**
+   * Get iterator of preferred node depends on requirement and/or availability.
+   *
+   * @param nodes
+   *          List of Nodes
+   * @param partition
+   *          node label
+   *
+   * @return iterator of preferred node
+   */
+  Iterator<N> getPreferredNodeIterator(Collection<N> nodes, String partition);
+
+  /**
+   * Refresh working nodes set for re-ordering based on the algorithm selected.
+   *
+   * @param nodes
+   *          a collection working nm's.
+   */
+  void addAndRefreshNodesSet(Collection<N> nodes, String partition);
+
+  /**
+   * Get sorted nodes per partition.
+   *
+   * @param partition
+   *          node label
+   *
+   * @return collection of sorted nodes
+   */
+  Set<N> getNodesPerPartition(String partition);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodePolicySpec.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodePolicySpec.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodePolicySpec.java
new file mode 100644
index 0000000..8386d78
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodePolicySpec.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
+
+/**
+ * MultiNodePolicySpec contains policyName and timeout.
+ */
+public class MultiNodePolicySpec {
+
+  private String policyName;
+  private long sortingInterval;
+
+  public MultiNodePolicySpec(String policyName, long timeout) {
+    this.setSortingInterval(timeout);
+    this.setPolicyName(policyName);
+  }
+
+  public long getSortingInterval() {
+    return sortingInterval;
+  }
+
+  public void setSortingInterval(long timeout) {
+    this.sortingInterval = timeout;
+  }
+
+  public String getPolicyName() {
+    return policyName;
+  }
+
+  public void setPolicyName(String policyName) {
+    this.policyName = policyName;
+  }
+
+  @Override
+  public String toString() {
+    return "MultiNodePolicySpec {" +
+        "policyName='" + policyName + '\'' +
+        ", sortingInterval=" + sortingInterval +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSorter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSorter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSorter.java
new file mode 100644
index 0000000..7e27c34
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSorter.java
@@ -0,0 +1,167 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Common node sorting class which will do sorting based on policy spec.
+ * @param <N> extends SchedulerNode.
+ */
+public class MultiNodeSorter<N extends SchedulerNode> extends AbstractService {
+
+  private MultiNodeLookupPolicy<N> multiNodePolicy;
+  private static final Log LOG = LogFactory.getLog(MultiNodeSorter.class);
+
+  // ScheduledExecutorService which schedules the PreemptionChecker to run
+  // periodically.
+  private ScheduledExecutorService ses;
+  private ScheduledFuture<?> handler;
+  private volatile boolean stopped;
+  private RMContext rmContext;
+  private MultiNodePolicySpec policySpec;
+
+  public MultiNodeSorter(RMContext rmContext,
+      MultiNodePolicySpec policy) {
+    super("MultiNodeLookupPolicy");
+    this.rmContext = rmContext;
+    this.policySpec = policy;
+  }
+
+  @VisibleForTesting
+  public synchronized MultiNodeLookupPolicy<N> getMultiNodeLookupPolicy() {
+    return multiNodePolicy;
+  }
+
+  public void serviceInit(Configuration conf) throws Exception {
+    LOG.info("Initializing MultiNodeSorter=" + policySpec.getPolicyName()
+        + ", with sorting interval=" + policySpec.getSortingInterval());
+    initPolicy(policySpec.getPolicyName());
+    super.serviceInit(conf);
+  }
+
+  @SuppressWarnings("unchecked")
+  void initPolicy(String policyName) throws YarnException {
+    Class<?> policyClass;
+    try {
+      policyClass = Class.forName(policyName);
+    } catch (ClassNotFoundException e) {
+      throw new YarnException(
+          "Invalid policy name:" + policyName + e.getMessage());
+    }
+    this.multiNodePolicy = (MultiNodeLookupPolicy<N>) ReflectionUtils
+        .newInstance(policyClass, null);
+  }
+
+  @Override
+  public void serviceStart() throws Exception {
+    LOG.info("Starting SchedulingMonitor=" + getName());
+    assert !stopped : "starting when already stopped";
+    ses = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() {
+      public Thread newThread(Runnable r) {
+        Thread t = new Thread(r);
+        t.setName(getName());
+        return t;
+      }
+    });
+
+    // Start sorter thread only if sorting interval is a +ve value.
+    if(policySpec.getSortingInterval() != 0) {
+      handler = ses.scheduleAtFixedRate(new SortingThread(),
+          0, policySpec.getSortingInterval(), TimeUnit.MILLISECONDS);
+    }
+    super.serviceStart();
+  }
+
+  @Override
+  public void serviceStop() throws Exception {
+    stopped = true;
+    if (handler != null) {
+      LOG.info("Stop " + getName());
+      handler.cancel(true);
+      ses.shutdown();
+    }
+    super.serviceStop();
+  }
+
+  @SuppressWarnings("unchecked")
+  @VisibleForTesting
+  public void reSortClusterNodes() {
+    Set<String> nodeLabels = new HashSet<>();
+    nodeLabels
+        .addAll(rmContext.getNodeLabelManager().getClusterNodeLabelNames());
+    nodeLabels.add(RMNodeLabelsManager.NO_LABEL);
+    for (String label : nodeLabels) {
+      Map<NodeId, SchedulerNode> nodesByPartition = new HashMap<>();
+      List<SchedulerNode> nodes = ((AbstractYarnScheduler) rmContext
+          .getScheduler()).getNodeTracker().getNodesPerPartition(label);
+      if (nodes != null && !nodes.isEmpty()) {
+        nodes.forEach(n -> nodesByPartition.put(n.getNodeID(), n));
+        multiNodePolicy.addAndRefreshNodesSet(
+            (Collection<N>) nodesByPartition.values(), label);
+      }
+    }
+  }
+
+  private class SortingThread implements Runnable {
+    @Override
+    public void run() {
+      try {
+        reSortClusterNodes();
+      } catch (Throwable t) {
+        // The preemption monitor does not alter structures nor do structures
+        // persist across invocations. Therefore, log, skip, and retry.
+        LOG.error("Exception raised while executing multinode"
+            + " sorter, skip this run..., exception=", t);
+      }
+    }
+  }
+
+  /**
+   * Verify whether sorter thread is running or not.
+   *
+   * @return true if sorter thread is running, false otherwise.
+   */
+  public boolean isSorterThreadRunning() {
+    return (handler != null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSortingManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSortingManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSortingManager.java
new file mode 100644
index 0000000..e872317
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSortingManager.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.collections.IteratorUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+
+/**
+ * Node Sorting Manager which runs all sorter threads and policies.
+ * @param <N> extends SchedulerNode
+ */
+public class MultiNodeSortingManager<N extends SchedulerNode>
+    extends AbstractService {
+
+  private static final Log LOG = LogFactory
+      .getLog(MultiNodeSortingManager.class);
+
+  private RMContext rmContext;
+  private Map<String, MultiNodeSorter<N>> runningMultiNodeSorters;
+  private Set<MultiNodePolicySpec> policySpecs = new HashSet<MultiNodePolicySpec>();
+  private Configuration conf;
+  private boolean multiNodePlacementEnabled;
+
+  public MultiNodeSortingManager() {
+    super("MultiNodeSortingManager");
+    this.runningMultiNodeSorters = new ConcurrentHashMap<>();
+  }
+
+  @Override
+  public void serviceInit(Configuration configuration) throws Exception {
+    LOG.info("Initializing NodeSortingService=" + getName());
+    super.serviceInit(configuration);
+    this.conf = configuration;
+  }
+
+  @Override
+  public void serviceStart() throws Exception {
+    LOG.info("Starting NodeSortingService=" + getName());
+    createAllPolicies();
+    super.serviceStart();
+  }
+
+  @Override
+  public void serviceStop() throws Exception {
+    for (MultiNodeSorter<N> sorter : runningMultiNodeSorters.values()) {
+      sorter.stop();
+    }
+    super.serviceStop();
+  }
+
+  private void createAllPolicies() {
+    if (!multiNodePlacementEnabled) {
+      return;
+    }
+    for (MultiNodePolicySpec policy : policySpecs) {
+      MultiNodeSorter<N> mon = new MultiNodeSorter<N>(rmContext, policy);
+      mon.init(conf);
+      mon.start();
+      runningMultiNodeSorters.put(policy.getPolicyName(), mon);
+    }
+  }
+
+  public MultiNodeSorter<N> getMultiNodePolicy(String name) {
+    return runningMultiNodeSorters.get(name);
+  }
+
+  public void setRMContext(RMContext context) {
+    this.rmContext = context;
+  }
+
+  public void registerMultiNodePolicyNames(
+      boolean isMultiNodePlacementEnabled,
+      Set<MultiNodePolicySpec> multiNodePlacementPolicies) {
+    this.policySpecs.addAll(multiNodePlacementPolicies);
+    this.multiNodePlacementEnabled = isMultiNodePlacementEnabled;
+    LOG.info("MultiNode scheduling is '" + multiNodePlacementEnabled +
+        "', and configured policies are " + StringUtils
+        .join(policySpecs.iterator(), ","));
+  }
+
+  public Iterator<N> getMultiNodeSortIterator(Collection<N> nodes,
+      String partition, String policyName) {
+    // nodeLookupPolicy can be null if app is configured with invalid policy.
+    // in such cases, use the the first node.
+    if(policyName == null) {
+      LOG.warn("Multi Node scheduling is enabled, however invalid class is"
+          + " configured. Valid sorting policy has to be configured in"
+          + " yarn.scheduler.capacity.<queue>.multi-node-sorting.policy");
+      return IteratorUtils.singletonIterator(
+          nodes.iterator().next());
+    }
+
+    MultiNodeSorter multiNodeSorter = getMultiNodePolicy(policyName);
+    if (multiNodeSorter == null) {
+      LOG.warn(
+          "MultiNode policy '" + policyName + "' is configured, however " +
+              "yarn.scheduler.capacity.multi-node-placement-enabled is false");
+      return IteratorUtils.singletonIterator(
+          nodes.iterator().next());
+    }
+
+    MultiNodeLookupPolicy<N> policy = multiNodeSorter
+        .getMultiNodeLookupPolicy();
+    // If sorter thread is not running, refresh node set.
+    if (!multiNodeSorter.isSorterThreadRunning()) {
+      policy.addAndRefreshNodesSet(nodes, partition);
+    }
+
+    return policy.getPreferredNodeIterator(nodes, partition);
+  }
+}


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

Branch: refs/heads/YARN-3409
Commit: 807be17d4e82ac0bbe6c7981519f16fc086b9d78
Parents: 9a3bf23
Author: bibinchundatt <bi...@apache.org>
Authored: Fri Feb 2 10:31:00 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:56 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/807be17d/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/807be17d/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/807be17d/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/807be17d/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


[14/50] [abbrv] hadoop git commit: HDDS-350. ContainerMapping#flushContainerInfo doesn't set containerId. Contributed by Ajay Kumar.

Posted by su...@apache.org.
HDDS-350. ContainerMapping#flushContainerInfo doesn't set containerId. 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/4c25f37c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4c25f37c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4c25f37c

Branch: refs/heads/YARN-3409
Commit: 4c25f37c6cc4e22a006cd095d6143b549bf4a0a8
Parents: 5aa15cf
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Wed Aug 22 10:53:54 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Wed Aug 22 10:54:10 2018 -0700

----------------------------------------------------------------------
 .../scm/container/common/helpers/ContainerInfo.java |  1 +
 .../hadoop/hdds/scm/container/ContainerMapping.java | 16 +---------------
 .../hdds/scm/container/TestContainerMapping.java    |  8 ++++++++
 3 files changed, 10 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c25f37c/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 427c08b..311c118 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
@@ -212,6 +212,7 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
   public HddsProtos.SCMContainerInfo getProtobuf() {
     HddsProtos.SCMContainerInfo.Builder builder =
         HddsProtos.SCMContainerInfo.newBuilder();
+    Preconditions.checkState(containerID > 0);
     return builder.setAllocatedBytes(getAllocatedBytes())
         .setContainerID(getContainerID())
         .setUsedBytes(getUsedBytes())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c25f37c/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 11863f2..4076dad 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
@@ -733,21 +733,7 @@ public class ContainerMapping implements Mapping {
         // return info of a deleted container. may revisit this in the future,
         // for now, just skip a not-found container
         if (containerBytes != null) {
-          HddsProtos.SCMContainerInfo oldInfoProto =
-              HddsProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes);
-          ContainerInfo oldInfo = ContainerInfo.fromProtobuf(oldInfoProto);
-          ContainerInfo newInfo = new ContainerInfo.Builder()
-              .setAllocatedBytes(info.getAllocatedBytes())
-              .setNumberOfKeys(oldInfo.getNumberOfKeys())
-              .setOwner(oldInfo.getOwner())
-              .setPipelineID(oldInfo.getPipelineID())
-              .setState(oldInfo.getState())
-              .setUsedBytes(oldInfo.getUsedBytes())
-              .setDeleteTransactionId(oldInfo.getDeleteTransactionId())
-              .setReplicationFactor(oldInfo.getReplicationFactor())
-              .setReplicationType(oldInfo.getReplicationType())
-              .build();
-          containerStore.put(dbKey, newInfo.getProtobuf().toByteArray());
+          containerStore.put(dbKey, info.getProtobuf().toByteArray());
         } else {
           LOG.debug("Container state manager has container {} but not found " +
                   "in container store, a deleted container?",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c25f37c/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 2dc7e99..1e9c35b 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
@@ -360,4 +360,12 @@ public class TestContainerMapping {
     return containerInfo;
   }
 
+  @Test
+  public void testFlushAllContainers() throws IOException {
+    ContainerInfo info = createContainer();
+    List<ContainerInfo> containers = mapping.getStateManager().getAllContainers();
+    Assert.assertTrue(containers.size() > 0);
+    mapping.flushContainerInfo();
+  }
+
 }


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

Branch: refs/heads/YARN-3409
Commit: 0d7a75e485cd42835c883c0491a1e766dba2227a
Parents: 4c25fa3
Author: Naganarasimha <na...@apache.org>
Authored: Tue Feb 27 18:46:16 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:56 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/0d7a75e4/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/0d7a75e4/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/0d7a75e4/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


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

Branch: refs/heads/YARN-3409
Commit: 5e1154bddfb300923d8a6240df9233eeb6bebbcc
Parents: a4121c7
Author: Naganarasimha <na...@apache.org>
Authored: Sun Jan 21 00:53:02 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:56 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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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/5e1154bd/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


[39/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/eed7de25/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/eed7de25/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/eed7de25/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/eed7de25/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


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

Branch: refs/heads/YARN-3409
Commit: 2b4c09017fdda6c58b845b2059bc404fa3621198
Parents: 0d7a75e
Author: Naganarasimha <na...@apache.org>
Authored: Mon Mar 12 08:05:53 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:56 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/2b4c0901/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 15f8b3b..b331381 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
@@ -3531,9 +3531,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.";
@@ -3565,6 +3568,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/2b4c0901/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/2b4c0901/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/2b4c0901/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 7278273..e6d708f 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
@@ -2897,6 +2897,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
@@ -2934,6 +2948,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/2b4c0901/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index b54a6b7..6eda4a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -66,6 +66,9 @@ import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.nodelabels.ConfigurationNodeLabelsProvider;
 import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider;
 import org.apache.hadoop.yarn.server.nodemanager.nodelabels.ScriptBasedNodeLabelsProvider;
+import org.apache.hadoop.yarn.server.nodemanager.nodelabels.ScriptBasedNodeAttributesProvider;
+import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeAttributesProvider;
+import org.apache.hadoop.yarn.server.nodemanager.nodelabels.ConfigurationNodeAttributesProvider;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMLeveldbStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
@@ -123,6 +126,7 @@ public class NodeManager extends CompositeService
   private ApplicationACLsManager aclsManager;
   private NodeHealthCheckerService nodeHealthChecker;
   private NodeLabelsProvider nodeLabelsProvider;
+  private NodeAttributesProvider nodeAttributesProvider;
   private LocalDirsHandlerService dirsHandler;
   private Context context;
   private AsyncDispatcher dispatcher;
@@ -162,14 +166,45 @@ public class NodeManager extends CompositeService
   protected NodeStatusUpdater createNodeStatusUpdater(Context context,
       Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
     return new NodeStatusUpdaterImpl(context, dispatcher, healthChecker,
-        metrics, nodeLabelsProvider);
+        metrics);
   }
 
-  protected NodeStatusUpdater createNodeStatusUpdater(Context context,
-      Dispatcher dispatcher, NodeHealthCheckerService healthChecker,
-      NodeLabelsProvider nodeLabelsProvider) {
-    return new NodeStatusUpdaterImpl(context, dispatcher, healthChecker,
-        metrics, nodeLabelsProvider);
+  protected NodeAttributesProvider createNodeAttributesProvider(
+      Configuration conf) throws IOException {
+    NodeAttributesProvider attributesProvider = null;
+    String providerString =
+        conf.get(YarnConfiguration.NM_NODE_ATTRIBUTES_PROVIDER_CONFIG, null);
+    if (providerString == null || providerString.trim().length() == 0) {
+      return attributesProvider;
+    }
+    switch (providerString.trim().toLowerCase()) {
+    case YarnConfiguration.CONFIG_NODE_DESCRIPTOR_PROVIDER:
+      attributesProvider = new ConfigurationNodeAttributesProvider();
+      break;
+    case YarnConfiguration.SCRIPT_NODE_DESCRIPTOR_PROVIDER:
+      attributesProvider = new ScriptBasedNodeAttributesProvider();
+      break;
+    default:
+      try {
+        Class<? extends NodeAttributesProvider> labelsProviderClass =
+            conf.getClass(YarnConfiguration.NM_NODE_ATTRIBUTES_PROVIDER_CONFIG,
+                null, NodeAttributesProvider.class);
+        attributesProvider = labelsProviderClass.newInstance();
+      } catch (InstantiationException | IllegalAccessException
+          | RuntimeException e) {
+        LOG.error("Failed to create NodeAttributesProvider"
+                + " based on Configuration", e);
+        throw new IOException(
+            "Failed to create NodeAttributesProvider : "
+                + e.getMessage(), e);
+      }
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Distributed Node Attributes is enabled"
+          + " with provider class as : "
+          + attributesProvider.getClass().toString());
+    }
+    return attributesProvider;
   }
 
   protected NodeLabelsProvider createNodeLabelsProvider(Configuration conf)
@@ -182,10 +217,10 @@ public class NodeManager extends CompositeService
       return provider;
     }
     switch (providerString.trim().toLowerCase()) {
-    case YarnConfiguration.CONFIG_NODE_LABELS_PROVIDER:
+    case YarnConfiguration.CONFIG_NODE_DESCRIPTOR_PROVIDER:
       provider = new ConfigurationNodeLabelsProvider();
       break;
-    case YarnConfiguration.SCRIPT_NODE_LABELS_PROVIDER:
+    case YarnConfiguration.SCRIPT_NODE_DESCRIPTOR_PROVIDER:
       provider = new ScriptBasedNodeLabelsProvider();
       break;
     default:
@@ -407,16 +442,19 @@ public class NodeManager extends CompositeService
     ((NMContext)context).setContainerExecutor(exec);
     ((NMContext)context).setDeletionService(del);
 
-    nodeLabelsProvider = createNodeLabelsProvider(conf);
+    nodeStatusUpdater =
+        createNodeStatusUpdater(context, dispatcher, nodeHealthChecker);
 
-    if (null == nodeLabelsProvider) {
-      nodeStatusUpdater =
-          createNodeStatusUpdater(context, dispatcher, nodeHealthChecker);
-    } else {
+    nodeLabelsProvider = createNodeLabelsProvider(conf);
+    if (nodeLabelsProvider != null) {
       addIfService(nodeLabelsProvider);
-      nodeStatusUpdater =
-          createNodeStatusUpdater(context, dispatcher, nodeHealthChecker,
-              nodeLabelsProvider);
+      nodeStatusUpdater.setNodeLabelsProvider(nodeLabelsProvider);
+    }
+
+    nodeAttributesProvider = createNodeAttributesProvider(conf);
+    if (nodeAttributesProvider != null) {
+      addIfService(nodeAttributesProvider);
+      nodeStatusUpdater.setNodeAttributesProvider(nodeAttributesProvider);
     }
 
     nodeResourceMonitor = createNodeResourceMonitor();

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

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


[22/50] [abbrv] hadoop git commit: HDDS-317. Use new StorageSize API for reading ozone.scm.container.size.gb. Contributed by Junjie Chen.

Posted by su...@apache.org.
HDDS-317. Use new StorageSize API for reading ozone.scm.container.size.gb. Contributed by Junjie Chen.


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

Branch: refs/heads/YARN-3409
Commit: 55b69310590723c5665f3e04589ef5e721d8261d
Parents: 96c4575
Author: Márton Elek <el...@apache.org>
Authored: Fri Aug 24 13:54:38 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Fri Aug 24 13:56:30 2018 +0200

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java    |  6 +++---
 hadoop-hdds/common/src/main/resources/ozone-default.xml   |  7 +++----
 .../hadoop/ozone/container/common/impl/ContainerData.java |  2 +-
 .../ozone/container/keyvalue/KeyValueContainerData.java   | 10 +++++++---
 .../hadoop/ozone/container/keyvalue/KeyValueHandler.java  |  7 ++++---
 .../apache/hadoop/hdds/scm/block/BlockManagerImpl.java    |  9 +++++----
 .../hadoop/hdds/scm/container/ContainerMapping.java       |  8 ++++----
 .../hadoop/hdds/scm/container/ContainerStateManager.java  |  9 +++++----
 .../hadoop/hdds/scm/pipelines/PipelineSelector.java       |  9 +++++----
 .../scm/container/TestCloseContainerEventHandler.java     |  8 ++++----
 .../hdds/scm/container/closer/TestContainerCloser.java    |  7 ++++---
 .../main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java  |  8 +++++---
 .../commandhandler/TestCloseContainerHandler.java         |  4 ++--
 .../commandhandler/TestReplicateContainerHandler.java     |  5 ++---
 14 files changed, 54 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
index e337d2f..2834883 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
@@ -215,9 +215,9 @@ public final class ScmConfigKeys {
       "ozone.scm.db.cache.size.mb";
   public static final int OZONE_SCM_DB_CACHE_SIZE_DEFAULT = 128;
 
-  public static final String OZONE_SCM_CONTAINER_SIZE_GB =
-      "ozone.scm.container.size.gb";
-  public static final int OZONE_SCM_CONTAINER_SIZE_DEFAULT = 5;
+  public static final String OZONE_SCM_CONTAINER_SIZE =
+      "ozone.scm.container.size";
+  public static final String OZONE_SCM_CONTAINER_SIZE_DEFAULT = "5GB";
 
   public static final String OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY =
       "ozone.scm.container.placement.impl";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/common/src/main/resources/ozone-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 568e38d..37a845e 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -611,12 +611,11 @@
     </description>
   </property>
   <property>
-    <name>ozone.scm.container.size.gb</name>
-    <value>5</value>
+    <name>ozone.scm.container.size</name>
+    <value>5GB</value>
     <tag>OZONE, PERFORMANCE, MANAGEMENT</tag>
     <description>
-      Default container size used by Ozone. This value is specified
-      in GB.
+      Default container size used by Ozone.
       There are two considerations while picking this number. The speed at which
       a container can be replicated, determined by the network speed and the
       metadata that each container generates. So selecting a large number

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/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 47894dc..afd1407 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
@@ -111,7 +111,7 @@ public abstract class ContainerData {
    * @param type - ContainerType
    * @param containerId - ContainerId
    * @param layOutVersion - Container layOutVersion
-   * @param size - Container maximum size
+   * @param size - Container maximum size in GB
    */
   protected ContainerData(ContainerType type, long containerId,
     int layOutVersion, int size) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/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 1d37437..e4cb5f3 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
@@ -21,6 +21,8 @@ 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.conf.StorageSize;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -82,7 +84,7 @@ public class KeyValueContainerData extends ContainerData {
   /**
    * Constructs KeyValueContainerData object.
    * @param id - ContainerId
-   * @param size - maximum size of the container
+   * @param size - maximum size in GB of the container
    */
   public KeyValueContainerData(long id, int size) {
     super(ContainerProtos.ContainerType.KeyValueContainer, id, size);
@@ -94,7 +96,7 @@ public class KeyValueContainerData extends ContainerData {
    * Constructs KeyValueContainerData object.
    * @param id - ContainerId
    * @param layOutVersion
-   * @param size - maximum size of the container
+   * @param size - maximum size in GB of the container
    */
   public KeyValueContainerData(long id, int layOutVersion, int size) {
     super(ContainerProtos.ContainerType.KeyValueContainer, id, layOutVersion,
@@ -266,9 +268,11 @@ public class KeyValueContainerData extends ContainerData {
   public static KeyValueContainerData getFromProtoBuf(
       ContainerProtos.ContainerData protoData) throws IOException {
     // TODO: Add containerMaxSize to ContainerProtos.ContainerData
+    StorageSize storageSize = StorageSize.parse(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
     KeyValueContainerData data = new KeyValueContainerData(
         protoData.getContainerID(),
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+        (int)storageSize.getUnit().toBytes(storageSize.getValue()));
     for (int x = 0; x < protoData.getMetadataCount(); x++) {
       data.addMetadata(protoData.getMetadata(x).getKey(),
           protoData.getMetadata(x).getValue());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/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 29c359e..8409561 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
@@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 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
@@ -149,9 +150,9 @@ public class KeyValueHandler extends Handler {
     volumeChoosingPolicy = ReflectionUtils.newInstance(conf.getClass(
         HDDS_DATANODE_VOLUME_CHOOSING_POLICY, RoundRobinVolumeChoosingPolicy
             .class, VolumeChoosingPolicy.class), conf);
-    maxContainerSizeGB = config.getInt(ScmConfigKeys
-            .OZONE_SCM_CONTAINER_SIZE_GB, ScmConfigKeys
-        .OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    maxContainerSizeGB = (int)config.getStorageSize(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+            ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.GB);
     // this handler lock is used for synchronizing createContainer Requests,
     // so using a fair lock here.
     handlerLock = new AutoCloseableLock(new ReentrantLock(true));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
index f3a111f..ca2a6a0 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
@@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.scm.block;
 
 import java.util.UUID;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.Mapping;
 import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
@@ -30,7 +31,6 @@ 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.metrics2.util.MBeans;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
@@ -98,9 +98,10 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
     this.nodeManager = nodeManager;
     this.containerManager = containerManager;
 
-    this.containerSize = OzoneConsts.GB * conf.getInt(
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    this.containerSize = (long)conf.getStorageSize(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT,
+        StorageUnit.BYTES);
 
     this.containerProvisionBatchSize =
         conf.getInt(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/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 4076dad..ba95764 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
@@ -20,6 +20,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.primitives.Longs;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.SCMContainerInfo;
@@ -66,7 +67,7 @@ import java.util.concurrent.locks.ReentrantLock;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
     .OZONE_SCM_CONTAINER_SIZE_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_SIZE_GB;
+    .OZONE_SCM_CONTAINER_SIZE;
 import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
     .FAILED_TO_CHANGE_CONTAINER_STATE;
 import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath;
@@ -129,9 +130,8 @@ public class ContainerMapping implements Mapping {
 
     this.lock = new ReentrantLock();
 
-    // 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;
+    size = (long)conf.getStorageSize(OZONE_SCM_CONTAINER_SIZE,
+        OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
     this.containerStateManager =
         new ContainerStateManager(conf, this);
     LOG.trace("Container State Manager created.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/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 5df7dc7..5eb8195 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdds.scm.container;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
@@ -35,7 +36,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
 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.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.statemachine
     .InvalidStateTransitionException;
 import org.apache.hadoop.ozone.common.statemachine.StateMachine;
@@ -148,9 +148,10 @@ public class ContainerStateManager implements Closeable {
         finalStates);
     initializeStateMachine();
 
-    this.containerSize = OzoneConsts.GB * configuration.getInt(
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    this.containerSize =(long)configuration.getStorageSize(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT,
+        StorageUnit.BYTES);
 
     lastUsedMap = new ConcurrentHashMap<>();
     containerCount = new AtomicLong(0);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/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 028d14b..5343bce 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
@@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.scm.pipelines;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
@@ -38,7 +39,6 @@ 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;
 import org.apache.hadoop.ozone.common.statemachine.StateMachine;
@@ -94,9 +94,10 @@ public class PipelineSelector {
     this.conf = conf;
     this.eventPublisher = eventPublisher;
     this.placementPolicy = createContainerPlacementPolicy(nodeManager, conf);
-    this.containerSize = OzoneConsts.GB * this.conf.getInt(
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    this.containerSize = (long)this.conf.getStorageSize(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT,
+        StorageUnit.BYTES);
     node2PipelineMap = new Node2PipelineMap();
     this.standaloneManager =
         new StandaloneManagerImpl(this.nodeManager, placementPolicy,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/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 543cad3..4790c82 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdds.scm.container;
 
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -39,7 +40,7 @@ import java.io.IOException;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATE;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATED;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;
 import static org.apache.hadoop.hdds.scm.events.SCMEvents.CLOSE_CONTAINER;
 import static org.apache.hadoop.hdds.scm.events.SCMEvents.DATANODE_COMMAND;
 
@@ -58,9 +59,8 @@ public class TestCloseContainerEventHandler {
   @BeforeClass
   public static void setUp() throws Exception {
     configuration = SCMTestUtils.getConf();
-    size = configuration
-        .getLong(OZONE_SCM_CONTAINER_SIZE_GB, OZONE_SCM_CONTAINER_SIZE_DEFAULT)
-        * 1024 * 1024 * 1024;
+    size = (long)configuration.getStorageSize(OZONE_SCM_CONTAINER_SIZE,
+        OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
     testDir = GenericTestUtils
         .getTestDir(TestCloseContainerEventHandler.class.getSimpleName());
     configuration

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/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 0c0f25d..608bb92 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
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdds.scm.container.closer;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.scm.TestUtils;
 import org.apache.hadoop.hdds.scm.container.ContainerMapping;
@@ -50,7 +51,7 @@ import static org.apache.hadoop.hdds.HddsConfigKeys
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
     .OZONE_SCM_CONTAINER_SIZE_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_SIZE_GB;
+    .OZONE_SCM_CONTAINER_SIZE;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent
     .CREATE;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent
@@ -71,8 +72,8 @@ public class TestContainerCloser {
   @BeforeClass
   public static void setUp() throws Exception {
     configuration = SCMTestUtils.getConf();
-    size = configuration.getLong(OZONE_SCM_CONTAINER_SIZE_GB,
-        OZONE_SCM_CONTAINER_SIZE_DEFAULT) * 1024 * 1024 * 1024;
+    size = (long)configuration.getStorageSize(OZONE_SCM_CONTAINER_SIZE,
+        OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
     configuration.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL,
         1, TimeUnit.SECONDS);
     testDir = GenericTestUtils

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
index 8d71d00..f54322c 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
@@ -23,6 +23,7 @@ import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
 import org.apache.hadoop.hdds.scm.cli.container.ContainerCommandHandler;
 import org.apache.hadoop.hdds.scm.cli.container.CreateContainerHandler;
@@ -49,7 +50,7 @@ import java.util.Arrays;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
     .OZONE_SCM_CONTAINER_SIZE_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_SIZE_GB;
+    .OZONE_SCM_CONTAINER_SIZE;
 import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
 import static org.apache.hadoop.hdds.scm.cli.ResultCode.EXECUTION_ERROR;
 import static org.apache.hadoop.hdds.scm.cli.ResultCode.SUCCESS;
@@ -107,8 +108,9 @@ public class SCMCLI extends OzoneBaseCLI {
         StorageContainerLocationProtocolPB.class);
     InetSocketAddress scmAddress =
         getScmAddressForClients(ozoneConf);
-    int containerSizeGB = ozoneConf.getInt(OZONE_SCM_CONTAINER_SIZE_GB,
-        OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    int containerSizeGB = (int)ozoneConf.getStorageSize(
+        OZONE_SCM_CONTAINER_SIZE, OZONE_SCM_CONTAINER_SIZE_DEFAULT,
+        StorageUnit.GB);
     ContainerOperationClient.setContainerSizeB(containerSizeGB*OzoneConsts.GB);
 
     RPC.setProtocolEngine(ozoneConf, StorageContainerLocationProtocolPB.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
index 3d39dbb..84b7b76 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Test;
@@ -52,7 +52,7 @@ public class TestCloseContainerHandler {
 
     //setup a cluster (1G free space is enough for a unit test)
     OzoneConfiguration conf = new OzoneConfiguration();
-    conf.set(OZONE_SCM_CONTAINER_SIZE_GB, "1");
+    conf.set(OZONE_SCM_CONTAINER_SIZE, "1GB");
     MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(1).build();
     cluster.waitForClusterToBeReady();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReplicateContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReplicateContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReplicateContainerHandler.java
index a5b101f..9e08212 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReplicateContainerHandler.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReplicateContainerHandler.java
@@ -23,14 +23,13 @@ import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand;
 import org.apache.hadoop.test.GenericTestUtils;
 
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_SIZE_GB;
+    .OZONE_SCM_CONTAINER_SIZE;
 import org.junit.Test;
 
 /**
@@ -47,7 +46,7 @@ public class TestReplicateContainerHandler {
         .captureLogs(ReplicateContainerCommandHandler.LOG);
 
     OzoneConfiguration conf = new OzoneConfiguration();
-    conf.set(OZONE_SCM_CONTAINER_SIZE_GB, "1");
+    conf.set(OZONE_SCM_CONTAINER_SIZE, "1GB");
     MiniOzoneCluster cluster =
         MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
     cluster.waitForClusterToBeReady();


---------------------------------------------------------------------
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: HDDS-364. Update open container replica information in SCM during DN register. Contributed by Ajay Kumar.

Posted by su...@apache.org.
HDDS-364. Update open container replica information in SCM during DN register. 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/a5eba255
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a5eba255
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a5eba255

Branch: refs/heads/YARN-3409
Commit: a5eba25506a4ca7ac9efa9b60b204c8cf1aa4160
Parents: 8563fd6
Author: Márton Elek <el...@apache.org>
Authored: Fri Aug 24 22:27:43 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Fri Aug 24 22:42:51 2018 +0200

----------------------------------------------------------------------
 .../hdds/scm/container/ContainerMapping.java    | 25 ++++++++++++++------
 .../scm/container/ContainerReportHandler.java   |  3 ++-
 .../hadoop/hdds/scm/container/Mapping.java      |  2 +-
 .../scm/server/SCMDatanodeProtocolServer.java   |  2 +-
 .../scm/container/TestContainerMapping.java     | 19 ++++++++++++---
 .../container/closer/TestContainerCloser.java   |  2 +-
 .../commandhandler/TestBlockDeletion.java       |  3 ++-
 7 files changed, 41 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/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 ba95764..b000bfd 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
@@ -505,15 +505,26 @@ public class ContainerMapping implements Mapping {
    */
   @Override
   public void processContainerReports(DatanodeDetails datanodeDetails,
-                                      ContainerReportsProto reports)
+      ContainerReportsProto reports, boolean isRegisterCall)
       throws IOException {
     List<StorageContainerDatanodeProtocolProtos.ContainerInfo>
         containerInfos = reports.getReportsList();
     PendingDeleteStatusList pendingDeleteStatusList =
         new PendingDeleteStatusList(datanodeDetails);
-    for (StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState :
+    for (StorageContainerDatanodeProtocolProtos.ContainerInfo contInfo :
         containerInfos) {
-      byte[] dbKey = Longs.toByteArray(datanodeState.getContainerID());
+      // Update replica info during registration process.
+      if (isRegisterCall) {
+        try {
+          getStateManager().addContainerReplica(ContainerID.
+              valueof(contInfo.getContainerID()), datanodeDetails);
+        } catch (Exception ex) {
+          // Continue to next one after logging the error.
+          LOG.error("Error while adding replica for containerId {}.",
+              contInfo.getContainerID(), ex);
+        }
+      }
+      byte[] dbKey = Longs.toByteArray(contInfo.getContainerID());
       lock.lock();
       try {
         byte[] containerBytes = containerStore.get(dbKey);
@@ -522,12 +533,12 @@ public class ContainerMapping implements Mapping {
               HddsProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes);
 
           HddsProtos.SCMContainerInfo newState =
-              reconcileState(datanodeState, knownState, datanodeDetails);
+              reconcileState(contInfo, knownState, datanodeDetails);
 
-          if (knownState.getDeleteTransactionId() > datanodeState
+          if (knownState.getDeleteTransactionId() > contInfo
               .getDeleteTransactionId()) {
             pendingDeleteStatusList
-                .addPendingDeleteStatus(datanodeState.getDeleteTransactionId(),
+                .addPendingDeleteStatus(contInfo.getDeleteTransactionId(),
                     knownState.getDeleteTransactionId(),
                     knownState.getContainerID());
           }
@@ -558,7 +569,7 @@ public class ContainerMapping implements Mapping {
           LOG.error("Error while processing container report from datanode :" +
                   " {}, for container: {}, reason: container doesn't exist in" +
                   "container database.", datanodeDetails,
-              datanodeState.getContainerID());
+              contInfo.getContainerID());
         }
       } finally {
         lock.unlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
index b26eed2..5a9e726 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
@@ -84,7 +84,8 @@ public class ContainerReportHandler implements
     try {
 
       //update state in container db and trigger close container events
-      containerMapping.processContainerReports(datanodeOrigin, containerReport);
+      containerMapping
+          .processContainerReports(datanodeOrigin, containerReport, false);
 
       Set<ContainerID> containerIds = containerReport.getReportsList().stream()
           .map(containerProto -> containerProto.getContainerID())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
index ac84be4..f4b5bb2 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
@@ -115,7 +115,7 @@ public interface Mapping extends Closeable {
    * @param reports Container report
    */
   void processContainerReports(DatanodeDetails datanodeDetails,
-                               ContainerReportsProto reports)
+      ContainerReportsProto reports, boolean isRegisterCall)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
index 0d34787..9215803 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
@@ -196,7 +196,7 @@ public class SCMDatanodeProtocolServer implements
     if (registeredCommand.getError()
         == SCMRegisteredResponseProto.ErrorCode.success) {
       scm.getScmContainerManager().processContainerReports(datanodeDetails,
-          containerReportsProto);
+          containerReportsProto, true);
     }
     return getRegisteredResponse(registeredCommand);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/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 1e9c35b..b0b39f1 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
@@ -242,7 +242,7 @@ public class TestContainerMapping {
   }
 
   @Test
-  public void testFullContainerReport() throws IOException {
+  public void testFullContainerReport() throws Exception {
     ContainerInfo info = createContainer();
     DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails();
     List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports =
@@ -266,13 +266,26 @@ public class TestContainerMapping {
         .newBuilder();
     crBuilder.addAllReports(reports);
 
-    mapping.processContainerReports(datanodeDetails, crBuilder.build());
+    mapping.processContainerReports(datanodeDetails, crBuilder.build(), false);
 
     ContainerInfo updatedContainer =
         mapping.getContainer(info.getContainerID());
     Assert.assertEquals(100000000L,
         updatedContainer.getNumberOfKeys());
     Assert.assertEquals(2000000000L, updatedContainer.getUsedBytes());
+
+    for (StorageContainerDatanodeProtocolProtos.ContainerInfo c : reports) {
+      LambdaTestUtils.intercept(SCMException.class, "No entry "
+          + "exist for containerId:", () -> mapping.getStateManager()
+          .getContainerReplicas(ContainerID.valueof(c.getContainerID())));
+    }
+
+    mapping.processContainerReports(TestUtils.randomDatanodeDetails(),
+        crBuilder.build(), true);
+    for (StorageContainerDatanodeProtocolProtos.ContainerInfo c : reports) {
+      Assert.assertTrue(mapping.getStateManager().getContainerReplicas(
+          ContainerID.valueof(c.getContainerID())).size() > 0);
+    }
   }
 
   @Test
@@ -301,7 +314,7 @@ public class TestContainerMapping {
         ContainerReportsProto.newBuilder();
     crBuilder.addAllReports(reports);
 
-    mapping.processContainerReports(datanodeDetails, crBuilder.build());
+    mapping.processContainerReports(datanodeDetails, crBuilder.build(), false);
 
     ContainerInfo updatedContainer =
         mapping.getContainer(info.getContainerID());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/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 608bb92..210df08 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
@@ -223,6 +223,6 @@ public class TestContainerCloser {
         .setDeleteTransactionId(0);
     reports.addReports(ciBuilder);
     mapping.processContainerReports(TestUtils.randomDatanodeDetails(),
-        reports.build());
+        reports.build(), false);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
index 94cdf61..5794151 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
@@ -187,7 +187,8 @@ public class TestBlockDeletion {
 
     logCapturer.clearOutput();
     scm.getScmContainerManager().processContainerReports(
-        cluster.getHddsDatanodes().get(0).getDatanodeDetails(), dummyReport);
+        cluster.getHddsDatanodes().get(0).getDatanodeDetails(), dummyReport,
+        false);
     // wait for event to be handled by event handler
     Thread.sleep(1000);
     String output = logCapturer.getOutput();


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

Branch: refs/heads/YARN-3409
Commit: 7c30fa0dc2b760bf32e6db916ea96b79d06dcc70
Parents: 807be17
Author: Naganarasimha <na...@apache.org>
Authored: Mon Feb 5 05:47:02 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:56 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/7c30fa0d/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 148edb9..15f8b3b 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,6 +3525,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";
 
@@ -3535,6 +3538,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";
 
@@ -3606,6 +3612,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/7c30fa0d/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 72e42d8..7278273 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
@@ -2894,6 +2894,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/7c30fa0d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index faf7adb..33096bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -932,7 +932,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
     @Override
     public Set<NodeLabel> getNodeLabelsForRegistration() {
-      Set<NodeLabel> nodeLabels = nodeLabelsProvider.getNodeLabels();
+      Set<NodeLabel> nodeLabels = nodeLabelsProvider.getDescriptors();
       nodeLabels = (null == nodeLabels)
           ? CommonNodeLabelsManager.EMPTY_NODELABEL_SET : nodeLabels;
       previousNodeLabels = nodeLabels;
@@ -967,7 +967,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     @Override
     public Set<NodeLabel> getNodeLabelsForHeartbeat() {
       Set<NodeLabel> nodeLabelsForHeartbeat =
-          nodeLabelsProvider.getNodeLabels();
+          nodeLabelsProvider.getDescriptors();
       // if the provider returns null then consider empty labels are set
       nodeLabelsForHeartbeat = (nodeLabelsForHeartbeat == null)
           ? CommonNodeLabelsManager.EMPTY_NODELABEL_SET

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c30fa0d/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/7c30fa0d/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/7c30fa0d/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/7c30fa0d/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/7c30fa0d/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/7c30fa0d/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/7c30fa0d/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/7c30fa0d/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/7c30fa0d/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/7c30fa0d/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/7c30fa0d/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/7c30fa0d/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/7c30fa0d/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/7c30fa0d/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: HDFS-13805. Journal Nodes should allow to format non-empty directories with -force option. Contributed by Surendra Singh Lilhore.

Posted by su...@apache.org.
HDFS-13805. Journal Nodes should allow to format non-empty directories with -force option. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/YARN-3409
Commit: 96c4575d7373079becfa3e3db29ba98e6fb86388
Parents: ca29fb7
Author: Surendra Singh Lilhore <su...@apache.org>
Authored: Fri Aug 24 08:14:57 2018 +0530
Committer: Surendra Singh Lilhore <su...@apache.org>
Committed: Fri Aug 24 08:14:57 2018 +0530

----------------------------------------------------------------------
 .../hadoop/hdfs/qjournal/client/AsyncLogger.java   |  3 ++-
 .../hdfs/qjournal/client/AsyncLoggerSet.java       |  4 ++--
 .../hdfs/qjournal/client/IPCLoggerChannel.java     |  5 +++--
 .../hdfs/qjournal/client/QuorumJournalManager.java |  4 ++--
 .../hdfs/qjournal/protocol/QJournalProtocol.java   |  2 +-
 .../QJournalProtocolServerSideTranslatorPB.java    |  2 +-
 .../protocolPB/QJournalProtocolTranslatorPB.java   |  6 ++++--
 .../hadoop/hdfs/qjournal/server/JNStorage.java     |  4 ++--
 .../hadoop/hdfs/qjournal/server/Journal.java       |  6 +++---
 .../hdfs/qjournal/server/JournalNodeRpcServer.java |  5 +++--
 .../hdfs/server/namenode/BackupJournalManager.java |  2 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java     |  5 +++--
 .../hadoop/hdfs/server/namenode/FSImage.java       |  5 +++--
 .../hadoop/hdfs/server/namenode/FSNamesystem.java  |  4 ++--
 .../hdfs/server/namenode/FileJournalManager.java   |  2 +-
 .../hdfs/server/namenode/JournalManager.java       |  2 +-
 .../hadoop/hdfs/server/namenode/JournalSet.java    |  2 +-
 .../hadoop/hdfs/server/namenode/NameNode.java      |  4 ++--
 .../src/main/proto/QJournalProtocol.proto          |  1 +
 .../hdfs/qjournal/client/TestEpochsAreUnique.java  |  2 +-
 .../hdfs/qjournal/client/TestQJMWithFaults.java    |  6 +++---
 .../qjournal/client/TestQuorumJournalManager.java  |  4 ++--
 .../client/TestQuorumJournalManagerUnit.java       |  4 +++-
 .../hadoop/hdfs/qjournal/server/TestJournal.java   | 17 ++++++++++++++---
 .../hdfs/qjournal/server/TestJournalNode.java      |  4 ++--
 .../qjournal/server/TestJournalNodeMXBean.java     |  2 +-
 .../hdfs/qjournal/server/TestJournalNodeSync.java  |  2 +-
 .../server/namenode/TestGenericJournalConf.java    |  2 +-
 .../tools/contract/AbstractContractDistCpTest.java |  2 +-
 29 files changed, 67 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
index d2b48cc..2633723 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
@@ -89,8 +89,9 @@ interface AsyncLogger {
   /**
    * Format the log directory.
    * @param nsInfo the namespace info to format with
+   * @param force the force option to format
    */
-  public ListenableFuture<Void> format(NamespaceInfo nsInfo);
+  public ListenableFuture<Void> format(NamespaceInfo nsInfo, boolean force);
 
   /**
    * @return whether or not the remote node has any valid data.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
index d46c2cf..b52e312 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
@@ -299,12 +299,12 @@ class AsyncLoggerSet {
     return QuorumCall.create(calls);
   }
 
-  QuorumCall<AsyncLogger,Void> format(NamespaceInfo nsInfo) {
+  QuorumCall<AsyncLogger, Void> format(NamespaceInfo nsInfo, boolean force) {
     Map<AsyncLogger, ListenableFuture<Void>> calls =
         Maps.newHashMap();
     for (AsyncLogger logger : loggers) {
       ListenableFuture<Void> future =
-          logger.format(nsInfo);
+          logger.format(nsInfo, force);
       calls.put(logger, future);
     }
     return QuorumCall.create(calls);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
index 3036735..4fca1bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
@@ -502,11 +502,12 @@ public class IPCLoggerChannel implements AsyncLogger {
   }
 
   @Override
-  public ListenableFuture<Void> format(final NamespaceInfo nsInfo) {
+  public ListenableFuture<Void> format(final NamespaceInfo nsInfo,
+      final boolean force) {
     return singleThreadExecutor.submit(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        getProxy().format(journalId, nameServiceId, nsInfo);
+        getProxy().format(journalId, nameServiceId, nsInfo, force);
         return null;
       }
     });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
index 4faaa98..bd45292 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
@@ -220,8 +220,8 @@ public class QuorumJournalManager implements JournalManager {
   }
   
   @Override
-  public void format(NamespaceInfo nsInfo) throws IOException {
-    QuorumCall<AsyncLogger,Void> call = loggers.format(nsInfo);
+  public void format(NamespaceInfo nsInfo, boolean force) throws IOException {
+    QuorumCall<AsyncLogger, Void> call = loggers.format(nsInfo, force);
     try {
       call.waitFor(loggers.size(), loggers.size(), 0, timeoutMs,
           "format");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
index 5558bd5..8dad261 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
@@ -68,7 +68,7 @@ public interface QJournalProtocol {
    * Format the underlying storage for the given namespace.
    */
   void format(String journalId, String nameServiceId,
-      NamespaceInfo nsInfo) throws IOException;
+      NamespaceInfo nsInfo, boolean force) throws IOException;
 
   /**
    * Begin a new epoch. See the HDFS-3077 design doc for details.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
index 865d296..2ad19da 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
@@ -147,7 +147,7 @@ public class QJournalProtocolServerSideTranslatorPB implements QJournalProtocolP
     try {
       impl.format(request.getJid().getIdentifier(),
           request.hasNameServiceId() ? request.getNameServiceId() : null,
-          PBHelper.convert(request.getNsInfo()));
+          PBHelper.convert(request.getNsInfo()), request.getForce());
       return FormatResponseProto.getDefaultInstance();
     } catch (IOException ioe) {
       throw new ServiceException(ioe);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
index d7cd7b5..42d35f5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
@@ -136,11 +136,13 @@ public class QJournalProtocolTranslatorPB implements ProtocolMetaInterface,
   @Override
   public void format(String jid,
                      String nameServiceId,
-                     NamespaceInfo nsInfo) throws IOException {
+                     NamespaceInfo nsInfo,
+                     boolean force) throws IOException {
     try {
       FormatRequestProto.Builder req = FormatRequestProto.newBuilder()
           .setJid(convertJournalId(jid))
-          .setNsInfo(PBHelper.convert(nsInfo));
+          .setNsInfo(PBHelper.convert(nsInfo))
+          .setForce(force);
       if(nameServiceId != null) {
         req.setNameServiceId(nameServiceId);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java
index 6bf4903..612fd3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java
@@ -204,10 +204,10 @@ class JNStorage extends Storage {
     }
   }
 
-  void format(NamespaceInfo nsInfo) throws IOException {
+  void format(NamespaceInfo nsInfo, boolean force) throws IOException {
     unlockAll();
     try {
-      sd.analyzeStorage(StartupOption.FORMAT, this, true);
+      sd.analyzeStorage(StartupOption.FORMAT, this, !force);
     } finally {
       sd.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
index 8f25d26..7e88afa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
@@ -227,13 +227,13 @@ public class Journal implements Closeable {
   /**
    * Format the local storage with the given namespace.
    */
-  void format(NamespaceInfo nsInfo) throws IOException {
+  void format(NamespaceInfo nsInfo, boolean force) throws IOException {
     Preconditions.checkState(nsInfo.getNamespaceID() != 0,
         "can't format with uninitialized namespace info: %s",
         nsInfo);
     LOG.info("Formatting journal id : " + journalId + " with namespace info: " +
-        nsInfo);
-    storage.format(nsInfo);
+        nsInfo + " and force: " + force);
+    storage.format(nsInfo, force);
     refreshCachedData();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
index b1a3c96..0f11026 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
@@ -176,9 +176,10 @@ public class JournalNodeRpcServer implements QJournalProtocol,
   @Override
   public void format(String journalId,
                      String nameServiceId,
-                     NamespaceInfo nsInfo)
+                     NamespaceInfo nsInfo,
+                     boolean force)
       throws IOException {
-    jn.getOrCreateJournal(journalId, nameServiceId).format(nsInfo);
+    jn.getOrCreateJournal(journalId, nameServiceId).format(nsInfo, force);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
index e1ddfb9..eac91bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
@@ -42,7 +42,7 @@ class BackupJournalManager implements JournalManager {
   }
 
   @Override
-  public void format(NamespaceInfo nsInfo) {
+  public void format(NamespaceInfo nsInfo, boolean force) {
     // format() should only get called at startup, before any BNs
     // can register with the NN.
     throw new UnsupportedOperationException(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index d6fb212..547ad57 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -417,13 +417,14 @@ public class FSEditLog implements LogsPurgeable {
    * File-based journals are skipped, since they are formatted by the
    * Storage format code.
    */
-  synchronized void formatNonFileJournals(NamespaceInfo nsInfo) throws IOException {
+  synchronized void formatNonFileJournals(NamespaceInfo nsInfo, boolean force)
+      throws IOException {
     Preconditions.checkState(state == State.BETWEEN_LOG_SEGMENTS,
         "Bad state: %s", state);
     
     for (JournalManager jm : journalSet.getJournalManagers()) {
       if (!(jm instanceof FileJournalManager)) {
-        jm.format(nsInfo);
+        jm.format(nsInfo, force);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
index 5cfc017..6d107be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
@@ -160,7 +160,8 @@ public class FSImage implements Closeable {
     archivalManager = new NNStorageRetentionManager(conf, storage, editLog);
   }
  
-  void format(FSNamesystem fsn, String clusterId) throws IOException {
+  void format(FSNamesystem fsn, String clusterId, boolean force)
+      throws IOException {
     long fileCount = fsn.getFilesTotal();
     // Expect 1 file, which is the root inode
     Preconditions.checkState(fileCount == 1,
@@ -171,7 +172,7 @@ public class FSImage implements Closeable {
     ns.clusterID = clusterId;
     
     storage.format(ns);
-    editLog.formatNonFileJournals(ns);
+    editLog.formatNonFileJournals(ns, force);
     saveFSImageInAllDirs(fsn, 0);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/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 b0fb26c..06bf008 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
@@ -1078,8 +1078,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     // format before starting up if requested
     if (startOpt == StartupOption.FORMAT) {
-      
-      fsImage.format(this, fsImage.getStorage().determineClusterId());// reuse current id
+      // reuse current id
+      fsImage.format(this, fsImage.getStorage().determineClusterId(), false);
 
       startOpt = StartupOption.REGULAR;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
index c71c09a..185ad73 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
@@ -100,7 +100,7 @@ public class FileJournalManager implements JournalManager {
   public void close() throws IOException {}
   
   @Override
-  public void format(NamespaceInfo ns) throws IOException {
+  public void format(NamespaceInfo ns, boolean force) throws IOException {
     // Formatting file journals is done by the StorageDirectory
     // format code, since they may share their directory with
     // checkpoints, etc.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
index ae1bc3b..d6d2094 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
@@ -43,7 +43,7 @@ public interface JournalManager extends Closeable, FormatConfirmable,
    * Format the underlying storage, removing any previously
    * stored data.
    */
-  void format(NamespaceInfo ns) throws IOException;
+  void format(NamespaceInfo ns, boolean force) throws IOException;
 
   /**
    * Begin writing to a new segment of the log stream, which starts at

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
index e7f2adb..868df01 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
@@ -188,7 +188,7 @@ public class JournalSet implements JournalManager {
   }
   
   @Override
-  public void format(NamespaceInfo nsInfo) throws IOException {
+  public void format(NamespaceInfo nsInfo, boolean force) throws IOException {
     // The operation is done by FSEditLog itself
     throw new UnsupportedOperationException();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 7f78d2f..a8034da 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -1159,7 +1159,7 @@ public class NameNode extends ReconfigurableBase implements
         return true; // aborted
       }
 
-      fsImage.format(fsn, clusterId);
+      fsImage.format(fsn, clusterId, force);
     } catch (IOException ioe) {
       LOG.warn("Encountered exception during format: ", ioe);
       fsImage.close();
@@ -1262,7 +1262,7 @@ public class NameNode extends ReconfigurableBase implements
       // actually want to save a checkpoint - just prime the dirs with
       // the existing namespace info
       newSharedStorage.format(nsInfo);
-      sharedEditsImage.getEditLog().formatNonFileJournals(nsInfo);
+      sharedEditsImage.getEditLog().formatNonFileJournals(nsInfo, force);
 
       // Need to make sure the edit log segments are in good shape to initialize
       // the shared edits dir.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
index a37c723..625966f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
@@ -242,6 +242,7 @@ message FormatRequestProto {
   required JournalIdProto jid = 1;
   required NamespaceInfoProto nsInfo = 2;
   optional string nameServiceId = 3;
+  optional bool force = 4 [ default = false ];
 }
 
 message FormatResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java
index 5101a41..0fc1429 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java
@@ -56,7 +56,7 @@ public class TestEpochsAreUnique {
     QuorumJournalManager qjm = new QuorumJournalManager(
         conf, uri, FAKE_NSINFO);
     try {
-      qjm.format(FAKE_NSINFO);
+      qjm.format(FAKE_NSINFO, false);
     } finally {
       qjm.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java
index 6ad43f5..40f213e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java
@@ -105,7 +105,7 @@ public class TestQJMWithFaults {
     long ret;
     try {
       qjm = createInjectableQJM(cluster);
-      qjm.format(FAKE_NSINFO);
+      qjm.format(FAKE_NSINFO, false);
       doWorkload(cluster, qjm);
       
       SortedSet<Integer> ipcCounts = Sets.newTreeSet();
@@ -156,7 +156,7 @@ public class TestQJMWithFaults {
         QuorumJournalManager qjm = null;
         try {
           qjm = createInjectableQJM(cluster);
-          qjm.format(FAKE_NSINFO);
+          qjm.format(FAKE_NSINFO, false);
           List<AsyncLogger> loggers = qjm.getLoggerSetForTests().getLoggersForTests();
           failIpcNumber(loggers.get(0), failA);
           failIpcNumber(loggers.get(1), failB);
@@ -240,7 +240,7 @@ public class TestQJMWithFaults {
     // Format the cluster using a non-faulty QJM.
     QuorumJournalManager qjmForInitialFormat =
         createInjectableQJM(cluster);
-    qjmForInitialFormat.format(FAKE_NSINFO);
+    qjmForInitialFormat.format(FAKE_NSINFO, false);
     qjmForInitialFormat.close();
     
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
index 69856ae..00bec22 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
@@ -100,7 +100,7 @@ public class TestQuorumJournalManager {
     qjm = createSpyingQJM();
     spies = qjm.getLoggerSetForTests().getLoggersForTests();
 
-    qjm.format(QJMTestUtil.FAKE_NSINFO);
+    qjm.format(QJMTestUtil.FAKE_NSINFO, false);
     qjm.recoverUnfinalizedSegments();
     assertEquals(1, qjm.getLoggerSetForTests().getEpoch());
   }
@@ -149,7 +149,7 @@ public class TestQuorumJournalManager {
     QuorumJournalManager qjm = closeLater(new QuorumJournalManager(
         conf, cluster.getQuorumJournalURI("testFormat-jid"), FAKE_NSINFO));
     assertFalse(qjm.hasSomeData());
-    qjm.format(FAKE_NSINFO);
+    qjm.format(FAKE_NSINFO, false);
     assertTrue(qjm.hasSomeData());
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
index 75dcf2f..9e1e3bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.qjournal.client;
 
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.eq;
 
 import java.io.IOException;
@@ -89,7 +90,8 @@ public class TestQuorumJournalManagerUnit {
           NewEpochResponseProto.newBuilder().build()
           ).when(logger).newEpoch(Mockito.anyLong());
       
-      futureReturns(null).when(logger).format(Mockito.<NamespaceInfo>any());
+      futureReturns(null).when(logger).format(Mockito.<NamespaceInfo>any(),
+          anyBoolean());
     }
     
     qjm.recoverUnfinalizedSegments();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
index b71d694..b8d2652 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
@@ -73,7 +73,7 @@ public class TestJournal {
     conf = new Configuration();
     journal = new Journal(conf, TEST_LOG_DIR, JID, StartupOption.REGULAR,
       mockErrorReporter);
-    journal.format(FAKE_NSINFO);
+    journal.format(FAKE_NSINFO, false);
   }
   
   @After
@@ -207,7 +207,7 @@ public class TestJournal {
     // Clear the storage directory before reformatting it
     journal.getStorage().getJournalManager()
         .getStorageDirectory().clearDirectory();
-    journal.format(FAKE_NSINFO_2);
+    journal.format(FAKE_NSINFO_2, false);
     
     assertEquals(0, journal.getLastPromisedEpoch());
     assertEquals(0, journal.getLastWriterEpoch());
@@ -425,7 +425,7 @@ public class TestJournal {
     try {
       // Format again here and to format the non-empty directories in
       // journal node.
-      journal.format(FAKE_NSINFO);
+      journal.format(FAKE_NSINFO, false);
       fail("Did not fail to format non-empty directories in journal node.");
     } catch (IOException ioe) {
       GenericTestUtils.assertExceptionContains(
@@ -434,4 +434,15 @@ public class TestJournal {
     }
   }
 
+  @Test
+  public void testFormatNonEmptyStorageDirectoriesWhenforceOptionIsTrue()
+      throws Exception {
+    try {
+      // Format again here and to format the non-empty directories in
+      // journal node.
+      journal.format(FAKE_NSINFO, true);
+    } catch (IOException ioe) {
+      fail("Format should be success with force option.");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
index 8d58792..4cc5968 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
@@ -159,11 +159,11 @@ public class TestJournalNode {
             HdfsServerConstants.StartupOption.REGULAR);
         NamespaceInfo fakeNameSpaceInfo = new NamespaceInfo(
             12345, "mycluster", "my-bp"+nsId, 0L);
-        journal.format(fakeNameSpaceInfo);
+        journal.format(fakeNameSpaceInfo, false);
       }
     } else {
       journal = jn.getOrCreateJournal(journalId);
-      journal.format(FAKE_NSINFO);
+      journal.format(FAKE_NSINFO, false);
     }
 
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java
index 1de37a4..7550c4e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java
@@ -82,7 +82,7 @@ public class TestJournalNodeMXBean {
     // format the journal ns1
     final NamespaceInfo FAKE_NSINFO = new NamespaceInfo(12345, "mycluster",
         "my-bp", 0L);
-    jn.getOrCreateJournal(NAMESERVICE).format(FAKE_NSINFO);
+    jn.getOrCreateJournal(NAMESERVICE).format(FAKE_NSINFO, false);
 
     // check again after format
     // getJournalsStatus

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java
index 8de9641..c23604b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java
@@ -341,7 +341,7 @@ public class TestJournalNodeSync {
     }
 
     // Format the JN
-    journal1.format(nsInfo);
+    journal1.format(nsInfo, false);
 
     // Roll some more edits
     for (int i = 4; i < 10; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
index 020ecb5..edcf9e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
@@ -155,7 +155,7 @@ public class TestGenericJournalConf {
     }
     
     @Override
-    public void format(NamespaceInfo nsInfo) throws IOException {
+    public void format(NamespaceInfo nsInfo, boolean force) throws IOException {
       formatCalled = true;
     }
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java
index 5c74430..0757a66 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java
@@ -249,7 +249,7 @@ public abstract class AbstractContractDistCpTest
     Counter c = job.getCounters().findCounter(counter);
     long value = c.getValue();
     String description =
-        String.format("%s value %s", c.getDisplayName(), value);
+        String.format("%s value %s", c.getDisplayName(), value, false);
 
     if (min >= 0) {
       assertTrue(description + " too below minimum " + min,


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

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


[08/50] [abbrv] hadoop git commit: YARN-7494. Add muti-node lookup mechanism and pluggable nodes sorting policies to optimize placement decision. Contributed by Sunil Govindan.

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceUsageMultiNodeLookupPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceUsageMultiNodeLookupPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceUsageMultiNodeLookupPolicy.java
new file mode 100644
index 0000000..d765af8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceUsageMultiNodeLookupPolicy.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.yarn.server.resourcemanager.scheduler.placement;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+
+import java.util.Comparator;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+
+/**
+ * <p>
+ * This class has the following functionality:
+ *
+ * <p>
+ * ResourceUsageMultiNodeLookupPolicy holds sorted nodes list based on the
+ * resource usage of nodes at given time.
+ * </p>
+ */
+public class ResourceUsageMultiNodeLookupPolicy<N extends SchedulerNode>
+    implements MultiNodeLookupPolicy<N> {
+
+  protected Map<String, Set<N>> nodesPerPartition = new ConcurrentHashMap<>();
+  protected Comparator<N> comparator;
+
+  public ResourceUsageMultiNodeLookupPolicy() {
+    this.comparator = new Comparator<N>() {
+      @Override
+      public int compare(N o1, N o2) {
+        int allocatedDiff = o1.getAllocatedResource()
+            .compareTo(o2.getAllocatedResource());
+        if (allocatedDiff == 0) {
+          return o1.getNodeID().compareTo(o2.getNodeID());
+        }
+        return allocatedDiff;
+      }
+    };
+  }
+
+  @Override
+  public Iterator<N> getPreferredNodeIterator(Collection<N> nodes,
+      String partition) {
+    return getNodesPerPartition(partition).iterator();
+  }
+
+  @Override
+  public void addAndRefreshNodesSet(Collection<N> nodes,
+      String partition) {
+    Set<N> nodeList = new ConcurrentSkipListSet<N>(comparator);
+    nodeList.addAll(nodes);
+    nodesPerPartition.put(partition, Collections.unmodifiableSet(nodeList));
+  }
+
+  @Override
+  public Set<N> getNodesPerPartition(String partition) {
+    return nodesPerPartition.getOrDefault(partition, Collections.emptySet());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.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/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
index eef86a4..09d3327 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -295,6 +296,8 @@ public class ReservationSystemTestUtil {
         });
 
     mockRmContext.setNodeLabelManager(nlm);
+    mockRmContext
+        .setMultiNodeSortingManager(mock(MultiNodeSortingManager.class));
     return mockRmContext;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.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/TestAppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
index b7b0eb7..df8309b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSLeafQueue;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
@@ -118,7 +119,7 @@ public class TestAppSchedulingInfo {
     doReturn(mock(QueueMetrics.class)).when(queue).getMetrics();
     AppSchedulingInfo  info = new AppSchedulingInfo(
         appAttemptId, "test", queue, mock(ActiveUsersManager.class), 0,
-        new ResourceUsage(), new HashMap<>(), null);
+        new ResourceUsage(), new HashMap<>(), mock(RMContext.class));
     Assert.assertEquals(0, info.getSchedulerKeys().size());
 
     Priority pri1 = Priority.newInstance(1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerTestBase.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/CapacitySchedulerTestBase.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/CapacitySchedulerTestBase.java
index 5cea3a2..60e25ed 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/CapacitySchedulerTestBase.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/CapacitySchedulerTestBase.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import com.google.common.collect.Sets;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.junit.Assert;
 
 import java.util.Set;
@@ -76,4 +77,16 @@ public class CapacitySchedulerTestBase {
         .getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label)
         .getMemorySize() > 0);
   }
+
+  protected void waitforNMRegistered(ResourceScheduler scheduler, int nodecount,
+      int timesec) throws InterruptedException {
+    long start = System.currentTimeMillis();
+    while (System.currentTimeMillis() - start < timesec * 1000) {
+      if (scheduler.getNumClusterNodes() < nodecount) {
+        Thread.sleep(100);
+      } else {
+        break;
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/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 8d948b5..e77d8e2 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
@@ -106,8 +106,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyConta
 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.placement
-    .UserGroupMappingPlacementRule;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
@@ -172,7 +170,6 @@ import org.mockito.Mockito;
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -4871,18 +4868,6 @@ public class TestCapacityScheduler extends CapacitySchedulerTestBase {
     return cs;
   }
 
-  private void waitforNMRegistered(ResourceScheduler scheduler, int nodecount,
-      int timesec) throws InterruptedException {
-    long start = System.currentTimeMillis();
-    while (System.currentTimeMillis() - start < timesec * 1000) {
-      if (scheduler.getNumClusterNodes() < nodecount) {
-        Thread.sleep(100);
-      } else {
-        break;
-      }
-    }
-  }
-
   @Test (timeout = 60000)
   public void testClearRequestsBeforeApplyTheProposal()
       throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerMultiNodes.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/TestCapacitySchedulerMultiNodes.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/TestCapacitySchedulerMultiNodes.java
new file mode 100644
index 0000000..c90af94
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerMultiNodes.java
@@ -0,0 +1,166 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.NodeId;
+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.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSorter;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
+import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test class for Multi Node scheduling related tests.
+ */
+public class TestCapacitySchedulerMultiNodes extends CapacitySchedulerTestBase {
+
+  private static final Log LOG = LogFactory
+      .getLog(TestCapacitySchedulerMultiNodes.class);
+  private CapacitySchedulerConfiguration conf;
+  private static final String POLICY_CLASS_NAME =
+      "org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.ResourceUsageMultiNodeLookupPolicy";
+
+  @Before
+  public void setUp() {
+    CapacitySchedulerConfiguration config =
+        new CapacitySchedulerConfiguration();
+    config.set(CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS,
+        DominantResourceCalculator.class.getName());
+    conf = new CapacitySchedulerConfiguration(config);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.set(CapacitySchedulerConfiguration.MULTI_NODE_SORTING_POLICIES,
+        "resource-based");
+    conf.set(CapacitySchedulerConfiguration.MULTI_NODE_SORTING_POLICY_NAME,
+        "resource-based");
+    String policyName =
+        CapacitySchedulerConfiguration.MULTI_NODE_SORTING_POLICY_NAME
+            + ".resource-based" + ".class";
+    conf.set(policyName, POLICY_CLASS_NAME);
+    conf.setBoolean(CapacitySchedulerConfiguration.MULTI_NODE_PLACEMENT_ENABLED,
+        true);
+    conf.setInt("yarn.scheduler.minimum-allocation-mb", 512);
+    conf.setInt("yarn.scheduler.minimum-allocation-vcores", 1);
+  }
+
+  @Test
+  public void testMultiNodeSorterForScheduling() throws Exception {
+    MockRM rm = new MockRM(conf);
+    rm.start();
+    rm.registerNode("127.0.0.1:1234", 10 * GB);
+    rm.registerNode("127.0.0.1:1235", 10 * GB);
+    rm.registerNode("127.0.0.1:1236", 10 * GB);
+    rm.registerNode("127.0.0.1:1237", 10 * GB);
+    ResourceScheduler scheduler = rm.getRMContext().getScheduler();
+    waitforNMRegistered(scheduler, 4, 5);
+    MultiNodeSortingManager<SchedulerNode> mns = rm.getRMContext()
+        .getMultiNodeSortingManager();
+    MultiNodeSorter<SchedulerNode> sorter = mns
+        .getMultiNodePolicy(POLICY_CLASS_NAME);
+    sorter.reSortClusterNodes();
+    Set<SchedulerNode> nodes = sorter.getMultiNodeLookupPolicy()
+        .getNodesPerPartition("");
+    Assert.assertEquals(4, nodes.size());
+    rm.stop();
+  }
+
+  @Test
+  public void testMultiNodeSorterForSchedulingWithOrdering() throws Exception {
+    MockRM rm = new MockRM(conf);
+    rm.start();
+    MockNM nm1 = rm.registerNode("127.0.0.1:1234", 10 * GB, 10);
+    MockNM nm2 = rm.registerNode("127.0.0.2:1235", 10 * GB, 10);
+    MockNM nm3 = rm.registerNode("127.0.0.3:1236", 10 * GB, 10);
+    MockNM nm4 = rm.registerNode("127.0.0.4:1237", 10 * GB, 10);
+    ResourceScheduler scheduler = rm.getRMContext().getScheduler();
+    waitforNMRegistered(scheduler, 4, 5);
+
+    MultiNodeSortingManager<SchedulerNode> mns = rm.getRMContext()
+        .getMultiNodeSortingManager();
+    MultiNodeSorter<SchedulerNode> sorter = mns
+        .getMultiNodePolicy(POLICY_CLASS_NAME);
+    sorter.reSortClusterNodes();
+
+    Set<SchedulerNode> nodes = sorter.getMultiNodeLookupPolicy()
+        .getNodesPerPartition("");
+    Assert.assertEquals(4, nodes.size());
+
+    RMApp app1 = rm.submitApp(2048, "app-1", "user1", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
+    SchedulerNodeReport reportNm1 =
+        rm.getResourceScheduler().getNodeReport(nm1.getNodeId());
+
+    // check node report
+    Assert.assertEquals(2 * GB, reportNm1.getUsedResource().getMemorySize());
+    Assert.assertEquals(8 * GB,
+        reportNm1.getAvailableResource().getMemorySize());
+
+    // Ideally thread will invoke this, but thread operates every 1sec.
+    // Hence forcefully recompute nodes.
+    sorter.reSortClusterNodes();
+
+    RMApp app2 = rm.submitApp(1024, "app-2", "user2", null, "default");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
+    SchedulerNodeReport reportNm2 =
+        rm.getResourceScheduler().getNodeReport(nm2.getNodeId());
+
+    // check node report
+    Assert.assertEquals(1 * GB, reportNm2.getUsedResource().getMemorySize());
+    Assert.assertEquals(9 * GB,
+        reportNm2.getAvailableResource().getMemorySize());
+
+    // Ideally thread will invoke this, but thread operates every 1sec.
+    // Hence forcefully recompute nodes.
+    sorter.reSortClusterNodes();
+
+    // Node1 and Node2 are now having used resources. Hence ensure these 2 comes
+    // latter in the list.
+    nodes = sorter.getMultiNodeLookupPolicy()
+        .getNodesPerPartition("");
+    List<NodeId> currentNodes = new ArrayList<>();
+    currentNodes.add(nm3.getNodeId());
+    currentNodes.add(nm4.getNodeId());
+    currentNodes.add(nm2.getNodeId());
+    currentNodes.add(nm1.getNodeId());
+    Iterator<SchedulerNode> it = nodes.iterator();
+    SchedulerNode current;
+    int i = 0;
+    while (it.hasNext()) {
+      current = it.next();
+      Assert.assertEquals(current.getNodeID(), currentNodes.get(i++));
+    }
+    rm.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.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/TestCapacitySchedulerNodeLabelUpdate.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/TestCapacitySchedulerNodeLabelUpdate.java
index b4ebd15..e239191 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/TestCapacitySchedulerNodeLabelUpdate.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/TestCapacitySchedulerNodeLabelUpdate.java
@@ -817,4 +817,74 @@ public class TestCapacitySchedulerNodeLabelUpdate {
     }
     return memorySize;
   }
+
+  private long waitForNodeLabelSchedulerEventUpdate(MockRM rm, String partition,
+      long expectedNodeCount, long timeout) throws InterruptedException {
+    long start = System.currentTimeMillis();
+    long size = 0;
+    while (System.currentTimeMillis() - start < timeout) {
+      CapacityScheduler scheduler = (CapacityScheduler) rm
+          .getResourceScheduler();
+      size = scheduler.getNodeTracker().getNodesPerPartition(partition).size();
+      if (size == expectedNodeCount) {
+        return size;
+      }
+      Thread.sleep(100);
+    }
+    return size;
+  }
+
+  @Test
+  public void testNodeCountBasedOnNodeLabelsFromClusterNodeTracker()
+      throws Exception {
+    // set node -> label
+    mgr.addToCluserNodeLabelsWithDefaultExclusivity(
+        ImmutableSet.of("x", "y", "z"));
+
+    // set mapping:
+    // h1 -> x
+    // h2 -> y
+    mgr.addLabelsToNode(
+        ImmutableMap.of(NodeId.newInstance("h1", 1234), toSet("x")));
+    mgr.addLabelsToNode(
+        ImmutableMap.of(NodeId.newInstance("h2", 1234), toSet("x")));
+
+    // inject node label manager
+    MockRM rm = new MockRM(getConfigurationWithQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm.getRMContext().setNodeLabelManager(mgr);
+    rm.start();
+    MockNM nm1 = rm.registerNode("h1:1234", 8000);
+    rm.registerNode("h2:1234", 8000);
+    rm.registerNode("h3:1234", 8000);
+
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    // Ensure that cluster node tracker is updated with correct set of node
+    // after Node registration.
+    Assert.assertEquals(2,
+        cs.getNodeTracker().getNodesPerPartition("x").size());
+    Assert.assertEquals(1, cs.getNodeTracker().getNodesPerPartition("").size());
+
+    rm.unRegisterNode(nm1);
+    rm.registerNode("h4:1234", 8000);
+
+    // Ensure that cluster node tracker is updated with correct set of node
+    // after new Node registration and old node label change.
+    Assert.assertEquals(1,
+        cs.getNodeTracker().getNodesPerPartition("x").size());
+    Assert.assertEquals(2, cs.getNodeTracker().getNodesPerPartition("").size());
+
+    mgr.replaceLabelsOnNode(
+        ImmutableMap.of(NodeId.newInstance("h2", 1234), toSet("")));
+
+    // Last node with label x is replaced by CLI or REST.
+    Assert.assertEquals(0,
+        waitForNodeLabelSchedulerEventUpdate(rm, "x", 0, 3000L));
+  }
 }


---------------------------------------------------------------------
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: HDFS-13772. Erasure coding: Unnecessary NameNode Logs displaying for Enabling/Disabling Erasure coding policies which are already enabled/disabled. Contributed by Ayush Saxena

Posted by su...@apache.org.
HDFS-13772. Erasure coding: Unnecessary NameNode Logs displaying for Enabling/Disabling Erasure coding policies which are already enabled/disabled. Contributed by Ayush Saxena


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

Branch: refs/heads/YARN-3409
Commit: 770d9d9bb686bacef601ec8c4f884dc5afa9d4e2
Parents: 34577d2
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Aug 21 09:33:19 2018 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue Aug 21 09:33:19 2018 +0530

----------------------------------------------------------------------
 .../namenode/ErasureCodingPolicyManager.java    | 15 +++++++----
 .../server/namenode/FSDirErasureCodingOp.java   | 22 ++++++++++++-----
 .../hdfs/server/namenode/FSNamesystem.java      | 26 +++++++++++---------
 .../hdfs/server/namenode/NameNodeRpcServer.java |  8 +++---
 .../server/namenode/TestNamenodeRetryCache.java |  2 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |  2 +-
 6 files changed, 47 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/770d9d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index 3a310da..e7de05b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -356,7 +356,7 @@ public final class ErasureCodingPolicyManager {
   /**
    * Disable an erasure coding policy by policyName.
    */
-  public synchronized void disablePolicy(String name) {
+  public synchronized boolean disablePolicy(String name) {
     ErasureCodingPolicyInfo info = policiesByName.get(name);
     if (info == null) {
       throw new HadoopIllegalArgumentException("The policy name " +
@@ -367,27 +367,32 @@ public final class ErasureCodingPolicyManager {
       enabledPoliciesByName.remove(name);
       enabledPolicies =
           enabledPoliciesByName.values().toArray(new ErasureCodingPolicy[0]);
+      info.setState(ErasureCodingPolicyState.DISABLED);
+      LOG.info("Disable the erasure coding policy " + name);
+      return true;
     }
-    info.setState(ErasureCodingPolicyState.DISABLED);
-    LOG.info("Disable the erasure coding policy " + name);
+    return false;
   }
 
   /**
    * Enable an erasure coding policy by policyName.
    */
-  public synchronized void enablePolicy(String name) {
+  public synchronized boolean enablePolicy(String name) {
     final ErasureCodingPolicyInfo info = policiesByName.get(name);
     if (info == null) {
       throw new HadoopIllegalArgumentException("The policy name " +
           name + " does not exist");
     }
-
+    if (enabledPoliciesByName.containsKey(name)) {
+      return false;
+    }
     final ErasureCodingPolicy ecPolicy = info.getPolicy();
     enabledPoliciesByName.put(name, ecPolicy);
     info.setState(ErasureCodingPolicyState.ENABLED);
     enabledPolicies =
         enabledPoliciesByName.values().toArray(new ErasureCodingPolicy[0]);
     LOG.info("Enable the erasure coding policy " + name);
+    return true;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/770d9d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
index 769c137..2ba840e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
@@ -253,11 +253,16 @@ final class FSDirErasureCodingOp {
    *                      rebuilding
    * @throws IOException
    */
-  static void enableErasureCodingPolicy(final FSNamesystem fsn,
+  static boolean enableErasureCodingPolicy(final FSNamesystem fsn,
       String ecPolicyName, final boolean logRetryCache) throws IOException {
     Preconditions.checkNotNull(ecPolicyName);
-    fsn.getErasureCodingPolicyManager().enablePolicy(ecPolicyName);
-    fsn.getEditLog().logEnableErasureCodingPolicy(ecPolicyName, logRetryCache);
+    boolean success =
+        fsn.getErasureCodingPolicyManager().enablePolicy(ecPolicyName);
+    if (success) {
+      fsn.getEditLog().logEnableErasureCodingPolicy(ecPolicyName,
+          logRetryCache);
+    }
+    return success;
   }
 
   /**
@@ -269,11 +274,16 @@ final class FSDirErasureCodingOp {
    *                      rebuilding
    * @throws IOException
    */
-  static void disableErasureCodingPolicy(final FSNamesystem fsn,
+  static boolean disableErasureCodingPolicy(final FSNamesystem fsn,
       String ecPolicyName, final boolean logRetryCache) throws IOException {
     Preconditions.checkNotNull(ecPolicyName);
-    fsn.getErasureCodingPolicyManager().disablePolicy(ecPolicyName);
-    fsn.getEditLog().logDisableErasureCodingPolicy(ecPolicyName, logRetryCache);
+    boolean success =
+        fsn.getErasureCodingPolicyManager().disablePolicy(ecPolicyName);
+    if (success) {
+      fsn.getEditLog().logDisableErasureCodingPolicy(ecPolicyName,
+          logRetryCache);
+    }
+    return success;
   }
 
   private static List<XAttr> removeErasureCodingPolicyXAttr(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/770d9d9b/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 cdd7d48..b0fb26c 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
@@ -7604,29 +7604,31 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param ecPolicyName the name of the policy to be enabled
    * @param logRetryCache whether to record RPC ids in editlog for retry cache
    *                      rebuilding
+   * @return
    * @throws IOException
    */
-  void enableErasureCodingPolicy(String ecPolicyName,
+  boolean enableErasureCodingPolicy(String ecPolicyName,
       final boolean logRetryCache) throws IOException {
     final String operationName = "enableErasureCodingPolicy";
     checkOperation(OperationCategory.WRITE);
     boolean success = false;
-    LOG.info("Enable the erasure coding policy " + ecPolicyName);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot enable erasure coding policy "
           + ecPolicyName);
-      FSDirErasureCodingOp.enableErasureCodingPolicy(this, ecPolicyName,
-          logRetryCache);
-      success = true;
+      success = FSDirErasureCodingOp.enableErasureCodingPolicy(this,
+          ecPolicyName, logRetryCache);
+    } catch (AccessControlException ace) {
+      logAuditEvent(false, operationName, ecPolicyName, null, null);
     } finally {
       writeUnlock(operationName);
       if (success) {
         getEditLog().logSync();
+        logAuditEvent(success, operationName, ecPolicyName, null, null);
       }
-      logAuditEvent(success, operationName, ecPolicyName, null, null);
     }
+    return success;
   }
 
   /**
@@ -7636,7 +7638,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    *                      rebuilding
    * @throws IOException
    */
-  void disableErasureCodingPolicy(String ecPolicyName,
+  boolean disableErasureCodingPolicy(String ecPolicyName,
       final boolean logRetryCache) throws IOException {
     final String operationName = "disableErasureCodingPolicy";
     checkOperation(OperationCategory.WRITE);
@@ -7647,16 +7649,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot disable erasure coding policy "
           + ecPolicyName);
-      FSDirErasureCodingOp.disableErasureCodingPolicy(this, ecPolicyName,
-          logRetryCache);
-      success = true;
+      success = FSDirErasureCodingOp.disableErasureCodingPolicy(this,
+          ecPolicyName, logRetryCache);
+    } catch (AccessControlException ace) {
+      logAuditEvent(false, operationName, ecPolicyName, null, null);
     } finally {
       writeUnlock(operationName);
       if (success) {
         getEditLog().logSync();
+        logAuditEvent(success, operationName, ecPolicyName, null, null);
       }
-      logAuditEvent(success, operationName, ecPolicyName, null, null);
     }
+    return success;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/770d9d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 318f801..ec5ce9d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -2477,8 +2477,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     }
     boolean success = false;
     try {
-      namesystem.enableErasureCodingPolicy(ecPolicyName, cacheEntry != null);
-      success = true;
+      success = namesystem.enableErasureCodingPolicy(ecPolicyName,
+          cacheEntry != null);
     } finally {
       RetryCache.setState(cacheEntry, success);
     }
@@ -2495,8 +2495,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     }
     boolean success = false;
     try {
-      namesystem.disableErasureCodingPolicy(ecPolicyName, cacheEntry != null);
-      success = true;
+      success = namesystem.disableErasureCodingPolicy(ecPolicyName,
+          cacheEntry != null);
     } finally {
       RetryCache.setState(cacheEntry, success);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/770d9d9b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
index 0995f13..a1946c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
@@ -463,7 +463,7 @@ public class TestNamenodeRetryCache {
     assertTrue(namesystem.hasRetryCache());
     cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) namesystem
         .getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 39, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 38, cacheSet.size());
     iter = cacheSet.iterator();
     while (iter.hasNext()) {
       CacheEntry entry = iter.next();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/770d9d9b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
index 43aaa92..bcdf511 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
@@ -194,7 +194,7 @@ public class TestRetryCacheWithHA {
     FSNamesystem fsn1 = cluster.getNamesystem(1);
     cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) fsn1
         .getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 39, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 38, cacheSet.size());
     iter = cacheSet.iterator();
     while (iter.hasNext()) {
       CacheEntry entry = iter.next();


---------------------------------------------------------------------
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: YARN-8129. Improve error message for invalid value in fields attribute. Contributed by Abhishek Modi.

Posted by su...@apache.org.
YARN-8129. Improve error message for invalid value in fields attribute. Contributed by Abhishek Modi.


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

Branch: refs/heads/YARN-3409
Commit: d3fef7a5c5b83d27e87b5e49928254a7d1b935e5
Parents: 770d9d9
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Tue Aug 21 11:58:07 2018 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Tue Aug 21 11:58:07 2018 +0530

----------------------------------------------------------------------
 .../timelineservice/reader/TimelineReaderWebServicesUtils.java | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fef7a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
index efaecd2..63529a4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
@@ -214,7 +214,11 @@ public final class TimelineReaderWebServicesUtils {
     String[] strs = str.split(delimiter);
     EnumSet<Field> fieldList = EnumSet.noneOf(Field.class);
     for (String s : strs) {
-      fieldList.add(Field.valueOf(s.trim().toUpperCase()));
+      try {
+        fieldList.add(Field.valueOf(s.trim().toUpperCase()));
+      } catch (IllegalArgumentException e) {
+        throw new IllegalArgumentException(s + " is not a valid field.");
+      }
     }
     return fieldList;
   }


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

Branch: refs/heads/YARN-3409
Commit: 743de584e1acc0b71be613f2909b903a9328c56a
Parents: 140f22b
Author: bibinchundatt <bi...@apache.org>
Authored: Sun Apr 1 19:24:00 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:57 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/743de584/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/743de584/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/743de584/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/743de584/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/743de584/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


[02/50] [abbrv] hadoop git commit: YARN-8673. [AMRMProxy] More robust responseId resync after an YarnRM master slave switch. Contributed by Botong Huang.

Posted by su...@apache.org.
YARN-8673. [AMRMProxy] More robust responseId resync after an YarnRM master slave switch. Contributed by Botong Huang.


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

Branch: refs/heads/YARN-3409
Commit: 8736fc39ac3b3de168d2c216f3d1c0edb48fb3f9
Parents: 65e7469
Author: Giovanni Matteo Fumarola <gi...@apache.com>
Authored: Mon Aug 20 12:22:36 2018 -0700
Committer: Giovanni Matteo Fumarola <gi...@apache.com>
Committed: Mon Aug 20 12:22:36 2018 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/client/AMRMClientUtils.java     |  47 +++++++
 .../hadoop/yarn/server/AMRMClientRelayer.java   | 130 +++++++++++++------
 .../server/uam/UnmanagedApplicationManager.java |   2 +-
 .../yarn/server/TestAMRMClientRelayer.java      |  53 +++++++-
 .../amrmproxy/FederationInterceptor.java        |   6 +-
 .../ApplicationMasterService.java               |   9 +-
 6 files changed, 190 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8736fc39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/AMRMClientUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/AMRMClientUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/AMRMClientUtils.java
index 5d4ab4a6..b8319cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/AMRMClientUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/AMRMClientUtils.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.security.SaslRpcServer;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -54,6 +55,10 @@ public final class AMRMClientUtils {
   public static final String APP_ALREADY_REGISTERED_MESSAGE =
       "Application Master is already registered : ";
 
+  public static final String EXPECTED_HB_RESPONSEID_MESSAGE =
+      " expect responseId to be ";
+  public static final String RECEIVED_HB_RESPONSEID_MESSAGE = " but get ";
+
   private AMRMClientUtils() {
   }
 
@@ -105,6 +110,48 @@ public final class AMRMClientUtils {
         SaslRpcServer.AuthMethod.TOKEN.toString());
   }
 
+  /**
+   * Generate the exception message when RM receives an AM heartbeat with
+   * invalid responseId.
+   *
+   * @param appAttemptId the app attempt
+   * @param expected the expected responseId value
+   * @param received the received responseId value
+   * @return the assembled exception message
+   */
+  public static String assembleInvalidResponseIdExceptionMessage(
+      ApplicationAttemptId appAttemptId, int expected, int received) {
+    return "Invalid responseId in AllocateRequest from application attempt: "
+        + appAttemptId + EXPECTED_HB_RESPONSEID_MESSAGE + expected
+        + RECEIVED_HB_RESPONSEID_MESSAGE + received;
+  }
+
+  /**
+   * Parse the expected responseId from the exception generated by RM when
+   * processing AM heartbeat.
+   *
+   * @param exceptionMessage the exception message thrown by RM
+   * @return the parsed expected responseId, -1 if failed
+   */
+  public static int parseExpectedResponseIdFromException(
+      String exceptionMessage) {
+    if (exceptionMessage == null) {
+      return -1;
+    }
+    int start = exceptionMessage.indexOf(EXPECTED_HB_RESPONSEID_MESSAGE);
+    int end = exceptionMessage.indexOf(RECEIVED_HB_RESPONSEID_MESSAGE);
+    if (start == -1 || end == -1) {
+      return -1;
+    }
+    start += EXPECTED_HB_RESPONSEID_MESSAGE.length();
+
+    try {
+      return Integer.parseInt(exceptionMessage.substring(start, end));
+    } catch (NumberFormatException ex) {
+      return -1;
+    }
+  }
+
   public static void addToOutstandingSchedulingRequests(
       Collection<SchedulingRequest> requests,
       Map<Set<String>, List<SchedulingRequest>> outstandingSchedRequests) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8736fc39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMRMClientRelayer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMRMClientRelayer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMRMClientRelayer.java
index 0d1a27e..1e2060c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMRMClientRelayer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMRMClientRelayer.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
@@ -48,6 +49,7 @@ import org.apache.hadoop.yarn.client.AMRMClientUtils;
 import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
+import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.scheduler.ResourceRequestSet;
@@ -111,13 +113,22 @@ public class AMRMClientRelayer extends AbstractService
       new HashMap<>();
   private List<SchedulingRequest> schedulingRequest = new ArrayList<>();
 
+  private ApplicationId appId;
+
+  // Normally -1, otherwise will override responseId with this value in the next
+  // heartbeat
+  private volatile int resetResponseId;
+
   public AMRMClientRelayer() {
     super(AMRMClientRelayer.class.getName());
+    this.resetResponseId = -1;
   }
 
-  public AMRMClientRelayer(ApplicationMasterProtocol rmClient) {
+  public AMRMClientRelayer(ApplicationMasterProtocol rmClient,
+      ApplicationId appId) {
     this();
     this.rmClient = rmClient;
+    this.appId = appId;
   }
 
   @Override
@@ -167,59 +178,63 @@ public class AMRMClientRelayer extends AbstractService
     try {
       return this.rmClient.finishApplicationMaster(request);
     } catch (ApplicationMasterNotRegisteredException e) {
-      LOG.warn("Out of sync with ResourceManager, hence resyncing.");
+      LOG.warn("Out of sync with RM for " + this.appId + ", hence resyncing.");
       // re register with RM
       registerApplicationMaster(this.amRegistrationRequest);
       return finishApplicationMaster(request);
     }
   }
 
+  private void addNewAllocateRequest(AllocateRequest allocateRequest)
+      throws YarnException {
+    // update the data structures first
+    addNewAsks(allocateRequest.getAskList());
+
+    if (allocateRequest.getReleaseList() != null) {
+      this.remotePendingRelease.addAll(allocateRequest.getReleaseList());
+      this.release.addAll(allocateRequest.getReleaseList());
+    }
+
+    if (allocateRequest.getResourceBlacklistRequest() != null) {
+      if (allocateRequest.getResourceBlacklistRequest()
+          .getBlacklistAdditions() != null) {
+        this.remoteBlacklistedNodes.addAll(allocateRequest
+            .getResourceBlacklistRequest().getBlacklistAdditions());
+        this.blacklistAdditions.addAll(allocateRequest
+            .getResourceBlacklistRequest().getBlacklistAdditions());
+      }
+      if (allocateRequest.getResourceBlacklistRequest()
+          .getBlacklistRemovals() != null) {
+        this.remoteBlacklistedNodes.removeAll(allocateRequest
+            .getResourceBlacklistRequest().getBlacklistRemovals());
+        this.blacklistRemovals.addAll(allocateRequest
+            .getResourceBlacklistRequest().getBlacklistRemovals());
+      }
+    }
+
+    if (allocateRequest.getUpdateRequests() != null) {
+      for (UpdateContainerRequest update : allocateRequest
+          .getUpdateRequests()) {
+        this.remotePendingChange.put(update.getContainerId(), update);
+        this.change.put(update.getContainerId(), update);
+      }
+    }
+
+    if (allocateRequest.getSchedulingRequests() != null) {
+      AMRMClientUtils.addToOutstandingSchedulingRequests(
+          allocateRequest.getSchedulingRequests(),
+          this.remotePendingSchedRequest);
+      this.schedulingRequest.addAll(allocateRequest.getSchedulingRequests());
+    }
+  }
+
   @Override
   public AllocateResponse allocate(AllocateRequest allocateRequest)
       throws YarnException, IOException {
     AllocateResponse allocateResponse = null;
     try {
       synchronized (this) {
-        // update the data structures first
-        addNewAsks(allocateRequest.getAskList());
-
-        if (allocateRequest.getReleaseList() != null) {
-          this.remotePendingRelease.addAll(allocateRequest.getReleaseList());
-          this.release.addAll(allocateRequest.getReleaseList());
-        }
-
-        if (allocateRequest.getResourceBlacklistRequest() != null) {
-          if (allocateRequest.getResourceBlacklistRequest()
-              .getBlacklistAdditions() != null) {
-            this.remoteBlacklistedNodes.addAll(allocateRequest
-                .getResourceBlacklistRequest().getBlacklistAdditions());
-            this.blacklistAdditions.addAll(allocateRequest
-                .getResourceBlacklistRequest().getBlacklistAdditions());
-          }
-          if (allocateRequest.getResourceBlacklistRequest()
-              .getBlacklistRemovals() != null) {
-            this.remoteBlacklistedNodes.removeAll(allocateRequest
-                .getResourceBlacklistRequest().getBlacklistRemovals());
-            this.blacklistRemovals.addAll(allocateRequest
-                .getResourceBlacklistRequest().getBlacklistRemovals());
-          }
-        }
-
-        if (allocateRequest.getUpdateRequests() != null) {
-          for (UpdateContainerRequest update : allocateRequest
-              .getUpdateRequests()) {
-            this.remotePendingChange.put(update.getContainerId(), update);
-            this.change.put(update.getContainerId(), update);
-          }
-        }
-
-        if (allocateRequest.getSchedulingRequests() != null) {
-          AMRMClientUtils.addToOutstandingSchedulingRequests(
-              allocateRequest.getSchedulingRequests(),
-              this.remotePendingSchedRequest);
-          this.schedulingRequest
-              .addAll(allocateRequest.getSchedulingRequests());
-        }
+        addNewAllocateRequest(allocateRequest);
 
         ArrayList<ResourceRequest> askList = new ArrayList<>(ask.size());
         for (ResourceRequest r : ask) {
@@ -238,13 +253,23 @@ public class AMRMClientRelayer extends AbstractService
             .updateRequests(new ArrayList<>(this.change.values()))
             .schedulingRequests(new ArrayList<>(this.schedulingRequest))
             .build();
+
+        if (this.resetResponseId != -1) {
+          LOG.info("Override allocate responseId from "
+              + allocateRequest.getResponseId() + " to " + this.resetResponseId
+              + " for " + this.appId);
+          allocateRequest.setResponseId(this.resetResponseId);
+        }
       }
 
       // Do the actual allocate call
       try {
         allocateResponse = this.rmClient.allocate(allocateRequest);
+
+        // Heartbeat succeeded, wipe out responseId overriding
+        this.resetResponseId = -1;
       } catch (ApplicationMasterNotRegisteredException e) {
-        LOG.warn("ApplicationMaster is out of sync with ResourceManager,"
+        LOG.warn("ApplicationMaster is out of sync with RM for " + this.appId
             + " hence resyncing.");
 
         synchronized (this) {
@@ -269,6 +294,25 @@ public class AMRMClientRelayer extends AbstractService
         // Reset responseId after re-register
         allocateRequest.setResponseId(0);
         return allocate(allocateRequest);
+      } catch (Throwable t) {
+
+        // If RM is complaining about responseId out of sync, force reset next
+        // time
+        if (t instanceof InvalidApplicationMasterRequestException) {
+          int responseId = AMRMClientUtils
+              .parseExpectedResponseIdFromException(t.getMessage());
+          if (responseId != -1) {
+            this.resetResponseId = responseId;
+            LOG.info("ResponseId out of sync with RM, expect " + responseId
+                + " but " + allocateRequest.getResponseId() + " used by "
+                + this.appId + ". Will override in the next allocate.");
+          } else {
+            LOG.warn("Failed to parse expected responseId out of exception for "
+                + this.appId);
+          }
+        }
+
+        throw t;
       }
 
       synchronized (this) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8736fc39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
index 856a818..abdec19 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
@@ -193,7 +193,7 @@ public class UnmanagedApplicationManager {
         this.applicationId.toString(), UserGroupInformation.getCurrentUser());
     this.rmProxyRelayer =
         new AMRMClientRelayer(createRMProxy(ApplicationMasterProtocol.class,
-            this.conf, this.userUgi, amrmToken));
+            this.conf, this.userUgi, amrmToken), this.applicationId);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8736fc39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/TestAMRMClientRelayer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/TestAMRMClientRelayer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/TestAMRMClientRelayer.java
index 22bb1f9..4c84f0b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/TestAMRMClientRelayer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/TestAMRMClientRelayer.java
@@ -40,7 +40,9 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.client.AMRMClientUtils;
 import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
+import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.scheduler.ResourceRequestSet;
 import org.apache.hadoop.yarn.util.Records;
@@ -62,6 +64,7 @@ public class TestAMRMClientRelayer {
     // Whether this mockRM will throw failover exception upon next heartbeat
     // from AM
     private boolean failover = false;
+    private int responseIdReset = -1;
     private List<ResourceRequest> lastAsk;
     private List<ContainerId> lastRelease;
     private List<String> lastBlacklistAdditions;
@@ -92,26 +95,40 @@ public class TestAMRMClientRelayer {
         this.failover = false;
         throw new ApplicationMasterNotRegisteredException("Mock RM restarted");
       }
+      if (this.responseIdReset != -1) {
+        String errorMessage =
+            AMRMClientUtils.assembleInvalidResponseIdExceptionMessage(null,
+                this.responseIdReset, request.getResponseId());
+        this.responseIdReset = -1;
+        throw new InvalidApplicationMasterRequestException(errorMessage);
+      }
+
       this.lastAsk = request.getAskList();
       this.lastRelease = request.getReleaseList();
       this.lastBlacklistAdditions =
           request.getResourceBlacklistRequest().getBlacklistAdditions();
       this.lastBlacklistRemovals =
           request.getResourceBlacklistRequest().getBlacklistRemovals();
-      return AllocateResponse.newInstance(0, null, null,
-          new ArrayList<NodeReport>(), Resource.newInstance(0, 0), null, 0,
-          null, null);
+      return AllocateResponse.newInstance(request.getResponseId() + 1, null,
+          null, new ArrayList<NodeReport>(), Resource.newInstance(0, 0), null,
+          0, null, null);
     }
 
     public void setFailoverFlag() {
       this.failover = true;
     }
+
+    public void setResponseIdReset(int expectedResponseId) {
+      this.responseIdReset = expectedResponseId;
+    }
   }
 
   private Configuration conf;
   private MockApplicationMasterService mockAMS;
   private AMRMClientRelayer relayer;
 
+  private int responseId = 0;
+
   // Buffer of asks that will be sent to RM in the next AM heartbeat
   private List<ResourceRequest> asks = new ArrayList<>();
   private List<ContainerId> releases = new ArrayList<>();
@@ -123,7 +140,7 @@ public class TestAMRMClientRelayer {
     this.conf = new Configuration();
 
     this.mockAMS = new MockApplicationMasterService();
-    this.relayer = new AMRMClientRelayer(this.mockAMS);
+    this.relayer = new AMRMClientRelayer(this.mockAMS, null);
 
     this.relayer.init(conf);
     this.relayer.start();
@@ -150,7 +167,7 @@ public class TestAMRMClientRelayer {
   private AllocateRequest getAllocateRequest() {
     // Need to create a new one every time because rather than directly
     // referring the lists, the protobuf impl makes a copy of the lists
-    return AllocateRequest.newInstance(0, 0, asks, releases,
+    return AllocateRequest.newInstance(responseId, 0, asks, releases,
         ResourceBlacklistRequest.newInstance(blacklistAdditions,
             blacklistRemoval));
   }
@@ -272,4 +289,30 @@ public class TestAMRMClientRelayer {
     clearAllocateRequestLists();
   }
 
+  @Test
+  public void testResponseIdResync() throws YarnException, IOException {
+    this.responseId = 10;
+
+    AllocateResponse response = this.relayer.allocate(getAllocateRequest());
+    Assert.assertEquals(this.responseId + 1, response.getResponseId());
+
+    int expected = 5;
+    this.mockAMS.setResponseIdReset(expected);
+
+    try {
+      this.relayer.allocate(getAllocateRequest());
+      Assert.fail("Expecting exception from RM");
+    } catch (InvalidApplicationMasterRequestException e) {
+      // Expected exception
+    }
+
+    // Verify that the responseId is overridden
+    response = this.relayer.allocate(getAllocateRequest());
+    Assert.assertEquals(expected + 1, response.getResponseId());
+
+    // Verify it is no longer overriden
+    this.responseId = response.getResponseId();
+    response = this.relayer.allocate(getAllocateRequest());
+    Assert.assertEquals(this.responseId + 1, response.getResponseId());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8736fc39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.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/amrmproxy/FederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
index 645e47e..65a2277 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
@@ -249,8 +249,10 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
 
     this.homeSubClusterId =
         SubClusterId.newInstance(YarnConfiguration.getClusterId(conf));
-    this.homeRMRelayer = new AMRMClientRelayer(createHomeRMProxy(appContext,
-        ApplicationMasterProtocol.class, this.appOwner));
+    this.homeRMRelayer = new AMRMClientRelayer(
+        createHomeRMProxy(appContext, ApplicationMasterProtocol.class,
+            this.appOwner),
+        getApplicationContext().getApplicationAttemptId().getApplicationId());
 
     this.federationFacade = FederationStateStoreFacade.getInstance();
     this.subClusterResolver = this.federationFacade.getSubClusterResolver();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8736fc39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.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/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
index 7dac2cd..6fe0aa9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
@@ -420,12 +420,9 @@ public class ApplicationMasterService extends AbstractService implements
         // heartbeat one step old, simply return lastReponse
         return lastResponse;
       } else if (request.getResponseId() != lastResponse.getResponseId()) {
-        String message =
-            "Invalid responseId in AllocateRequest from application attempt: "
-                + appAttemptId + ", expect responseId to be "
-                + lastResponse.getResponseId() + ", but get "
-                + request.getResponseId();
-        throw new InvalidApplicationMasterRequestException(message);
+        throw new InvalidApplicationMasterRequestException(AMRMClientUtils
+            .assembleInvalidResponseIdExceptionMessage(appAttemptId,
+                lastResponse.getResponseId(), request.getResponseId()));
       }
 
       AllocateResponse response =


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

Branch: refs/heads/YARN-3409
Commit: 009cec0713872648277aabc1ad5350845737f387
Parents: 5e1154b
Author: Naganarasimha <na...@apache.org>
Authored: Tue Jan 23 07:18:20 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:56 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/009cec07/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/009cec07/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/009cec07/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/009cec07/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


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

Branch: refs/heads/YARN-3409
Commit: 339fd5f4ca791a2d1601b42f45595bbff4d257f6
Parents: da2fd55
Author: Naganarasimha <na...@apache.org>
Authored: Thu Jun 28 08:13:09 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:57 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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/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/339fd5f4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index 20c9603..a600895 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.yarn.client.cli;
 
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyInt;
@@ -1544,8 +1546,8 @@ public class TestYarnCLI {
   public void testNodeStatus() throws Exception {
     NodeId nodeId = NodeId.newInstance("host0", 0);
     NodeCLI cli = new NodeCLI();
-    when(client.getNodeReports()).thenReturn(
-                    getNodeReports(3, NodeState.RUNNING, false));
+    when(client.getNodeReports())
+        .thenReturn(getNodeReports(3, NodeState.RUNNING, false, false, false));
     cli.setClient(client);
     cli.setSysOutPrintStream(sysOut);
     cli.setSysErrPrintStream(sysErr);
@@ -1568,6 +1570,8 @@ public class TestYarnCLI {
     pw.println("\tCPU-Used : 0 vcores");
     pw.println("\tCPU-Capacity : 0 vcores");
     pw.println("\tNode-Labels : a,b,c,x,y,z");
+    pw.println("\tNode Attributes : rm.yarn.io/GPU(STRING)=ARM");
+    pw.println("\t                  rm.yarn.io/CPU(STRING)=ARM");
     pw.println("\tResource Utilization by Node : PMem:2048 MB, VMem:4096 MB, VCores:8.0");
     pw.println("\tResource Utilization by Containers : PMem:1024 MB, VMem:2048 MB, VCores:4.0");
     pw.close();
@@ -1604,6 +1608,7 @@ public class TestYarnCLI {
     pw.println("\tCPU-Used : 0 vcores");
     pw.println("\tCPU-Capacity : 0 vcores");
     pw.println("\tNode-Labels : ");
+    pw.println("\tNode Attributes : ");
     pw.println("\tResource Utilization by Node : PMem:2048 MB, VMem:4096 MB, VCores:8.0");
     pw.println("\tResource Utilization by Containers : PMem:1024 MB, VMem:2048 MB, VCores:4.0");
     pw.close();
@@ -1616,8 +1621,8 @@ public class TestYarnCLI {
   public void testNodeStatusWithEmptyResourceUtilization() throws Exception {
     NodeId nodeId = NodeId.newInstance("host0", 0);
     NodeCLI cli = new NodeCLI();
-    when(client.getNodeReports()).thenReturn(
-                    getNodeReports(3, NodeState.RUNNING, false, true));
+    when(client.getNodeReports())
+        .thenReturn(getNodeReports(3, NodeState.RUNNING, false, true, true));
     cli.setClient(client);
     cli.setSysOutPrintStream(sysOut);
     cli.setSysErrPrintStream(sysErr);
@@ -1640,6 +1645,7 @@ public class TestYarnCLI {
     pw.println("\tCPU-Used : 0 vcores");
     pw.println("\tCPU-Capacity : 0 vcores");
     pw.println("\tNode-Labels : a,b,c,x,y,z");
+    pw.println("\tNode Attributes : ");
     pw.println("\tResource Utilization by Node : ");
     pw.println("\tResource Utilization by Containers : ");
     pw.close();
@@ -2049,18 +2055,20 @@ public class TestYarnCLI {
     cli.run(new String[] { "application" });
     verify(sysErr).println("Invalid Command Usage : ");
   }
-  
+
   private List<NodeReport> getNodeReports(int noOfNodes, NodeState state) {
-    return getNodeReports(noOfNodes, state, true, false);
+    return getNodeReports(noOfNodes, state, true, false, true);
   }
 
   private List<NodeReport> getNodeReports(int noOfNodes, NodeState state,
-      boolean emptyNodeLabel) {
-    return getNodeReports(noOfNodes, state, emptyNodeLabel, false);
+      boolean emptyNodeLabel, boolean emptyAttributes) {
+    return getNodeReports(noOfNodes, state, emptyNodeLabel, false,
+        emptyAttributes);
   }
 
   private List<NodeReport> getNodeReports(int noOfNodes, NodeState state,
-      boolean emptyNodeLabel, boolean emptyResourceUtilization) {
+      boolean emptyNodeLabel, boolean emptyResourceUtilization,
+      boolean emptyAttributes) {
     List<NodeReport> nodeReports = new ArrayList<NodeReport>();
 
     for (int i = 0; i < noOfNodes; i++) {
@@ -2082,6 +2090,11 @@ public class TestYarnCLI {
         nodeReport.setAggregatedContainersUtilization(containersUtilization);
         nodeReport.setNodeUtilization(nodeUtilization);
       }
+      if (!emptyAttributes) {
+        nodeReport.setNodeAttributes(ImmutableSet.of(NodeAttribute
+                .newInstance("GPU", NodeAttributeType.STRING, "ARM"),
+            NodeAttribute.newInstance("CPU", NodeAttributeType.STRING, "ARM")));
+      }
       nodeReports.add(nodeReport);
     }
     return nodeReports;

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


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

Branch: refs/heads/YARN-3409
Commit: d4440fd44f97a4908f71d31881cc26137937e451
Parents: 2b4c090
Author: Sunil G <su...@apache.org>
Authored: Thu Mar 29 17:12:08 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:56 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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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/d4440fd4/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


[19/50] [abbrv] hadoop git commit: YARN-8649. NPE in localizer hearbeat processing if a container is killed while localizing. Contributed by lujie

Posted by su...@apache.org.
YARN-8649. NPE in localizer hearbeat processing if a container is killed while localizing. Contributed by lujie


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

Branch: refs/heads/YARN-3409
Commit: 585ebd873a55bedd2a364d256837f08ada8ba032
Parents: bed8cb6
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Aug 23 09:29:46 2018 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Aug 23 09:29:46 2018 -0500

----------------------------------------------------------------------
 .../localizer/LocalResourcesTrackerImpl.java            |  5 +++++
 .../localizer/ResourceLocalizationService.java          | 12 ++++++++----
 .../localizer/TestResourceLocalizationService.java      | 12 +++++++++++-
 3 files changed, 24 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/585ebd87/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.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/containermanager/localizer/LocalResourcesTrackerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
index dd31543..ad24c62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
@@ -500,6 +500,11 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
 
     Path localPath = new Path(rPath, req.getPath().getName());
     LocalizedResource rsrc = localrsrc.get(req);
+    if (rsrc == null) {
+      LOG.warn("Resource " + req + " has been removed"
+          + " and will no longer be localized");
+      return null;
+    }
     rsrc.setLocalPath(localPath);
     LocalResource lr = LocalResource.newInstance(req.getResource(),
         req.getType(), req.getVisibility(), req.getSize(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/585ebd87/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.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/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index 2892d1f..d9b887f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -885,6 +885,9 @@ public class ResourceLocalizationService extends CompositeService
             Path publicDirDestPath =
                 publicRsrc.getPathForLocalization(key, publicRootPath,
                     delService);
+            if (publicDirDestPath == null) {
+              return;
+            }
             if (!publicDirDestPath.getParent().equals(publicRootPath)) {
               createParentDirs(publicDirDestPath, publicRootPath);
               if (diskValidator != null) {
@@ -1175,10 +1178,11 @@ public class ResourceLocalizationService extends CompositeService
           LocalResourcesTracker tracker = getLocalResourcesTracker(
               next.getVisibility(), user, applicationId);
           if (tracker != null) {
-            ResourceLocalizationSpec resource =
-                NodeManagerBuilderUtils.newResourceLocalizationSpec(next,
-                getPathForLocalization(next, tracker));
-            rsrcs.add(resource);
+            Path localPath = getPathForLocalization(next, tracker);
+            if (localPath != null) {
+              rsrcs.add(NodeManagerBuilderUtils.newResourceLocalizationSpec(
+                  next, localPath));
+            }
           }
         } catch (IOException e) {
           LOG.error("local path for PRIVATE localization could not be " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/585ebd87/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.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/containermanager/localizer/TestResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
index 2b9148e..21896ca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
@@ -1717,8 +1717,18 @@ public class TestResourceLocalizationService {
       assertEquals("NM should tell localizer to be LIVE in Heartbeat.",
           LocalizerAction.LIVE, response.getLocalizerAction());
 
-      // Cleanup application.
+      // Cleanup container.
       spyService.handle(new ContainerLocalizationCleanupEvent(c, rsrcs));
+      dispatcher.await();
+      try {
+        /*Directly send heartbeat to introduce race as container
+          is being cleaned up.*/
+        locRunnerForContainer.processHeartbeat(
+              Collections.singletonList(rsrcSuccess));
+      } catch (Exception e) {
+        fail("Exception should not have been thrown on processing heartbeat");
+      }
+      // Cleanup application.
       spyService.handle(new ApplicationLocalizationEvent(
           LocalizationEventType.DESTROY_APPLICATION_RESOURCES, app));
       dispatcher.await();


---------------------------------------------------------------------
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-363. Faster datanode registration during the first startup. Contributed by Elek, Marton.

Posted by su...@apache.org.
HDDS-363. Faster datanode registration during the first startup. 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/138b0c14
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/138b0c14
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/138b0c14

Branch: refs/heads/YARN-3409
Commit: 138b0c1443ae485ba90b953362d6dea89c77a758
Parents: 55b6931
Author: Márton Elek <el...@apache.org>
Authored: Fri Aug 24 15:36:10 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Fri Aug 24 15:36:10 2018 +0200

----------------------------------------------------------------------
 .../statemachine/DatanodeStateMachine.java      |  5 +---
 .../common/statemachine/StateContext.java       | 20 ++++++++++++++
 .../states/datanode/RunningDatanodeState.java   |  1 +
 .../states/endpoint/RegisterEndpointTask.java   | 28 ++++++++++++++++----
 .../ozone/container/common/TestEndPoint.java    |  3 ++-
 5 files changed, 47 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/138b0c14/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
index cf6f1ca..0a23912 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
@@ -48,8 +48,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
-import static org.apache.hadoop.hdds.scm.HddsServerUtil.getScmHeartbeatInterval;
-
 /**
  * State Machine Class.
  */
@@ -60,7 +58,6 @@ public class DatanodeStateMachine implements Closeable {
   private final ExecutorService executorService;
   private final Configuration conf;
   private final SCMConnectionManager connectionManager;
-  private final long heartbeatFrequency;
   private StateContext context;
   private final OzoneContainer container;
   private DatanodeDetails datanodeDetails;
@@ -86,7 +83,6 @@ public class DatanodeStateMachine implements Closeable {
             .setNameFormat("Datanode State Machine Thread - %d").build());
     connectionManager = new SCMConnectionManager(conf);
     context = new StateContext(this.conf, DatanodeStates.getInitState(), this);
-    heartbeatFrequency = getScmHeartbeatInterval(conf);
     container = new OzoneContainer(this.datanodeDetails,
         new OzoneConfiguration(conf), context);
     nextHB = new AtomicLong(Time.monotonicNow());
@@ -147,6 +143,7 @@ public class DatanodeStateMachine implements Closeable {
     while (context.getState() != DatanodeStates.SHUTDOWN) {
       try {
         LOG.debug("Executing cycle Number : {}", context.getExecutionCount());
+        long heartbeatFrequency = context.getHeartbeatFrequency();
         nextHB.set(Time.monotonicNow() + heartbeatFrequency);
         context.execute(executorService, heartbeatFrequency,
             TimeUnit.MILLISECONDS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/138b0c14/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
index 19c9496..a342294 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
@@ -33,6 +33,8 @@ import org.apache.hadoop.ozone.protocol.commands.CommandStatus;
 import org.apache.hadoop.ozone.protocol.commands.CommandStatus
     .CommandStatusBuilder;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+
+import static org.apache.hadoop.hdds.scm.HddsServerUtil.getScmHeartbeatInterval;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -67,6 +69,13 @@ public class StateContext {
   private DatanodeStateMachine.DatanodeStates state;
 
   /**
+   * Starting with a 2 sec heartbeat frequency which will be updated to the
+   * real HB frequency after scm registration. With this method the
+   * initial registration could be significant faster.
+   */
+  private AtomicLong heartbeatFrequency = new AtomicLong(2000);
+
+  /**
    * Constructs a StateContext.
    *
    * @param conf   - Configration
@@ -398,4 +407,15 @@ public class StateContext {
     }
     return false;
   }
+
+  public void configureHeartbeatFrequency(){
+    heartbeatFrequency.set(getScmHeartbeatInterval(conf));
+  }
+
+  /**
+   * Return current heartbeat frequency in ms.
+   */
+  public long getHeartbeatFrequency() {
+    return heartbeatFrequency.get();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/138b0c14/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
index 1758c03..ec2358a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
@@ -101,6 +101,7 @@ public class RunningDatanodeState implements DatanodeState {
       return  RegisterEndpointTask.newBuilder()
           .setConfig(conf)
           .setEndpointStateMachine(endpoint)
+          .setContext(context)
           .setDatanodeDetails(context.getParent().getDatanodeDetails())
           .setOzoneContainer(context.getParent().getContainer())
           .build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/138b0c14/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 25af4a1..ccab095 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
@@ -29,6 +29,7 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -50,6 +51,7 @@ public final class RegisterEndpointTask implements
   private Future<EndpointStateMachine.EndPointStates> result;
   private DatanodeDetails datanodeDetails;
   private final OzoneContainer datanodeContainerManager;
+  private StateContext stateContext;
 
   /**
    * Creates a register endpoint task.
@@ -60,10 +62,12 @@ public final class RegisterEndpointTask implements
    */
   @VisibleForTesting
   public RegisterEndpointTask(EndpointStateMachine rpcEndPoint,
-      Configuration conf, OzoneContainer ozoneContainer) {
+      Configuration conf, OzoneContainer ozoneContainer,
+      StateContext context) {
     this.rpcEndPoint = rpcEndPoint;
     this.conf = conf;
     this.datanodeContainerManager = ozoneContainer;
+    this.stateContext = context;
 
   }
 
@@ -124,6 +128,7 @@ public final class RegisterEndpointTask implements
           rpcEndPoint.getState().getNextState();
       rpcEndPoint.setState(nextState);
       rpcEndPoint.zeroMissedCount();
+      this.stateContext.configureHeartbeatFrequency();
     } catch (IOException ex) {
       rpcEndPoint.logIfNeeded(ex);
     } finally {
@@ -150,6 +155,7 @@ public final class RegisterEndpointTask implements
     private Configuration conf;
     private DatanodeDetails datanodeDetails;
     private OzoneContainer container;
+    private StateContext context;
 
     /**
      * Constructs the builder class.
@@ -200,6 +206,10 @@ public final class RegisterEndpointTask implements
       return this;
     }
 
+    public Builder setContext(StateContext stateContext) {
+      this.context = stateContext;
+      return this;
+    }
 
     public RegisterEndpointTask build() {
       if (endPointStateMachine == null) {
@@ -210,8 +220,9 @@ public final class RegisterEndpointTask implements
 
       if (conf == null) {
         LOG.error("No config specified.");
-        throw new IllegalArgumentException("A valid configration is needed to" +
-            " construct RegisterEndpoint task");
+        throw new IllegalArgumentException(
+            "A valid configuration is needed to construct RegisterEndpoint "
+                + "task");
       }
 
       if (datanodeDetails == null) {
@@ -223,13 +234,20 @@ public final class RegisterEndpointTask implements
       if (container == null) {
         LOG.error("Container is not specified");
         throw new IllegalArgumentException("Container is not specified to " +
-            "constrict RegisterEndpoint task");
+            "construct RegisterEndpoint task");
+      }
+
+      if (context == null) {
+        LOG.error("StateContext is not specified");
+        throw new IllegalArgumentException("Container is not specified to " +
+            "construct RegisterEndpoint task");
       }
 
       RegisterEndpointTask task = new RegisterEndpointTask(this
-          .endPointStateMachine, this.conf, this.container);
+          .endPointStateMachine, this.conf, this.container, this.context);
       task.setDatanodeDetails(datanodeDetails);
       return task;
     }
+
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/138b0c14/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
index 5071d8d..5efcdd1 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
@@ -309,7 +309,8 @@ public class TestEndPoint {
     when(ozoneContainer.getContainerReport()).thenReturn(
         TestUtils.getRandomContainerReports(10));
     RegisterEndpointTask endpointTask =
-        new RegisterEndpointTask(rpcEndPoint, conf, ozoneContainer);
+        new RegisterEndpointTask(rpcEndPoint, conf, ozoneContainer,
+            mock(StateContext.class));
     if (!clearDatanodeDetails) {
       DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails();
       endpointTask.setDatanodeDetails(datanodeDetails);


---------------------------------------------------------------------
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: HDFS-13848. Refactor NameNode failover proxy providers. Contributed by Konstantin Shvachko.

Posted by su...@apache.org.
HDFS-13848. Refactor NameNode failover proxy providers. Contributed by Konstantin Shvachko.

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

Branch: refs/heads/YARN-3409
Commit: a4121c71c29d13866a605d9c0d013e5de9c147c3
Parents: a5eba25
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Fri Aug 24 18:27:30 2018 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Fri Aug 24 18:27:30 2018 -0700

----------------------------------------------------------------------
 .../hadoop/io/retry/FailoverProxyProvider.java  |  15 +-
 .../ha/AbstractNNFailoverProxyProvider.java     | 152 ++++++++++++++++++-
 .../ha/ConfiguredFailoverProxyProvider.java     | 141 ++---------------
 .../namenode/ha/IPFailoverProxyProvider.java    |  51 +------
 4 files changed, 176 insertions(+), 183 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4121c71/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
index c73e083..f2fa3af 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
@@ -30,27 +30,30 @@ import org.apache.hadoop.classification.InterfaceStability;
  */
 @InterfaceStability.Evolving
 public interface FailoverProxyProvider<T> extends Closeable {
-  public static final class ProxyInfo<T> {
-    public final T proxy;
+  static class ProxyInfo<T> {
+    public T proxy;
     /*
      * The information (e.g., the IP address) of the current proxy object. It
      * provides information for debugging purposes.
      */
-    public final String proxyInfo;
+    public String proxyInfo;
 
     public ProxyInfo(T proxy, String proxyInfo) {
       this.proxy = proxy;
       this.proxyInfo = proxyInfo;
     }
 
+    private String proxyName() {
+      return proxy != null ? proxy.getClass().getSimpleName() : "UnknownProxy";
+    }
+
     public String getString(String methodName) {
-      return proxy.getClass().getSimpleName() + "." + methodName
-          + " over " + proxyInfo;
+      return proxyName() + "." + methodName + " over " + proxyInfo;
     }
 
     @Override
     public String toString() {
-      return proxy.getClass().getSimpleName() + " over " + proxyInfo;
+      return proxyName() + " over " + proxyInfo;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4121c71/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
index e0fdb32..252b70d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
@@ -18,14 +18,68 @@
 
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.HAUtilClient;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.io.retry.FailoverProxyProvider;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class AbstractNNFailoverProxyProvider<T> implements
     FailoverProxyProvider <T> {
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(AbstractNNFailoverProxyProvider.class);
 
-  private AtomicBoolean fallbackToSimpleAuth;
+  protected Configuration conf;
+  protected Class<T> xface;
+  protected HAProxyFactory<T> factory;
+  protected UserGroupInformation ugi;
+  protected AtomicBoolean fallbackToSimpleAuth;
+
+  protected AbstractNNFailoverProxyProvider() {
+  }
+
+  protected AbstractNNFailoverProxyProvider(Configuration conf, URI uri,
+      Class<T> xface, HAProxyFactory<T> factory) {
+    this.conf = new Configuration(conf);
+    this.xface = xface;
+    this.factory = factory;
+    try {
+      this.ugi = UserGroupInformation.getCurrentUser();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    int maxRetries = this.conf.getInt(
+        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_KEY,
+        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_DEFAULT);
+    this.conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
+        maxRetries);
+
+    int maxRetriesOnSocketTimeouts = this.conf.getInt(
+        HdfsClientConfigKeys
+        .Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+        HdfsClientConfigKeys
+        .Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
+    this.conf.setInt(
+        CommonConfigurationKeysPublic
+        .IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+        maxRetriesOnSocketTimeouts);
+  }
 
   /**
    * Inquire whether logical HA URI is used for the implementation. If it is
@@ -51,4 +105,100 @@ public abstract class AbstractNNFailoverProxyProvider<T> implements
   public synchronized AtomicBoolean getFallbackToSimpleAuth() {
     return fallbackToSimpleAuth;
   }
+
+  /**
+   * ProxyInfo to a NameNode. Includes its address.
+   */
+  public static class NNProxyInfo<T> extends ProxyInfo<T> {
+    private InetSocketAddress address;
+
+    public NNProxyInfo(InetSocketAddress address) {
+      super(null, address.toString());
+      this.address = address;
+    }
+
+    public InetSocketAddress getAddress() {
+      return address;
+    }
+  }
+
+  @Override
+  public Class<T> getInterface() {
+    return xface;
+  }
+
+  /**
+   * Create a proxy if it has not been created yet.
+   */
+  protected NNProxyInfo<T> createProxyIfNeeded(NNProxyInfo<T> pi) {
+    if (pi.proxy == null) {
+      assert pi.getAddress() != null : "Proxy address is null";
+      try {
+        pi.proxy = factory.createProxy(conf,
+            pi.getAddress(), xface, ugi, false, getFallbackToSimpleAuth());
+      } catch (IOException ioe) {
+        LOG.error("{} Failed to create RPC proxy to NameNode",
+            this.getClass().getSimpleName(), ioe);
+        throw new RuntimeException(ioe);
+      }
+    }
+    return pi;
+  }
+
+  /**
+   * Get list of configured NameNode proxy addresses.
+   * Randomize the list if requested.
+   */
+  protected List<NNProxyInfo<T>> getProxyAddresses(URI uri, String addressKey) {
+    final List<NNProxyInfo<T>> proxies = new ArrayList<NNProxyInfo<T>>();
+    Map<String, Map<String, InetSocketAddress>> map =
+        DFSUtilClient.getAddresses(conf, null, addressKey);
+    Map<String, InetSocketAddress> addressesInNN = map.get(uri.getHost());
+
+    if (addressesInNN == null || addressesInNN.size() == 0) {
+      throw new RuntimeException("Could not find any configured addresses " +
+          "for URI " + uri);
+    }
+
+    Collection<InetSocketAddress> addressesOfNns = addressesInNN.values();
+    for (InetSocketAddress address : addressesOfNns) {
+      proxies.add(new NNProxyInfo<T>(address));
+    }
+    // Randomize the list to prevent all clients pointing to the same one
+    boolean randomized = getRandomOrder(conf, uri);
+    if (randomized) {
+      Collections.shuffle(proxies);
+    }
+
+    // The client may have a delegation token set for the logical
+    // URI of the cluster. Clone this token to apply to each of the
+    // underlying IPC addresses so that the IPC code can find it.
+    HAUtilClient.cloneDelegationTokenForLogicalUri(ugi, uri, addressesOfNns);
+    return proxies;
+  }
+
+  /**
+   * Check whether random order is configured for failover proxy provider
+   * for the namenode/nameservice.
+   *
+   * @param conf Configuration
+   * @param nameNodeUri The URI of namenode/nameservice
+   * @return random order configuration
+   */
+  public static boolean getRandomOrder(
+      Configuration conf, URI nameNodeUri) {
+    String host = nameNodeUri.getHost();
+    String configKeyWithHost = HdfsClientConfigKeys.Failover.RANDOM_ORDER
+        + "." + host;
+
+    if (conf.get(configKeyWithHost) != null) {
+      return conf.getBoolean(
+          configKeyWithHost,
+          HdfsClientConfigKeys.Failover.RANDOM_ORDER_DEFAULT);
+    }
+
+    return conf.getBoolean(
+        HdfsClientConfigKeys.Failover.RANDOM_ORDER,
+        HdfsClientConfigKeys.Failover.RANDOM_ORDER_DEFAULT);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4121c71/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
index f46532a..92e75ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
@@ -19,23 +19,11 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.hdfs.DFSUtilClient;
-import org.apache.hadoop.hdfs.HAUtilClient;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 
@@ -48,17 +36,9 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_RP
 public class ConfiguredFailoverProxyProvider<T> extends
     AbstractNNFailoverProxyProvider<T> {
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ConfiguredFailoverProxyProvider.class);
-
-  protected final Configuration conf;
-  protected final List<AddressRpcProxyPair<T>> proxies =
-      new ArrayList<AddressRpcProxyPair<T>>();
-  protected final UserGroupInformation ugi;
-  protected final Class<T> xface;
+  protected final List<NNProxyInfo<T>> proxies;
 
   private int currentProxyIndex = 0;
-  protected final HAProxyFactory<T> factory;
 
   public ConfiguredFailoverProxyProvider(Configuration conf, URI uri,
       Class<T> xface, HAProxyFactory<T> factory) {
@@ -67,83 +47,8 @@ public class ConfiguredFailoverProxyProvider<T> extends
 
   public ConfiguredFailoverProxyProvider(Configuration conf, URI uri,
       Class<T> xface, HAProxyFactory<T> factory, String addressKey) {
-    this.xface = xface;
-    this.conf = new Configuration(conf);
-    int maxRetries = this.conf.getInt(
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_KEY,
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_DEFAULT);
-    this.conf.setInt(
-        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
-        maxRetries);
-
-    int maxRetriesOnSocketTimeouts = this.conf.getInt(
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
-    this.conf.setInt(
-            CommonConfigurationKeysPublic
-                    .IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
-            maxRetriesOnSocketTimeouts);
-
-    try {
-      ugi = UserGroupInformation.getCurrentUser();
-
-      Map<String, Map<String, InetSocketAddress>> map =
-          DFSUtilClient.getAddresses(conf, null, addressKey);
-      Map<String, InetSocketAddress> addressesInNN = map.get(uri.getHost());
-
-      if (addressesInNN == null || addressesInNN.size() == 0) {
-        throw new RuntimeException("Could not find any configured addresses " +
-            "for URI " + uri);
-      }
-
-      Collection<InetSocketAddress> addressesOfNns = addressesInNN.values();
-      for (InetSocketAddress address : addressesOfNns) {
-        proxies.add(new AddressRpcProxyPair<T>(address));
-      }
-      // Randomize the list to prevent all clients pointing to the same one
-      boolean randomized = getRandomOrder(conf, uri);
-      if (randomized) {
-        Collections.shuffle(proxies);
-      }
-
-      // The client may have a delegation token set for the logical
-      // URI of the cluster. Clone this token to apply to each of the
-      // underlying IPC addresses so that the IPC code can find it.
-      HAUtilClient.cloneDelegationTokenForLogicalUri(ugi, uri, addressesOfNns);
-      this.factory = factory;
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Check whether random order is configured for failover proxy provider
-   * for the namenode/nameservice.
-   *
-   * @param conf Configuration
-   * @param nameNodeUri The URI of namenode/nameservice
-   * @return random order configuration
-   */
-  private static boolean getRandomOrder(
-      Configuration conf, URI nameNodeUri) {
-    String host = nameNodeUri.getHost();
-    String configKeyWithHost = HdfsClientConfigKeys.Failover.RANDOM_ORDER
-        + "." + host;
-
-    if (conf.get(configKeyWithHost) != null) {
-      return conf.getBoolean(
-          configKeyWithHost,
-          HdfsClientConfigKeys.Failover.RANDOM_ORDER_DEFAULT);
-    }
-
-    return conf.getBoolean(
-        HdfsClientConfigKeys.Failover.RANDOM_ORDER,
-        HdfsClientConfigKeys.Failover.RANDOM_ORDER_DEFAULT);
-  }
-
-  @Override
-  public Class<T> getInterface() {
-    return xface;
+    super(conf, uri, xface, factory);
+    this.proxies = getProxyAddresses(uri, addressKey);
   }
 
   /**
@@ -151,21 +56,8 @@ public class ConfiguredFailoverProxyProvider<T> extends
    */
   @Override
   public synchronized ProxyInfo<T> getProxy() {
-    AddressRpcProxyPair<T> current = proxies.get(currentProxyIndex);
-    return getProxy(current);
-  }
-
-  protected ProxyInfo<T> getProxy(AddressRpcProxyPair<T> current) {
-    if (current.namenode == null) {
-      try {
-        current.namenode = factory.createProxy(conf,
-            current.address, xface, ugi, false, getFallbackToSimpleAuth());
-      } catch (IOException e) {
-        LOG.error("Failed to create RPC proxy to NameNode", e);
-        throw new RuntimeException(e);
-      }
-    }
-    return new ProxyInfo<T>(current.namenode, current.address.toString());
+    NNProxyInfo<T> current = proxies.get(currentProxyIndex);
+    return createProxyIfNeeded(current);
   }
 
   @Override
@@ -178,30 +70,17 @@ public class ConfiguredFailoverProxyProvider<T> extends
   }
 
   /**
-   * A little pair object to store the address and connected RPC proxy object to
-   * an NN. Note that {@link AddressRpcProxyPair#namenode} may be null.
-   */
-  protected static class AddressRpcProxyPair<T> {
-    public final InetSocketAddress address;
-    public T namenode;
-
-    public AddressRpcProxyPair(InetSocketAddress address) {
-      this.address = address;
-    }
-  }
-
-  /**
    * Close all the proxy objects which have been opened over the lifetime of
    * this proxy provider.
    */
   @Override
   public synchronized void close() throws IOException {
-    for (AddressRpcProxyPair<T> proxy : proxies) {
-      if (proxy.namenode != null) {
-        if (proxy.namenode instanceof Closeable) {
-          ((Closeable)proxy.namenode).close();
+    for (ProxyInfo<T> proxy : proxies) {
+      if (proxy.proxy != null) {
+        if (proxy.proxy instanceof Closeable) {
+          ((Closeable)proxy.proxy).close();
         } else {
-          RPC.stopProxy(proxy.namenode);
+          RPC.stopProxy(proxy.proxy);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4121c71/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
index ed250a0..e703740 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
@@ -19,15 +19,11 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hdfs.DFSUtilClient;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.UserGroupInformation;
 
 /**
  * A NNFailoverProxyProvider implementation which works on IP failover setup.
@@ -47,53 +43,18 @@ import org.apache.hadoop.security.UserGroupInformation;
  */
 public class IPFailoverProxyProvider<T> extends
     AbstractNNFailoverProxyProvider<T> {
-  private final Configuration conf;
-  private final Class<T> xface;
-  private final URI nameNodeUri;
-  private final HAProxyFactory<T> factory;
-  private ProxyInfo<T> nnProxyInfo = null;
+  private final NNProxyInfo<T> nnProxyInfo;
 
   public IPFailoverProxyProvider(Configuration conf, URI uri,
       Class<T> xface, HAProxyFactory<T> factory) {
-    this.xface = xface;
-    this.nameNodeUri = uri;
-    this.factory = factory;
-
-    this.conf = new Configuration(conf);
-    int maxRetries = this.conf.getInt(
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_KEY,
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_DEFAULT);
-    this.conf.setInt(
-        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
-        maxRetries);
-
-    int maxRetriesOnSocketTimeouts = this.conf.getInt(
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
-    this.conf.setInt(
-        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
-        maxRetriesOnSocketTimeouts);
-  }
-
-  @Override
-  public Class<T> getInterface() {
-    return xface;
+    super(conf, uri, xface, factory);
+    this.nnProxyInfo = new NNProxyInfo<T>(DFSUtilClient.getNNAddress(uri));
   }
 
   @Override
-  public synchronized ProxyInfo<T> getProxy() {
+  public synchronized NNProxyInfo<T> getProxy() {
     // Create a non-ha proxy if not already created.
-    if (nnProxyInfo == null) {
-      try {
-        // Create a proxy that is not wrapped in RetryProxy
-        InetSocketAddress nnAddr = DFSUtilClient.getNNAddress(nameNodeUri);
-        nnProxyInfo = new ProxyInfo<T>(factory.createProxy(conf, nnAddr, xface,
-          UserGroupInformation.getCurrentUser(), false), nnAddr.toString());
-      } catch (IOException ioe) {
-        throw new RuntimeException(ioe);
-      }
-    }
-    return nnProxyInfo;
+    return createProxyIfNeeded(nnProxyInfo);
   }
 
   /** Nothing to do for IP failover */
@@ -106,7 +67,7 @@ public class IPFailoverProxyProvider<T> extends
    */
   @Override
   public synchronized void close() throws IOException {
-    if (nnProxyInfo == null) {
+    if (nnProxyInfo.proxy == null) {
       return;
     }
     if (nnProxyInfo.proxy instanceof Closeable) {


---------------------------------------------------------------------
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: YARN-8581. [AMRMProxy] Add sub-cluster timeout in LocalityMulticastAMRMProxyPolicy. Contributed by Botong Huang.

Posted by su...@apache.org.
YARN-8581. [AMRMProxy] Add sub-cluster timeout in LocalityMulticastAMRMProxyPolicy. Contributed by Botong Huang.


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

Branch: refs/heads/YARN-3409
Commit: e0f6ffdbad6f43fd43ec57fb68ebf5275b8b9ba0
Parents: 8736fc3
Author: Giovanni Matteo Fumarola <gi...@apache.com>
Authored: Mon Aug 20 14:33:16 2018 -0700
Committer: Giovanni Matteo Fumarola <gi...@apache.com>
Committed: Mon Aug 20 14:33:16 2018 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  8 +-
 .../yarn/conf/TestYarnConfigurationFields.java  |  2 +
 .../LocalityMulticastAMRMProxyPolicy.java       | 64 ++++++++++++--
 .../utils/FederationStateStoreFacade.java       |  9 ++
 .../TestLocalityMulticastAMRMProxyPolicy.java   | 91 +++++++++++++++++---
 .../utils/FederationPoliciesTestUtil.java       |  7 +-
 6 files changed, 162 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f6ffdb/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 78e28f7..148edb9 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
@@ -3209,8 +3209,14 @@ public class YarnConfiguration extends Configuration {
       "org.apache.hadoop.yarn.server.federation.resolver."
           + "DefaultSubClusterResolverImpl";
 
-  public static final String DEFAULT_FEDERATION_POLICY_KEY = "*";
+  // AMRMProxy split-merge timeout for active sub-clusters. We will not route
+  // new asks to expired sub-clusters.
+  public static final String FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT =
+      FEDERATION_PREFIX + "amrmproxy.subcluster.timeout.ms";
+  public static final long DEFAULT_FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT =
+      60000; // one minute
 
+  public static final String DEFAULT_FEDERATION_POLICY_KEY = "*";
   public static final String FEDERATION_POLICY_MANAGER = FEDERATION_PREFIX
       + "policy-manager";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f6ffdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index 9249ed4..d63933c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -105,6 +105,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
         .add(YarnConfiguration.DEFAULT_FEDERATION_POLICY_MANAGER);
     configurationPropsToSkipCompare
         .add(YarnConfiguration.DEFAULT_FEDERATION_POLICY_MANAGER_PARAMS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT);
 
     // Federation StateStore ZK implementation configs to be ignored
     configurationPropsToSkipCompare.add(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f6ffdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
index 1481f34..1ccd61c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
@@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
@@ -126,6 +127,8 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
   private SubClusterResolver resolver;
 
   private Map<SubClusterId, Resource> headroom;
+  private Map<SubClusterId, Long> lastHeartbeatTimeStamp;
+  private long subClusterTimeOut;
   private float hrAlpha;
   private FederationStateStoreFacade federationFacade;
   private AllocationBookkeeper bookkeeper;
@@ -178,6 +181,7 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
 
     if (headroom == null) {
       headroom = new ConcurrentHashMap<>();
+      lastHeartbeatTimeStamp = new ConcurrentHashMap<>();
     }
     hrAlpha = policy.getHeadroomAlpha();
 
@@ -185,13 +189,29 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
         policyContext.getFederationStateStoreFacade();
     this.homeSubcluster = policyContext.getHomeSubcluster();
 
+    this.subClusterTimeOut = this.federationFacade.getConf().getLong(
+        YarnConfiguration.FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT,
+        YarnConfiguration.DEFAULT_FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT);
+    if (this.subClusterTimeOut <= 0) {
+      LOG.info(
+          "{} configured to be {}, should be positive. Using default of {}.",
+          YarnConfiguration.FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT,
+          this.subClusterTimeOut,
+          YarnConfiguration.DEFAULT_FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT);
+      this.subClusterTimeOut =
+          YarnConfiguration.DEFAULT_FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT;
+    }
   }
 
   @Override
   public void notifyOfResponse(SubClusterId subClusterId,
       AllocateResponse response) throws YarnException {
-    // stateless policy does not care about responses except tracking headroom
-    headroom.put(subClusterId, response.getAvailableResources());
+    if (response.getAvailableResources() != null) {
+      headroom.put(subClusterId, response.getAvailableResources());
+      LOG.info("Subcluster {} updated with {} memory headroom", subClusterId,
+          response.getAvailableResources().getMemorySize());
+    }
+    lastHeartbeatTimeStamp.put(subClusterId, System.currentTimeMillis());
   }
 
   @Override
@@ -281,6 +301,15 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
     // handle all non-localized requests (ANY)
     splitAnyRequests(nonLocalizedRequests, bookkeeper);
 
+    for (Map.Entry<SubClusterId, List<ResourceRequest>> entry : bookkeeper
+        .getAnswer().entrySet()) {
+      // A new-cluster here will trigger new UAM luanch, which might take a long
+      // time. We don't want too many requests stuck in this UAM before it is
+      // ready and starts heartbeating
+      if (!lastHeartbeatTimeStamp.containsKey(entry.getKey())) {
+        lastHeartbeatTimeStamp.put(entry.getKey(), System.currentTimeMillis());
+      }
+    }
     return bookkeeper.getAnswer();
   }
 
@@ -519,13 +548,10 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
       policyWeights = weights;
       totPolicyWeight = 0;
 
-      // pre-compute the set of subclusters that are both active and enabled by
-      // the policy weights, and accumulate their total weight
       for (Map.Entry<SubClusterId, Float> entry : policyWeights.entrySet()) {
         if (entry.getValue() > 0
             && activeSubclusters.containsKey(entry.getKey())) {
           activeAndEnabledSC.add(entry.getKey());
-          totPolicyWeight += entry.getValue();
         }
       }
 
@@ -535,6 +561,34 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
                 + "currently active we cannot forward the ResourceRequest(s)");
       }
 
+      Set<SubClusterId> tmpSCSet = new HashSet<>(activeAndEnabledSC);
+      for (Map.Entry<SubClusterId, Long> entry : lastHeartbeatTimeStamp
+          .entrySet()) {
+        long duration = System.currentTimeMillis() - entry.getValue();
+        if (duration > subClusterTimeOut) {
+          LOG.warn(
+              "Subcluster {} does not have a success heartbeat for {}s, "
+                  + "skip routing asks there for this request",
+              entry.getKey(), (double) duration / 1000);
+          tmpSCSet.remove(entry.getKey());
+        }
+      }
+      if (tmpSCSet.size() < 1) {
+        LOG.warn("All active and enabled subclusters have expired last "
+            + "heartbeat time. Ignore the expiry check for this request");
+      } else {
+        activeAndEnabledSC = tmpSCSet;
+      }
+
+      LOG.info("{} subcluster active, {} subclusters active and enabled",
+          activeSubclusters.size(), activeAndEnabledSC.size());
+
+      // pre-compute the set of subclusters that are both active and enabled by
+      // the policy weights, and accumulate their total weight
+      for (SubClusterId sc : activeAndEnabledSC) {
+        totPolicyWeight += policyWeights.get(sc);
+      }
+
       // pre-compute headroom-based weights for active/enabled subclusters
       for (Map.Entry<SubClusterId, Resource> r : headroom.entrySet()) {
         if (activeAndEnabledSC.contains(r.getKey())) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f6ffdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
index 1bcb0f4..5d9702f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
@@ -392,6 +392,15 @@ public final class FederationStateStoreFacade {
   }
 
   /**
+   * Get the configuration.
+   *
+   * @return configuration object
+   */
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  /**
    * Helper method to create instances of Object using the class name defined in
    * the configuration object. The instances creates {@link RetryProxy} using
    * the specific {@link RetryPolicy}.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f6ffdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
index f66bbb6..cf9ac53 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
@@ -32,11 +32,13 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
@@ -106,6 +108,10 @@ public class TestLocalityMulticastAMRMProxyPolicy
   }
 
   private void initializePolicy() throws YarnException {
+    initializePolicy(new YarnConfiguration());
+  }
+
+  private void initializePolicy(Configuration conf) throws YarnException {
     setFederationPolicyContext(new FederationPolicyInitializationContext());
     SubClusterResolver resolver = FederationPoliciesTestUtil.initResolver();
     getFederationPolicyContext().setFederationSubclusterResolver(resolver);
@@ -116,7 +122,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     getFederationPolicyContext().setHomeSubcluster(getHomeSubCluster());
     FederationPoliciesTestUtil.initializePolicyContext(
         getFederationPolicyContext(), getPolicy(), getPolicyInfo(),
-        getActiveSubclusters());
+        getActiveSubclusters(), conf);
   }
 
   @Test(expected = FederationPolicyInitializationException.class)
@@ -145,7 +151,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     initializePolicy();
     List<ResourceRequest> resourceRequests = createSimpleRequest();
 
-    prepPolicyWithHeadroom();
+    prepPolicyWithHeadroom(true);
 
     Map<SubClusterId, List<ResourceRequest>> response =
         ((FederationAMRMProxyPolicy) getPolicy())
@@ -205,7 +211,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     int numRR = 1000;
     List<ResourceRequest> resourceRequests = createLargeRandomList(numRR);
 
-    prepPolicyWithHeadroom();
+    prepPolicyWithHeadroom(true);
 
     int numIterations = 1000;
     long tstart = System.currentTimeMillis();
@@ -233,7 +239,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     List<ResourceRequest> resourceRequests = createZeroSizedANYRequest();
 
     // this receives responses from sc0,sc1,sc2
-    prepPolicyWithHeadroom();
+    prepPolicyWithHeadroom(true);
 
     Map<SubClusterId, List<ResourceRequest>> response =
         ((FederationAMRMProxyPolicy) getPolicy())
@@ -269,7 +275,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     initializePolicy();
     List<ResourceRequest> resourceRequests = createSimpleRequest();
 
-    prepPolicyWithHeadroom();
+    prepPolicyWithHeadroom(true);
 
     Map<SubClusterId, List<ResourceRequest>> response =
         ((FederationAMRMProxyPolicy) getPolicy())
@@ -292,10 +298,14 @@ public class TestLocalityMulticastAMRMProxyPolicy
     checkTotalContainerAllocation(response, 100);
   }
 
-  private void prepPolicyWithHeadroom() throws YarnException {
+  private void prepPolicyWithHeadroom(boolean setSubCluster0)
+      throws YarnException {
     AllocateResponse ar = getAllocateResponseWithTargetHeadroom(40);
-    ((FederationAMRMProxyPolicy) getPolicy())
-        .notifyOfResponse(SubClusterId.newInstance("subcluster0"), ar);
+
+    if (setSubCluster0) {
+      ((FederationAMRMProxyPolicy) getPolicy())
+          .notifyOfResponse(SubClusterId.newInstance("subcluster0"), ar);
+    }
 
     ar = getAllocateResponseWithTargetHeadroom(0);
     ((FederationAMRMProxyPolicy) getPolicy())
@@ -333,7 +343,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
 
     FederationPoliciesTestUtil.initializePolicyContext(
         getFederationPolicyContext(), getPolicy(), getPolicyInfo(),
-        getActiveSubclusters());
+        getActiveSubclusters(), new Configuration());
 
     List<ResourceRequest> resourceRequests = createComplexRequest();
 
@@ -669,7 +679,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     List<ResourceRequest> resourceRequests = new ArrayList<>();
 
     // Initialize the headroom map
-    prepPolicyWithHeadroom();
+    prepPolicyWithHeadroom(true);
 
     // Cancel at ANY level only
     resourceRequests.add(FederationPoliciesTestUtil.createResourceRequest(0L,
@@ -716,4 +726,65 @@ public class TestLocalityMulticastAMRMProxyPolicy
     checkExpectedAllocation(response, "subcluster5", 1, 25);
     checkTotalContainerAllocation(response, 100);
   }
+
+  @Test
+  public void testSubClusterExpiry() throws Exception {
+
+    // Tests how the headroom info are used to split based on the capacity
+    // each RM claims to give us.
+    // Configure policy to be 100% headroom based
+    getPolicyInfo().setHeadroomAlpha(1.0f);
+
+    YarnConfiguration conf = new YarnConfiguration();
+    // Set expiry to 500ms
+    conf.setLong(YarnConfiguration.FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT,
+        500);
+
+    initializePolicy(conf);
+    List<ResourceRequest> resourceRequests = createSimpleRequest();
+
+    // Update the response timestamp for the first time
+    prepPolicyWithHeadroom(true);
+
+    Map<SubClusterId, List<ResourceRequest>> response =
+        ((FederationAMRMProxyPolicy) getPolicy())
+            .splitResourceRequests(resourceRequests);
+
+    // pretty print requests
+    prettyPrintRequests(response);
+
+    validateSplit(response, resourceRequests);
+
+    /*
+     * based on headroom, we expect 75 containers to got to subcluster0 (60) and
+     * subcluster2 (15) according to the advertised headroom (40 and 10), no
+     * containers for sublcuster1 as it advertise zero headroom, and 25 to
+     * subcluster5 which has unknown headroom, and so it gets 1/4th of the load
+     */
+    checkExpectedAllocation(response, "subcluster0", 1, 60);
+    checkExpectedAllocation(response, "subcluster1", 1, -1);
+    checkExpectedAllocation(response, "subcluster2", 1, 15);
+    checkExpectedAllocation(response, "subcluster5", 1, 25);
+    checkTotalContainerAllocation(response, 100);
+
+    Thread.sleep(800);
+
+    // Update the response timestamp for the second time, skipping sc0 and sc5
+    prepPolicyWithHeadroom(false);
+
+    response = ((FederationAMRMProxyPolicy) getPolicy())
+        .splitResourceRequests(resourceRequests);
+
+    // pretty print requests
+    prettyPrintRequests(response);
+
+    validateSplit(response, resourceRequests);
+
+    checkExpectedAllocation(response, "subcluster0", 1, -1);
+    checkExpectedAllocation(response, "subcluster1", 1, -1);
+    checkExpectedAllocation(response, "subcluster2", 1, 100);
+    checkExpectedAllocation(response, "subcluster5", 1, -1);
+    checkTotalContainerAllocation(response, 100);
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f6ffdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
index 24399cb..9cc6d06 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
@@ -117,7 +117,7 @@ public final class FederationPoliciesTestUtil {
   public static void initializePolicyContext(
       FederationPolicyInitializationContext fpc, ConfigurableFederationPolicy
       policy, WeightedPolicyInfo policyInfo,
-      Map<SubClusterId, SubClusterInfo> activeSubclusters)
+      Map<SubClusterId, SubClusterInfo> activeSubclusters, Configuration conf)
       throws YarnException {
     ByteBuffer buf = policyInfo.toByteBuffer();
     fpc.setSubClusterPolicyConfiguration(SubClusterPolicyConfiguration
@@ -133,7 +133,7 @@ public final class FederationPoliciesTestUtil {
         .newInstance(new ArrayList<SubClusterInfo>(activeSubclusters.values()));
 
     when(fss.getSubClusters(any())).thenReturn(response);
-    facade.reinitialize(fss, new Configuration());
+    facade.reinitialize(fss, conf);
     fpc.setFederationStateStoreFacade(facade);
     policy.reinitialize(fpc);
   }
@@ -155,7 +155,8 @@ public final class FederationPoliciesTestUtil {
     FederationPolicyInitializationContext context =
         new FederationPolicyInitializationContext(null, initResolver(),
             initFacade(), SubClusterId.newInstance(subclusterId));
-    initializePolicyContext(context, policy, policyInfo, activeSubclusters);
+    initializePolicyContext(context, policy, policyInfo, activeSubclusters,
+        new Configuration());
   }
 
   /**


---------------------------------------------------------------------
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-265. Move numPendingDeletionBlocks and deleteTransactionId from ContainerData to KeyValueContainerData. Contributed by LiXin Ge.

Posted by su...@apache.org.
HDDS-265. Move numPendingDeletionBlocks and deleteTransactionId from ContainerData to KeyValueContainerData. Contributed by LiXin Ge.


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

Branch: refs/heads/YARN-3409
Commit: 5aa15cfaffbf294b5025989c20d905b01da52c2b
Parents: 8184739
Author: Hanisha Koneru <ha...@apache.org>
Authored: Wed Aug 22 10:04:15 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Wed Aug 22 10:04:15 2018 -0700

----------------------------------------------------------------------
 .../common/helpers/ContainerReport.java         |  14 ---
 .../common/helpers/KeyValueContainerReport.java | 117 +++++++++++++++++++
 .../container/common/impl/ContainerData.java    |  53 ---------
 .../container/common/impl/ContainerSet.java     |  50 +-------
 .../RandomContainerDeletionChoosingPolicy.java  |   3 +-
 ...NOrderedContainerDeletionChoosingPolicy.java |  24 ++--
 .../container/common/interfaces/Container.java  |   6 +
 .../ContainerDeletionChoosingPolicy.java        |  13 +++
 .../container/keyvalue/KeyValueContainer.java   |  46 ++++++++
 .../keyvalue/KeyValueContainerData.java         |  54 +++++++++
 .../common/TestKeyValueContainerData.java       |   4 +-
 .../common/TestBlockDeletingService.java        |   8 +-
 .../commandhandler/TestBlockDeletion.java       |   9 +-
 13 files changed, 268 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
index b242754..a4c1f2f 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.container.common.helpers;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerInfo;
 
-import static java.lang.Math.max;
 
 /**
  * Container Report iterates the closed containers and sends a container report
@@ -37,7 +36,6 @@ public class ContainerReport {
   private long readBytes;
   private long writeBytes;
   private long containerID;
-  private long deleteTransactionId;
 
   public long getContainerID() {
     return containerID;
@@ -47,9 +45,6 @@ public class ContainerReport {
     this.containerID = containerID;
   }
 
-
-
-
   /**
    * Constructs the ContainerReport.
    *
@@ -66,7 +61,6 @@ public class ContainerReport {
     this.readBytes = 0L;
     this.writeCount = 0L;
     this.writeBytes = 0L;
-    this.deleteTransactionId = 0;
   }
 
   /**
@@ -100,9 +94,6 @@ public class ContainerReport {
     if (info.hasWriteBytes()) {
       report.setWriteBytes(info.getWriteBytes());
     }
-    if (info.hasDeleteTransactionId()) {
-      report.updateDeleteTransactionId(info.getDeleteTransactionId());
-    }
 
     report.setContainerID(info.getContainerID());
     return report;
@@ -193,10 +184,6 @@ public class ContainerReport {
     this.bytesUsed = bytesUsed;
   }
 
-  public void updateDeleteTransactionId(long transactionId) {
-    this.deleteTransactionId = max(transactionId, deleteTransactionId);
-  }
-
   /**
    * Gets a containerInfo protobuf message from ContainerReports.
    *
@@ -213,7 +200,6 @@ public class ContainerReport {
         .setWriteBytes(this.getWriteBytes())
         .setFinalhash(this.getFinalhash())
         .setContainerID(this.getContainerID())
-        .setDeleteTransactionId(this.deleteTransactionId)
         .build();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyValueContainerReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyValueContainerReport.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyValueContainerReport.java
new file mode 100644
index 0000000..b03487b
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyValueContainerReport.java
@@ -0,0 +1,117 @@
+/*
+ * 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.container.common.helpers;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerInfo;
+
+import static java.lang.Math.max;
+
+/**
+ * KeyValueContainer Report iterates the closed containers and sends a
+ * container report to SCM.
+ */
+public class KeyValueContainerReport extends ContainerReport{
+  private long deleteTransactionId;
+
+  /**
+   * Constructs the KeyValueContainerReport.
+   *
+   * @param containerID - Container ID.
+   * @param finalhash - Final Hash.
+   */
+  public KeyValueContainerReport(long containerID, String finalhash) {
+    super(containerID, finalhash);
+    this.deleteTransactionId = 0;
+  }
+
+  /**
+   * Sets the deleteTransactionId if it is greater than existing.
+   * @param transactionId - deleteTransactionId
+   */
+  public void updateDeleteTransactionId(long transactionId) {
+    this.deleteTransactionId = max(transactionId, deleteTransactionId);
+  }
+
+  /**
+   * Gets the deleteTransactionId.
+   * @return - deleteTransactionId.
+   */
+  public long getDeleteTransactionId() {
+    return this.deleteTransactionId;
+  }
+
+  /**
+   * Gets a containerReport from protobuf class.
+   *
+   * @param info - ContainerInfo.
+   * @return - ContainerReport.
+   */
+  public static KeyValueContainerReport getFromProtoBuf(ContainerInfo info) {
+    Preconditions.checkNotNull(info);
+    KeyValueContainerReport report = new KeyValueContainerReport(
+        info.getContainerID(), info.getFinalhash());
+    if (info.hasSize()) {
+      report.setSize(info.getSize());
+    }
+    if (info.hasKeyCount()) {
+      report.setKeyCount(info.getKeyCount());
+    }
+    if (info.hasUsed()) {
+      report.setBytesUsed(info.getUsed());
+    }
+    if (info.hasReadCount()) {
+      report.setReadCount(info.getReadCount());
+    }
+    if (info.hasReadBytes()) {
+      report.setReadBytes(info.getReadBytes());
+    }
+    if (info.hasWriteCount()) {
+      report.setWriteCount(info.getWriteCount());
+    }
+    if (info.hasWriteBytes()) {
+      report.setWriteBytes(info.getWriteBytes());
+    }
+    if (info.hasDeleteTransactionId()) {
+      report.updateDeleteTransactionId(info.getDeleteTransactionId());
+    }
+    report.setContainerID(info.getContainerID());
+    return report;
+  }
+
+  /**
+   * Gets a containerInfo protobuf message from ContainerReports.
+   *
+   * @return ContainerInfo
+   */
+  @Override
+  public ContainerInfo getProtoBufMessage() {
+    return ContainerInfo.newBuilder()
+        .setKeyCount(this.getKeyCount())
+        .setSize(this.getSize())
+        .setUsed(this.getBytesUsed())
+        .setReadCount(this.getReadCount())
+        .setReadBytes(this.getReadBytes())
+        .setWriteCount(this.getWriteCount())
+        .setWriteBytes(this.getWriteBytes())
+        .setFinalhash(this.getFinalhash())
+        .setContainerID(this.getContainerID())
+        .setDeleteTransactionId(this.getDeleteTransactionId())
+        .build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/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 26954a7..47894dc 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
@@ -33,11 +33,9 @@ import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 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;
@@ -81,8 +79,6 @@ public abstract class ContainerData {
 
   private HddsVolume volume;
 
-  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],
@@ -99,12 +95,6 @@ public abstract class ContainerData {
       MAX_SIZE_GB,
       CHECKSUM));
 
-
-  /**
-   * Number of pending deletion blocks in container.
-   */
-  private final AtomicInteger numPendingDeletionBlocks;
-
   /**
    * Creates a ContainerData Object, which holds metadata of the container.
    * @param type - ContainerType
@@ -139,8 +129,6 @@ public abstract class ContainerData {
     this.bytesUsed = new AtomicLong(0L);
     this.keyCount = new AtomicLong(0L);
     this.maxSizeGB = size;
-    this.numPendingDeletionBlocks = new AtomicInteger(0);
-    this.deleteTransactionId = 0;
     setChecksumTo0ByteArray();
   }
 
@@ -403,31 +391,6 @@ public abstract class ContainerData {
     this.keyCount.set(count);
   }
 
-  /**
-   * Increase the count of pending deletion blocks.
-   *
-   * @param numBlocks increment number
-   */
-  public void incrPendingDeletionBlocks(int numBlocks) {
-    this.numPendingDeletionBlocks.addAndGet(numBlocks);
-  }
-
-  /**
-   * Decrease the count of pending deletion blocks.
-   *
-   * @param numBlocks decrement number
-   */
-  public void decrPendingDeletionBlocks(int numBlocks) {
-    this.numPendingDeletionBlocks.addAndGet(-1 * numBlocks);
-  }
-
-  /**
-   * Get the number of pending deletion blocks.
-   */
-  public int getNumPendingDeletionBlocks() {
-    return this.numPendingDeletionBlocks.get();
-  }
-
   public void setChecksumTo0ByteArray() {
     this.checksum = DUMMY_CHECKSUM;
   }
@@ -469,20 +432,4 @@ public abstract class ContainerData {
    * @return Protocol Buffer Message
    */
   public abstract ContainerProtos.ContainerData getProtoBufMessage();
-
-  /**
-   * Sets deleteTransactionId to latest delete transactionId for the container.
-   *
-   * @param transactionId latest transactionId of the container.
-   */
-  public void updateDeleteTransactionId(long transactionId) {
-    deleteTransactionId = max(transactionId, deleteTransactionId);
-  }
-
-  /**
-   * Return the latest deleteTransactionId of the container.
-   */
-  public long getDeleteTransactionId() {
-    return deleteTransactionId;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
index 3da09f2..f92ab52 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
@@ -22,9 +22,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerInfo;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.hdds.scm.container.common.helpers
@@ -43,8 +40,6 @@ import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.stream.Collectors;
 
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.INVALID_CONTAINER_STATE;
 
 /**
  * Class that manages Containers created on the datanode.
@@ -204,58 +199,19 @@ public class ContainerSet {
     ContainerReportsProto.Builder crBuilder =
         ContainerReportsProto.newBuilder();
 
-
     for (Container container: containers) {
-      long containerId = container.getContainerData().getContainerID();
-      ContainerInfo.Builder ciBuilder = ContainerInfo.newBuilder();
-      ContainerData containerData = container.getContainerData();
-      ciBuilder.setContainerID(containerId)
-          .setReadCount(containerData.getReadCount())
-          .setWriteCount(containerData.getWriteCount())
-          .setReadBytes(containerData.getReadBytes())
-          .setWriteBytes(containerData.getWriteBytes())
-          .setUsed(containerData.getBytesUsed())
-          .setState(getState(containerData))
-          .setDeleteTransactionId(containerData.getDeleteTransactionId());
-
-      crBuilder.addReports(ciBuilder.build());
+      crBuilder.addReports(container.getContainerReport());
     }
 
     return crBuilder.build();
   }
 
-  /**
-   * Returns LifeCycle State of the container.
-   * @param containerData - ContainerData
-   * @return LifeCycle State of the container
-   * @throws StorageContainerException
-   */
-  private HddsProtos.LifeCycleState getState(ContainerData containerData)
-      throws StorageContainerException {
-    HddsProtos.LifeCycleState state;
-    switch (containerData.getState()) {
-    case OPEN:
-      state = HddsProtos.LifeCycleState.OPEN;
-      break;
-    case CLOSING:
-      state = HddsProtos.LifeCycleState.CLOSING;
-      break;
-    case CLOSED:
-      state = HddsProtos.LifeCycleState.CLOSED;
-      break;
-    default:
-      throw new StorageContainerException("Invalid Container state found: " +
-          containerData.getContainerID(), INVALID_CONTAINER_STATE);
-    }
-    return state;
-  }
-
   public List<ContainerData> chooseContainerForBlockDeletion(int count,
       ContainerDeletionChoosingPolicy deletionPolicy)
       throws StorageContainerException {
     Map<Long, ContainerData> containerDataMap = containerMap.entrySet().stream()
-        .filter(e -> e.getValue().getContainerType()
-            == ContainerProtos.ContainerType.KeyValueContainer)
+        .filter(e -> deletionPolicy.isValidContainerType(
+            e.getValue().getContainerType()))
         .collect(Collectors.toMap(Map.Entry::getKey,
             e -> e.getValue().getContainerData()));
     return deletionPolicy

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
index 83d746b..5c6c319 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ozone.container.common.interfaces
     .ContainerDeletionChoosingPolicy;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -58,7 +59,7 @@ public class RandomContainerDeletionChoosingPolicy
         LOG.debug("Select container {} for block deletion, "
             + "pending deletion blocks num: {}.",
             entry.getContainerID(),
-            entry.getNumPendingDeletionBlocks());
+            ((KeyValueContainerData)entry).getNumPendingDeletionBlocks());
       } else {
         break;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
index 68074fc..b17680c 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.ozone.container.common.interfaces
     .ContainerDeletionChoosingPolicy;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -41,14 +42,11 @@ public class TopNOrderedContainerDeletionChoosingPolicy
       LoggerFactory.getLogger(TopNOrderedContainerDeletionChoosingPolicy.class);
 
   /** customized comparator used to compare differentiate container data. **/
-  private static final Comparator<ContainerData> CONTAINER_DATA_COMPARATOR
-      = new Comparator<ContainerData>() {
-        @Override
-        public int compare(ContainerData c1, ContainerData c2) {
-          return Integer.compare(c2.getNumPendingDeletionBlocks(),
-              c1.getNumPendingDeletionBlocks());
-        }
-      };
+  private static final Comparator<KeyValueContainerData>
+        KEY_VALUE_CONTAINER_DATA_COMPARATOR = (KeyValueContainerData c1,
+                                               KeyValueContainerData c2) ->
+              Integer.compare(c2.getNumPendingDeletionBlocks(),
+                  c1.getNumPendingDeletionBlocks());
 
   @Override
   public List<ContainerData> chooseContainerForBlockDeletion(int count,
@@ -58,13 +56,15 @@ public class TopNOrderedContainerDeletionChoosingPolicy
         "Internal assertion: candidate containers cannot be null");
 
     List<ContainerData> result = new LinkedList<>();
-    List<ContainerData> orderedList = new LinkedList<>();
-    orderedList.addAll(candidateContainers.values());
-    Collections.sort(orderedList, CONTAINER_DATA_COMPARATOR);
+    List<KeyValueContainerData> orderedList = new LinkedList<>();
+    for (ContainerData entry : candidateContainers.values()) {
+      orderedList.add((KeyValueContainerData)entry);
+    }
+    Collections.sort(orderedList, KEY_VALUE_CONTAINER_DATA_COMPARATOR);
 
     // get top N list ordered by pending deletion blocks' number
     int currentCount = 0;
-    for (ContainerData entry : orderedList) {
+    for (KeyValueContainerData entry : orderedList) {
       if (currentCount < count) {
         if (entry.getNumPendingDeletionBlocks() > 0) {
           result.add(entry);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
index a7077d9..7f706b5 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.container.common.interfaces;
 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.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.
     StorageContainerException;
 
@@ -111,4 +112,9 @@ public interface Container extends RwLock {
    */
   BlockIterator blockIterator() throws IOException;
 
+  /**
+   * Returns containerReport for the container.
+   */
+  StorageContainerDatanodeProtocolProtos.ContainerInfo getContainerReport()
+      throws StorageContainerException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
index dce86e9..84c4f90 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.ozone.container.common.interfaces;
 
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
@@ -42,4 +43,16 @@ public interface ContainerDeletionChoosingPolicy {
   List<ContainerData> chooseContainerForBlockDeletion(int count,
       Map<Long, ContainerData> candidateContainers)
       throws StorageContainerException;
+
+  /**
+   * Determine if the container has suitable type for this policy.
+   * @param type  type of the container
+   * @return whether the container type suitable for this policy.
+   */
+  default boolean isValidContainerType(ContainerProtos.ContainerType type) {
+    if (type == ContainerProtos.ContainerType.KeyValueContainer) {
+      return true;
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/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 c96f997..0ea748a 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
@@ -28,6 +28,8 @@ 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.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.io.nativeio.NativeIO;
@@ -406,6 +408,50 @@ public class KeyValueContainer implements Container {
   }
 
   /**
+   * Returns KeyValueContainerReport for the KeyValueContainer.
+   */
+  @Override
+  public StorageContainerDatanodeProtocolProtos.ContainerInfo
+      getContainerReport() throws StorageContainerException{
+    StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
+        StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder();
+    ciBuilder.setContainerID(containerData.getContainerID())
+        .setReadCount(containerData.getReadCount())
+        .setWriteCount(containerData.getWriteCount())
+        .setReadBytes(containerData.getReadBytes())
+        .setWriteBytes(containerData.getWriteBytes())
+        .setUsed(containerData.getBytesUsed())
+        .setState(getHddsState())
+        .setDeleteTransactionId(containerData.getDeleteTransactionId());
+    return ciBuilder.build();
+  }
+
+  /**
+   * Returns LifeCycle State of the container.
+   * @return LifeCycle State of the container in HddsProtos format
+   * @throws StorageContainerException
+   */
+  private HddsProtos.LifeCycleState getHddsState()
+      throws StorageContainerException {
+    HddsProtos.LifeCycleState state;
+    switch (containerData.getState()) {
+    case OPEN:
+      state = HddsProtos.LifeCycleState.OPEN;
+      break;
+    case CLOSING:
+      state = HddsProtos.LifeCycleState.CLOSING;
+      break;
+    case CLOSED:
+      state = HddsProtos.LifeCycleState.CLOSED;
+      break;
+    default:
+      throw new StorageContainerException("Invalid Container state found: " +
+          containerData.getContainerID(), INVALID_CONTAINER_STATE);
+    }
+    return state;
+  }
+
+  /**
    * Returns container DB file.
    * @return
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/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 0705cf4..1d37437 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
@@ -32,7 +32,9 @@ import java.io.File;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import static java.lang.Math.max;
 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.METADATA_PATH;
@@ -61,6 +63,13 @@ public class KeyValueContainerData extends ContainerData {
 
   private File dbFile = null;
 
+  /**
+   * Number of pending deletion blocks in KeyValueContainer.
+   */
+  private final AtomicInteger numPendingDeletionBlocks;
+
+  private long deleteTransactionId;
+
   static {
     // Initialize YAML fields
     KV_YAML_FIELDS = Lists.newArrayList();
@@ -77,6 +86,8 @@ public class KeyValueContainerData extends ContainerData {
    */
   public KeyValueContainerData(long id, int size) {
     super(ContainerProtos.ContainerType.KeyValueContainer, id, size);
+    this.numPendingDeletionBlocks = new AtomicInteger(0);
+    this.deleteTransactionId = 0;
   }
 
   /**
@@ -88,6 +99,8 @@ public class KeyValueContainerData extends ContainerData {
   public KeyValueContainerData(long id, int layOutVersion, int size) {
     super(ContainerProtos.ContainerType.KeyValueContainer, id, layOutVersion,
         size);
+    this.numPendingDeletionBlocks = new AtomicInteger(0);
+    this.deleteTransactionId = 0;
   }
 
 
@@ -169,6 +182,47 @@ public class KeyValueContainerData extends ContainerData {
   }
 
   /**
+   * Increase the count of pending deletion blocks.
+   *
+   * @param numBlocks increment number
+   */
+  public void incrPendingDeletionBlocks(int numBlocks) {
+    this.numPendingDeletionBlocks.addAndGet(numBlocks);
+  }
+
+  /**
+   * Decrease the count of pending deletion blocks.
+   *
+   * @param numBlocks decrement number
+   */
+  public void decrPendingDeletionBlocks(int numBlocks) {
+    this.numPendingDeletionBlocks.addAndGet(-1 * numBlocks);
+  }
+
+  /**
+   * Get the number of pending deletion blocks.
+   */
+  public int getNumPendingDeletionBlocks() {
+    return this.numPendingDeletionBlocks.get();
+  }
+
+  /**
+   * Sets deleteTransactionId to latest delete transactionId for the container.
+   *
+   * @param transactionId latest transactionId of the container.
+   */
+  public void updateDeleteTransactionId(long transactionId) {
+    deleteTransactionId = max(transactionId, deleteTransactionId);
+  }
+
+  /**
+   * Return the latest deleteTransactionId of the container.
+   */
+  public long getDeleteTransactionId() {
+    return deleteTransactionId;
+  }
+
+  /**
    * Returns a ProtoBuf Message from ContainerData.
    *
    * @return Protocol Buffer Message

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
index 42db66d..12ce163 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
@@ -57,6 +57,7 @@ public class TestKeyValueContainerData {
     assertEquals(val.get(), kvData.getReadCount());
     assertEquals(val.get(), kvData.getWriteCount());
     assertEquals(val.get(), kvData.getKeyCount());
+    assertEquals(val.get(), kvData.getNumPendingDeletionBlocks());
     assertEquals(MAXSIZE, kvData.getMaxSizeGB());
 
     kvData.setState(state);
@@ -68,6 +69,7 @@ public class TestKeyValueContainerData {
     kvData.incrReadCount();
     kvData.incrWriteCount();
     kvData.incrKeyCount();
+    kvData.incrPendingDeletionBlocks(1);
 
     assertEquals(state, kvData.getState());
     assertEquals(containerDBType, kvData.getContainerDBType());
@@ -79,7 +81,7 @@ public class TestKeyValueContainerData {
     assertEquals(1, kvData.getReadCount());
     assertEquals(1, kvData.getWriteCount());
     assertEquals(1, kvData.getKeyCount());
-
+    assertEquals(1, kvData.getNumPendingDeletionBlocks());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/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 4ca4124..dcf4022 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
@@ -203,8 +203,12 @@ 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();
+    // NOTE: this test assumes that all the container is KetValueContainer and
+    // have DeleteTransactionId in KetValueContainerData. If other
+    // types is going to be added, this test should be checked.
+    long transactionId = ((KeyValueContainerData)containerMap
+        .get(containerData.get(0).getContainerID()).getContainerData())
+        .getDeleteTransactionId();
 
 
     // Number of deleted blocks in container should be equal to 0 before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
index 2524de6..94cdf61 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
@@ -139,7 +139,9 @@ public class TestBlockDeletion {
     Assert.assertTrue(verifyBlocksCreated(omKeyLocationInfoGroupList));
     // No containers with deleted blocks
     Assert.assertTrue(containerIdsWithDeletedBlocks.isEmpty());
-    // Delete transactionIds for the containers should be 0
+    // Delete transactionIds for the containers should be 0.
+    // NOTE: this test assumes that all the container is KetValueContainer. If
+    // other container types is going to be added, this test should be checked.
     matchContainerTransactionIds();
     om.deleteKey(keyArgs);
     Thread.sleep(5000);
@@ -215,8 +217,9 @@ public class TestBlockDeletion {
         Assert.assertEquals(
             scm.getContainerInfo(containerId).getDeleteTransactionId(), 0);
       }
-      Assert.assertEquals(dnContainerSet.getContainer(containerId)
-              .getContainerData().getDeleteTransactionId(),
+      Assert.assertEquals(((KeyValueContainerData)dnContainerSet
+              .getContainer(containerId).getContainerData())
+              .getDeleteTransactionId(),
           scm.getContainerInfo(containerId).getDeleteTransactionId());
     }
   }


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

Branch: refs/heads/YARN-3409
Commit: 140f22b67f852d737d123d73208f7607ea92df85
Parents: 2c0a6c8
Author: Sunil G <su...@apache.org>
Authored: Sun Apr 1 18:07:07 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:57 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/140f22b6/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/140f22b6/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/140f22b6/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/140f22b6/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/140f22b6/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/140f22b6/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/140f22b6/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/140f22b6/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/140f22b6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
index c444b6e..b6c0cc1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.net.Node;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -284,6 +285,16 @@ public class MockNodes {
     public Map<String, Long> getAllocationTagsWithCount() {
       return null;
     }
+    
+    public void setNodeAttributes(String prefix,
+        Set<NodeAttribute> nodeAttributes) {
+
+    }
+
+    @Override
+    public Map<String, Set<NodeAttribute>> getAllNodeAttributes() {
+      return null;
+    }
 
     @Override
     public RMContext getRMContext() {


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


[16/50] [abbrv] hadoop git commit: HDDS-356. Support ColumnFamily based RockDBStore and TableStore. Contributed by Anu Engineer.

Posted by su...@apache.org.
HDDS-356. Support ColumnFamily based RockDBStore and TableStore.
Contributed by Anu Engineer.


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

Branch: refs/heads/YARN-3409
Commit: b021249ac84abe31c9d30d73ed483bea2acdbaab
Parents: af4b705
Author: Anu Engineer <ae...@apache.org>
Authored: Wed Aug 22 18:55:14 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Wed Aug 22 18:55:14 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/utils/db/DBStore.java     |  93 +++++++
 .../org/apache/hadoop/utils/db/RDBStore.java    | 252 +++++++++++++++++++
 .../hadoop/utils/db/RDBStoreIterator.java       |  88 +++++++
 .../org/apache/hadoop/utils/db/RDBTable.java    | 173 +++++++++++++
 .../java/org/apache/hadoop/utils/db/Table.java  | 150 +++++++++++
 .../apache/hadoop/utils/db/TableIterator.java   |  50 ++++
 .../apache/hadoop/utils/db/package-info.java    |  22 ++
 .../apache/hadoop/utils/db/TestRDBStore.java    | 246 ++++++++++++++++++
 .../hadoop/utils/db/TestRDBTableStore.java      | 189 ++++++++++++++
 .../apache/hadoop/utils/db/package-info.java    |  22 ++
 10 files changed, 1285 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java
new file mode 100644
index 0000000..a817f4f
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java
@@ -0,0 +1,93 @@
+/*
+ * 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.db;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+/**
+ * The DBStore interface provides the ability to create Tables, which store
+ * a specific type of Key-Value pair. Some DB interfaces like LevelDB will not
+ * be able to do this. In those case a Table creation will map to a default
+ * store.
+ *
+ */
+@InterfaceStability.Evolving
+public interface DBStore extends AutoCloseable {
+
+  /**
+   * Gets an existing TableStore.
+   *
+   * @param name - Name of the TableStore to get
+   * @return - TableStore.
+   * @throws IOException on Failure
+   */
+  Table getTable(String name) throws IOException;
+
+  /**
+   * Lists the Known list of Tables in a DB.
+   *
+   * @return List of Tables, in case of Rocks DB and LevelDB we will return at
+   * least one entry called DEFAULT.
+   * @throws IOException on Failure
+   */
+  ArrayList<Table> listTables() throws IOException;
+
+  /**
+   * Compact the entire database.
+   *
+   * @throws IOException on Failure
+   */
+  void compactDB() throws IOException;
+
+  /**
+   * Moves a key from the Source Table to the destination Table.
+   *
+   * @param key - Key to move.
+   * @param source - Source Table.
+   * @param dest - Destination Table.
+   * @throws IOException on Failure
+   */
+  void move(byte[] key, Table source, Table dest) throws IOException;
+
+  /**
+   * Moves a key from the Source Table to the destination Table and updates the
+   * destination to the new value.
+   *
+   * @param key - Key to move.
+   * @param value - new value to write to the destination table.
+   * @param source - Source Table.
+   * @param dest - Destination Table.
+   * @throws IOException on Failure
+   */
+  void move(byte[] key, byte[] value, Table source, Table dest)
+      throws IOException;
+
+  /**
+   * Returns an estimated count of keys in this DB.
+   *
+   * @return long, estimate of keys in the DB.
+   */
+  long getEstimatedKeyCount() throws IOException;
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
new file mode 100644
index 0000000..c719d31
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
@@ -0,0 +1,252 @@
+/*
+ * 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.db;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.utils.RocksDBStoreMBean;
+import org.apache.ratis.shaded.com.google.common.annotations.VisibleForTesting;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+import org.rocksdb.WriteOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.management.ObjectName;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * RocksDB Store that supports creating Tables in DB.
+ */
+public class RDBStore implements DBStore {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RDBStore.class);
+  private final RocksDB db;
+  private final File dbLocation;
+  private final WriteOptions writeOptions;
+  private final DBOptions dbOptions;
+  private final Hashtable<String, ColumnFamilyHandle> handleTable;
+  private ObjectName statMBeanName;
+
+  public RDBStore(File dbFile, DBOptions options, List<String> families)
+      throws IOException {
+    Preconditions.checkNotNull(dbFile, "DB file location cannot be null");
+    Preconditions.checkNotNull(families);
+    Preconditions.checkArgument(families.size() > 0);
+    handleTable = new Hashtable<>();
+
+    final List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        new ArrayList<>();
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    for (String family : families) {
+      columnFamilyDescriptors.add(
+          new ColumnFamilyDescriptor(family.getBytes(StandardCharsets.UTF_8),
+              new ColumnFamilyOptions()));
+    }
+
+    dbOptions = options;
+    dbLocation = dbFile;
+    // TODO: Read from the next Config.
+    writeOptions = new WriteOptions();
+
+    try {
+      db = RocksDB.open(dbOptions, dbLocation.getAbsolutePath(),
+          columnFamilyDescriptors, columnFamilyHandles);
+
+      for (int x = 0; x < columnFamilyHandles.size(); x++) {
+        handleTable.put(
+            DFSUtil.bytes2String(columnFamilyHandles.get(x).getName()),
+            columnFamilyHandles.get(x));
+      }
+
+      if (dbOptions.statistics() != null) {
+        Map<String, String> jmxProperties = new HashMap<>();
+        jmxProperties.put("dbName", dbFile.getName());
+        statMBeanName = MBeans.register("Ozone", "RocksDbStore", jmxProperties,
+            new RocksDBStoreMBean(dbOptions.statistics()));
+        if (statMBeanName == null) {
+          LOG.warn("jmx registration failed during RocksDB init, db path :{}",
+              dbFile.getAbsolutePath());
+        }
+      }
+
+    } catch (RocksDBException e) {
+      throw toIOException(
+          "Failed init RocksDB, db path : " + dbFile.getAbsolutePath(), e);
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("RocksDB successfully opened.");
+      LOG.debug("[Option] dbLocation= {}", dbLocation.getAbsolutePath());
+      LOG.debug("[Option] createIfMissing = {}", options.createIfMissing());
+      LOG.debug("[Option] maxOpenFiles= {}", options.maxOpenFiles());
+    }
+  }
+
+  public static IOException toIOException(String msg, RocksDBException e) {
+    String statusCode = e.getStatus() == null ? "N/A" :
+        e.getStatus().getCodeString();
+    String errMessage = e.getMessage() == null ? "Unknown error" :
+        e.getMessage();
+    String output = msg + "; status : " + statusCode
+        + "; message : " + errMessage;
+    return new IOException(output, e);
+  }
+
+  @Override
+  public void compactDB() throws IOException {
+    if (db != null) {
+      try {
+        db.compactRange();
+      } catch (RocksDBException e) {
+        throw toIOException("Failed to compact db", e);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+
+    for (final ColumnFamilyHandle handle : handleTable.values()) {
+      handle.close();
+    }
+    if (dbOptions != null) {
+      dbOptions.close();
+    }
+    if (writeOptions != null) {
+      writeOptions.close();
+    }
+    if (statMBeanName != null) {
+      MBeans.unregister(statMBeanName);
+      statMBeanName = null;
+    }
+    if (db != null) {
+      db.close();
+    }
+  }
+
+  @Override
+  public void move(byte[] key, Table source, Table dest) throws IOException {
+    RDBTable sourceTable;
+    RDBTable destTable;
+    if (source instanceof RDBTable) {
+      sourceTable = (RDBTable) source;
+    } else {
+      LOG.error("Unexpected Table type. Expected RocksTable Store for Source.");
+      throw new IOException("Unexpected TableStore Type in source. Expected "
+          + "RocksDBTable.");
+    }
+
+    if (dest instanceof RDBTable) {
+      destTable = (RDBTable) dest;
+    } else {
+      LOG.error("Unexpected Table type. Expected RocksTable Store for Dest.");
+      throw new IOException("Unexpected TableStore Type in dest. Expected "
+          + "RocksDBTable.");
+    }
+    try (WriteBatch batch = new WriteBatch()) {
+      byte[] value = sourceTable.get(key);
+      batch.put(destTable.getHandle(), key, value);
+      batch.delete(sourceTable.getHandle(), key);
+      db.write(writeOptions, batch);
+    } catch (RocksDBException rockdbException) {
+      LOG.error("Move of key failed. Key:{}", DFSUtil.bytes2String(key));
+      throw toIOException("Unable to move key: " + DFSUtil.bytes2String(key),
+          rockdbException);
+    }
+  }
+
+  @Override
+  public void move(byte[] key, byte[] value, Table source,
+      Table dest) throws IOException {
+    RDBTable sourceTable;
+    RDBTable destTable;
+    if (source instanceof RDBTable) {
+      sourceTable = (RDBTable) source;
+    } else {
+      LOG.error("Unexpected Table type. Expected RocksTable Store for Source.");
+      throw new IOException("Unexpected TableStore Type in source. Expected "
+          + "RocksDBTable.");
+    }
+
+    if (dest instanceof RDBTable) {
+      destTable = (RDBTable) dest;
+    } else {
+      LOG.error("Unexpected Table type. Expected RocksTable Store for Dest.");
+      throw new IOException("Unexpected TableStore Type in dest. Expected "
+          + "RocksDBTable.");
+    }
+    try (WriteBatch batch = new WriteBatch()) {
+      batch.put(destTable.getHandle(), key, value);
+      batch.delete(sourceTable.getHandle(), key);
+      db.write(writeOptions, batch);
+    } catch (RocksDBException rockdbException) {
+      LOG.error("Move of key failed. Key:{}", DFSUtil.bytes2String(key));
+      throw toIOException("Unable to move key: " + DFSUtil.bytes2String(key),
+          rockdbException);
+    }
+  }
+
+  @Override
+  public long getEstimatedKeyCount() throws IOException {
+    try {
+      return Long.parseLong(db.getProperty("rocksdb.estimate-num-keys"));
+    } catch (RocksDBException e) {
+      throw toIOException("Unable to get the estimated count.", e);
+    }
+  }
+
+  @VisibleForTesting
+  protected ObjectName getStatMBeanName() {
+    return statMBeanName;
+  }
+
+  @Override
+  public Table getTable(String name) throws IOException {
+    ColumnFamilyHandle handle = handleTable.get(name);
+    if (handle == null) {
+      throw new IOException("No such table in this DB. TableName : " + name);
+    }
+    return new RDBTable(this.db, handle, this.writeOptions);
+  }
+
+  @Override
+  public ArrayList<Table> listTables() throws IOException {
+    ArrayList<Table> returnList = new ArrayList<>();
+    for (ColumnFamilyHandle handle: handleTable.values())  {
+      returnList.add(new RDBTable(db, handle, writeOptions));
+    }
+    return returnList;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStoreIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStoreIterator.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStoreIterator.java
new file mode 100644
index 0000000..f1f2df6
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStoreIterator.java
@@ -0,0 +1,88 @@
+/*
+ * 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.db;
+
+import org.apache.hadoop.utils.db.Table.KeyValue;
+import org.rocksdb.RocksIterator;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+import java.util.function.Consumer;
+
+/**
+ * RocksDB store iterator.
+ */
+public class RDBStoreIterator implements TableIterator<KeyValue> {
+
+  private RocksIterator rocksDBIterator;
+
+  public RDBStoreIterator(RocksIterator iterator) {
+    this.rocksDBIterator = iterator;
+    rocksDBIterator.seekToFirst();
+  }
+
+  @Override
+  public void forEachRemaining(Consumer<? super KeyValue> action) {
+    while(hasNext()) {
+      action.accept(next());
+    }
+  }
+
+  @Override
+  public boolean hasNext() {
+    return rocksDBIterator.isValid();
+  }
+
+  @Override
+  public Table.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();
+  }
+
+  @Override
+  public KeyValue seek(byte[] key) {
+    rocksDBIterator.seek(key);
+    if (rocksDBIterator.isValid()) {
+      return KeyValue.create(rocksDBIterator.key(),
+          rocksDBIterator.value());
+    }
+    return null;
+  }
+
+  @Override
+  public void close() throws IOException {
+    rocksDBIterator.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBTable.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBTable.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBTable.java
new file mode 100644
index 0000000..8cf6b35
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBTable.java
@@ -0,0 +1,173 @@
+/*
+ * 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.db;
+
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+import org.rocksdb.WriteOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+/**
+ * RocksDB implementation of ozone metadata store.
+ */
+public class RDBTable implements Table {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RDBTable.class);
+
+  private final RocksDB db;
+  private final ColumnFamilyHandle handle;
+  private final WriteOptions writeOptions;
+
+  /**
+   * Constructs a TableStore.
+   *
+   * @param db - DBstore that we are using.
+   * @param handle - ColumnFamily Handle.
+   * @param writeOptions - RocksDB write Options.
+   */
+  public RDBTable(RocksDB db, ColumnFamilyHandle handle,
+      WriteOptions writeOptions) {
+    this.db = db;
+    this.handle = handle;
+    this.writeOptions = writeOptions;
+  }
+
+  /**
+   * Converts RocksDB exception to IOE.
+   * @param msg  - Message to add to exception.
+   * @param e - Original Exception.
+   * @return  IOE.
+   */
+  public static IOException toIOException(String msg, RocksDBException e) {
+    String statusCode = e.getStatus() == null ? "N/A" :
+        e.getStatus().getCodeString();
+    String errMessage = e.getMessage() == null ? "Unknown error" :
+        e.getMessage();
+    String output = msg + "; status : " + statusCode
+        + "; message : " + errMessage;
+    return new IOException(output, e);
+  }
+
+  /**
+   * Returns the Column family Handle.
+   *
+   * @return ColumnFamilyHandle.
+   */
+  @Override
+  public ColumnFamilyHandle getHandle() {
+    return handle;
+  }
+
+  @Override
+  public void put(byte[] key, byte[] value) throws IOException {
+    try {
+      db.put(handle, writeOptions, key, value);
+    } catch (RocksDBException e) {
+      LOG.error("Failed to write to DB. Key: {}", new String(key,
+          StandardCharsets.UTF_8));
+      throw toIOException("Failed to put key-value to metadata "
+          + "store", e);
+    }
+  }
+
+  @Override
+  public boolean isEmpty() throws IOException {
+    try (TableIterator<KeyValue> keyIter = iterator()) {
+      keyIter.seekToFirst();
+      return !keyIter.hasNext();
+    }
+  }
+
+  @Override
+  public byte[] get(byte[] key) throws IOException {
+    try {
+      return db.get(handle, key);
+    } catch (RocksDBException e) {
+      throw toIOException(
+          "Failed to get the value for the given key", e);
+    }
+  }
+
+  @Override
+  public void delete(byte[] key) throws IOException {
+    try {
+      db.delete(handle, key);
+    } catch (RocksDBException e) {
+      throw toIOException("Failed to delete the given key", e);
+    }
+  }
+
+  @Override
+  public void writeBatch(WriteBatch operation) throws IOException {
+    try {
+      db.write(writeOptions, operation);
+    } catch (RocksDBException e) {
+      throw toIOException("Batch write operation failed", e);
+    }
+  }
+
+//  @Override
+//  public void iterate(byte[] from, EntryConsumer consumer)
+//      throws IOException {
+//
+//    try (RocksIterator it = db.newIterator(handle)) {
+//      if (from != null) {
+//        it.seek(from);
+//      } else {
+//        it.seekToFirst();
+//      }
+//      while (it.isValid()) {
+//        if (!consumer.consume(it.key(), it.value())) {
+//          break;
+//        }
+//        it.next();
+//      }
+//    }
+//  }
+
+  @Override
+  public TableIterator<KeyValue> iterator() {
+    ReadOptions readOptions = new ReadOptions();
+    return new RDBStoreIterator(db.newIterator(handle, readOptions));
+  }
+
+  @Override
+  public String getName() throws IOException {
+    try {
+      return DFSUtil.bytes2String(this.getHandle().getName());
+    } catch (RocksDBException rdbEx) {
+      throw toIOException("Unable to get the table name.", rdbEx);
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    // Nothing do for a Column Family.
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/Table.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/Table.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/Table.java
new file mode 100644
index 0000000..3942585
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/Table.java
@@ -0,0 +1,150 @@
+/*
+ * 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.db;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.WriteBatch;
+
+import java.io.IOException;
+
+/**
+ * Interface for key-value store that stores ozone metadata. Ozone metadata is
+ * stored as key value pairs, both key and value are arbitrary byte arrays. Each
+ * Table Stores a certain kind of keys and values. This allows a DB to have
+ * different kind of tables.
+ */
+@InterfaceStability.Evolving
+public interface Table extends AutoCloseable {
+
+  /**
+   * Puts a key-value pair into the store.
+   *
+   * @param key metadata key
+   * @param value metadata value
+   */
+  void put(byte[] key, byte[] value) throws IOException;
+
+  /**
+   * @return true if the metadata store is empty.
+   * @throws IOException on Failure
+   */
+  boolean isEmpty() throws IOException;
+
+  /**
+   * Returns the value mapped to the given key in byte array or returns null
+   * if the key is not found.
+   *
+   * @param key metadata key
+   * @return value in byte array or null if the key is not found.
+   * @throws IOException on Failure
+   */
+  byte[] get(byte[] key) throws IOException;
+
+  /**
+   * Deletes a key from the metadata store.
+   *
+   * @param key metadata key
+   * @throws IOException on Failure
+   */
+  void delete(byte[] key) throws IOException;
+
+  /**
+   * Return the Column Family handle. TODO: This leaks an RockDB abstraction
+   * into Ozone code, cleanup later.
+   *
+   * @return ColumnFamilyHandle
+   */
+  ColumnFamilyHandle getHandle();
+
+  /**
+   * A batch of PUT, DELETE operations handled as a single atomic write.
+   *
+   * @throws IOException write fails
+   */
+  void writeBatch(WriteBatch operation) throws IOException;
+
+  /**
+   * Returns the iterator for this metadata store.
+   *
+   * @return MetaStoreIterator
+   */
+  TableIterator<KeyValue> iterator();
+
+  /**
+   * Returns the Name of this Table.
+   * @return - Table Name.
+   * @throws IOException on failure.
+   */
+  String getName() throws IOException;
+
+  /**
+   * 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 - Key Bytes
+     * @param value - Value bytes
+     */
+    private KeyValue(byte[] key, byte[] value) {
+      this.key = key;
+      this.value = value;
+    }
+
+    /**
+     * Create a KeyValue pair.
+     *
+     * @param key - Key Bytes
+     * @param value - Value bytes
+     * @return KeyValue object.
+     */
+    public static KeyValue create(byte[] key, byte[] value) {
+      return new KeyValue(key, 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;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableIterator.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableIterator.java
new file mode 100644
index 0000000..83a8f3c
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableIterator.java
@@ -0,0 +1,50 @@
+/*
+ * 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.db;
+
+import java.io.Closeable;
+import java.util.Iterator;
+
+/**
+ * Iterator for MetaDataStore DB.
+ *
+ * @param <T>
+ */
+public interface TableIterator<T> extends Iterator<T>, Closeable {
+
+  /**
+   * seek to first entry.
+   */
+  void seekToFirst();
+
+  /**
+   * seek to last entry.
+   */
+  void seekToLast();
+
+  /**
+   * Seek to the specific key.
+   *
+   * @param key - Bytes that represent the key.
+   * @return T.
+   */
+  T seek(byte[] key);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/package-info.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/package-info.java
new file mode 100644
index 0000000..17d676d
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ *
+ */
+/**
+ * Database interfaces for Ozone.
+ */
+package org.apache.hadoop.utils.db;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
new file mode 100644
index 0000000..94a650b
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
@@ -0,0 +1,246 @@
+/*
+ * 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.db;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+import org.rocksdb.DBOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.Statistics;
+import org.rocksdb.StatsLevel;
+
+import javax.management.MBeanServer;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * RDBStore Tests.
+ */
+public class TestRDBStore {
+  private final List<String> families =
+      Arrays.asList(DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY),
+          "First", "Second", "Third",
+          "Fourth", "Fifth",
+          "Sixth");
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+  private RDBStore rdbStore = null;
+  private DBOptions options = null;
+
+  @Before
+  public void setUp() throws Exception {
+    options = new DBOptions();
+    options.setCreateIfMissing(true);
+    options.setCreateMissingColumnFamilies(true);
+
+    Statistics statistics = new Statistics();
+    statistics.setStatsLevel(StatsLevel.ALL);
+    options = options.setStatistics(statistics);
+    rdbStore = new RDBStore(folder.newFolder(), options, families);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (rdbStore != null) {
+      rdbStore.close();
+    }
+  }
+
+  @Test
+  public void compactDB() throws Exception {
+    try (RDBStore newStore =
+             new RDBStore(folder.newFolder(), options, families)) {
+      Assert.assertNotNull("DB Store cannot be null", newStore);
+      try (Table firstTable = newStore.getTable(families.get(1))) {
+        Assert.assertNotNull("Table cannot be null", firstTable);
+        for (int x = 0; x < 100; x++) {
+          byte[] key =
+              RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+          byte[] value =
+              RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+          firstTable.put(key, value);
+        }
+      }
+      // This test does not assert anything if there is any error this test
+      // will throw and fail.
+      newStore.compactDB();
+    }
+  }
+
+  @Test
+  public void close() throws Exception {
+    RDBStore newStore =
+        new RDBStore(folder.newFolder(), options, families);
+    Assert.assertNotNull("DBStore cannot be null", newStore);
+    // This test does not assert anything if there is any error this test
+    // will throw and fail.
+    newStore.close();
+  }
+
+  @Test
+  public void moveKey() throws Exception {
+    byte[] key =
+        RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+    byte[] value =
+        RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+
+    try (Table firstTable = rdbStore.getTable(families.get(1))) {
+      firstTable.put(key, value);
+      try (Table secondTable = rdbStore.getTable(families.get(2))) {
+        rdbStore.move(key, firstTable, secondTable);
+        byte[] newvalue = secondTable.get(key);
+        // Make sure we have value in the second table
+        Assert.assertNotNull(newvalue);
+        //and it is same as what we wrote to the FirstTable
+        Assert.assertArrayEquals(value, newvalue);
+      }
+      // After move this key must not exist in the first table.
+      Assert.assertNull(firstTable.get(key));
+    }
+  }
+
+  @Test
+  public void moveWithValue() throws Exception {
+    byte[] key =
+        RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+    byte[] value =
+        RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+
+    byte[] nextValue =
+        RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+    try (Table firstTable = rdbStore.getTable(families.get(1))) {
+      firstTable.put(key, value);
+      try (Table secondTable = rdbStore.getTable(families.get(2))) {
+        rdbStore.move(key, nextValue, firstTable, secondTable);
+        byte[] newvalue = secondTable.get(key);
+        // Make sure we have value in the second table
+        Assert.assertNotNull(newvalue);
+        //and it is not same as what we wrote to the FirstTable, and equals
+        // the new value.
+        Assert.assertArrayEquals(nextValue, nextValue);
+      }
+    }
+
+  }
+
+  @Test
+  public void getEstimatedKeyCount() throws Exception {
+    try (RDBStore newStore =
+             new RDBStore(folder.newFolder(), options, families)) {
+      Assert.assertNotNull("DB Store cannot be null", newStore);
+      // Write 100 keys to the first table.
+      try (Table firstTable = newStore.getTable(families.get(1))) {
+        Assert.assertNotNull("Table cannot be null", firstTable);
+        for (int x = 0; x < 100; x++) {
+          byte[] key =
+              RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+          byte[] value =
+              RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+          firstTable.put(key, value);
+        }
+      }
+
+      // Write 100 keys to the secondTable table.
+      try (Table secondTable = newStore.getTable(families.get(2))) {
+        Assert.assertNotNull("Table cannot be null", secondTable);
+        for (int x = 0; x < 100; x++) {
+          byte[] key =
+              RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+          byte[] value =
+              RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+          secondTable.put(key, value);
+        }
+      }
+      // Let us make sure that our estimate is not off by 10%
+      Assert.assertTrue(newStore.getEstimatedKeyCount() > 180
+          || newStore.getEstimatedKeyCount() < 220);
+    }
+  }
+
+  @Test
+  public void getStatMBeanName() throws Exception {
+
+    try (Table firstTable = rdbStore.getTable(families.get(1))) {
+      for (int y = 0; y < 100; y++) {
+        byte[] key =
+            RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+        byte[] value =
+            RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+        firstTable.put(key, value);
+      }
+    }
+    MBeanServer platformMBeanServer =
+        ManagementFactory.getPlatformMBeanServer();
+    Thread.sleep(2000);
+
+    Object keysWritten = platformMBeanServer
+        .getAttribute(rdbStore.getStatMBeanName(), "NUMBER_KEYS_WRITTEN");
+
+    Assert.assertTrue(((Long) keysWritten) >= 99L);
+
+    Object dbWriteAverage = platformMBeanServer
+        .getAttribute(rdbStore.getStatMBeanName(), "DB_WRITE_AVERAGE");
+    Assert.assertTrue((double) dbWriteAverage > 0);
+  }
+
+  @Test
+  public void getTable() throws Exception {
+    for (String tableName : families) {
+      try (Table table = rdbStore.getTable(tableName)) {
+        Assert.assertNotNull(tableName + "is null", table);
+      }
+    }
+    thrown.expect(IOException.class);
+    rdbStore.getTable("ATableWithNoName");
+  }
+
+  @Test
+  public void listTables() throws Exception {
+    List<Table> tableList = rdbStore.listTables();
+    Assert.assertNotNull("Table list cannot be null", tableList);
+    Map<String, Table> hashTable = new HashMap<>();
+
+    for (Table t : tableList) {
+      hashTable.put(t.getName(), t);
+    }
+
+    int count = families.size();
+    // Assert that we have all the tables in the list and no more.
+    for (String name : families) {
+      Assert.assertTrue(hashTable.containsKey(name));
+      count--;
+    }
+    Assert.assertEquals(0, count);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
new file mode 100644
index 0000000..677a1f9
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
@@ -0,0 +1,189 @@
+/*
+ * 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.db;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.rocksdb.DBOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.Statistics;
+import org.rocksdb.StatsLevel;
+import org.rocksdb.WriteBatch;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Tests for RocksDBTable Store.
+ */
+public class TestRDBTableStore {
+  private static int count = 0;
+  private final List<String> families =
+      Arrays.asList(DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY),
+          "First", "Second", "Third",
+          "Fourth", "Fifth",
+          "Sixth");
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private RDBStore rdbStore = null;
+  private DBOptions options = null;
+
+  @Before
+  public void setUp() throws Exception {
+    options = new DBOptions();
+    options.setCreateIfMissing(true);
+    options.setCreateMissingColumnFamilies(true);
+
+    Statistics statistics = new Statistics();
+    statistics.setStatsLevel(StatsLevel.ALL);
+    options = options.setStatistics(statistics);
+    rdbStore = new RDBStore(folder.newFolder(), options, families);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (rdbStore != null) {
+      rdbStore.close();
+    }
+  }
+
+  @Test
+  public void toIOException() {
+  }
+
+  @Test
+  public void getHandle() throws Exception {
+    try (Table testTable = rdbStore.getTable("First")) {
+      Assert.assertNotNull(testTable);
+      Assert.assertNotNull(testTable.getHandle());
+    }
+  }
+
+  @Test
+  public void putGetAndEmpty() throws Exception {
+    try (Table testTable = rdbStore.getTable("First")) {
+      byte[] key =
+          RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+      byte[] value =
+          RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+      testTable.put(key, value);
+      Assert.assertFalse(testTable.isEmpty());
+      byte[] readValue = testTable.get(key);
+      Assert.assertArrayEquals(value, readValue);
+    }
+    try (Table secondTable = rdbStore.getTable("Second")) {
+      Assert.assertTrue(secondTable.isEmpty());
+    }
+  }
+
+  @Test
+  public void delete() throws Exception {
+    List<byte[]> deletedKeys = new LinkedList<>();
+    List<byte[]> validKeys = new LinkedList<>();
+    byte[] value =
+        RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+    for (int x = 0; x < 100; x++) {
+      deletedKeys.add(
+          RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8));
+    }
+
+    for (int x = 0; x < 100; x++) {
+      validKeys.add(
+          RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8));
+    }
+
+    // Write all the keys and delete the keys scheduled for delete.
+    //Assert we find only expected keys in the Table.
+    try (Table testTable = rdbStore.getTable("Fourth")) {
+      for (int x = 0; x < deletedKeys.size(); x++) {
+        testTable.put(deletedKeys.get(x), value);
+        testTable.delete(deletedKeys.get(x));
+      }
+
+      for (int x = 0; x < validKeys.size(); x++) {
+        testTable.put(validKeys.get(x), value);
+      }
+
+      for (int x = 0; x < validKeys.size(); x++) {
+        Assert.assertNotNull(testTable.get(validKeys.get(0)));
+      }
+
+      for (int x = 0; x < deletedKeys.size(); x++) {
+        Assert.assertNull(testTable.get(deletedKeys.get(0)));
+      }
+    }
+  }
+
+  @Test
+  public void writeBatch() throws Exception {
+    WriteBatch batch = new WriteBatch();
+    try (Table testTable = rdbStore.getTable("Fifth")) {
+      byte[] key =
+          RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+      byte[] value =
+          RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+      batch.put(testTable.getHandle(), key, value);
+      testTable.writeBatch(batch);
+      Assert.assertNotNull(testTable.get(key));
+    }
+    batch.close();
+  }
+
+  private static boolean consume(Table.KeyValue keyValue) {
+    count++;
+    Assert.assertNotNull(keyValue.getKey());
+    return true;
+  }
+
+  @Test
+  public void forEachAndIterator() throws Exception {
+    final int iterCount = 100;
+    try (Table testTable = rdbStore.getTable("Sixth")) {
+      for (int x = 0; x < iterCount; x++) {
+        byte[] key =
+            RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+        byte[] value =
+            RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+        testTable.put(key, value);
+      }
+      int localCount = 0;
+      try (TableIterator<Table.KeyValue> iter = testTable.iterator()) {
+        while (iter.hasNext()) {
+          Table.KeyValue keyValue = iter.next();
+          localCount++;
+        }
+
+        Assert.assertEquals(iterCount, localCount);
+        iter.seekToFirst();
+        iter.forEachRemaining(TestRDBTableStore::consume);
+        Assert.assertEquals(iterCount, count);
+
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/package-info.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/package-info.java
new file mode 100644
index 0000000..f06855e
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ *
+ */
+/**
+ * Tests for the DB Utilities.
+ */
+package org.apache.hadoop.utils.db;
\ 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


[10/50] [abbrv] hadoop git commit: YARN-8298. Added express upgrade for YARN service. Contributed by Chandni Singh

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/utils/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/utils/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/utils/TestServiceApiUtil.java
new file mode 100644
index 0000000..98e7474
--- /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/utils/TestServiceApiUtil.java
@@ -0,0 +1,743 @@
+/*
+ * 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.utils;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.service.ServiceTestUtils;
+import org.apache.hadoop.yarn.service.api.records.Artifact;
+import org.apache.hadoop.yarn.service.api.records.Component;
+import org.apache.hadoop.yarn.service.api.records.KerberosPrincipal;
+import org.apache.hadoop.yarn.service.api.records.PlacementConstraint;
+import org.apache.hadoop.yarn.service.api.records.PlacementPolicy;
+import org.apache.hadoop.yarn.service.api.records.PlacementScope;
+import org.apache.hadoop.yarn.service.api.records.PlacementType;
+import org.apache.hadoop.yarn.service.api.records.Resource;
+import org.apache.hadoop.yarn.service.api.records.Service;
+import org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.hadoop.yarn.service.conf.RestApiConstants.DEFAULT_UNLIMITED_LIFETIME;
+import static org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test for ServiceApiUtil helper methods.
+ */
+public class TestServiceApiUtil extends ServiceTestUtils {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestServiceApiUtil.class);
+  private static final String EXCEPTION_PREFIX = "Should have thrown " +
+      "exception: ";
+  private static final String NO_EXCEPTION_PREFIX = "Should not have thrown " +
+      "exception: ";
+
+  private static final String LEN_64_STR =
+      "abcdefghijklmnopqrstuvwxyz0123456789abcdefghijklmnopqrstuvwxyz01";
+
+  private static final YarnConfiguration CONF_DEFAULT_DNS = new
+      YarnConfiguration();
+  private static final YarnConfiguration CONF_DNS_ENABLED = new
+      YarnConfiguration();
+
+  @BeforeClass
+  public static void init() {
+    CONF_DNS_ENABLED.setBoolean(RegistryConstants.KEY_DNS_ENABLED, true);
+  }
+
+  @Test(timeout = 90000)
+  public void testResourceValidation() throws Exception {
+    assertEquals(RegistryConstants.MAX_FQDN_LABEL_LENGTH + 1, LEN_64_STR
+        .length());
+
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+
+    Service app = new Service();
+
+    // no name
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with no name");
+    } catch (IllegalArgumentException e) {
+      assertEquals(ERROR_APPLICATION_NAME_INVALID, e.getMessage());
+    }
+
+    app.setName("test");
+    // no version
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + " service with no version");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(ERROR_APPLICATION_VERSION_INVALID,
+          app.getName()), e.getMessage());
+    }
+
+    app.setVersion("v1");
+    // bad format name
+    String[] badNames = {"4finance", "Finance", "finance@home", LEN_64_STR};
+    for (String badName : badNames) {
+      app.setName(badName);
+      try {
+        ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+        Assert.fail(EXCEPTION_PREFIX + "service with bad name " + badName);
+      } catch (IllegalArgumentException e) {
+
+      }
+    }
+
+    // launch command not specified
+    app.setName(LEN_64_STR);
+    Component comp = new Component().name("comp1");
+    app.addComponent(comp);
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DEFAULT_DNS);
+      Assert.fail(EXCEPTION_PREFIX + "service with no launch command");
+    } catch (IllegalArgumentException e) {
+      assertEquals(RestApiErrorMessages.ERROR_ABSENT_LAUNCH_COMMAND,
+          e.getMessage());
+    }
+
+    // launch command not specified
+    app.setName(LEN_64_STR.substring(0, RegistryConstants
+        .MAX_FQDN_LABEL_LENGTH));
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with no launch command");
+    } catch (IllegalArgumentException e) {
+      assertEquals(RestApiErrorMessages.ERROR_ABSENT_LAUNCH_COMMAND,
+          e.getMessage());
+    }
+
+    // memory not specified
+    comp.setLaunchCommand("sleep 1");
+    Resource res = new Resource();
+    app.setResource(res);
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with no memory");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_RESOURCE_MEMORY_FOR_COMP_INVALID,
+          comp.getName()), e.getMessage());
+    }
+
+    // invalid no of cpus
+    res.setMemory("100mb");
+    res.setCpus(-2);
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(
+          EXCEPTION_PREFIX + "service with invalid no of cpus");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_RESOURCE_CPUS_FOR_COMP_INVALID_RANGE,
+          comp.getName()), e.getMessage());
+    }
+
+    // number of containers not specified
+    res.setCpus(2);
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with no container count");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage()
+          .contains(ERROR_CONTAINERS_COUNT_INVALID));
+    }
+
+    // specifying profile along with cpus/memory raises exception
+    res.setProfile("hbase_finance_large");
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX
+          + "service with resource profile along with cpus/memory");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(RestApiErrorMessages
+              .ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_FOR_COMP_NOT_SUPPORTED,
+          comp.getName()),
+          e.getMessage());
+    }
+
+    // currently resource profile alone is not supported.
+    // TODO: remove the next test once resource profile alone is supported.
+    res.setCpus(null);
+    res.setMemory(null);
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with resource profile only");
+    } catch (IllegalArgumentException e) {
+      assertEquals(ERROR_RESOURCE_PROFILE_NOT_SUPPORTED_YET,
+          e.getMessage());
+    }
+
+    // unset profile here and add cpus/memory back
+    res.setProfile(null);
+    res.setCpus(2);
+    res.setMemory("2gb");
+
+    // null number of containers
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "null number of containers");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith(ERROR_CONTAINERS_COUNT_INVALID));
+    }
+  }
+
+  @Test
+  public void testArtifacts() throws IOException {
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+
+    Service app = new Service();
+    app.setName("service1");
+    app.setVersion("v1");
+    Resource res = new Resource();
+    app.setResource(res);
+    res.setMemory("512M");
+
+    // no artifact id fails with default type
+    Artifact artifact = new Artifact();
+    app.setArtifact(artifact);
+    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(String.format(ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName),
+          e.getMessage());
+    }
+
+    // no artifact id fails with SERVICE type
+    artifact.setType(Artifact.TypeEnum.SERVICE);
+    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());
+    }
+
+    // no artifact id fails with TARBALL type
+    artifact.setType(Artifact.TypeEnum.TARBALL);
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with no artifact id");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName),
+          e.getMessage());
+    }
+
+    // everything valid here
+    artifact.setType(Artifact.TypeEnum.DOCKER);
+    artifact.setId("docker.io/centos:centos7");
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      LOG.error("service attributes specified should be valid here", e);
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    assertEquals(app.getLifetime(), DEFAULT_UNLIMITED_LIFETIME);
+  }
+
+  private static Resource createValidResource() {
+    Resource res = new Resource();
+    res.setMemory("512M");
+    return res;
+  }
+
+  private static Component createValidComponent(String compName) {
+    Component comp = new Component();
+    comp.setName(compName);
+    comp.setResource(createValidResource());
+    comp.setNumberOfContainers(1L);
+    comp.setLaunchCommand("sleep 1");
+    return comp;
+  }
+
+  private static Service createValidApplication(String compName) {
+    Service app = new Service();
+    app.setName("name");
+    app.setVersion("v1");
+    app.setResource(createValidResource());
+    if (compName != null) {
+      app.addComponent(createValidComponent(compName));
+    }
+    return app;
+  }
+
+  @Test
+  public void testExternalApplication() throws IOException {
+    Service ext = createValidApplication("comp1");
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs(ext);
+
+    Service app = createValidApplication(null);
+
+    Artifact artifact = new Artifact();
+    artifact.setType(Artifact.TypeEnum.SERVICE);
+    artifact.setId("id");
+    app.setArtifact(artifact);
+    app.addComponent(ServiceTestUtils.createComponent("comp2"));
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    assertEquals(1, app.getComponents().size());
+    assertNotNull(app.getComponent("comp2"));
+  }
+
+  @Test
+  public void testDuplicateComponents() throws IOException {
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+
+    String compName = "comp1";
+    Service app = createValidApplication(compName);
+    app.addComponent(createValidComponent(compName));
+
+    // duplicate component name fails
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with component collision");
+    } catch (IllegalArgumentException e) {
+      assertEquals("Component name collision: " + compName, e.getMessage());
+    }
+  }
+
+  @Test
+  public void testComponentNameSameAsServiceName() throws IOException {
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+    Service app = new Service();
+    app.setName("test");
+    app.setVersion("v1");
+    app.addComponent(createValidComponent("test"));
+
+    //component name same as service name
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "component name matches service name");
+    } catch (IllegalArgumentException e) {
+      assertEquals("Component name test must not be same as service name test",
+          e.getMessage());
+    }
+  }
+
+  @Test
+  public void testExternalDuplicateComponent() throws IOException {
+    Service ext = createValidApplication("comp1");
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs(ext);
+
+    Service app = createValidApplication("comp1");
+    Artifact artifact = new Artifact();
+    artifact.setType(Artifact.TypeEnum.SERVICE);
+    artifact.setId("id");
+    app.getComponent("comp1").setArtifact(artifact);
+
+    // duplicate component name okay in the case of SERVICE component
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+  }
+
+  @Test
+  public void testExternalComponent() throws IOException {
+    Service ext = createValidApplication("comp1");
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs(ext);
+
+    Service app = createValidApplication("comp2");
+    Artifact artifact = new Artifact();
+    artifact.setType(Artifact.TypeEnum.SERVICE);
+    artifact.setId("id");
+    app.setArtifact(artifact);
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    assertEquals(1, app.getComponents().size());
+    // artifact ID not inherited from global
+    assertNotNull(app.getComponent("comp2"));
+
+    // set SERVICE artifact id on component
+    app.getComponent("comp2").setArtifact(artifact);
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    assertEquals(1, app.getComponents().size());
+    // original component replaced by external component
+    assertNotNull(app.getComponent("comp1"));
+  }
+
+  public static void verifyDependencySorting(List<Component> components,
+      Component... expectedSorting) {
+    Collection<Component> actualSorting = ServiceApiUtil.sortByDependencies(
+        components);
+    assertEquals(expectedSorting.length, actualSorting.size());
+    int i = 0;
+    for (Component component : actualSorting) {
+      assertEquals(expectedSorting[i++], component);
+    }
+  }
+
+  @Test
+  public void testDependencySorting() throws IOException {
+    Component a = ServiceTestUtils.createComponent("a");
+    Component b = ServiceTestUtils.createComponent("b");
+    Component c = ServiceTestUtils.createComponent("c");
+    Component d =
+        ServiceTestUtils.createComponent("d").dependencies(Arrays.asList("c"));
+    Component e = ServiceTestUtils.createComponent("e")
+        .dependencies(Arrays.asList("b", "d"));
+
+    verifyDependencySorting(Arrays.asList(a, b, c), a, b, c);
+    verifyDependencySorting(Arrays.asList(c, a, b), c, a, b);
+    verifyDependencySorting(Arrays.asList(a, b, c, d, e), a, b, c, d, e);
+    verifyDependencySorting(Arrays.asList(e, d, c, b, a), c, b, a, d, e);
+
+    c.setDependencies(Arrays.asList("e"));
+    try {
+      verifyDependencySorting(Arrays.asList(a, b, c, d, e));
+      Assert.fail(EXCEPTION_PREFIX + "components with dependency cycle");
+    } catch (IllegalArgumentException ex) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_DEPENDENCY_CYCLE, Arrays.asList(c, d,
+              e)), ex.getMessage());
+    }
+
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+    Service service = createValidApplication(null);
+    service.setComponents(Arrays.asList(c, d, e));
+    try {
+      ServiceApiUtil.validateAndResolveService(service, sfs,
+          CONF_DEFAULT_DNS);
+      Assert.fail(EXCEPTION_PREFIX + "components with bad dependencies");
+    } catch (IllegalArgumentException ex) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_DEPENDENCY_INVALID, "b", "e"), ex
+          .getMessage());
+    }
+  }
+
+  @Test
+  public void testInvalidComponent() throws IOException {
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+    testComponent(sfs);
+  }
+
+  @Test
+  public void testValidateCompName() {
+    String[] invalidNames = {
+        "EXAMPLE", // UPPER case not allowed
+        "example_app" // underscore not allowed.
+    };
+    for (String name : invalidNames) {
+      try {
+        ServiceApiUtil.validateNameFormat(name, new Configuration());
+        Assert.fail();
+      } catch (IllegalArgumentException ex) {
+        ex.printStackTrace();
+      }
+    }
+  }
+
+  private static void testComponent(SliderFileSystem sfs)
+      throws IOException {
+    int maxLen = RegistryConstants.MAX_FQDN_LABEL_LENGTH;
+    assertEquals(19, Long.toString(Long.MAX_VALUE).length());
+    maxLen = maxLen - Long.toString(Long.MAX_VALUE).length();
+
+    String compName = LEN_64_STR.substring(0, maxLen + 1);
+    Service app = createValidApplication(null);
+    app.addComponent(createValidComponent(compName));
+
+    // invalid component name fails if dns is enabled
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with invalid component name");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(RestApiErrorMessages
+          .ERROR_COMPONENT_NAME_INVALID, maxLen, compName), e.getMessage());
+    }
+
+    // does not fail if dns is disabled
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DEFAULT_DNS);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    compName = LEN_64_STR.substring(0, maxLen);
+    app = createValidApplication(null);
+    app.addComponent(createValidComponent(compName));
+
+    // does not fail
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+  }
+
+  @Test
+  public void testPlacementPolicy() throws IOException {
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+    Service app = createValidApplication("comp-a");
+    Component comp = app.getComponents().get(0);
+    PlacementPolicy pp = new PlacementPolicy();
+    PlacementConstraint pc = new PlacementConstraint();
+    pc.setName("CA1");
+    pp.setConstraints(Collections.singletonList(pc));
+    comp.setPlacementPolicy(pp);
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "constraint with no type");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_TYPE_NULL,
+          "CA1 ", "comp-a"), e.getMessage());
+    }
+
+    // Set the type
+    pc.setType(PlacementType.ANTI_AFFINITY);
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "constraint with no scope");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_SCOPE_NULL,
+          "CA1 ", "comp-a"), e.getMessage());
+    }
+
+    // Set the scope
+    pc.setScope(PlacementScope.NODE);
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "constraint with no tag(s)");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_TAGS_NULL,
+          "CA1 ", "comp-a"), e.getMessage());
+    }
+
+    // Set a target tag - but an invalid one
+    pc.setTargetTags(Collections.singletonList("comp-invalid"));
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "constraint with invalid tag name");
+    } catch (IllegalArgumentException e) {
+      assertEquals(
+          String.format(
+              RestApiErrorMessages.ERROR_PLACEMENT_POLICY_TAG_NAME_NOT_SAME,
+              "comp-invalid", "comp-a", "comp-a", "comp-a"),
+          e.getMessage());
+    }
+
+    // Set valid target tags now
+    pc.setTargetTags(Collections.singletonList("comp-a"));
+
+    // Finally it should succeed
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+  }
+
+  @Test
+  public void testKerberosPrincipal() throws IOException {
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+    Service app = createValidApplication("comp-a");
+    KerberosPrincipal kp = new KerberosPrincipal();
+    kp.setKeytab("/some/path");
+    kp.setPrincipalName("user/_HOST@domain.com");
+    app.setKerberosPrincipal(kp);
+
+    try {
+      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
+      Assert.fail(EXCEPTION_PREFIX + "service with invalid keytab URI scheme");
+    } catch (IllegalArgumentException e) {
+      assertEquals(
+          String.format(RestApiErrorMessages.ERROR_KEYTAB_URI_SCHEME_INVALID,
+              kp.getKeytab()),
+          e.getMessage());
+    }
+
+    kp.setKeytab("/ blank / in / paths");
+    try {
+      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
+      Assert.fail(EXCEPTION_PREFIX + "service with invalid keytab");
+    } catch (IllegalArgumentException e) {
+      // strip out the %s at the end of the RestApiErrorMessages string constant
+      assertTrue(e.getMessage().contains(
+          RestApiErrorMessages.ERROR_KEYTAB_URI_INVALID.substring(0,
+              RestApiErrorMessages.ERROR_KEYTAB_URI_INVALID.length() - 2)));
+    }
+
+    kp.setKeytab("file:///tmp/a.keytab");
+    // now it should succeed
+    try {
+      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+  }
+
+  @Test
+  public void testKerberosPrincipalNameFormat() throws IOException {
+    Service app = createValidApplication("comp-a");
+    KerberosPrincipal kp = new KerberosPrincipal();
+    kp.setPrincipalName("user@domain.com");
+    app.setKerberosPrincipal(kp);
+
+    try {
+      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
+      Assert.fail(EXCEPTION_PREFIX + "service with invalid principal name " +
+          "format.");
+    } catch (IllegalArgumentException e) {
+      assertEquals(
+          String.format(
+              RestApiErrorMessages.ERROR_KERBEROS_PRINCIPAL_NAME_FORMAT,
+              kp.getPrincipalName()),
+          e.getMessage());
+    }
+
+    kp.setPrincipalName("user/_HOST@domain.com");
+    try {
+      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+  }
+
+  @Test
+  public void testResolveCompsDependency() {
+    Service service = createExampleApplication();
+    List<String> dependencies = new ArrayList<String>();
+    dependencies.add("compb");
+    Component compa = createComponent("compa");
+    compa.setDependencies(dependencies);
+    Component compb = createComponent("compb");
+    service.addComponent(compa);
+    service.addComponent(compb);
+    List<String> order = ServiceApiUtil.resolveCompsDependency(service);
+    List<String> expected = new ArrayList<String>();
+    expected.add("compb");
+    expected.add("compa");
+    for (int i = 0; i < expected.size(); i++) {
+      Assert.assertEquals("Components are not equal.", expected.get(i),
+          order.get(i));
+    }
+  }
+
+  @Test
+  public void testResolveCompsDependencyReversed() {
+    Service service = createExampleApplication();
+    List<String> dependencies = new ArrayList<String>();
+    dependencies.add("compa");
+    Component compa = createComponent("compa");
+    Component compb = createComponent("compb");
+    compb.setDependencies(dependencies);
+    service.addComponent(compa);
+    service.addComponent(compb);
+    List<String> order = ServiceApiUtil.resolveCompsDependency(service);
+    List<String> expected = new ArrayList<String>();
+    expected.add("compa");
+    expected.add("compb");
+    for (int i = 0; i < expected.size(); i++) {
+      Assert.assertEquals("Components are not equal.", expected.get(i),
+          order.get(i));
+    }
+  }
+
+  @Test
+  public void testResolveCompsCircularDependency() {
+    Service service = createExampleApplication();
+    List<String> dependencies = new ArrayList<String>();
+    List<String> dependencies2 = new ArrayList<String>();
+    dependencies.add("compb");
+    dependencies2.add("compa");
+    Component compa = createComponent("compa");
+    compa.setDependencies(dependencies);
+    Component compb = createComponent("compb");
+    compa.setDependencies(dependencies2);
+    service.addComponent(compa);
+    service.addComponent(compb);
+    List<String> order = ServiceApiUtil.resolveCompsDependency(service);
+    List<String> expected = new ArrayList<String>();
+    expected.add("compa");
+    expected.add("compb");
+    for (int i = 0; i < expected.size(); i++) {
+      Assert.assertEquals("Components are not equal.", expected.get(i),
+          order.get(i));
+    }
+  }
+
+  @Test
+  public void testResolveNoCompsDependency() {
+    Service service = createExampleApplication();
+    Component compa = createComponent("compa");
+    Component compb = createComponent("compb");
+    service.addComponent(compa);
+    service.addComponent(compb);
+    List<String> order = ServiceApiUtil.resolveCompsDependency(service);
+    List<String> expected = new ArrayList<String>();
+    expected.add("compa");
+    expected.add("compb");
+    for (int i = 0; i < expected.size(); i++) {
+      Assert.assertEquals("Components are not equal.", expected.get(i),
+          order.get(i));
+    }
+  }
+
+  public static Service createExampleApplication() {
+
+    Service exampleApp = new Service();
+    exampleApp.setName("example-app");
+    exampleApp.setVersion("v1");
+    return exampleApp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
index 807938c..a0e4e02 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.yarn.client.cli;
 
 import java.io.ByteArrayOutputStream;
+import java.io.File;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
@@ -100,6 +101,7 @@ public class ApplicationCLI extends YarnCLI {
   public static final String COMPONENT = "component";
   public static final String ENABLE_FAST_LAUNCH = "enableFastLaunch";
   public static final String UPGRADE_CMD = "upgrade";
+  public static final String UPGRADE_EXPRESS = "express";
   public static final String UPGRADE_INITIATE = "initiate";
   public static final String UPGRADE_AUTO_FINALIZE = "autoFinalize";
   public static final String UPGRADE_FINALIZE = "finalize";
@@ -247,6 +249,9 @@ public class ApplicationCLI extends YarnCLI {
       opts.addOption(UPGRADE_CMD, true, "Upgrades an application/long-" +
           "running service. It requires either -initiate, -instances, or " +
           "-finalize options.");
+      opts.addOption(UPGRADE_EXPRESS, true, "Works with -upgrade option to " +
+          "perform express upgrade.  It requires the upgraded application " +
+          "specification file.");
       opts.addOption(UPGRADE_INITIATE, true, "Works with -upgrade option to " +
           "initiate the application upgrade. It requires the upgraded " +
           "application specification file.");
@@ -639,9 +644,9 @@ public class ApplicationCLI extends YarnCLI {
       moveApplicationAcrossQueues(cliParser.getOptionValue(APP_ID),
           cliParser.getOptionValue(CHANGE_APPLICATION_QUEUE));
     } else if (cliParser.hasOption(UPGRADE_CMD)) {
-      if (hasAnyOtherCLIOptions(cliParser, opts, UPGRADE_CMD, UPGRADE_INITIATE,
-          UPGRADE_AUTO_FINALIZE, UPGRADE_FINALIZE, COMPONENT_INSTS, COMPONENTS,
-          APP_TYPE_CMD)) {
+      if (hasAnyOtherCLIOptions(cliParser, opts, UPGRADE_CMD, UPGRADE_EXPRESS,
+          UPGRADE_INITIATE, UPGRADE_AUTO_FINALIZE, UPGRADE_FINALIZE,
+          COMPONENT_INSTS, COMPONENTS, APP_TYPE_CMD)) {
         printUsage(title, opts);
         return exitCode;
       }
@@ -649,7 +654,14 @@ public class ApplicationCLI extends YarnCLI {
       AppAdminClient client =  AppAdminClient.createAppAdminClient(appType,
           getConf());
       String appName = cliParser.getOptionValue(UPGRADE_CMD);
-      if (cliParser.hasOption(UPGRADE_INITIATE)) {
+      if (cliParser.hasOption(UPGRADE_EXPRESS)) {
+        File file = new File(cliParser.getOptionValue(UPGRADE_EXPRESS));
+        if (!file.exists()) {
+          System.err.println(file.getAbsolutePath() + " does not exist.");
+          return exitCode;
+        }
+        return client.actionUpgradeExpress(appName, file);
+      } else if (cliParser.hasOption(UPGRADE_INITIATE)) {
         if (hasAnyOtherCLIOptions(cliParser, opts, UPGRADE_CMD,
             UPGRADE_INITIATE, UPGRADE_AUTO_FINALIZE, APP_TYPE_CMD)) {
           printUsage(title, opts);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/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 526adfd..20c9603 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
@@ -2161,6 +2161,10 @@ public class TestYarnCLI {
     pw.println("                                          Optionally a destination folder");
     pw.println("                                          for the tarball can be");
     pw.println("                                          specified.");
+    pw.println(" -express <arg>                           Works with -upgrade option to");
+    pw.println("                                          perform express upgrade.  It");
+    pw.println("                                          requires the upgraded");
+    pw.println("                                          application specification file.");
     pw.println(" -finalize                                Works with -upgrade option to");
     pw.println("                                          finalize the upgrade.");
     pw.println(" -flex <Application Name or ID>           Changes number of running");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
index 3fb4778..232666d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
@@ -288,4 +289,15 @@ public abstract class AppAdminClient extends CompositeService {
       List<String> components, String version, List<String> containerStates)
       throws IOException, YarnException;
 
+  /**
+   * Express upgrade a long running service.
+   *
+   * @param appName  the name of the application
+   * @param fileName specification of application upgrade to save.
+   * @return exit code
+   */
+  @Public
+  @Unstable
+  public abstract int actionUpgradeExpress(String appName, File fileName)
+      throws IOException, YarnException;
 }


---------------------------------------------------------------------
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: HDDS-328. Support export and import of the KeyValueContainer. Contributed by Elek Marton.

Posted by su...@apache.org.
HDDS-328. Support export and import of the KeyValueContainer. 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/ca29fb75
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ca29fb75
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ca29fb75

Branch: refs/heads/YARN-3409
Commit: ca29fb754e8a162edba380a5f1deb48699e14d8b
Parents: 585ebd8
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Thu Aug 23 11:30:28 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Thu Aug 23 11:30:28 2018 -0700

----------------------------------------------------------------------
 .../common/impl/ContainerDataYaml.java          |  94 ++++---
 .../container/common/interfaces/Container.java  |  33 ++-
 .../common/interfaces/ContainerPacker.java      |  58 +++++
 .../container/keyvalue/KeyValueContainer.java   | 128 ++++++++--
 .../container/keyvalue/KeyValueHandler.java     |  19 +-
 .../container/keyvalue/TarContainerPacker.java  | 249 +++++++++++++++++++
 .../keyvalue/helpers/KeyValueContainerUtil.java |  22 +-
 .../keyvalue/TestKeyValueContainer.java         |  95 ++++++-
 .../keyvalue/TestTarContainerPacker.java        | 231 +++++++++++++++++
 9 files changed, 849 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/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 aed75d3..ec6d642 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
@@ -18,31 +18,34 @@
 
 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;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.Writer;
+import java.io.ByteArrayInputStream;
+import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStreamWriter;
-import java.io.File;
+import java.io.Writer;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
-import java.util.Map;
 
+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 com.google.common.base.Preconditions;
+import static org.apache.hadoop.ozone.container.keyvalue
+    .KeyValueContainerData.KEYVALUE_YAML_TAG;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.Yaml;
 import org.yaml.snakeyaml.constructor.AbstractConstruct;
 import org.yaml.snakeyaml.constructor.Constructor;
 import org.yaml.snakeyaml.introspector.BeanAccess;
@@ -54,9 +57,6 @@ import org.yaml.snakeyaml.nodes.ScalarNode;
 import org.yaml.snakeyaml.nodes.Tag;
 import org.yaml.snakeyaml.representer.Representer;
 
-import static org.apache.hadoop.ozone.container.keyvalue
-    .KeyValueContainerData.KEYVALUE_YAML_TAG;
-
 /**
  * Class for creating and reading .container files.
  */
@@ -106,36 +106,52 @@ public final class ContainerDataYaml {
   /**
    * Read the yaml file, and return containerData.
    *
-   * @param containerFile
    * @throws IOException
    */
   public static ContainerData readContainerFile(File containerFile)
       throws IOException {
     Preconditions.checkNotNull(containerFile, "containerFile cannot be null");
+    try (FileInputStream inputFileStream = new FileInputStream(containerFile)) {
+      return readContainer(inputFileStream);
+    }
+
+  }
+
+  /**
+   * Read the yaml file content, and return containerData.
+   *
+   * @throws IOException
+   */
+  public static ContainerData readContainer(byte[] containerFileContent)
+      throws IOException {
+    return readContainer(
+        new ByteArrayInputStream(containerFileContent));
+  }
+
+  /**
+   * Read the yaml content, and return containerData.
+   *
+   * @throws IOException
+   */
+  public static ContainerData readContainer(InputStream input)
+      throws IOException {
 
-    InputStream input = null;
     ContainerData containerData;
-    try {
-      PropertyUtils propertyUtils = new PropertyUtils();
-      propertyUtils.setBeanAccess(BeanAccess.FIELD);
-      propertyUtils.setAllowReadOnlyProperties(true);
+    PropertyUtils propertyUtils = new PropertyUtils();
+    propertyUtils.setBeanAccess(BeanAccess.FIELD);
+    propertyUtils.setAllowReadOnlyProperties(true);
 
-      Representer representer = new ContainerDataRepresenter();
-      representer.setPropertyUtils(propertyUtils);
+    Representer representer = new ContainerDataRepresenter();
+    representer.setPropertyUtils(propertyUtils);
 
-      Constructor containerDataConstructor = new ContainerDataConstructor();
+    Constructor containerDataConstructor = new ContainerDataConstructor();
 
-      Yaml yaml = new Yaml(containerDataConstructor, representer);
-      yaml.setBeanAccess(BeanAccess.FIELD);
+    Yaml yaml = new Yaml(containerDataConstructor, representer);
+    yaml.setBeanAccess(BeanAccess.FIELD);
+
+    containerData = (ContainerData)
+        yaml.load(input);
 
-      input = new FileInputStream(containerFile);
-      containerData = (ContainerData)
-          yaml.load(input);
-    } finally {
-      if (input!= null) {
-        input.close();
-      }
-    }
     return containerData;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
index 7f706b5..9380f0c 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
@@ -18,26 +18,27 @@
 
 package org.apache.hadoop.ozone.container.common.interfaces;
 
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Map;
+
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerLifeCycleState;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
-import org.apache.hadoop.hdds.scm.container.common.helpers.
-    StorageContainerException;
 
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.Map;
-
-
 /**
  * Interface for Container Operations.
  */
-public interface Container extends RwLock {
+public interface Container<CONTAINERDATA extends ContainerData> extends RwLock {
 
   /**
    * Creates a container.
@@ -71,7 +72,7 @@ public interface Container extends RwLock {
    * @return ContainerData - Container Data.
    * @throws StorageContainerException
    */
-  ContainerData getContainerData();
+  CONTAINERDATA getContainerData();
 
   /**
    * Get the Container Lifecycle state.
@@ -113,6 +114,20 @@ public interface Container extends RwLock {
   BlockIterator blockIterator() throws IOException;
 
   /**
+   * Import the container from an external archive.
+   */
+  void importContainerData(InputStream stream,
+      ContainerPacker<CONTAINERDATA> packer) throws IOException;
+
+  /**
+   * Export all the data of the container to one output archive with the help
+   * of the packer.
+   *
+   */
+  void exportContainerData(OutputStream stream,
+      ContainerPacker<CONTAINERDATA> packer) throws IOException;
+
+  /**
    * Returns containerReport for the container.
    */
   StorageContainerDatanodeProtocolProtos.ContainerInfo getContainerReport()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerPacker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerPacker.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerPacker.java
new file mode 100644
index 0000000..8308c23
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerPacker.java
@@ -0,0 +1,58 @@
+/**
+ * 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.container.common.interfaces;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+
+/**
+ * Service to pack/unpack ContainerData container data to/from a single byte
+ * stream.
+ */
+public interface ContainerPacker<CONTAINERDATA extends ContainerData> {
+
+  /**
+   * Extract the container data to the path defined by the container.
+   * <p>
+   * This doesn't contain the extraction of the container descriptor file.
+   *
+   * @return the byte content of the descriptor (which won't be written to a
+   * file but returned).
+   */
+  byte[] unpackContainerData(Container<CONTAINERDATA> container,
+      InputStream inputStream)
+      throws IOException;
+
+  /**
+   * Compress all the container data (chunk data, metadata db AND container
+   * descriptor) to one single archive.
+   */
+  void pack(Container<CONTAINERDATA> container, OutputStream destination)
+      throws IOException;
+
+  /**
+   * Read the descriptor from the finished archive to get the data before
+   * importing the container.
+   */
+  byte[] unpackContainerDescriptor(InputStream inputStream)
+      throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/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 0ea748a..8108a11 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
@@ -18,9 +18,15 @@
 
 package org.apache.hadoop.ozone.container.keyvalue;
 
-import com.google.common.base.Preconditions;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.nio.file.Files;
 import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileUtil;
@@ -37,32 +43,27 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
-import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
-import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerPacker;
 import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
 import org.apache.hadoop.ozone.container.keyvalue.helpers
     .KeyValueContainerLocationUtil;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.utils.MetadataStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-
-import java.util.Map;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.FileUtils;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.CONTAINER_ALREADY_EXISTS;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CONTAINER_INTERNAL_ERROR;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.CONTAINER_FILES_CREATE_ERROR;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CONTAINER_INTERNAL_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.DISK_OUT_OF_SPACE;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.ERROR_IN_COMPACT_DB;
@@ -70,11 +71,13 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.INVALID_CONTAINER_STATE;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.UNSUPPORTED_REQUEST;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Class to perform KeyValue Container operations.
  */
-public class KeyValueContainer implements Container {
+public class KeyValueContainer implements Container<KeyValueContainerData> {
 
   private static final Logger LOG = LoggerFactory.getLogger(Container.class);
 
@@ -167,6 +170,34 @@ public class KeyValueContainer implements Container {
   }
 
   /**
+   * Set all of the path realted container data fields based on the name
+   * conventions.
+   *
+   * @param scmId
+   * @param containerVolume
+   * @param hddsVolumeDir
+   */
+  public void populatePathFields(String scmId,
+      HddsVolume containerVolume, String hddsVolumeDir) {
+
+    long containerId = containerData.getContainerID();
+
+    File containerMetaDataPath = KeyValueContainerLocationUtil
+        .getContainerMetaDataPath(hddsVolumeDir, scmId, containerId);
+
+    File chunksPath = KeyValueContainerLocationUtil.getChunksLocationPath(
+        hddsVolumeDir, scmId, containerId);
+    File dbFile = KeyValueContainerLocationUtil.getContainerDBFile(
+        containerMetaDataPath, containerId);
+
+    //Set containerData for the KeyValueContainer.
+    containerData.setMetadataPath(containerMetaDataPath.getPath());
+    containerData.setChunksPath(chunksPath.getPath());
+    containerData.setDbFile(dbFile);
+    containerData.setVolume(containerVolume);
+  }
+
+  /**
    * Writes to .container file.
    *
    * @param containerFile container file name
@@ -334,6 +365,75 @@ public class KeyValueContainer implements Container {
         containerData.getContainerPath()));
   }
 
+  @Override
+  public void importContainerData(InputStream input,
+      ContainerPacker<KeyValueContainerData> packer) throws IOException {
+    writeLock();
+    try {
+      if (getContainerFile().exists()) {
+        String errorMessage = String.format(
+            "Can't import container (cid=%d) data to a specific location"
+                + " as the container descriptor (%s) has already been exist.",
+            getContainerData().getContainerID(),
+            getContainerFile().getAbsolutePath());
+        throw new IOException(errorMessage);
+      }
+      //copy the values from the input stream to the final destination
+      // directory.
+      byte[] descriptorContent = packer.unpackContainerData(this, input);
+
+      Preconditions.checkNotNull(descriptorContent,
+          "Container descriptor is missing from the container archive: "
+              + getContainerData().getContainerID());
+
+      //now, we have extracted the container descriptor from the previous
+      //datanode. We can load it and upload it with the current data
+      // (original metadata + current filepath fields)
+      KeyValueContainerData originalContainerData =
+          (KeyValueContainerData) ContainerDataYaml
+              .readContainer(descriptorContent);
+
+
+      containerData.setState(originalContainerData.getState());
+      containerData
+          .setContainerDBType(originalContainerData.getContainerDBType());
+      containerData.setBytesUsed(originalContainerData.getBytesUsed());
+
+      //rewriting the yaml file with new checksum calculation.
+      update(originalContainerData.getMetadata(), true);
+
+      //fill in memory stat counter (keycount, byte usage)
+      KeyValueContainerUtil.parseKVContainerData(containerData, config);
+
+    } catch (Exception ex) {
+      //delete all the temporary data in case of any exception.
+      try {
+        FileUtils.deleteDirectory(new File(containerData.getMetadataPath()));
+        FileUtils.deleteDirectory(new File(containerData.getChunksPath()));
+        FileUtils.deleteDirectory(getContainerFile());
+      } catch (Exception deleteex) {
+        LOG.error(
+            "Can not cleanup destination directories after a container import"
+                + " error (cid" +
+                containerData.getContainerID() + ")", deleteex);
+      }
+      throw ex;
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  @Override
+  public void exportContainerData(OutputStream destination,
+      ContainerPacker<KeyValueContainerData> packer) throws IOException {
+    if (getContainerData().getState() != ContainerLifeCycleState.CLOSED) {
+      throw new IllegalStateException(
+          "Only closed containers could be exported: ContainerId="
+              + getContainerData().getContainerID());
+    }
+    packer.pack(this, destination);
+  }
+
   /**
    * Acquire read lock.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/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 9ddb474..29c359e 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
@@ -47,6 +47,7 @@ import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
 import org.apache.hadoop.ozone.container.common.impl.OpenContainerBlockMap;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.SmallFileUtils;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
@@ -162,7 +163,8 @@ public class KeyValueHandler extends Handler {
     return volumeChoosingPolicy;
   }
   /**
-   * Returns OpenContainerBlockMap instance
+   * Returns OpenContainerBlockMap instance.
+   *
    * @return OpenContainerBlockMap
    */
   public OpenContainerBlockMap getOpenContainerBlockMap() {
@@ -269,6 +271,19 @@ public class KeyValueHandler extends Handler {
     return ContainerUtils.getSuccessResponse(request);
   }
 
+  public void populateContainerPathFields(KeyValueContainer container,
+      long maxSize) throws IOException {
+    volumeSet.acquireLock();
+    try {
+      HddsVolume containerVolume = volumeChoosingPolicy.chooseVolume(volumeSet
+          .getVolumesList(), maxSize);
+      String hddsVolumeDir = containerVolume.getHddsRootDir().toString();
+      container.populatePathFields(scmID, containerVolume, hddsVolumeDir);
+    } finally {
+      volumeSet.releaseLock();
+    }
+  }
+
   /**
    * Handles Read Container Request. Returns the ContainerData as response.
    */
@@ -322,7 +337,7 @@ public class KeyValueHandler extends Handler {
    * Open containers cannot be deleted.
    * Holds writeLock on ContainerSet till the container is removed from
    * containerMap. On disk deletion of container files will happen
-   * asynchornously without the lock.
+   * asynchronously without the lock.
    */
   ContainerCommandResponseProto handleDeleteContainer(
       ContainerCommandRequestProto request, KeyValueContainer kvContainer) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java
new file mode 100644
index 0000000..13689a7
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java
@@ -0,0 +1,249 @@
+/**
+ * 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.container.keyvalue;
+
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerPacker;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.compress.archivers.ArchiveEntry;
+import org.apache.commons.compress.archivers.ArchiveOutputStream;
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
+import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
+import org.apache.commons.compress.compressors.CompressorException;
+import org.apache.commons.compress.compressors.CompressorInputStream;
+import org.apache.commons.compress.compressors.CompressorOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.io.IOUtils;
+
+/**
+ * Compress/uncompress KeyValueContainer data to a tar.gz archive.
+ */
+public class TarContainerPacker
+    implements ContainerPacker<KeyValueContainerData> {
+
+  private static final String CHUNKS_DIR_NAME = OzoneConsts.STORAGE_DIR_CHUNKS;
+
+  private static final String DB_DIR_NAME = "db";
+
+  private static final String CONTAINER_FILE_NAME = "container.yaml";
+
+
+
+  /**
+   * Given an input stream (tar file) extract the data to the specified
+   * directories.
+   *
+   * @param container container which defines the destination structure.
+   * @param inputStream the input stream.
+   * @throws IOException
+   */
+  @Override
+  public byte[] unpackContainerData(Container<KeyValueContainerData> container,
+      InputStream inputStream)
+      throws IOException {
+    byte[] descriptorFileContent = null;
+    try {
+      KeyValueContainerData containerData = container.getContainerData();
+      CompressorInputStream compressorInputStream =
+          new CompressorStreamFactory()
+              .createCompressorInputStream(CompressorStreamFactory.GZIP,
+                  inputStream);
+
+      TarArchiveInputStream tarInput =
+          new TarArchiveInputStream(compressorInputStream);
+
+      TarArchiveEntry entry = tarInput.getNextTarEntry();
+      while (entry != null) {
+        String name = entry.getName();
+        if (name.startsWith(DB_DIR_NAME + "/")) {
+          Path destinationPath = containerData.getDbFile().toPath()
+              .resolve(name.substring(DB_DIR_NAME.length() + 1));
+          extractEntry(tarInput, entry.getSize(), destinationPath);
+        } else if (name.startsWith(CHUNKS_DIR_NAME + "/")) {
+          Path destinationPath = Paths.get(containerData.getChunksPath())
+              .resolve(name.substring(CHUNKS_DIR_NAME.length() + 1));
+          extractEntry(tarInput, entry.getSize(), destinationPath);
+        } else if (name.equals(CONTAINER_FILE_NAME)) {
+          //Don't do anything. Container file should be unpacked in a
+          //separated step by unpackContainerDescriptor call.
+          descriptorFileContent = readEntry(tarInput, entry);
+        } else {
+          throw new IllegalArgumentException(
+              "Unknown entry in the tar file: " + "" + name);
+        }
+        entry = tarInput.getNextTarEntry();
+      }
+      return descriptorFileContent;
+
+    } catch (CompressorException e) {
+      throw new IOException(
+          "Can't uncompress the given container: " + container
+              .getContainerData().getContainerID(),
+          e);
+    }
+  }
+
+  private void extractEntry(TarArchiveInputStream tarInput, long size,
+      Path path) throws IOException {
+    Preconditions.checkNotNull(path, "Path element should not be null");
+    Path parent = Preconditions.checkNotNull(path.getParent(),
+        "Path element should have a parent directory");
+    Files.createDirectories(parent);
+    try (BufferedOutputStream bos = new BufferedOutputStream(
+        new FileOutputStream(path.toAbsolutePath().toString()))) {
+      int bufferSize = 1024;
+      byte[] buffer = new byte[bufferSize + 1];
+      long remaining = size;
+      while (remaining > 0) {
+        int read =
+            tarInput.read(buffer, 0, (int) Math.min(remaining, bufferSize));
+        if (read >= 0) {
+          remaining -= read;
+          bos.write(buffer, 0, read);
+        } else {
+          remaining = 0;
+        }
+      }
+    }
+
+  }
+
+  /**
+   * Given a containerData include all the required container data/metadata
+   * in a tar file.
+   *
+   * @param container Container to archive (data + metadata).
+   * @param destination   Destination tar file/stream.
+   * @throws IOException
+   */
+  @Override
+  public void pack(Container<KeyValueContainerData> container,
+      OutputStream destination)
+      throws IOException {
+
+    KeyValueContainerData containerData = container.getContainerData();
+
+    try (CompressorOutputStream gzippedOut = new CompressorStreamFactory()
+          .createCompressorOutputStream(CompressorStreamFactory.GZIP,
+              destination)) {
+
+      try (ArchiveOutputStream archiveOutputStream = new TarArchiveOutputStream(
+          gzippedOut)) {
+
+        includePath(containerData.getDbFile().toString(), DB_DIR_NAME,
+            archiveOutputStream);
+
+        includePath(containerData.getChunksPath(), CHUNKS_DIR_NAME,
+            archiveOutputStream);
+
+        includeFile(container.getContainerFile(),
+            CONTAINER_FILE_NAME,
+            archiveOutputStream);
+      }
+    } catch (CompressorException e) {
+      throw new IOException(
+          "Can't compress the container: " + containerData.getContainerID(),
+          e);
+    }
+
+  }
+
+  @Override
+  public byte[] unpackContainerDescriptor(InputStream inputStream)
+      throws IOException {
+    try {
+      CompressorInputStream compressorInputStream =
+          new CompressorStreamFactory()
+              .createCompressorInputStream(CompressorStreamFactory.GZIP,
+                  inputStream);
+
+      TarArchiveInputStream tarInput =
+          new TarArchiveInputStream(compressorInputStream);
+
+      TarArchiveEntry entry = tarInput.getNextTarEntry();
+      while (entry != null) {
+        String name = entry.getName();
+        if (name.equals(CONTAINER_FILE_NAME)) {
+          return readEntry(tarInput, entry);
+        }
+        entry = tarInput.getNextTarEntry();
+      }
+
+    } catch (CompressorException e) {
+      throw new IOException(
+          "Can't read the container descriptor from the container archive",
+          e);
+    }
+    throw new IOException(
+        "Container descriptor is missing from the container archive.");
+  }
+
+  private byte[] readEntry(TarArchiveInputStream tarInput,
+      TarArchiveEntry entry) throws IOException {
+    ByteArrayOutputStream bos = new ByteArrayOutputStream();
+    int bufferSize = 1024;
+    byte[] buffer = new byte[bufferSize + 1];
+    long remaining = entry.getSize();
+    while (remaining > 0) {
+      int read =
+          tarInput.read(buffer, 0, (int) Math.min(remaining, bufferSize));
+      remaining -= read;
+      bos.write(buffer, 0, read);
+    }
+    return bos.toByteArray();
+  }
+
+  private void includePath(String containerPath, String subdir,
+      ArchiveOutputStream archiveOutputStream) throws IOException {
+
+    for (Path path : Files.list(Paths.get(containerPath))
+        .collect(Collectors.toList())) {
+
+      includeFile(path.toFile(), subdir + "/" + path.getFileName(),
+          archiveOutputStream);
+    }
+  }
+
+  private void includeFile(File file, String entryName,
+      ArchiveOutputStream archiveOutputStream) throws IOException {
+    ArchiveEntry archiveEntry =
+        archiveOutputStream.createArchiveEntry(file, entryName);
+    archiveOutputStream.putArchiveEntry(archiveEntry);
+    try (FileInputStream fis = new FileInputStream(file)) {
+      IOUtils.copy(fis, archiveOutputStream);
+    }
+    archiveOutputStream.closeArchiveEntry();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/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 2352cf6..ed4536f 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
@@ -17,10 +17,14 @@
  */
 package org.apache.hadoop.ozone.container.keyvalue.helpers;
 
-import com.google.common.base.Preconditions;
-import org.apache.commons.io.FileUtils;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.conf.Configuration;
-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;
@@ -32,16 +36,12 @@ import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.utils.MetadataKeyFilters;
 import org.apache.hadoop.utils.MetadataStore;
 import org.apache.hadoop.utils.MetadataStoreBuilder;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.FileUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.List;
-import java.util.Map;
-
 /**
  * Class which defines utility methods for KeyValueContainer.
  */
@@ -157,7 +157,7 @@ public final class KeyValueContainerUtil {
    * @throws IOException
    */
   public static void parseKVContainerData(KeyValueContainerData kvContainerData,
-      OzoneConfiguration config) throws IOException {
+      Configuration config) throws IOException {
 
     long containerID = kvContainerData.getContainerID();
     File metadataPath = new File(kvContainerData.getMetadataPath());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/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 6ff2eca..7359868 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
@@ -23,7 +23,8 @@ 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.datanode.proto.ContainerProtos
+    .ContainerLifeCycleState;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
@@ -37,6 +38,8 @@ import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.utils.MetadataStore;
+
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -46,6 +49,8 @@ import org.mockito.Mockito;
 
 import java.io.File;
 
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
@@ -74,7 +79,6 @@ public class TestKeyValueContainer {
   private String scmId = UUID.randomUUID().toString();
   private VolumeSet volumeSet;
   private RoundRobinVolumeChoosingPolicy volumeChoosingPolicy;
-  private long containerID = 1L;
   private KeyValueContainerData keyValueContainerData;
   private KeyValueContainer keyValueContainer;
 
@@ -141,13 +145,14 @@ public class TestKeyValueContainer {
 
   }
 
+  @SuppressWarnings("RedundantCast")
   @Test
   public void testCreateContainer() throws Exception {
 
     // Create Container.
     keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
 
-    keyValueContainerData = (KeyValueContainerData) keyValueContainer
+    keyValueContainerData = keyValueContainer
         .getContainerData();
 
     String containerMetaDataPath = keyValueContainerData
@@ -167,6 +172,86 @@ public class TestKeyValueContainer {
   }
 
   @Test
+  public void testContainerImportExport() throws Exception {
+
+    long containerId = keyValueContainer.getContainerData().getContainerID();
+    // Create Container.
+    keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+
+
+    keyValueContainerData = keyValueContainer
+        .getContainerData();
+
+    keyValueContainerData.setState(ContainerLifeCycleState.CLOSED);
+
+    int numberOfKeysToWrite = 12;
+    //write one few keys to check the key count after import
+    MetadataStore metadataStore = KeyUtils.getDB(keyValueContainerData, conf);
+    for (int i = 0; i < numberOfKeysToWrite; i++) {
+      metadataStore.put(("test" + i).getBytes(), "test".getBytes());
+    }
+    metadataStore.close();
+
+    Map<String, String> metadata = new HashMap<>();
+    metadata.put("key1", "value1");
+    keyValueContainer.update(metadata, true);
+
+    //destination path
+    File folderToExport = folder.newFile("exported.tar.gz");
+
+    TarContainerPacker packer = new TarContainerPacker();
+
+    //export the container
+    try (FileOutputStream fos = new FileOutputStream(folderToExport)) {
+      keyValueContainer
+          .exportContainerData(fos, packer);
+    }
+
+    //delete the original one
+    keyValueContainer.delete(true);
+
+    //create a new one
+    KeyValueContainerData containerData =
+        new KeyValueContainerData(containerId, 1,
+            keyValueContainerData.getMaxSizeGB());
+    KeyValueContainer container = new KeyValueContainer(containerData, conf);
+
+    HddsVolume containerVolume = volumeChoosingPolicy.chooseVolume(volumeSet
+        .getVolumesList(), 1);
+    String hddsVolumeDir = containerVolume.getHddsRootDir().toString();
+
+    container.populatePathFields(scmId, containerVolume, hddsVolumeDir);
+    try (FileInputStream fis = new FileInputStream(folderToExport)) {
+      container.importContainerData(fis, packer);
+    }
+
+    Assert.assertEquals("value1", containerData.getMetadata().get("key1"));
+    Assert.assertEquals(keyValueContainerData.getContainerDBType(),
+        containerData.getContainerDBType());
+    Assert.assertEquals(keyValueContainerData.getState(),
+        containerData.getState());
+    Assert.assertEquals(numberOfKeysToWrite,
+        containerData.getKeyCount());
+    Assert.assertEquals(keyValueContainerData.getLayOutVersion(),
+        containerData.getLayOutVersion());
+    Assert.assertEquals(keyValueContainerData.getMaxSizeGB(),
+        containerData.getMaxSizeGB());
+    Assert.assertEquals(keyValueContainerData.getBytesUsed(),
+        containerData.getBytesUsed());
+
+    //Can't overwrite existing container
+    try {
+      try (FileInputStream fis = new FileInputStream(folderToExport)) {
+        container.importContainerData(fis, packer);
+      }
+      fail("Container is imported twice. Previous files are overwritten");
+    } catch (Exception ex) {
+      //all good
+    }
+
+  }
+
+  @Test
   public void testDuplicateContainer() throws Exception {
     try {
       // Create Container.
@@ -224,7 +309,7 @@ public class TestKeyValueContainer {
     keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
     keyValueContainer.close();
 
-    keyValueContainerData = (KeyValueContainerData) keyValueContainer
+    keyValueContainerData = keyValueContainer
         .getContainerData();
 
     assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED,
@@ -249,7 +334,7 @@ public class TestKeyValueContainer {
     metadata.put("OWNER", "hdfs");
     keyValueContainer.update(metadata, true);
 
-    keyValueContainerData = (KeyValueContainerData) keyValueContainer
+    keyValueContainerData = keyValueContainer
         .getContainerData();
 
     assertEquals(2, keyValueContainerData.getMetadata().size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java
new file mode 100644
index 0000000..a599f72
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java
@@ -0,0 +1,231 @@
+/*
+ * 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.container.keyvalue;
+
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerPacker;
+
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
+import org.apache.commons.compress.compressors.CompressorException;
+import org.apache.commons.compress.compressors.CompressorInputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test the tar/untar for a given container.
+ */
+public class TestTarContainerPacker {
+
+  private static final String TEST_DB_FILE_NAME = "test1";
+
+  private static final String TEST_DB_FILE_CONTENT = "test1";
+
+  private static final String TEST_CHUNK_FILE_NAME = "chunk1";
+
+  private static final String TEST_CHUNK_FILE_CONTENT = "This is a chunk";
+
+  private static final String TEST_DESCRIPTOR_FILE_CONTENT = "descriptor";
+
+  private ContainerPacker packer = new TarContainerPacker();
+
+  private static final Path SOURCE_CONTAINER_ROOT =
+      Paths.get("target/test/data/packer-source-dir");
+
+  private static final Path DEST_CONTAINER_ROOT =
+      Paths.get("target/test/data/packer-dest-dir");
+
+  @BeforeClass
+  public static void init() throws IOException {
+    initDir(SOURCE_CONTAINER_ROOT);
+    initDir(DEST_CONTAINER_ROOT);
+  }
+
+  private static void initDir(Path path) throws IOException {
+    if (path.toFile().exists()) {
+      FileUtils.deleteDirectory(path.toFile());
+    }
+    path.toFile().mkdirs();
+  }
+
+  private KeyValueContainerData createContainer(long id, Path dir,
+      OzoneConfiguration conf) throws IOException {
+
+    Path containerDir = dir.resolve("container" + id);
+    Path dbDir = containerDir.resolve("db");
+    Path dataDir = containerDir.resolve("data");
+    Files.createDirectories(dbDir);
+    Files.createDirectories(dataDir);
+
+    KeyValueContainerData containerData = new KeyValueContainerData(id, -1);
+    containerData.setChunksPath(dataDir.toString());
+    containerData.setMetadataPath(dbDir.getParent().toString());
+    containerData.setDbFile(dbDir.toFile());
+
+
+    return containerData;
+  }
+
+  @Test
+  public void pack() throws IOException, CompressorException {
+
+    //GIVEN
+    OzoneConfiguration conf = new OzoneConfiguration();
+
+    KeyValueContainerData sourceContainerData =
+        createContainer(1L, SOURCE_CONTAINER_ROOT, conf);
+
+    KeyValueContainer sourceContainer =
+        new KeyValueContainer(sourceContainerData, conf);
+
+    //sample db file in the metadata directory
+    try (FileWriter writer = new FileWriter(
+        sourceContainerData.getDbFile().toPath()
+            .resolve(TEST_DB_FILE_NAME)
+            .toFile())) {
+      IOUtils.write(TEST_DB_FILE_CONTENT, writer);
+    }
+
+    //sample chunk file in the chunk directory
+    try (FileWriter writer = new FileWriter(
+        Paths.get(sourceContainerData.getChunksPath())
+            .resolve(TEST_CHUNK_FILE_NAME)
+            .toFile())) {
+      IOUtils.write(TEST_CHUNK_FILE_CONTENT, writer);
+    }
+
+    //sample container descriptor file
+    try (FileWriter writer = new FileWriter(
+        sourceContainer.getContainerFile())) {
+      IOUtils.write(TEST_DESCRIPTOR_FILE_CONTENT, writer);
+    }
+
+    Path targetFile =
+        SOURCE_CONTAINER_ROOT.getParent().resolve("container.tar.gz");
+
+    //WHEN: pack it
+    try (FileOutputStream output = new FileOutputStream(targetFile.toFile())) {
+      packer.pack(sourceContainer, output);
+    }
+
+    //THEN: check the result
+    try (FileInputStream input = new FileInputStream(targetFile.toFile())) {
+      CompressorInputStream uncompressed = new CompressorStreamFactory()
+          .createCompressorInputStream(CompressorStreamFactory.GZIP, input);
+      TarArchiveInputStream tarStream = new TarArchiveInputStream(uncompressed);
+
+      TarArchiveEntry entry;
+      Map<String, TarArchiveEntry> entries = new HashMap<>();
+      while ((entry = tarStream.getNextTarEntry()) != null) {
+        entries.put(entry.getName(), entry);
+      }
+
+      Assert.assertTrue(
+          entries.containsKey("container.yaml"));
+
+    }
+
+    //read the container descriptor only
+    try (FileInputStream input = new FileInputStream(targetFile.toFile())) {
+      String containerYaml = new String(packer.unpackContainerDescriptor(input),
+          Charset.forName(StandardCharsets.UTF_8.name()));
+      Assert.assertEquals(TEST_DESCRIPTOR_FILE_CONTENT, containerYaml);
+    }
+
+    KeyValueContainerData destinationContainerData =
+        createContainer(2L, DEST_CONTAINER_ROOT, conf);
+
+    KeyValueContainer destinationContainer =
+        new KeyValueContainer(destinationContainerData, conf);
+
+    String descriptor = "";
+
+    //unpackContainerData
+    try (FileInputStream input = new FileInputStream(targetFile.toFile())) {
+      descriptor =
+          new String(packer.unpackContainerData(destinationContainer, input),
+              Charset.forName(StandardCharsets.UTF_8.name()));
+    }
+
+    assertExampleMetadataDbIsGood(
+        destinationContainerData.getDbFile().toPath());
+    assertExampleChunkFileIsGood(
+        Paths.get(destinationContainerData.getChunksPath()));
+    Assert.assertFalse(
+        "Descriptor file should not been exctarcted by the "
+            + "unpackContainerData Call",
+        destinationContainer.getContainerFile().exists());
+    Assert.assertEquals(TEST_DESCRIPTOR_FILE_CONTENT, descriptor);
+
+  }
+
+
+  private void assertExampleMetadataDbIsGood(Path dbPath)
+      throws IOException {
+
+    Path dbFile = dbPath.resolve(TEST_DB_FILE_NAME);
+
+    Assert.assertTrue(
+        "example DB file is missing after pack/unpackContainerData: " + dbFile,
+        Files.exists(dbFile));
+
+    try (FileInputStream testFile = new FileInputStream(dbFile.toFile())) {
+      List<String> strings = IOUtils
+          .readLines(testFile, Charset.forName(StandardCharsets.UTF_8.name()));
+      Assert.assertEquals(1, strings.size());
+      Assert.assertEquals(TEST_DB_FILE_CONTENT, strings.get(0));
+    }
+  }
+
+  private void assertExampleChunkFileIsGood(Path chunkDirPath)
+      throws IOException {
+
+    Path chunkFile = chunkDirPath.resolve(TEST_CHUNK_FILE_NAME);
+
+    Assert.assertTrue(
+        "example chunk file is missing after pack/unpackContainerData: "
+            + chunkFile,
+        Files.exists(chunkFile));
+
+    try (FileInputStream testFile = new FileInputStream(chunkFile.toFile())) {
+      List<String> strings = IOUtils
+          .readLines(testFile, Charset.forName(StandardCharsets.UTF_8.name()));
+      Assert.assertEquals(1, strings.size());
+      Assert.assertEquals(TEST_CHUNK_FILE_CONTENT, strings.get(0));
+    }
+  }
+
+}
\ 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


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

Branch: refs/heads/YARN-3409
Commit: da2fd55f58fd317d1bba7f656146f1b91d807ccf
Parents: 2b3d54b
Author: Sunil G <su...@apache.org>
Authored: Fri May 25 16:08:26 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:57 2018 +0530

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


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


[07/50] [abbrv] hadoop git commit: YARN-8683. Support to display pending scheduling requests in RM app attempt page. Contributed by Tao Yang.

Posted by su...@apache.org.
YARN-8683. Support to display pending scheduling requests in RM app attempt page. 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/54d0bf89
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/54d0bf89
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/54d0bf89

Branch: refs/heads/YARN-3409
Commit: 54d0bf8935e35aad0f4d67df358ceb970cfcd713
Parents: d3fef7a
Author: Weiwei Yang <ww...@apache.org>
Authored: Tue Aug 21 19:00:31 2018 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Tue Aug 21 19:00:31 2018 +0800

----------------------------------------------------------------------
 .../scheduler/AbstractYarnScheduler.java        | 10 ++++
 .../scheduler/AppSchedulingInfo.java            | 17 +++++++
 .../scheduler/YarnScheduler.java                | 10 ++++
 .../placement/AppPlacementAllocator.java        |  6 +++
 .../LocalityAppPlacementAllocator.java          |  5 ++
 .../SingleConstraintAppPlacementAllocator.java  |  4 +-
 .../webapp/RMAppAttemptBlock.java               | 24 +++++++--
 .../resourcemanager/webapp/dao/AppInfo.java     | 10 ++++
 .../webapp/dao/ResourceRequestInfo.java         | 52 ++++++++++++++++++--
 9 files changed, 129 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.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/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index d2e81a5..9d2b058 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -878,6 +878,16 @@ public abstract class AbstractYarnScheduler
   }
 
   @Override
+  public List<SchedulingRequest> getPendingSchedulingRequestsForAttempt(
+      ApplicationAttemptId attemptId) {
+    SchedulerApplicationAttempt attempt = getApplicationAttempt(attemptId);
+    if (attempt != null) {
+      return attempt.getAppSchedulingInfo().getAllSchedulingRequests();
+    }
+    return null;
+  }
+
+  @Override
   public Priority checkAndGetApplicationPriority(
       Priority priorityRequestedByApp, UserGroupInformation user,
       String queueName, ApplicationId applicationId) throws YarnException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.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/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index 8074f06..d63d2b82 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -476,6 +476,23 @@ public class AppSchedulingInfo {
     return ret;
   }
 
+  /**
+   * Fetch SchedulingRequests.
+   * @return All pending SchedulingRequests.
+   */
+  public List<SchedulingRequest> getAllSchedulingRequests() {
+    List<SchedulingRequest> ret = new ArrayList<>();
+    try {
+      this.readLock.lock();
+      schedulerKeyToAppPlacementAllocator.values().stream()
+          .filter(ap -> ap.getSchedulingRequest() != null)
+          .forEach(ap -> ret.add(ap.getSchedulingRequest()));
+    } finally {
+      this.readLock.unlock();
+    }
+    return ret;
+  }
+
   public PendingAsk getNextPendingAsk() {
     try {
       readLock.lock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.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/YarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
index 43d55c4..0f7a5b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
@@ -364,6 +364,16 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
       ApplicationAttemptId attemptId);
 
   /**
+   * Get pending scheduling request for specified application attempt.
+   *
+   * @param attemptId the id of the application attempt
+   *
+   * @return pending scheduling requests
+   */
+  List<SchedulingRequest> getPendingSchedulingRequestsForAttempt(
+      ApplicationAttemptId attemptId);
+
+  /**
    * Get cluster max priority.
    * 
    * @return maximum priority of cluster

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
index df58157..088b3dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
@@ -192,4 +192,10 @@ public abstract class AppPlacementAllocator<N extends SchedulerNode> {
     this.rmContext = rmContext;
     this.schedulerRequestKey = schedulerRequestKey;
   }
+
+  /**
+   * Get pending SchedulingRequest.
+   * @return SchedulingRequest
+   */
+  public abstract SchedulingRequest getSchedulingRequest();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
index e1239a9..f1df343 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
@@ -425,4 +425,9 @@ public class LocalityAppPlacementAllocator <N extends SchedulerNode>
       writeLock.unlock();
     }
   }
+
+  @Override
+  public SchedulingRequest getSchedulingRequest() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
index 2b610f2..914f35d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
@@ -505,8 +505,8 @@ public class SingleConstraintAppPlacementAllocator<N extends SchedulerNode>
     }
   }
 
-  @VisibleForTesting
-  SchedulingRequest getSchedulingRequest() {
+  @Override
+  public SchedulingRequest getSchedulingRequest() {
     return schedulingRequest;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.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/RMAppAttemptBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
index 43a6ac9..99a569a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
@@ -87,10 +87,14 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
         .h3("Total Outstanding Resource Requests: "
             + getTotalResource(resourceRequests))
         .table("#resourceRequests").thead().tr().th(".priority", "Priority")
+        .th(".allocationRequestId", "AllocationRequestId")
         .th(".resource", "ResourceName").th(".capacity", "Capability")
         .th(".containers", "NumContainers")
         .th(".relaxlocality", "RelaxLocality")
-        .th(".labelexpression", "NodeLabelExpression").__().__().tbody();
+        .th(".labelexpression", "NodeLabelExpression")
+        .th(".executiontype", "ExecutionType")
+        .th(".allocationTags", "AllocationTags")
+        .th(".placementConstraint", "PlacementConstraint").__().__().tbody();
 
     StringBuilder resourceRequestTableData = new StringBuilder("[\n");
     for (ResourceRequestInfo resourceRequest  : resourceRequests) {
@@ -99,7 +103,11 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
       }
       resourceRequestTableData.append("[\"")
           .append(String.valueOf(resourceRequest.getPriority())).append("\",\"")
-          .append(resourceRequest.getResourceName()).append("\",\"")
+          .append(String.valueOf(resourceRequest.getAllocationRequestId()))
+          .append("\",\"")
+          .append(resourceRequest.getResourceName() == null ? "N/A"
+              : resourceRequest.getResourceName())
+          .append("\",\"")
           .append(StringEscapeUtils.escapeEcmaScript(StringEscapeUtils
               .escapeHtml4(String.valueOf(resourceRequest.getCapability()))))
           .append("\",\"")
@@ -109,6 +117,15 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
           .append("\",\"")
           .append(resourceRequest.getNodeLabelExpression() == null ? "N/A"
               : resourceRequest.getNodeLabelExpression())
+          .append("\",\"")
+          .append(resourceRequest.getExecutionTypeRequest() == null ? "N/A"
+              : resourceRequest.getExecutionTypeRequest().getExecutionType())
+          .append("\",\"")
+          .append(resourceRequest.getAllocationTags() == null ? "N/A" :
+              StringUtils.join(resourceRequest.getAllocationTags(), ","))
+          .append("\",\"")
+          .append(resourceRequest.getPlacementConstraint() == null ? "N/A"
+              : resourceRequest.getPlacementConstraint())
           .append("\"],\n");
     }
     if (resourceRequestTableData
@@ -132,7 +149,8 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
       if (request.getNumContainers() == 0) {
         continue;
       }
-      if (request.getResourceName().equals(ResourceRequest.ANY)) {
+      if (request.getResourceName() == null || request.getResourceName()
+          .equals(ResourceRequest.ANY)) {
         Resources.addTo(
             totalResource,
             Resources.multiply(request.getCapability().getResource(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.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/AppInfo.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/AppInfo.java
index 9d82bc7..63b6fe0 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/AppInfo.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/AppInfo.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -230,6 +231,15 @@ public class AppInfo {
                 resourceRequests.add(new ResourceRequestInfo(req));
               }
             }
+
+            List<SchedulingRequest> schedulingRequestsRaw = rm.getRMContext()
+                .getScheduler().getPendingSchedulingRequestsForAttempt(
+                    attempt.getAppAttemptId());
+            if (schedulingRequestsRaw != null) {
+              for (SchedulingRequest req : schedulingRequestsRaw) {
+                resourceRequests.add(new ResourceRequestInfo(req));
+              }
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceRequestInfo.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/ResourceRequestInfo.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/ResourceRequestInfo.java
index 030af45..beab9d4 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/ResourceRequestInfo.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/ResourceRequestInfo.java
@@ -21,11 +21,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
 import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 
 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.Set;
 
 /**
  * Simple class representing a resource request.
@@ -36,6 +38,8 @@ public class ResourceRequestInfo {
 
   @XmlElement(name = "priority")
   private int priority;
+  @XmlElement(name = "allocationRequestId")
+  private long allocationRequestId;
   @XmlElement(name = "resourceName")
   private String resourceName;
   @XmlElement(name = "capability")
@@ -50,16 +54,17 @@ public class ResourceRequestInfo {
   @XmlElement(name = "executionTypeRequest")
   private ExecutionTypeRequestInfo executionTypeRequest;
 
-  @XmlElement(name = "executionType")
-  private String executionType;
-  @XmlElement(name = "enforceExecutionType")
-  private boolean enforceExecutionType;
+  @XmlElement(name = "placementConstraint")
+  private String placementConstraint;
+  @XmlElement(name = "allocationTags")
+  private Set<String> allocationTags;
 
   public ResourceRequestInfo() {
   }
 
   public ResourceRequestInfo(ResourceRequest request) {
     priority = request.getPriority().getPriority();
+    allocationRequestId = request.getAllocationRequestId();
     resourceName = request.getResourceName();
     capability = new ResourceInfo(request.getCapability());
     numContainers = request.getNumContainers();
@@ -71,6 +76,21 @@ public class ResourceRequestInfo {
     }
   }
 
+  public ResourceRequestInfo(SchedulingRequest request) {
+    priority = request.getPriority().getPriority();
+    allocationRequestId = request.getAllocationRequestId();
+    capability = new ResourceInfo(request.getResourceSizing().getResources());
+    numContainers = request.getResourceSizing().getNumAllocations();
+    if (request.getExecutionType() != null) {
+      executionTypeRequest =
+          new ExecutionTypeRequestInfo(request.getExecutionType());
+    }
+    allocationTags = request.getAllocationTags();
+    if (request.getPlacementConstraint() != null) {
+      placementConstraint = request.getPlacementConstraint().toString();
+    }
+  }
+
   public Priority getPriority() {
     return Priority.newInstance(priority);
   }
@@ -128,4 +148,28 @@ public class ResourceRequestInfo {
   public ExecutionTypeRequestInfo getExecutionTypeRequest() {
     return executionTypeRequest;
   }
+
+  public String getPlacementConstraint() {
+    return placementConstraint;
+  }
+
+  public void setPlacementConstraint(String placementConstraint) {
+    this.placementConstraint = placementConstraint;
+  }
+
+  public Set<String> getAllocationTags() {
+    return allocationTags;
+  }
+
+  public void setAllocationTags(Set<String> allocationTags) {
+    this.allocationTags = allocationTags;
+  }
+
+  public long getAllocationRequestId() {
+    return allocationRequestId;
+  }
+
+  public void setAllocationRequestId(long allocationRequestId) {
+    this.allocationRequestId = allocationRequestId;
+  }
 }


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

Branch: refs/heads/YARN-3409
Commit: 1c022888c3a19d98925d1a2ef4d263ee3fc67ead
Parents: c535be9
Author: Sunil G <su...@apache.org>
Authored: Fri Apr 6 07:09:27 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:57 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/1c022888/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 b331381..eeadd93 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
@@ -3464,6 +3464,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/1c022888/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/1c022888/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/1c022888/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/1c022888/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/1c022888/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/1c022888/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/1c022888/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/1c022888/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/1c022888/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/1c022888/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/1c022888/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/1c022888/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 e6d708f..b74fccd 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
@@ -3950,4 +3950,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/1c022888/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/1c022888/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/1c022888/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/1c022888/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/1c022888/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


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

Branch: refs/heads/YARN-3409
Commit: c190002a099529b829485549266ff8dd99adf8b9
Parents: eed7de2
Author: Naganarasimha <na...@apache.org>
Authored: Fri Apr 20 07:31:53 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:57 2018 +0530

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


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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c190002a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesRequest.java
new file mode 100644
index 0000000..8e91bca
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesRequest.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.Set;
+
+/**
+ * <p>
+ * The request from clients to get nodes to attributes mapping
+ * in the cluster from the <code>ResourceManager</code>.
+ * </p>
+ *
+ * @see ApplicationClientProtocol#getNodesToAttributes
+ * (GetNodesToAttributesRequest)
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class GetNodesToAttributesRequest {
+
+  public static GetNodesToAttributesRequest newInstance(Set<String> hostNames) {
+    GetNodesToAttributesRequest request =
+        Records.newRecord(GetNodesToAttributesRequest.class);
+    request.setHostNames(hostNames);
+    return request;
+  }
+
+  /**
+   * Set hostnames for which mapping is required.
+   *
+   * @param hostnames
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public abstract void setHostNames(Set<String> hostnames);
+
+  /**
+   * Get hostnames for which mapping is required.
+   *
+   * @return Set<String> of hostnames.
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public abstract Set<String> getHostNames();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c190002a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesResponse.java
new file mode 100644
index 0000000..acc07bb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNodesToAttributesResponse.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * <p>
+ * The response sent by the <code>ResourceManager</code> to a client requesting
+ * nodes to attributes mapping.
+ * </p>
+ *
+ * @see ApplicationClientProtocol#getNodesToAttributes
+ * (GetNodesToAttributesRequest)
+ */
+@Public
+@Evolving
+public abstract class GetNodesToAttributesResponse {
+
+  public static GetNodesToAttributesResponse newInstance(
+      Map<String, Set<NodeAttribute>> map) {
+    GetNodesToAttributesResponse response =
+        Records.newRecord(GetNodesToAttributesResponse.class);
+    response.setNodeToAttributes(map);
+    return response;
+  }
+
+  @Public
+  @Evolving
+  public abstract void setNodeToAttributes(Map<String, Set<NodeAttribute>> map);
+
+  /**
+   * Get hostnames to NodeAttributes mapping.
+   *
+   * @return Map<String, Set<NodeAttribute>> host to attributes.
+   */
+  @Public
+  @Evolving
+  public abstract Map<String, Set<NodeAttribute>> getNodeToAttributes();
+}

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c190002a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesRequestPBImpl.java
new file mode 100644
index 0000000..0d9b722
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesRequestPBImpl.java
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
+
+import com.google.protobuf.TextFormat;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNodesToAttributesRequestProto;
+
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Request to get hostname to attributes mapping.
+ */
+public class GetNodesToAttributesRequestPBImpl
+    extends GetNodesToAttributesRequest {
+
+  private GetNodesToAttributesRequestProto proto =
+      GetNodesToAttributesRequestProto.getDefaultInstance();
+  private GetNodesToAttributesRequestProto.Builder builder = null;
+
+  private Set<String> hostNames = null;
+  private boolean viaProto = false;
+
+  public GetNodesToAttributesRequestPBImpl() {
+    builder = GetNodesToAttributesRequestProto.newBuilder();
+  }
+
+  public GetNodesToAttributesRequestPBImpl(
+      GetNodesToAttributesRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetNodesToAttributesRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (hostNames != null && !hostNames.isEmpty()) {
+      builder.clearHostnames();
+      builder.addAllHostnames(hostNames);
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public void setHostNames(Set<String> hostnames) {
+    maybeInitBuilder();
+    if (hostNames == null) {
+      builder.clearHostnames();
+    }
+    this.hostNames = hostnames;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder =
+          YarnServiceProtos.GetNodesToAttributesRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public Set<String> getHostNames() {
+    initNodeToAttributes();
+    return this.hostNames;
+  }
+
+  private void initNodeToAttributes() {
+    if (this.hostNames != null) {
+      return;
+    }
+    YarnServiceProtos.GetNodesToAttributesRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<String> hostNamesList = p.getHostnamesList();
+    this.hostNames = new HashSet<>();
+    this.hostNames.addAll(hostNamesList);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c190002a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesResponsePBImpl.java
new file mode 100644
index 0000000..1114d14
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToAttributesResponsePBImpl.java
@@ -0,0 +1,181 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Nodes to attributes request response.
+ */
+public class GetNodesToAttributesResponsePBImpl
+    extends GetNodesToAttributesResponse {
+
+  private YarnServiceProtos.GetNodesToAttributesResponseProto proto =
+      YarnServiceProtos.GetNodesToAttributesResponseProto.getDefaultInstance();
+  private YarnServiceProtos.GetNodesToAttributesResponseProto.Builder builder =
+      null;
+  private boolean viaProto = false;
+
+  private Map<String, Set<NodeAttribute>> nodesToAttributes;
+
+  public GetNodesToAttributesResponsePBImpl() {
+    this.builder =
+        YarnServiceProtos.GetNodesToAttributesResponseProto.newBuilder();
+  }
+
+  public GetNodesToAttributesResponsePBImpl(
+      YarnServiceProtos.GetNodesToAttributesResponseProto proto) {
+    this.proto = proto;
+    this.viaProto = true;
+  }
+
+  private void initNodesToAttributes() {
+    if (this.nodesToAttributes != null) {
+      return;
+    }
+    YarnServiceProtos.GetNodesToAttributesResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<YarnProtos.NodeToAttributesProto> list = p.getNodesToAttributesList();
+    this.nodesToAttributes = new HashMap<>();
+    for (YarnProtos.NodeToAttributesProto c : list) {
+      HashSet<NodeAttribute> attributes = new HashSet<>();
+      for (YarnProtos.NodeAttributeProto nodeAttrProto : c
+          .getNodeAttributesList()) {
+        attributes.add(new NodeAttributePBImpl(nodeAttrProto));
+      }
+      nodesToAttributes.put(c.getNode(), attributes);
+    }
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder =
+          YarnServiceProtos.GetNodesToAttributesResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void addNodesToAttributesToProto() {
+    maybeInitBuilder();
+    builder.clearNodesToAttributes();
+    if (nodesToAttributes == null) {
+      return;
+    }
+    Iterable<YarnProtos.NodeToAttributesProto> iterable =
+        () -> new Iterator<YarnProtos.NodeToAttributesProto>() {
+
+          private Iterator<Map.Entry<String, Set<NodeAttribute>>> iter =
+              nodesToAttributes.entrySet().iterator();
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+
+          @Override
+          public YarnProtos.NodeToAttributesProto next() {
+            Map.Entry<String, Set<NodeAttribute>> now = iter.next();
+            Set<YarnProtos.NodeAttributeProto> protoSet = new HashSet<>();
+            for (NodeAttribute nodeAttribute : now.getValue()) {
+              protoSet.add(convertToProtoFormat(nodeAttribute));
+            }
+            return YarnProtos.NodeToAttributesProto.newBuilder()
+                .setNode(now.getKey()).addAllNodeAttributes(protoSet).build();
+          }
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+        };
+    builder.addAllNodesToAttributes(iterable);
+  }
+
+  private NodeAttributePBImpl convertFromProtoFormat(
+      YarnProtos.NodeAttributeProto p) {
+    return new NodeAttributePBImpl(p);
+  }
+
+  private YarnProtos.NodeAttributeProto convertToProtoFormat(NodeAttribute t) {
+    return ((NodeAttributePBImpl) t).getProto();
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.nodesToAttributes != null) {
+      addNodesToAttributesToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  public YarnServiceProtos.GetNodesToAttributesResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    assert false : "hashCode not designed";
+    return 0;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public void setNodeToAttributes(Map<String, Set<NodeAttribute>> map) {
+    initNodesToAttributes();
+    nodesToAttributes.clear();
+    nodesToAttributes.putAll(map);
+  }
+
+  @Override
+  public Map<String, Set<NodeAttribute>> getNodeToAttributes() {
+    initNodesToAttributes();
+    return nodesToAttributes;
+  }
+}

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c190002a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
index 4a64473..46f0e89 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
@@ -66,6 +66,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToAttributesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
@@ -749,4 +751,10 @@ public class FederationClientInterceptor
       throws YarnException, IOException {
     throw new NotImplementedException();
   }
+
+  @Override
+  public GetNodesToAttributesResponse getNodesToAttributes(
+      GetNodesToAttributesRequest request) throws YarnException, IOException {
+    throw new NotImplementedException();
+  }
 }

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

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


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


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

Branch: refs/heads/YARN-3409
Commit: c535be93ccb442b4818c5aee8e729a4708093a70
Parents: 743de58
Author: bibinchundatt <bi...@apache.org>
Authored: Thu Apr 5 14:09:50 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:57 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/c535be93/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


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

Branch: refs/heads/YARN-3409
Commit: eed7de25e0435c2f8d0bdc03e695a59d83040092
Parents: 1c02288
Author: Naganarasimha <na...@apache.org>
Authored: Tue Apr 10 07:28:53 2018 +0800
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:57 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/eed7de25/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/eed7de25/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/eed7de25/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/eed7de25/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/eed7de25/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/eed7de25/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/eed7de25/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/eed7de25/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 eeadd93..dd804a3 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
@@ -3474,7 +3474,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/eed7de25/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/eed7de25/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/eed7de25/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index acd452d..084457b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -260,6 +260,21 @@ message GetClusterNodeLabelsResponseProto {
   repeated NodeLabelProto nodeLabels = 2;
 }
 
+message GetClusterNodeAttributesRequestProto {
+}
+
+message GetClusterNodeAttributesResponseProto {
+  repeated NodeAttributeProto nodeAttributes = 1;
+}
+
+message GetAttributesToNodesRequestProto {
+  repeated NodeAttributeProto nodeAttributes = 1;
+}
+
+message GetAttributesToNodesResponseProto {
+  repeated AttributeToNodesProto attributeToNodes = 1;
+}
+
 message UpdateApplicationPriorityRequestProto {
   required ApplicationIdProto applicationId = 1;
   required PriorityProto applicationPriority = 2;

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


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


[11/50] [abbrv] hadoop git commit: YARN-8298. Added express upgrade for YARN service. Contributed by Chandni Singh

Posted by su...@apache.org.
YARN-8298.  Added express upgrade for YARN service.
            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/e557c6bd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e557c6bd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e557c6bd

Branch: refs/heads/YARN-3409
Commit: e557c6bd8de2811a561210f672f47b4d07a9d5c6
Parents: 9c3fc3e
Author: Eric Yang <ey...@apache.org>
Authored: Tue Aug 21 19:49:26 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Tue Aug 21 19:49:26 2018 -0400

----------------------------------------------------------------------
 .../yarn/service/client/ApiServiceClient.java   |  20 +
 .../hadoop/yarn/service/webapp/ApiServer.java   |  12 +-
 .../hadoop/yarn/service/ClientAMService.java    |   2 +-
 .../hadoop/yarn/service/ServiceEvent.java       |  25 +
 .../hadoop/yarn/service/ServiceManager.java     | 127 +++-
 .../hadoop/yarn/service/ServiceScheduler.java   |  15 +-
 .../yarn/service/api/records/ServiceState.java  |   2 +-
 .../yarn/service/client/ServiceClient.java      | 100 ++-
 .../yarn/service/component/Component.java       |  16 +-
 .../yarn/service/component/ComponentEvent.java  |  10 +
 .../component/instance/ComponentInstance.java   |   5 +
 .../yarn/service/utils/ServiceApiUtil.java      |  44 ++
 .../src/main/proto/ClientAMProtocol.proto       |   1 +
 .../hadoop/yarn/service/TestServiceApiUtil.java | 653 ----------------
 .../hadoop/yarn/service/TestServiceManager.java | 299 +++++---
 .../yarn/service/TestYarnNativeServices.java    |  35 +
 .../yarn/service/utils/TestServiceApiUtil.java  | 743 +++++++++++++++++++
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |  20 +-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |   4 +
 .../hadoop/yarn/client/api/AppAdminClient.java  |  12 +
 20 files changed, 1308 insertions(+), 837 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
index 9229446..ca6cc50 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
@@ -601,6 +601,26 @@ public class ApiServiceClient extends AppAdminClient {
   }
 
   @Override
+  public int actionUpgradeExpress(String appName, File path)
+      throws IOException, YarnException {
+    int result;
+    try {
+      Service service =
+          loadAppJsonFromLocalFS(path.getAbsolutePath(), appName, null, null);
+      service.setState(ServiceState.EXPRESS_UPGRADING);
+      String buffer = jsonSerDeser.toJson(service);
+      LOG.info("Upgrade in progress. Please wait..");
+      ClientResponse response = getApiClient(getServicePath(appName))
+          .put(ClientResponse.class, buffer);
+      result = processResponse(response);
+    } catch (Exception e) {
+      LOG.error("Failed to upgrade application: ", e);
+      result = EXIT_EXCEPTION_THROWN;
+    }
+    return result;
+  }
+
+  @Override
   public int initiateUpgrade(String appName,
       String fileName, boolean autoFinalize) throws IOException, YarnException {
     int result;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
index 4db0ac8..cd6f0d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
@@ -440,7 +440,8 @@ public class ApiServer {
       if (updateServiceData.getState() != null && (
           updateServiceData.getState() == ServiceState.UPGRADING ||
               updateServiceData.getState() ==
-                  ServiceState.UPGRADING_AUTO_FINALIZE)) {
+                  ServiceState.UPGRADING_AUTO_FINALIZE) ||
+          updateServiceData.getState() == ServiceState.EXPRESS_UPGRADING) {
         return upgradeService(updateServiceData, ugi);
       }
 
@@ -690,7 +691,11 @@ public class ApiServer {
       ServiceClient sc = getServiceClient();
       sc.init(YARN_CONFIG);
       sc.start();
-      sc.initiateUpgrade(service);
+      if (service.getState().equals(ServiceState.EXPRESS_UPGRADING)) {
+        sc.actionUpgradeExpress(service);
+      } else {
+        sc.initiateUpgrade(service);
+      }
       sc.close();
       return null;
     });
@@ -706,7 +711,8 @@ public class ApiServer {
       String serviceName, Set<String> compNames) throws YarnException,
       IOException, InterruptedException {
     Service service = getServiceFromClient(ugi, serviceName);
-    if (service.getState() != ServiceState.UPGRADING) {
+    if (!service.getState().equals(ServiceState.UPGRADING) &&
+        !service.getState().equals(ServiceState.UPGRADING_AUTO_FINALIZE)) {
       throw new YarnException(
           String.format("The upgrade of service %s has not been initiated.",
               service.getName()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.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/ClientAMService.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/ClientAMService.java
index 5bf1833..2ef8f7e 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/ClientAMService.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/ClientAMService.java
@@ -166,7 +166,7 @@ public class ClientAMService extends AbstractService
       LOG.info("Upgrading service to version {} by {}", request.getVersion(),
           UserGroupInformation.getCurrentUser());
       context.getServiceManager().processUpgradeRequest(request.getVersion(),
-          request.getAutoFinalize());
+          request.getAutoFinalize(), request.getExpressUpgrade());
       return UpgradeServiceResponseProto.newBuilder().build();
     } catch (Exception ex) {
       return UpgradeServiceResponseProto.newBuilder().setError(ex.getMessage())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceEvent.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/ServiceEvent.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/ServiceEvent.java
index 0196be2..3a55472 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/ServiceEvent.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/ServiceEvent.java
@@ -19,6 +19,9 @@
 package org.apache.hadoop.yarn.service;
 
 import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.hadoop.yarn.service.api.records.Component;
+
+import java.util.Queue;
 
 /**
  * Events are handled by {@link ServiceManager} to manage the service
@@ -29,6 +32,8 @@ public class ServiceEvent extends AbstractEvent<ServiceEventType> {
   private final ServiceEventType type;
   private String version;
   private boolean autoFinalize;
+  private boolean expressUpgrade;
+  private Queue<Component> compsToUpgradeInOrder;
 
   public ServiceEvent(ServiceEventType serviceEventType) {
     super(serviceEventType);
@@ -56,4 +61,24 @@ public class ServiceEvent extends AbstractEvent<ServiceEventType> {
     this.autoFinalize = autoFinalize;
     return this;
   }
+
+  public boolean isExpressUpgrade() {
+    return expressUpgrade;
+  }
+
+  public ServiceEvent setExpressUpgrade(boolean expressUpgrade) {
+    this.expressUpgrade = expressUpgrade;
+    return this;
+  }
+
+  public Queue<Component> getCompsToUpgradeInOrder() {
+    return compsToUpgradeInOrder;
+  }
+
+  public ServiceEvent setCompsToUpgradeInOrder(
+      Queue<Component> compsToUpgradeInOrder) {
+    this.compsToUpgradeInOrder = compsToUpgradeInOrder;
+    return this;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceManager.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/ServiceManager.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/ServiceManager.java
index 05ecb3f..04454b1 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/ServiceManager.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/ServiceManager.java
@@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.service.api.records.ComponentState;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.component.Component;
@@ -40,8 +41,11 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.text.MessageFormat;
 import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Queue;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import static org.apache.hadoop.yarn.service.utils.ServiceApiUtil.jsonSerDeser;
@@ -67,6 +71,8 @@ public class ServiceManager implements EventHandler<ServiceEvent> {
   private final SliderFileSystem fs;
 
   private String upgradeVersion;
+  private Queue<org.apache.hadoop.yarn.service.api.records
+        .Component> compsToUpgradeInOrder;
 
   private static final StateMachineFactory<ServiceManager, State,
       ServiceEventType, ServiceEvent> STATE_MACHINE_FACTORY =
@@ -141,14 +147,20 @@ public class ServiceManager implements EventHandler<ServiceEvent> {
     @Override
     public State transition(ServiceManager serviceManager,
         ServiceEvent event) {
+      serviceManager.upgradeVersion = event.getVersion();
       try {
-        if (!event.isAutoFinalize()) {
-          serviceManager.serviceSpec.setState(ServiceState.UPGRADING);
+        if (event.isExpressUpgrade()) {
+          serviceManager.serviceSpec.setState(ServiceState.EXPRESS_UPGRADING);
+          serviceManager.compsToUpgradeInOrder = event
+              .getCompsToUpgradeInOrder();
+          serviceManager.upgradeNextCompIfAny();
+        } else if (event.isAutoFinalize()) {
+          serviceManager.serviceSpec.setState(ServiceState
+              .UPGRADING_AUTO_FINALIZE);
         } else {
           serviceManager.serviceSpec.setState(
-              ServiceState.UPGRADING_AUTO_FINALIZE);
+              ServiceState.UPGRADING);
         }
-        serviceManager.upgradeVersion = event.getVersion();
         return State.UPGRADING;
       } catch (Throwable e) {
         LOG.error("[SERVICE]: Upgrade to version {} failed", event.getVersion(),
@@ -169,8 +181,19 @@ public class ServiceManager implements EventHandler<ServiceEvent> {
       if (currState.equals(ServiceState.STABLE)) {
         return State.STABLE;
       }
+      if (currState.equals(ServiceState.EXPRESS_UPGRADING)) {
+        org.apache.hadoop.yarn.service.api.records.Component component =
+            serviceManager.compsToUpgradeInOrder.peek();
+        if (!component.getState().equals(ComponentState.NEEDS_UPGRADE) &&
+            !component.getState().equals(ComponentState.UPGRADING)) {
+          serviceManager.compsToUpgradeInOrder.remove();
+        }
+        serviceManager.upgradeNextCompIfAny();
+      }
       if (currState.equals(ServiceState.UPGRADING_AUTO_FINALIZE) ||
-          event.getType().equals(ServiceEventType.START)) {
+          event.getType().equals(ServiceEventType.START) ||
+          (currState.equals(ServiceState.EXPRESS_UPGRADING) &&
+              serviceManager.compsToUpgradeInOrder.isEmpty())) {
         ServiceState targetState = checkIfStable(serviceManager.serviceSpec);
         if (targetState.equals(ServiceState.STABLE)) {
           if (serviceManager.finalizeUpgrade()) {
@@ -184,6 +207,19 @@ public class ServiceManager implements EventHandler<ServiceEvent> {
     }
   }
 
+  private void upgradeNextCompIfAny() {
+    if (!compsToUpgradeInOrder.isEmpty()) {
+      org.apache.hadoop.yarn.service.api.records.Component component =
+          compsToUpgradeInOrder.peek();
+
+      ComponentEvent needUpgradeEvent = new ComponentEvent(
+          component.getName(), ComponentEventType.UPGRADE).setTargetSpec(
+          component).setUpgradeVersion(upgradeVersion).setExpressUpgrade(true);
+      context.scheduler.getDispatcher().getEventHandler().handle(
+          needUpgradeEvent);
+    }
+  }
+
   /**
    * @return whether finalization of upgrade was successful.
    */
@@ -250,23 +286,18 @@ public class ServiceManager implements EventHandler<ServiceEvent> {
   }
 
   void processUpgradeRequest(String upgradeVersion,
-      boolean autoFinalize) throws IOException {
+      boolean autoFinalize, boolean expressUpgrade) throws IOException {
     Service targetSpec = ServiceApiUtil.loadServiceUpgrade(
         context.fs, context.service.getName(), upgradeVersion);
 
     List<org.apache.hadoop.yarn.service.api.records.Component>
-        compsThatNeedUpgrade = componentsFinder.
+        compsNeedUpgradeList = componentsFinder.
         findTargetComponentSpecs(context.service, targetSpec);
-    ServiceEvent event = new ServiceEvent(ServiceEventType.UPGRADE)
-        .setVersion(upgradeVersion)
-        .setAutoFinalize(autoFinalize);
-    context.scheduler.getDispatcher().getEventHandler().handle(event);
 
-    if (compsThatNeedUpgrade != null && !compsThatNeedUpgrade.isEmpty()) {
-      if (autoFinalize) {
-        event.setAutoFinalize(true);
-      }
-      compsThatNeedUpgrade.forEach(component -> {
+    // remove all components from need upgrade list if there restart policy
+    // doesn't all upgrade.
+    if (compsNeedUpgradeList != null) {
+      compsNeedUpgradeList.removeIf(component -> {
         org.apache.hadoop.yarn.service.api.records.Component.RestartPolicyEnum
             restartPolicy = component.getRestartPolicy();
 
@@ -274,25 +305,65 @@ public class ServiceManager implements EventHandler<ServiceEvent> {
             Component.getRestartPolicyHandler(restartPolicy);
         // Do not allow upgrades for components which have NEVER/ON_FAILURE
         // restart policy
-        if (restartPolicyHandler.allowUpgrades()) {
+        if (!restartPolicyHandler.allowUpgrades()) {
+          LOG.info("The component {} has a restart policy that doesnt " +
+                  "allow upgrades {} ", component.getName(),
+              component.getRestartPolicy().toString());
+          return true;
+        }
+
+        return false;
+      });
+    }
+
+    ServiceEvent event = new ServiceEvent(ServiceEventType.UPGRADE)
+        .setVersion(upgradeVersion)
+        .setAutoFinalize(autoFinalize)
+        .setExpressUpgrade(expressUpgrade);
+
+    if (expressUpgrade) {
+      // In case of express upgrade  components need to be upgraded in order.
+      // Once the service manager gets notified that a component finished
+      // upgrading, it then issues event to upgrade the next component.
+      Map<String, org.apache.hadoop.yarn.service.api.records.Component>
+          compsNeedUpgradeByName = new HashMap<>();
+      if (compsNeedUpgradeList != null) {
+        compsNeedUpgradeList.forEach(component ->
+            compsNeedUpgradeByName.put(component.getName(), component));
+      }
+      List<String> resolvedComps = ServiceApiUtil
+          .resolveCompsDependency(targetSpec);
+
+      Queue<org.apache.hadoop.yarn.service.api.records.Component>
+          orderedCompUpgrade = new LinkedList<>();
+      resolvedComps.forEach(compName -> {
+        org.apache.hadoop.yarn.service.api.records.Component component =
+            compsNeedUpgradeByName.get(compName);
+        if (component != null ) {
+          orderedCompUpgrade.add(component);
+        }
+      });
+      event.setCompsToUpgradeInOrder(orderedCompUpgrade);
+    }
+
+    context.scheduler.getDispatcher().getEventHandler().handle(event);
+
+    if (compsNeedUpgradeList != null && !compsNeedUpgradeList.isEmpty()) {
+      if (!expressUpgrade) {
+        compsNeedUpgradeList.forEach(component -> {
           ComponentEvent needUpgradeEvent = new ComponentEvent(
               component.getName(), ComponentEventType.UPGRADE).setTargetSpec(
               component).setUpgradeVersion(event.getVersion());
           context.scheduler.getDispatcher().getEventHandler().handle(
               needUpgradeEvent);
-        } else {
-          LOG.info("The component {} has a restart "
-              + "policy that doesnt allow upgrades {} ", component.getName(),
-              component.getRestartPolicy().toString());
-        }
-      });
-    } else {
+
+        });
+      }
+    }  else if (autoFinalize) {
       // nothing to upgrade if upgrade auto finalize is requested, trigger a
       // state check.
-      if (autoFinalize) {
-        context.scheduler.getDispatcher().getEventHandler().handle(
-            new ServiceEvent(ServiceEventType.CHECK_STABLE));
-      }
+      context.scheduler.getDispatcher().getEventHandler().handle(
+          new ServiceEvent(ServiceEventType.CHECK_STABLE));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/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 0801ad0..384659f 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
@@ -219,7 +219,7 @@ public class ServiceScheduler extends CompositeService {
     nmClient.getClient().cleanupRunningContainersOnStop(false);
     addIfService(nmClient);
 
-    dispatcher = new AsyncDispatcher("Component  dispatcher");
+    dispatcher = createAsyncDispatcher();
     dispatcher.register(ServiceEventType.class, new ServiceEventHandler());
     dispatcher.register(ComponentEventType.class,
         new ComponentEventHandler());
@@ -253,6 +253,9 @@ public class ServiceScheduler extends CompositeService {
         YarnServiceConf.CONTAINER_RECOVERY_TIMEOUT_MS,
         YarnServiceConf.DEFAULT_CONTAINER_RECOVERY_TIMEOUT_MS,
         app.getConfiguration(), getConfig());
+
+    serviceManager = createServiceManager();
+    context.setServiceManager(serviceManager);
   }
 
   protected YarnRegistryViewForProviders createYarnRegistryOperations(
@@ -262,6 +265,14 @@ public class ServiceScheduler extends CompositeService {
         context.attemptId);
   }
 
+  protected ServiceManager createServiceManager() {
+    return new ServiceManager(context);
+  }
+
+  protected AsyncDispatcher createAsyncDispatcher() {
+    return new AsyncDispatcher("Component  dispatcher");
+  }
+
   protected NMClientAsync createNMClient() {
     return NMClientAsync.createNMClientAsync(new NMClientCallback());
   }
@@ -344,8 +355,6 @@ public class ServiceScheduler extends CompositeService {
 
     // Since AM has been started and registered, the service is in STARTED state
     app.setState(ServiceState.STARTED);
-    serviceManager = new ServiceManager(context);
-    context.setServiceManager(serviceManager);
 
     // recover components based on containers sent from RM
     recoverComponents(response);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.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/api/records/ServiceState.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/api/records/ServiceState.java
index b6ae38b..0b3c037 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/api/records/ServiceState.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/api/records/ServiceState.java
@@ -30,5 +30,5 @@ import org.apache.hadoop.classification.InterfaceStability;
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 public enum ServiceState {
   ACCEPTED, STARTED, STABLE, STOPPED, FAILED, FLEX, UPGRADING,
-  UPGRADING_AUTO_FINALIZE;
+  UPGRADING_AUTO_FINALIZE, EXPRESS_UPGRADING;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.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/client/ServiceClient.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/client/ServiceClient.java
index 5668d9f..a27ed87 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/client/ServiceClient.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/client/ServiceClient.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.service.client;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -215,48 +216,31 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     return EXIT_SUCCESS;
   }
 
-  @Override
-  public int initiateUpgrade(String appName, String fileName,
-      boolean autoFinalize)
-      throws IOException, YarnException {
-    Service upgradeService = loadAppJsonFromLocalFS(fileName, appName,
-        null, null);
-    if (autoFinalize) {
-      upgradeService.setState(ServiceState.UPGRADING_AUTO_FINALIZE);
-    } else {
-      upgradeService.setState(ServiceState.UPGRADING);
-    }
-    return initiateUpgrade(upgradeService);
-  }
-
-  public int initiateUpgrade(Service service) throws YarnException,
-      IOException {
+  private ApplicationReport upgradePrecheck(Service service)
+      throws YarnException, IOException {
     boolean upgradeEnabled = getConfig().getBoolean(
-        YARN_SERVICE_UPGRADE_ENABLED,
-        YARN_SERVICE_UPGRADE_ENABLED_DEFAULT);
+        YARN_SERVICE_UPGRADE_ENABLED, YARN_SERVICE_UPGRADE_ENABLED_DEFAULT);
     if (!upgradeEnabled) {
       throw new YarnException(ErrorStrings.SERVICE_UPGRADE_DISABLED);
     }
-    Service persistedService =
-        ServiceApiUtil.loadService(fs, service.getName());
+    Service persistedService = ServiceApiUtil.loadService(fs,
+        service.getName());
     if (!StringUtils.isEmpty(persistedService.getId())) {
-      cachedAppInfo.put(persistedService.getName(), new AppInfo(
-          ApplicationId.fromString(persistedService.getId()),
-          persistedService.getKerberosPrincipal().getPrincipalName()));
+      cachedAppInfo.put(persistedService.getName(),
+          new AppInfo(ApplicationId.fromString(persistedService.getId()),
+              persistedService.getKerberosPrincipal().getPrincipalName()));
     }
 
     if (persistedService.getVersion().equals(service.getVersion())) {
-      String message =
-          service.getName() + " is already at version " + service.getVersion()
-              + ". There is nothing to upgrade.";
+      String message = service.getName() + " is already at version "
+          + service.getVersion() + ". There is nothing to upgrade.";
       LOG.error(message);
       throw new YarnException(message);
     }
 
     Service liveService = getStatus(service.getName());
     if (!liveService.getState().equals(ServiceState.STABLE)) {
-      String message = service.getName() + " is at " +
-          liveService.getState()
+      String message = service.getName() + " is at " + liveService.getState()
           + " state and upgrade can only be initiated when service is STABLE.";
       LOG.error(message);
       throw new YarnException(message);
@@ -266,11 +250,67 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     ServiceApiUtil.validateAndResolveService(service, fs, getConfig());
     ServiceApiUtil.createDirAndPersistApp(fs, serviceUpgradeDir, service);
 
-    ApplicationReport appReport =
-        yarnClient.getApplicationReport(getAppId(service.getName()));
+    ApplicationReport appReport = yarnClient
+        .getApplicationReport(getAppId(service.getName()));
     if (StringUtils.isEmpty(appReport.getHost())) {
       throw new YarnException(service.getName() + " AM hostname is empty");
     }
+    return appReport;
+  }
+
+  @Override
+  public int actionUpgradeExpress(String appName, File path)
+      throws IOException, YarnException {
+    Service service =
+        loadAppJsonFromLocalFS(path.getAbsolutePath(), appName, null, null);
+    service.setState(ServiceState.UPGRADING_AUTO_FINALIZE);
+    actionUpgradeExpress(service);
+    return EXIT_SUCCESS;
+  }
+
+  public int actionUpgradeExpress(Service service) throws YarnException,
+      IOException {
+    ApplicationReport appReport = upgradePrecheck(service);
+    ClientAMProtocol proxy = createAMProxy(service.getName(), appReport);
+    UpgradeServiceRequestProto.Builder requestBuilder =
+        UpgradeServiceRequestProto.newBuilder();
+    requestBuilder.setVersion(service.getVersion());
+    if (service.getState().equals(ServiceState.UPGRADING_AUTO_FINALIZE)) {
+      requestBuilder.setAutoFinalize(true);
+    }
+    if (service.getState().equals(ServiceState.EXPRESS_UPGRADING)) {
+      requestBuilder.setExpressUpgrade(true);
+      requestBuilder.setAutoFinalize(true);
+    }
+    UpgradeServiceResponseProto responseProto = proxy.upgrade(
+        requestBuilder.build());
+    if (responseProto.hasError()) {
+      LOG.error("Service {} express upgrade to version {} failed because {}",
+          service.getName(), service.getVersion(), responseProto.getError());
+      throw new YarnException("Failed to express upgrade service " +
+          service.getName() + " to version " + service.getVersion() +
+          " because " + responseProto.getError());
+    }
+    return EXIT_SUCCESS;
+  }
+
+  @Override
+  public int initiateUpgrade(String appName, String fileName,
+      boolean autoFinalize)
+      throws IOException, YarnException {
+    Service upgradeService = loadAppJsonFromLocalFS(fileName, appName,
+        null, null);
+    if (autoFinalize) {
+      upgradeService.setState(ServiceState.UPGRADING_AUTO_FINALIZE);
+    } else {
+      upgradeService.setState(ServiceState.UPGRADING);
+    }
+    return initiateUpgrade(upgradeService);
+  }
+
+  public int initiateUpgrade(Service service) throws YarnException,
+      IOException {
+    ApplicationReport appReport = upgradePrecheck(service);
     ClientAMProtocol proxy = createAMProxy(service.getName(), appReport);
 
     UpgradeServiceRequestProto.Builder requestBuilder =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/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 41a2fcd..acf3404 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
@@ -21,6 +21,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.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import static org.apache.hadoop.yarn.service.api.records.Component
@@ -43,6 +44,7 @@ import org.apache.hadoop.yarn.service.ServiceEventType;
 import org.apache.hadoop.yarn.service.api.records.ContainerState;
 import org.apache.hadoop.yarn.service.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceId;
 import org.apache.hadoop.yarn.service.ContainerFailureTracker;
 import org.apache.hadoop.yarn.service.ServiceContext;
@@ -546,13 +548,21 @@ public class Component implements EventHandler<ComponentEvent> {
     @Override
     public void transition(Component component, ComponentEvent event) {
       component.upgradeInProgress.set(true);
+      component.upgradeEvent = event;
       component.componentSpec.setState(org.apache.hadoop.yarn.service.api.
           records.ComponentState.NEEDS_UPGRADE);
       component.numContainersThatNeedUpgrade.set(
           component.componentSpec.getNumberOfContainers());
-      component.componentSpec.getContainers().forEach(container ->
-          container.setState(ContainerState.NEEDS_UPGRADE));
-      component.upgradeEvent = event;
+      component.componentSpec.getContainers().forEach(container -> {
+        container.setState(ContainerState.NEEDS_UPGRADE);
+        if (event.isExpressUpgrade()) {
+          ComponentInstanceEvent upgradeEvent = new ComponentInstanceEvent(
+              ContainerId.fromString(container.getId()),
+                  ComponentInstanceEventType.UPGRADE);
+          LOG.info("Upgrade container {}", container.getId());
+          component.dispatcher.getEventHandler().handle(upgradeEvent);
+        }
+      });
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/ComponentEvent.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/ComponentEvent.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/ComponentEvent.java
index 84caa77..643961d 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/ComponentEvent.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/ComponentEvent.java
@@ -35,6 +35,7 @@ public class ComponentEvent extends AbstractEvent<ComponentEventType> {
   private ContainerId containerId;
   private org.apache.hadoop.yarn.service.api.records.Component targetSpec;
   private String upgradeVersion;
+  private boolean expressUpgrade;
 
   public ContainerId getContainerId() {
     return containerId;
@@ -113,4 +114,13 @@ public class ComponentEvent extends AbstractEvent<ComponentEventType> {
     this.upgradeVersion = upgradeVersion;
     return this;
   }
+
+  public boolean isExpressUpgrade() {
+    return expressUpgrade;
+  }
+
+  public ComponentEvent setExpressUpgrade(boolean expressUpgrade) {
+    this.expressUpgrade = expressUpgrade;
+    return this;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/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 11a6caa..ed5e68e 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
@@ -380,6 +380,11 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
     @Override
     public void transition(ComponentInstance compInstance,
         ComponentInstanceEvent event) {
+      if (!compInstance.containerSpec.getState().equals(
+          ContainerState.NEEDS_UPGRADE)) {
+        //nothing to upgrade. this may happen with express upgrade.
+        return;
+      }
       compInstance.containerSpec.setState(ContainerState.UPGRADING);
       compInstance.component.decContainersReady(false);
       ComponentEvent upgradeEvent = compInstance.component.getUpgradeEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/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 9219569..b588e88 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
@@ -638,6 +638,32 @@ public class ServiceApiUtil {
     return containerNeedUpgrade;
   }
 
+  /**
+   * Validates the components that are requested are stable for upgrade.
+   * It returns the instances of the components which are in ready state.
+   */
+  public static List<Container> validateAndResolveCompsStable(
+      Service liveService, Collection<String> compNames) throws YarnException {
+    Preconditions.checkNotNull(compNames);
+    HashSet<String> requestedComps = Sets.newHashSet(compNames);
+    List<Container> containerNeedUpgrade = new ArrayList<>();
+    for (Component liveComp : liveService.getComponents()) {
+      if (requestedComps.contains(liveComp.getName())) {
+        if (!liveComp.getState().equals(ComponentState.STABLE)) {
+          // Nothing to upgrade
+          throw new YarnException(String.format(
+              ERROR_COMP_DOES_NOT_NEED_UPGRADE, liveComp.getName()));
+        }
+        liveComp.getContainers().forEach(liveContainer -> {
+          if (liveContainer.getState().equals(ContainerState.READY)) {
+            containerNeedUpgrade.add(liveContainer);
+          }
+        });
+      }
+    }
+    return containerNeedUpgrade;
+  }
+
   private static String parseComponentName(String componentInstanceName)
       throws YarnException {
     int idx = componentInstanceName.lastIndexOf('-');
@@ -651,4 +677,22 @@ public class ServiceApiUtil {
   public static String $(String s) {
     return "${" + s +"}";
   }
+
+  public static List<String> resolveCompsDependency(Service service) {
+    List<String> components = new ArrayList<String>();
+    for (Component component : service.getComponents()) {
+      int depSize = component.getDependencies().size();
+      if (!components.contains(component.getName())) {
+        components.add(component.getName());
+      }
+      if (depSize != 0) {
+        for (String depComp : component.getDependencies()) {
+          if (!components.contains(depComp)) {
+            components.add(0, depComp);
+          }
+        }
+      }
+    }
+    return components;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto
index 6166ded..169f765 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto
@@ -66,6 +66,7 @@ message StopResponseProto {
 message UpgradeServiceRequestProto {
   optional string version = 1;
   optional bool autoFinalize = 2;
+  optional bool expressUpgrade = 3;
 }
 
 message UpgradeServiceResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/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
deleted file mode 100644
index c2a80e7..0000000
--- 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
+++ /dev/null
@@ -1,653 +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.service;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.registry.client.api.RegistryConstants;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.service.api.records.Artifact;
-import org.apache.hadoop.yarn.service.api.records.Component;
-import org.apache.hadoop.yarn.service.api.records.KerberosPrincipal;
-import org.apache.hadoop.yarn.service.api.records.PlacementConstraint;
-import org.apache.hadoop.yarn.service.api.records.PlacementPolicy;
-import org.apache.hadoop.yarn.service.api.records.PlacementScope;
-import org.apache.hadoop.yarn.service.api.records.PlacementType;
-import org.apache.hadoop.yarn.service.api.records.Resource;
-import org.apache.hadoop.yarn.service.api.records.Service;
-import org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages;
-import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
-import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-import static org.apache.hadoop.yarn.service.conf.RestApiConstants.DEFAULT_UNLIMITED_LIFETIME;
-import static org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages.*;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Test for ServiceApiUtil helper methods.
- */
-public class TestServiceApiUtil {
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TestServiceApiUtil.class);
-  private static final String EXCEPTION_PREFIX = "Should have thrown " +
-      "exception: ";
-  private static final String NO_EXCEPTION_PREFIX = "Should not have thrown " +
-      "exception: ";
-
-  private static final String LEN_64_STR =
-      "abcdefghijklmnopqrstuvwxyz0123456789abcdefghijklmnopqrstuvwxyz01";
-
-  private static final YarnConfiguration CONF_DEFAULT_DNS = new
-      YarnConfiguration();
-  private static final YarnConfiguration CONF_DNS_ENABLED = new
-      YarnConfiguration();
-
-  @BeforeClass
-  public static void init() {
-    CONF_DNS_ENABLED.setBoolean(RegistryConstants.KEY_DNS_ENABLED, true);
-  }
-
-  @Test(timeout = 90000)
-  public void testResourceValidation() throws Exception {
-    assertEquals(RegistryConstants.MAX_FQDN_LABEL_LENGTH + 1, LEN_64_STR
-        .length());
-
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-
-    Service app = new Service();
-
-    // no name
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with no name");
-    } catch (IllegalArgumentException e) {
-      assertEquals(ERROR_APPLICATION_NAME_INVALID, e.getMessage());
-    }
-
-    app.setName("test");
-    // no version
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + " service with no version");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(ERROR_APPLICATION_VERSION_INVALID,
-          app.getName()), e.getMessage());
-    }
-
-    app.setVersion("v1");
-    // bad format name
-    String[] badNames = {"4finance", "Finance", "finance@home", LEN_64_STR};
-    for (String badName : badNames) {
-      app.setName(badName);
-      try {
-        ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-        Assert.fail(EXCEPTION_PREFIX + "service with bad name " + badName);
-      } catch (IllegalArgumentException e) {
-
-      }
-    }
-
-    // launch command not specified
-    app.setName(LEN_64_STR);
-    Component comp = new Component().name("comp1");
-    app.addComponent(comp);
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DEFAULT_DNS);
-      Assert.fail(EXCEPTION_PREFIX + "service with no launch command");
-    } catch (IllegalArgumentException e) {
-      assertEquals(RestApiErrorMessages.ERROR_ABSENT_LAUNCH_COMMAND,
-          e.getMessage());
-    }
-
-    // launch command not specified
-    app.setName(LEN_64_STR.substring(0, RegistryConstants
-        .MAX_FQDN_LABEL_LENGTH));
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with no launch command");
-    } catch (IllegalArgumentException e) {
-      assertEquals(RestApiErrorMessages.ERROR_ABSENT_LAUNCH_COMMAND,
-          e.getMessage());
-    }
-
-    // memory not specified
-    comp.setLaunchCommand("sleep 1");
-    Resource res = new Resource();
-    app.setResource(res);
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with no memory");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_RESOURCE_MEMORY_FOR_COMP_INVALID,
-          comp.getName()), e.getMessage());
-    }
-
-    // invalid no of cpus
-    res.setMemory("100mb");
-    res.setCpus(-2);
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(
-          EXCEPTION_PREFIX + "service with invalid no of cpus");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_RESOURCE_CPUS_FOR_COMP_INVALID_RANGE,
-          comp.getName()), e.getMessage());
-    }
-
-    // number of containers not specified
-    res.setCpus(2);
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with no container count");
-    } catch (IllegalArgumentException e) {
-      Assert.assertTrue(e.getMessage()
-          .contains(ERROR_CONTAINERS_COUNT_INVALID));
-    }
-
-    // specifying profile along with cpus/memory raises exception
-    res.setProfile("hbase_finance_large");
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX
-          + "service with resource profile along with cpus/memory");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(RestApiErrorMessages
-              .ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_FOR_COMP_NOT_SUPPORTED,
-          comp.getName()),
-          e.getMessage());
-    }
-
-    // currently resource profile alone is not supported.
-    // TODO: remove the next test once resource profile alone is supported.
-    res.setCpus(null);
-    res.setMemory(null);
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with resource profile only");
-    } catch (IllegalArgumentException e) {
-      assertEquals(ERROR_RESOURCE_PROFILE_NOT_SUPPORTED_YET,
-          e.getMessage());
-    }
-
-    // unset profile here and add cpus/memory back
-    res.setProfile(null);
-    res.setCpus(2);
-    res.setMemory("2gb");
-
-    // null number of containers
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "null number of containers");
-    } catch (IllegalArgumentException e) {
-      Assert.assertTrue(e.getMessage()
-          .startsWith(ERROR_CONTAINERS_COUNT_INVALID));
-    }
-  }
-
-  @Test
-  public void testArtifacts() throws IOException {
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-
-    Service app = new Service();
-    app.setName("service1");
-    app.setVersion("v1");
-    Resource res = new Resource();
-    app.setResource(res);
-    res.setMemory("512M");
-
-    // no artifact id fails with default type
-    Artifact artifact = new Artifact();
-    app.setArtifact(artifact);
-    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(String.format(ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName),
-          e.getMessage());
-    }
-
-    // no artifact id fails with SERVICE type
-    artifact.setType(Artifact.TypeEnum.SERVICE);
-    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());
-    }
-
-    // no artifact id fails with TARBALL type
-    artifact.setType(Artifact.TypeEnum.TARBALL);
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with no artifact id");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName),
-          e.getMessage());
-    }
-
-    // everything valid here
-    artifact.setType(Artifact.TypeEnum.DOCKER);
-    artifact.setId("docker.io/centos:centos7");
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      LOG.error("service attributes specified should be valid here", e);
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-
-    assertEquals(app.getLifetime(), DEFAULT_UNLIMITED_LIFETIME);
-  }
-
-  private static Resource createValidResource() {
-    Resource res = new Resource();
-    res.setMemory("512M");
-    return res;
-  }
-
-  private static Component createValidComponent(String compName) {
-    Component comp = new Component();
-    comp.setName(compName);
-    comp.setResource(createValidResource());
-    comp.setNumberOfContainers(1L);
-    comp.setLaunchCommand("sleep 1");
-    return comp;
-  }
-
-  private static Service createValidApplication(String compName) {
-    Service app = new Service();
-    app.setName("name");
-    app.setVersion("v1");
-    app.setResource(createValidResource());
-    if (compName != null) {
-      app.addComponent(createValidComponent(compName));
-    }
-    return app;
-  }
-
-  @Test
-  public void testExternalApplication() throws IOException {
-    Service ext = createValidApplication("comp1");
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs(ext);
-
-    Service app = createValidApplication(null);
-
-    Artifact artifact = new Artifact();
-    artifact.setType(Artifact.TypeEnum.SERVICE);
-    artifact.setId("id");
-    app.setArtifact(artifact);
-    app.addComponent(ServiceTestUtils.createComponent("comp2"));
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-
-    assertEquals(1, app.getComponents().size());
-    assertNotNull(app.getComponent("comp2"));
-  }
-
-  @Test
-  public void testDuplicateComponents() throws IOException {
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-
-    String compName = "comp1";
-    Service app = createValidApplication(compName);
-    app.addComponent(createValidComponent(compName));
-
-    // duplicate component name fails
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with component collision");
-    } catch (IllegalArgumentException e) {
-      assertEquals("Component name collision: " + compName, e.getMessage());
-    }
-  }
-
-  @Test
-  public void testComponentNameSameAsServiceName() throws IOException {
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-    Service app = new Service();
-    app.setName("test");
-    app.setVersion("v1");
-    app.addComponent(createValidComponent("test"));
-
-    //component name same as service name
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "component name matches service name");
-    } catch (IllegalArgumentException e) {
-      assertEquals("Component name test must not be same as service name test",
-          e.getMessage());
-    }
-  }
-
-  @Test
-  public void testExternalDuplicateComponent() throws IOException {
-    Service ext = createValidApplication("comp1");
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs(ext);
-
-    Service app = createValidApplication("comp1");
-    Artifact artifact = new Artifact();
-    artifact.setType(Artifact.TypeEnum.SERVICE);
-    artifact.setId("id");
-    app.getComponent("comp1").setArtifact(artifact);
-
-    // duplicate component name okay in the case of SERVICE component
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-  }
-
-  @Test
-  public void testExternalComponent() throws IOException {
-    Service ext = createValidApplication("comp1");
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs(ext);
-
-    Service app = createValidApplication("comp2");
-    Artifact artifact = new Artifact();
-    artifact.setType(Artifact.TypeEnum.SERVICE);
-    artifact.setId("id");
-    app.setArtifact(artifact);
-
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-
-    assertEquals(1, app.getComponents().size());
-    // artifact ID not inherited from global
-    assertNotNull(app.getComponent("comp2"));
-
-    // set SERVICE artifact id on component
-    app.getComponent("comp2").setArtifact(artifact);
-
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-
-    assertEquals(1, app.getComponents().size());
-    // original component replaced by external component
-    assertNotNull(app.getComponent("comp1"));
-  }
-
-  public static void verifyDependencySorting(List<Component> components,
-      Component... expectedSorting) {
-    Collection<Component> actualSorting = ServiceApiUtil.sortByDependencies(
-        components);
-    assertEquals(expectedSorting.length, actualSorting.size());
-    int i = 0;
-    for (Component component : actualSorting) {
-      assertEquals(expectedSorting[i++], component);
-    }
-  }
-
-  @Test
-  public void testDependencySorting() throws IOException {
-    Component a = ServiceTestUtils.createComponent("a");
-    Component b = ServiceTestUtils.createComponent("b");
-    Component c = ServiceTestUtils.createComponent("c");
-    Component d =
-        ServiceTestUtils.createComponent("d").dependencies(Arrays.asList("c"));
-    Component e = ServiceTestUtils.createComponent("e")
-        .dependencies(Arrays.asList("b", "d"));
-
-    verifyDependencySorting(Arrays.asList(a, b, c), a, b, c);
-    verifyDependencySorting(Arrays.asList(c, a, b), c, a, b);
-    verifyDependencySorting(Arrays.asList(a, b, c, d, e), a, b, c, d, e);
-    verifyDependencySorting(Arrays.asList(e, d, c, b, a), c, b, a, d, e);
-
-    c.setDependencies(Arrays.asList("e"));
-    try {
-      verifyDependencySorting(Arrays.asList(a, b, c, d, e));
-      Assert.fail(EXCEPTION_PREFIX + "components with dependency cycle");
-    } catch (IllegalArgumentException ex) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_DEPENDENCY_CYCLE, Arrays.asList(c, d,
-              e)), ex.getMessage());
-    }
-
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-    Service service = createValidApplication(null);
-    service.setComponents(Arrays.asList(c, d, e));
-    try {
-      ServiceApiUtil.validateAndResolveService(service, sfs,
-          CONF_DEFAULT_DNS);
-      Assert.fail(EXCEPTION_PREFIX + "components with bad dependencies");
-    } catch (IllegalArgumentException ex) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_DEPENDENCY_INVALID, "b", "e"), ex
-          .getMessage());
-    }
-  }
-
-  @Test
-  public void testInvalidComponent() throws IOException {
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-    testComponent(sfs);
-  }
-
-  @Test
-  public void testValidateCompName() {
-    String[] invalidNames = {
-        "EXAMPLE", // UPPER case not allowed
-        "example_app" // underscore not allowed.
-    };
-    for (String name : invalidNames) {
-      try {
-        ServiceApiUtil.validateNameFormat(name, new Configuration());
-        Assert.fail();
-      } catch (IllegalArgumentException ex) {
-        ex.printStackTrace();
-      }
-    }
-  }
-
-  private static void testComponent(SliderFileSystem sfs)
-      throws IOException {
-    int maxLen = RegistryConstants.MAX_FQDN_LABEL_LENGTH;
-    assertEquals(19, Long.toString(Long.MAX_VALUE).length());
-    maxLen = maxLen - Long.toString(Long.MAX_VALUE).length();
-
-    String compName = LEN_64_STR.substring(0, maxLen + 1);
-    Service app = createValidApplication(null);
-    app.addComponent(createValidComponent(compName));
-
-    // invalid component name fails if dns is enabled
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with invalid component name");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(RestApiErrorMessages
-          .ERROR_COMPONENT_NAME_INVALID, maxLen, compName), e.getMessage());
-    }
-
-    // does not fail if dns is disabled
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DEFAULT_DNS);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-
-    compName = LEN_64_STR.substring(0, maxLen);
-    app = createValidApplication(null);
-    app.addComponent(createValidComponent(compName));
-
-    // does not fail
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-  }
-
-  @Test
-  public void testPlacementPolicy() throws IOException {
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-    Service app = createValidApplication("comp-a");
-    Component comp = app.getComponents().get(0);
-    PlacementPolicy pp = new PlacementPolicy();
-    PlacementConstraint pc = new PlacementConstraint();
-    pc.setName("CA1");
-    pp.setConstraints(Collections.singletonList(pc));
-    comp.setPlacementPolicy(pp);
-
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "constraint with no type");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_TYPE_NULL,
-          "CA1 ", "comp-a"), e.getMessage());
-    }
-
-    // Set the type
-    pc.setType(PlacementType.ANTI_AFFINITY);
-
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "constraint with no scope");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_SCOPE_NULL,
-          "CA1 ", "comp-a"), e.getMessage());
-    }
-
-    // Set the scope
-    pc.setScope(PlacementScope.NODE);
-
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "constraint with no tag(s)");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_TAGS_NULL,
-          "CA1 ", "comp-a"), e.getMessage());
-    }
-
-    // Set a target tag - but an invalid one
-    pc.setTargetTags(Collections.singletonList("comp-invalid"));
-
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "constraint with invalid tag name");
-    } catch (IllegalArgumentException e) {
-      assertEquals(
-          String.format(
-              RestApiErrorMessages.ERROR_PLACEMENT_POLICY_TAG_NAME_NOT_SAME,
-              "comp-invalid", "comp-a", "comp-a", "comp-a"),
-          e.getMessage());
-    }
-
-    // Set valid target tags now
-    pc.setTargetTags(Collections.singletonList("comp-a"));
-
-    // Finally it should succeed
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-  }
-
-  @Test
-  public void testKerberosPrincipal() throws IOException {
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-    Service app = createValidApplication("comp-a");
-    KerberosPrincipal kp = new KerberosPrincipal();
-    kp.setKeytab("/some/path");
-    kp.setPrincipalName("user/_HOST@domain.com");
-    app.setKerberosPrincipal(kp);
-
-    try {
-      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
-      Assert.fail(EXCEPTION_PREFIX + "service with invalid keytab URI scheme");
-    } catch (IllegalArgumentException e) {
-      assertEquals(
-          String.format(RestApiErrorMessages.ERROR_KEYTAB_URI_SCHEME_INVALID,
-              kp.getKeytab()),
-          e.getMessage());
-    }
-
-    kp.setKeytab("/ blank / in / paths");
-    try {
-      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
-      Assert.fail(EXCEPTION_PREFIX + "service with invalid keytab");
-    } catch (IllegalArgumentException e) {
-      // strip out the %s at the end of the RestApiErrorMessages string constant
-      assertTrue(e.getMessage().contains(
-          RestApiErrorMessages.ERROR_KEYTAB_URI_INVALID.substring(0,
-              RestApiErrorMessages.ERROR_KEYTAB_URI_INVALID.length() - 2)));
-    }
-
-    kp.setKeytab("file:///tmp/a.keytab");
-    // now it should succeed
-    try {
-      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-  }
-
-  @Test
-  public void testKerberosPrincipalNameFormat() throws IOException {
-    Service app = createValidApplication("comp-a");
-    KerberosPrincipal kp = new KerberosPrincipal();
-    kp.setPrincipalName("user@domain.com");
-    app.setKerberosPrincipal(kp);
-
-    try {
-      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
-      Assert.fail(EXCEPTION_PREFIX + "service with invalid principal name format.");
-    } catch (IllegalArgumentException e) {
-      assertEquals(
-          String.format(RestApiErrorMessages.ERROR_KERBEROS_PRINCIPAL_NAME_FORMAT,
-              kp.getPrincipalName()),
-          e.getMessage());
-    }
-
-    kp.setPrincipalName("user/_HOST@domain.com");
-    try {
-      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceManager.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/TestServiceManager.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/TestServiceManager.java
index fc509f1..a37cabe 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/TestServiceManager.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/TestServiceManager.java
@@ -19,23 +19,26 @@
 package org.apache.hadoop.yarn.service;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.service.api.records.Artifact;
 import org.apache.hadoop.yarn.service.api.records.ComponentState;
+import org.apache.hadoop.yarn.service.api.records.ContainerState;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.api.records.ServiceState;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType;
 import org.apache.hadoop.yarn.service.exceptions.SliderException;
-import org.apache.hadoop.yarn.service.registry.YarnRegistryViewForProviders;
 import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.Map;
-
-import static org.mockito.Mockito.mock;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
 
 /**
  * Tests for {@link ServiceManager}.
@@ -46,117 +49,120 @@ public class TestServiceManager {
   public ServiceTestUtils.ServiceFSWatcher rule =
       new ServiceTestUtils.ServiceFSWatcher();
 
-  @Test
-  public void testUpgrade() throws IOException, SliderException {
-    ServiceManager serviceManager = createTestServiceManager("testUpgrade");
-    upgrade(serviceManager, "v2", false, false);
+  @Test (timeout = TIMEOUT)
+  public void testUpgrade() throws Exception {
+    ServiceContext context = createServiceContext("testUpgrade");
+    initUpgrade(context, "v2", false, false, false);
     Assert.assertEquals("service not upgraded", ServiceState.UPGRADING,
-        serviceManager.getServiceSpec().getState());
+        context.getServiceManager().getServiceSpec().getState());
   }
 
-  @Test
+  @Test (timeout = TIMEOUT)
   public void testRestartNothingToUpgrade()
-      throws IOException, SliderException {
-    ServiceManager serviceManager = createTestServiceManager(
+      throws Exception {
+    ServiceContext context = createServiceContext(
         "testRestartNothingToUpgrade");
-    upgrade(serviceManager, "v2", false, false);
-
-    //make components stable
-    serviceManager.getServiceSpec().getComponents().forEach(comp -> {
-      comp.setState(ComponentState.STABLE);
-    });
-    serviceManager.handle(new ServiceEvent(ServiceEventType.START));
+    initUpgrade(context, "v2", false, false, false);
+    ServiceManager manager = context.getServiceManager();
+    //make components stable by upgrading all instances
+    upgradeAllInstances(context);
+
+    context.scheduler.getDispatcher().getEventHandler().handle(
+        new ServiceEvent(ServiceEventType.START));
+    GenericTestUtils.waitFor(()->
+        context.service.getState().equals(ServiceState.STABLE),
+        CHECK_EVERY_MILLIS, TIMEOUT);
     Assert.assertEquals("service not re-started", ServiceState.STABLE,
-        serviceManager.getServiceSpec().getState());
+        manager.getServiceSpec().getState());
   }
 
-  @Test
-  public void testAutoFinalizeNothingToUpgrade() throws IOException,
-      SliderException {
-    ServiceManager serviceManager = createTestServiceManager(
+  @Test(timeout = TIMEOUT)
+  public void testAutoFinalizeNothingToUpgrade() throws Exception {
+    ServiceContext context = createServiceContext(
         "testAutoFinalizeNothingToUpgrade");
-    upgrade(serviceManager, "v2", false, true);
-
-    //make components stable
-    serviceManager.getServiceSpec().getComponents().forEach(comp ->
-        comp.setState(ComponentState.STABLE));
-    serviceManager.handle(new ServiceEvent(ServiceEventType.CHECK_STABLE));
+    initUpgrade(context, "v2", false, true, false);
+    ServiceManager manager = context.getServiceManager();
+    //make components stable by upgrading all instances
+    upgradeAllInstances(context);
+
+    GenericTestUtils.waitFor(()->
+        context.service.getState().equals(ServiceState.STABLE),
+        CHECK_EVERY_MILLIS, TIMEOUT);
     Assert.assertEquals("service stable", ServiceState.STABLE,
-        serviceManager.getServiceSpec().getState());
+        manager.getServiceSpec().getState());
   }
 
-  @Test
+  @Test(timeout = TIMEOUT)
   public void testRestartWithPendingUpgrade()
-      throws IOException, SliderException {
-    ServiceManager serviceManager = createTestServiceManager("testRestart");
-    upgrade(serviceManager, "v2", true, false);
-    serviceManager.handle(new ServiceEvent(ServiceEventType.START));
+      throws Exception {
+    ServiceContext context = createServiceContext("testRestart");
+    initUpgrade(context, "v2", true, false, false);
+    ServiceManager manager = context.getServiceManager();
+
+    context.scheduler.getDispatcher().getEventHandler().handle(
+        new ServiceEvent(ServiceEventType.START));
+    context.scheduler.getDispatcher().stop();
     Assert.assertEquals("service should still be upgrading",
-        ServiceState.UPGRADING, serviceManager.getServiceSpec().getState());
+        ServiceState.UPGRADING, manager.getServiceSpec().getState());
   }
 
-  @Test
-  public void testCheckState() throws IOException, SliderException {
-    ServiceManager serviceManager = createTestServiceManager(
-        "testCheckState");
-    upgrade(serviceManager, "v2", true, false);
+  @Test(timeout = TIMEOUT)
+  public void testFinalize() throws Exception {
+    ServiceContext context = createServiceContext("testCheckState");
+    initUpgrade(context, "v2", true, false, false);
+    ServiceManager manager = context.getServiceManager();
     Assert.assertEquals("service not upgrading", ServiceState.UPGRADING,
-        serviceManager.getServiceSpec().getState());
+        manager.getServiceSpec().getState());
 
-    // make components stable
-    serviceManager.getServiceSpec().getComponents().forEach(comp -> {
-      comp.setState(ComponentState.STABLE);
-    });
-    ServiceEvent checkStable = new ServiceEvent(ServiceEventType.CHECK_STABLE);
-    serviceManager.handle(checkStable);
-    Assert.assertEquals("service should still be upgrading",
-        ServiceState.UPGRADING, serviceManager.getServiceSpec().getState());
+    //make components stable by upgrading all instances
+    upgradeAllInstances(context);
 
     // finalize service
-    ServiceEvent restart = new ServiceEvent(ServiceEventType.START);
-    serviceManager.handle(restart);
-    Assert.assertEquals("service not stable",
-        ServiceState.STABLE, serviceManager.getServiceSpec().getState());
+    context.scheduler.getDispatcher().getEventHandler().handle(
+        new ServiceEvent(ServiceEventType.START));
+    GenericTestUtils.waitFor(()->
+        context.service.getState().equals(ServiceState.STABLE),
+        CHECK_EVERY_MILLIS, TIMEOUT);
+    Assert.assertEquals("service not re-started", ServiceState.STABLE,
+        manager.getServiceSpec().getState());
 
-    validateUpgradeFinalization(serviceManager.getName(), "v2");
+    validateUpgradeFinalization(manager.getName(), "v2");
   }
 
-  @Test
-  public void testCheckStateAutoFinalize() throws IOException, SliderException {
-    ServiceManager serviceManager = createTestServiceManager(
-        "testCheckState");
-    serviceManager.getServiceSpec().setState(
+  @Test(timeout = TIMEOUT)
+  public void testAutoFinalize() throws Exception {
+    ServiceContext context = createServiceContext("testCheckStateAutoFinalize");
+    ServiceManager manager = context.getServiceManager();
+    manager.getServiceSpec().setState(
         ServiceState.UPGRADING_AUTO_FINALIZE);
-    upgrade(serviceManager, "v2", true, true);
-    Assert.assertEquals("service not upgrading",
-        ServiceState.UPGRADING_AUTO_FINALIZE,
-        serviceManager.getServiceSpec().getState());
+    initUpgrade(context, "v2", true, true, false);
 
     // make components stable
-    serviceManager.getServiceSpec().getComponents().forEach(comp ->
-        comp.setState(ComponentState.STABLE));
-    ServiceEvent checkStable = new ServiceEvent(ServiceEventType.CHECK_STABLE);
-    serviceManager.handle(checkStable);
+    upgradeAllInstances(context);
+
+    GenericTestUtils.waitFor(() ->
+        context.service.getState().equals(ServiceState.STABLE),
+        CHECK_EVERY_MILLIS, TIMEOUT);
     Assert.assertEquals("service not stable",
-        ServiceState.STABLE, serviceManager.getServiceSpec().getState());
+        ServiceState.STABLE, manager.getServiceSpec().getState());
 
-    validateUpgradeFinalization(serviceManager.getName(), "v2");
+    validateUpgradeFinalization(manager.getName(), "v2");
   }
 
   @Test
-  public void testInvalidUpgrade() throws IOException, SliderException {
-    ServiceManager serviceManager = createTestServiceManager(
-        "testInvalidUpgrade");
-    serviceManager.getServiceSpec().setState(
+  public void testInvalidUpgrade() throws Exception {
+    ServiceContext serviceContext = createServiceContext("testInvalidUpgrade");
+    ServiceManager manager = serviceContext.getServiceManager();
+    manager.getServiceSpec().setState(
         ServiceState.UPGRADING_AUTO_FINALIZE);
     Service upgradedDef = ServiceTestUtils.createExampleApplication();
-    upgradedDef.setName(serviceManager.getName());
+    upgradedDef.setName(manager.getName());
     upgradedDef.setVersion("v2");
     upgradedDef.setLifetime(2L);
     writeUpgradedDef(upgradedDef);
 
     try {
-      serviceManager.processUpgradeRequest("v2", true);
+      manager.processUpgradeRequest("v2", true, false);
     } catch (Exception ex) {
       Assert.assertTrue(ex instanceof UnsupportedOperationException);
       return;
@@ -164,6 +170,32 @@ public class TestServiceManager {
     Assert.fail();
   }
 
+  @Test(timeout = TIMEOUT)
+  public void testExpressUpgrade() throws Exception {
+    ServiceContext context = createServiceContext("testExpressUpgrade");
+    ServiceManager manager = context.getServiceManager();
+    manager.getServiceSpec().setState(
+        ServiceState.EXPRESS_UPGRADING);
+    initUpgrade(context, "v2", true, true, true);
+
+    List<String> comps = ServiceApiUtil.resolveCompsDependency(context.service);
+    // wait till instances of first component are in upgrade
+    String comp1 = comps.get(0);
+    upgradeInstancesOf(context, comp1);
+
+    // wait till instances of second component are in upgrade
+    String comp2 = comps.get(1);
+    upgradeInstancesOf(context, comp2);
+
+    GenericTestUtils.waitFor(() ->
+            context.service.getState().equals(ServiceState.STABLE),
+        CHECK_EVERY_MILLIS, TIMEOUT);
+
+    Assert.assertEquals("service not stable",
+        ServiceState.STABLE, manager.getServiceSpec().getState());
+    validateUpgradeFinalization(manager.getName(), "v2");
+  }
+
   private void validateUpgradeFinalization(String serviceName,
       String expectedVersion) throws IOException {
     Service savedSpec = ServiceApiUtil.loadService(rule.getFs(), serviceName);
@@ -172,15 +204,16 @@ public class TestServiceManager {
     Assert.assertNotNull("app id not present", savedSpec.getId());
     Assert.assertEquals("state not stable", ServiceState.STABLE,
         savedSpec.getState());
-    savedSpec.getComponents().forEach(compSpec -> {
-      Assert.assertEquals("comp not stable", ComponentState.STABLE,
-          compSpec.getState());
-    });
+    savedSpec.getComponents().forEach(compSpec ->
+        Assert.assertEquals("comp not stable", ComponentState.STABLE,
+        compSpec.getState()));
   }
 
-  private void upgrade(ServiceManager serviceManager, String version,
-      boolean upgradeArtifact, boolean autoFinalize)
-      throws IOException, SliderException {
+  private void initUpgrade(ServiceContext context, String version,
+      boolean upgradeArtifact, boolean autoFinalize, boolean expressUpgrade)
+      throws IOException, SliderException, TimeoutException,
+      InterruptedException {
+    ServiceManager serviceManager = context.getServiceManager();
     Service upgradedDef = ServiceTestUtils.createExampleApplication();
     upgradedDef.setName(serviceManager.getName());
     upgradedDef.setVersion(version);
@@ -191,39 +224,81 @@ public class TestServiceManager {
       });
     }
     writeUpgradedDef(upgradedDef);
-    serviceManager.processUpgradeRequest(version, autoFinalize);
+    serviceManager.processUpgradeRequest(version, autoFinalize, expressUpgrade);
     ServiceEvent upgradeEvent = new ServiceEvent(ServiceEventType.UPGRADE);
-    upgradeEvent.setVersion(version);
-    if (autoFinalize) {
-      upgradeEvent.setAutoFinalize(true);
-    }
-    serviceManager.handle(upgradeEvent);
+    upgradeEvent.setVersion(version).setExpressUpgrade(expressUpgrade)
+        .setAutoFinalize(autoFinalize);
+
+    GenericTestUtils.waitFor(()-> {
+      ServiceState serviceState = context.service.getState();
+      if (serviceState.equals(ServiceState.UPGRADING) ||
+          serviceState.equals(ServiceState.UPGRADING_AUTO_FINALIZE) ||
+          serviceState.equals(ServiceState.EXPRESS_UPGRADING)) {
+        return true;
+      }
+      return false;
+    }, CHECK_EVERY_MILLIS, TIMEOUT);
+  }
+
+  private void upgradeAllInstances(ServiceContext context) throws
+      TimeoutException, InterruptedException {
+    // upgrade the instances
+    context.scheduler.getLiveInstances().forEach(((containerId, instance) -> {
+      ComponentInstanceEvent event = new ComponentInstanceEvent(containerId,
+          ComponentInstanceEventType.UPGRADE);
+      context.scheduler.getDispatcher().getEventHandler().handle(event);
+    }));
+
+    // become ready
+    context.scheduler.getLiveInstances().forEach(((containerId, instance) -> {
+      ComponentInstanceEvent event = new ComponentInstanceEvent(containerId,
+          ComponentInstanceEventType.BECOME_READY);
+
+      context.scheduler.getDispatcher().getEventHandler().handle(event);
+    }));
+    GenericTestUtils.waitFor(()-> {
+      for (ComponentInstance instance:
+          context.scheduler.getLiveInstances().values()) {
+        if (!instance.getContainerState().equals(ContainerState.READY)) {
+          return false;
+        }
+      }
+      return true;
+    }, CHECK_EVERY_MILLIS, TIMEOUT);
   }
 
-  private ServiceManager createTestServiceManager(String name)
-      throws IOException {
-    ServiceContext context = new ServiceContext();
-    context.service = createBaseDef(name);
-    context.fs = rule.getFs();
-
-    context.scheduler = new ServiceScheduler(context) {
-      @Override
-      protected YarnRegistryViewForProviders createYarnRegistryOperations(
-          ServiceContext context, RegistryOperations registryClient) {
-        return mock(YarnRegistryViewForProviders.class);
+  private void upgradeInstancesOf(ServiceContext context, String compName)
+      throws TimeoutException, InterruptedException {
+    Collection<ComponentInstance> compInstances = context.scheduler
+        .getAllComponents().get(compName).getAllComponentInstances();
+    GenericTestUtils.waitFor(() -> {
+      for (ComponentInstance instance : compInstances) {
+        if (!instance.getContainerState().equals(ContainerState.UPGRADING)) {
+          return false;
+        }
       }
-    };
+      return true;
+    }, CHECK_EVERY_MILLIS, TIMEOUT);
 
-    context.scheduler.init(rule.getConf());
+    // instances of comp1 get upgraded and become ready event is triggered
+    // become ready
+    compInstances.forEach(instance -> {
+      ComponentInstanceEvent event = new ComponentInstanceEvent(
+          instance.getContainer().getId(),
+          ComponentInstanceEventType.BECOME_READY);
 
-    Map<String, org.apache.hadoop.yarn.service.component.Component>
-        componentState = context.scheduler.getAllComponents();
-    context.service.getComponents().forEach(component -> {
-      componentState.put(component.getName(),
-          new org.apache.hadoop.yarn.service.component.Component(component,
-              1L, context));
+      context.scheduler.getDispatcher().getEventHandler().handle(event);
     });
-    return new ServiceManager(context);
+  }
+
+  private ServiceContext createServiceContext(String name)
+      throws Exception {
+    Service service  = createBaseDef(name);
+    ServiceContext context = new MockRunningServiceContext(rule,
+        service);
+    context.scheduler.getDispatcher().setDrainEventsOnStop();
+    context.scheduler.getDispatcher().start();
+    return context;
   }
 
   public static Service createBaseDef(String name) {
@@ -257,4 +332,6 @@ public class TestServiceManager {
         upgradedDef);
   }
 
+  private static final int TIMEOUT = 200000;
+  private static final int CHECK_EVERY_MILLIS = 100;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.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/TestYarnNativeServices.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/TestYarnNativeServices.java
index 8b13b24..216d88f 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/TestYarnNativeServices.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/TestYarnNativeServices.java
@@ -415,6 +415,41 @@ public class TestYarnNativeServices extends ServiceTestUtils {
     client.actionDestroy(service.getName());
   }
 
+  @Test(timeout = 200000)
+  public void testExpressUpgrade() throws Exception {
+    setupInternal(NUM_NMS);
+    getConf().setBoolean(YARN_SERVICE_UPGRADE_ENABLED, true);
+    ServiceClient client = createClient(getConf());
+
+    Service service = createExampleApplication();
+    client.actionCreate(service);
+    waitForServiceToBeStable(client, service);
+
+    // upgrade the service
+    Component component = service.getComponents().iterator().next();
+    service.setState(ServiceState.EXPRESS_UPGRADING);
+    service.setVersion("v2");
+    component.getConfiguration().getEnv().put("key1", "val1");
+    Component component2 = service.getComponent("compb");
+    component2.getConfiguration().getEnv().put("key2", "val2");
+    client.actionUpgradeExpress(service);
+
+    // wait for upgrade to complete
+    waitForServiceToBeStable(client, service);
+    Service active = client.getStatus(service.getName());
+    Assert.assertEquals("component not stable", ComponentState.STABLE,
+        active.getComponent(component.getName()).getState());
+    Assert.assertEquals("compa does not have new env", "val1",
+        active.getComponent(component.getName()).getConfiguration()
+            .getEnv("key1"));
+    Assert.assertEquals("compb does not have new env", "val2",
+        active.getComponent(component2.getName()).getConfiguration()
+            .getEnv("key2"));
+    LOG.info("Stop/destroy service {}", service);
+    client.actionStop(service.getName(), true);
+    client.actionDestroy(service.getName());
+  }
+
   // Test to verify ANTI_AFFINITY placement policy
   // 1. Start mini cluster with 3 NMs and scheduler placement-constraint handler
   // 2. Create an example service with 3 containers


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

Branch: refs/heads/YARN-3409
Commit: 4c25fa3d8ed4da8b8e6b013469fab4252f0c2547
Parents: e20f6b6
Author: Sunil G <su...@apache.org>
Authored: Tue Feb 27 08:15:42 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Sat Aug 25 21:10:56 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/4c25fa3d/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/4c25fa3d/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/4c25fa3d/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/4c25fa3d/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


[24/50] [abbrv] hadoop git commit: YARN-8632. Threads in SLS quit without logging exception. Contributed by Xianghao Lu.

Posted by su...@apache.org.
YARN-8632. Threads in SLS quit without logging exception. Contributed by Xianghao Lu.


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

Branch: refs/heads/YARN-3409
Commit: 8563fd67befeb0026a95ae268395baf04cd3c22f
Parents: 138b0c1
Author: Yufei Gu <yu...@apple.com>
Authored: Fri Aug 24 11:23:17 2018 -0700
Committer: Yufei Gu <yu...@apple.com>
Committed: Fri Aug 24 11:23:17 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java    | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8563fd67/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java
index b8bc8be..2957d23 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java
@@ -32,7 +32,6 @@ import java.util.List;
 import java.util.SortedMap;
 import java.util.Locale;
 import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.Lock;
@@ -48,6 +47,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.concurrent.HadoopScheduledThreadPoolExecutor;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
@@ -169,7 +169,7 @@ public abstract class SchedulerMetrics {
     web.start();
 
     // a thread to update histogram timer
-    pool = new ScheduledThreadPoolExecutor(2);
+    pool = new HadoopScheduledThreadPoolExecutor(2);
     pool.scheduleAtFixedRate(new HistogramsRunnable(), 0, 1000,
         TimeUnit.MILLISECONDS);
 
@@ -518,7 +518,8 @@ public abstract class SchedulerMetrics {
 
     @Override
     public void run() {
-      if(running) {
+      SchedulerWrapper wrapper = (SchedulerWrapper) scheduler;
+      if(running && wrapper.getTracker().getQueueSet() != null) {
         // all WebApp to get real tracking json
         String trackingMetrics = web.generateRealTimeTrackingMetrics();
         // output


---------------------------------------------------------------------
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: HADOOP-15679. ShutdownHookManager shutdown time needs to be configurable & extended. Contributed by Steve Loughran.

Posted by su...@apache.org.
HADOOP-15679. ShutdownHookManager shutdown time needs to be configurable & extended.
Contributed by Steve Loughran.


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

Branch: refs/heads/YARN-3409
Commit: 34577d2c21096046861d2deefdbb2638b411c687
Parents: e0f6ffd
Author: Steve Loughran <st...@apache.org>
Authored: Mon Aug 20 18:36:24 2018 -0700
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Aug 20 18:36:24 2018 -0700

----------------------------------------------------------------------
 .../fs/CommonConfigurationKeysPublic.java       |   9 +
 .../apache/hadoop/util/ShutdownHookManager.java | 169 ++++++++--
 .../src/main/resources/core-default.xml         |  16 +
 .../hadoop/util/TestShutdownHookManager.java    | 328 +++++++++++++++----
 4 files changed, 418 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/34577d2c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index c7f32f9..b101b3b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -905,5 +905,14 @@ public class CommonConfigurationKeysPublic {
 
   public static final String HADOOP_TAGS_SYSTEM = "hadoop.tags.system";
   public static final String HADOOP_TAGS_CUSTOM = "hadoop.tags.custom";
+
+  /** Configuration option for the shutdown hook manager shutdown time:
+   *  {@value}. */
+  public static final String SERVICE_SHUTDOWN_TIMEOUT =
+      "hadoop.service.shutdown.timeout";
+
+  /** Default shutdown hook timeout: {@value} seconds. */
+  public static final long SERVICE_SHUTDOWN_TIMEOUT_DEFAULT = 30;
+
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/34577d2c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java
index 153f92b..2ca8e55 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java
@@ -17,11 +17,17 @@
  */
 package org.apache.hadoop.util;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -34,6 +40,9 @@ import java.util.concurrent.TimeoutException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.SERVICE_SHUTDOWN_TIMEOUT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.SERVICE_SHUTDOWN_TIMEOUT_DEFAULT;
+
 /**
  * The <code>ShutdownHookManager</code> enables running shutdownHook
  * in a deterministic order, higher priority first.
@@ -42,53 +51,55 @@ import java.util.concurrent.atomic.AtomicBoolean;
  * This class registers a single JVM shutdownHook and run all the
  * shutdownHooks registered to it (to this class) in order based on their
  * priority.
+ *
+ * Unless a hook was registered with a shutdown explicitly set through
+ * {@link #addShutdownHook(Runnable, int, long, TimeUnit)},
+ * the shutdown time allocated to it is set by the configuration option
+ * {@link CommonConfigurationKeysPublic#SERVICE_SHUTDOWN_TIMEOUT} in
+ * {@code core-site.xml}, with a default value of
+ * {@link CommonConfigurationKeysPublic#SERVICE_SHUTDOWN_TIMEOUT_DEFAULT}
+ * seconds.
  */
-public class ShutdownHookManager {
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ShutdownHookManager {
 
   private static final ShutdownHookManager MGR = new ShutdownHookManager();
 
   private static final Logger LOG =
       LoggerFactory.getLogger(ShutdownHookManager.class);
-  private static final long TIMEOUT_DEFAULT = 10;
-  private static final TimeUnit TIME_UNIT_DEFAULT = TimeUnit.SECONDS;
+
+  /** Minimum shutdown timeout: {@value} second(s). */
+  public static final long TIMEOUT_MINIMUM = 1;
+
+  /** The default time unit used: seconds. */
+  public static final TimeUnit TIME_UNIT_DEFAULT = TimeUnit.SECONDS;
 
   private static final ExecutorService EXECUTOR =
       HadoopExecutors.newSingleThreadExecutor(new ThreadFactoryBuilder()
-          .setDaemon(true).build());
+          .setDaemon(true)
+          .setNameFormat("shutdown-hook-%01d")
+          .build());
+
   static {
     try {
       Runtime.getRuntime().addShutdownHook(
         new Thread() {
           @Override
           public void run() {
-            MGR.shutdownInProgress.set(true);
-            for (HookEntry entry: MGR.getShutdownHooksInOrder()) {
-              Future<?> future = EXECUTOR.submit(entry.getHook());
-              try {
-                future.get(entry.getTimeout(), entry.getTimeUnit());
-              } catch (TimeoutException ex) {
-                future.cancel(true);
-                LOG.warn("ShutdownHook '" + entry.getHook().getClass().
-                    getSimpleName() + "' timeout, " + ex.toString(), ex);
-              } catch (Throwable ex) {
-                LOG.warn("ShutdownHook '" + entry.getHook().getClass().
-                    getSimpleName() + "' failed, " + ex.toString(), ex);
-              }
-            }
-            try {
-              EXECUTOR.shutdown();
-              if (!EXECUTOR.awaitTermination(TIMEOUT_DEFAULT,
-                  TIME_UNIT_DEFAULT)) {
-                LOG.error("ShutdownHookManger shutdown forcefully.");
-                EXECUTOR.shutdownNow();
-              }
-              LOG.debug("ShutdownHookManger complete shutdown.");
-            } catch (InterruptedException ex) {
-              LOG.error("ShutdownHookManger interrupted while waiting for " +
-                  "termination.", ex);
-              EXECUTOR.shutdownNow();
-              Thread.currentThread().interrupt();
+            if (MGR.shutdownInProgress.getAndSet(true)) {
+              LOG.info("Shutdown process invoked a second time: ignoring");
+              return;
             }
+            long started = System.currentTimeMillis();
+            int timeoutCount = executeShutdown();
+            long ended = System.currentTimeMillis();
+            LOG.debug(String.format(
+                "Completed shutdown in %.3f seconds; Timeouts: %d",
+                (ended-started)/1000.0, timeoutCount));
+            // each of the hooks have executed; now shut down the
+            // executor itself.
+            shutdownExecutor(new Configuration());
           }
         }
       );
@@ -99,18 +110,92 @@ public class ShutdownHookManager {
   }
 
   /**
+   * Execute the shutdown.
+   * This is exposed purely for testing: do not invoke it.
+   * @return the number of shutdown hooks which timed out.
+   */
+  @InterfaceAudience.Private
+  @VisibleForTesting
+  static int executeShutdown() {
+    int timeouts = 0;
+    for (HookEntry entry: MGR.getShutdownHooksInOrder()) {
+      Future<?> future = EXECUTOR.submit(entry.getHook());
+      try {
+        future.get(entry.getTimeout(), entry.getTimeUnit());
+      } catch (TimeoutException ex) {
+        timeouts++;
+        future.cancel(true);
+        LOG.warn("ShutdownHook '" + entry.getHook().getClass().
+            getSimpleName() + "' timeout, " + ex.toString(), ex);
+      } catch (Throwable ex) {
+        LOG.warn("ShutdownHook '" + entry.getHook().getClass().
+            getSimpleName() + "' failed, " + ex.toString(), ex);
+      }
+    }
+    return timeouts;
+  }
+
+  /**
+   * Shutdown the executor thread itself.
+   * @param conf the configuration containing the shutdown timeout setting.
+   */
+  private static void shutdownExecutor(final Configuration conf) {
+    try {
+      EXECUTOR.shutdown();
+      long shutdownTimeout = getShutdownTimeout(conf);
+      if (!EXECUTOR.awaitTermination(
+          shutdownTimeout,
+          TIME_UNIT_DEFAULT)) {
+        // timeout waiting for the
+        LOG.error("ShutdownHookManger shutdown forcefully after"
+            + " {} seconds.", shutdownTimeout);
+        EXECUTOR.shutdownNow();
+      }
+      LOG.debug("ShutdownHookManger completed shutdown.");
+    } catch (InterruptedException ex) {
+      // interrupted.
+      LOG.error("ShutdownHookManger interrupted while waiting for " +
+          "termination.", ex);
+      EXECUTOR.shutdownNow();
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  /**
    * Return <code>ShutdownHookManager</code> singleton.
    *
    * @return <code>ShutdownHookManager</code> singleton.
    */
+  @InterfaceAudience.Public
   public static ShutdownHookManager get() {
     return MGR;
   }
 
   /**
+   * Get the shutdown timeout in seconds, from the supplied
+   * configuration.
+   * @param conf configuration to use.
+   * @return a timeout, always greater than or equal to {@link #TIMEOUT_MINIMUM}
+   */
+  @InterfaceAudience.Private
+  @VisibleForTesting
+  static long getShutdownTimeout(Configuration conf) {
+    long duration = conf.getTimeDuration(
+        SERVICE_SHUTDOWN_TIMEOUT,
+        SERVICE_SHUTDOWN_TIMEOUT_DEFAULT,
+        TIME_UNIT_DEFAULT);
+    if (duration < TIMEOUT_MINIMUM) {
+      duration = TIMEOUT_MINIMUM;
+    }
+    return duration;
+  }
+
+  /**
    * Private structure to store ShutdownHook, its priority and timeout
    * settings.
    */
+  @InterfaceAudience.Private
+  @VisibleForTesting
   static class HookEntry {
     private final Runnable hook;
     private final int priority;
@@ -118,7 +203,9 @@ public class ShutdownHookManager {
     private final TimeUnit unit;
 
     HookEntry(Runnable hook, int priority) {
-      this(hook, priority, TIMEOUT_DEFAULT, TIME_UNIT_DEFAULT);
+      this(hook, priority,
+          getShutdownTimeout(new Configuration()),
+          TIME_UNIT_DEFAULT);
     }
 
     HookEntry(Runnable hook, int priority, long timeout, TimeUnit unit) {
@@ -176,10 +263,12 @@ public class ShutdownHookManager {
    *
    * @return the list of shutdownHooks in order of execution.
    */
+  @InterfaceAudience.Private
+  @VisibleForTesting
   List<HookEntry> getShutdownHooksInOrder() {
     List<HookEntry> list;
     synchronized (MGR.hooks) {
-      list = new ArrayList<HookEntry>(MGR.hooks);
+      list = new ArrayList<>(MGR.hooks);
     }
     Collections.sort(list, new Comparator<HookEntry>() {
 
@@ -200,6 +289,8 @@ public class ShutdownHookManager {
    * @param shutdownHook shutdownHook <code>Runnable</code>
    * @param priority priority of the shutdownHook.
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Stable
   public void addShutdownHook(Runnable shutdownHook, int priority) {
     if (shutdownHook == null) {
       throw new IllegalArgumentException("shutdownHook cannot be NULL");
@@ -223,6 +314,8 @@ public class ShutdownHookManager {
    * @param timeout timeout of the shutdownHook
    * @param unit unit of the timeout <code>TimeUnit</code>
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Stable
   public void addShutdownHook(Runnable shutdownHook, int priority, long timeout,
       TimeUnit unit) {
     if (shutdownHook == null) {
@@ -242,6 +335,8 @@ public class ShutdownHookManager {
    * @return TRUE if the shutdownHook was registered and removed,
    * FALSE otherwise.
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Stable
   public boolean removeShutdownHook(Runnable shutdownHook) {
     if (shutdownInProgress.get()) {
       throw new IllegalStateException("Shutdown in progress, cannot remove a " +
@@ -256,6 +351,8 @@ public class ShutdownHookManager {
    * @param shutdownHook shutdownHook to check if registered.
    * @return TRUE/FALSE depending if the shutdownHook is is registered.
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Stable
   public boolean hasShutdownHook(Runnable shutdownHook) {
     return hooks.contains(new HookEntry(shutdownHook, 0));
   }
@@ -265,6 +362,8 @@ public class ShutdownHookManager {
    * 
    * @return TRUE if the shutdown is in progress, otherwise FALSE.
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Stable
   public boolean isShutdownInProgress() {
     return shutdownInProgress.get();
   }
@@ -272,7 +371,9 @@ public class ShutdownHookManager {
   /**
    * clear all registered shutdownHooks.
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Stable
   public void clearShutdownHooks() {
     hooks.clear();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/34577d2c/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 29c2bc2..7a0e1a8 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -553,6 +553,22 @@
     </description>
 </property>
 
+  <property>
+    <name>hadoop.service.shutdown.timeout</name>
+    <value>30s</value>
+    <description>
+      Timeout to wait for each shutdown operation to complete.
+      If a hook takes longer than this time to complete, it will be interrupted,
+      so the service will shutdown. This allows the service shutdown
+      to recover from a blocked operation.
+      Some shutdown hooks may need more time than this, for example when
+      a large amount of data needs to be uploaded to an object store.
+      In this situation: increase the timeout.
+
+      The minimum duration of the timeout is 1 second, "1s".
+    </description>
+</property>
+
 <property>
   <name>hadoop.rpc.protection</name>
   <value>authentication</value>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/34577d2c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShutdownHookManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShutdownHookManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShutdownHookManager.java
index d539823..03fa903 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShutdownHookManager.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShutdownHookManager.java
@@ -17,97 +17,285 @@
  */
 package org.apache.hadoop.util;
 
-import org.apache.commons.lang3.exception.ExceptionUtils;
-import org.slf4j.LoggerFactory;
-import org.junit.Assert;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.junit.After;
 import org.junit.Test;
 import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
 
 import static java.lang.Thread.sleep;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.SERVICE_SHUTDOWN_TIMEOUT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.SERVICE_SHUTDOWN_TIMEOUT_DEFAULT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 public class TestShutdownHookManager {
+
   static final Logger LOG =
       LoggerFactory.getLogger(TestShutdownHookManager.class.getName());
 
+  /**
+   * remove all the shutdown hooks so that they never get invoked later
+   * on in this test process.
+   */
+  @After
+  public void clearShutdownHooks() {
+    ShutdownHookManager.get().clearShutdownHooks();
+  }
+
+  /**
+   * Verify hook registration, then execute the hook callback stage
+   * of shutdown to verify invocation, execution order and timeout
+   * processing.
+   */
   @Test
   public void shutdownHookManager() {
     ShutdownHookManager mgr = ShutdownHookManager.get();
-    Assert.assertNotNull(mgr);
-    Assert.assertEquals(0, mgr.getShutdownHooksInOrder().size());
-    Runnable hook1 = new Runnable() {
-      @Override
-      public void run() {
-        LOG.info("Shutdown hook1 complete.");
-      }
-    };
-    Runnable hook2 = new Runnable() {
-      @Override
-      public void run() {
-        LOG.info("Shutdown hook2 complete.");
-      }
-    };
-
-    Runnable hook3 = new Runnable() {
-      @Override
-      public void run() {
-        try {
-          sleep(3000);
-          LOG.info("Shutdown hook3 complete.");
-        } catch (InterruptedException ex) {
-          LOG.info("Shutdown hook3 interrupted exception:",
-              ExceptionUtils.getStackTrace(ex));
-          Assert.fail("Hook 3 should not timeout.");
-        }
-      }
-    };
-
-    Runnable hook4 = new Runnable() {
-      @Override
-      public void run() {
-        try {
-          sleep(3500);
-          LOG.info("Shutdown hook4 complete.");
-          Assert.fail("Hook 4 should timeout");
-        } catch (InterruptedException ex) {
-          LOG.info("Shutdown hook4 interrupted exception:",
-              ExceptionUtils.getStackTrace(ex));
-        }
-      }
-    };
+    assertNotNull("No ShutdownHookManager", mgr);
+    assertEquals(0, mgr.getShutdownHooksInOrder().size());
+    Hook hook1 = new Hook("hook1", 0, false);
+    Hook hook2 = new Hook("hook2", 0, false);
+    Hook hook3 = new Hook("hook3", 1000, false);
+    Hook hook4 = new Hook("hook4", 25000, true);
+    Hook hook5 = new Hook("hook5",
+        (SERVICE_SHUTDOWN_TIMEOUT_DEFAULT + 1) * 1000, true);
 
     mgr.addShutdownHook(hook1, 0);
-    Assert.assertTrue(mgr.hasShutdownHook(hook1));
-    Assert.assertEquals(1, mgr.getShutdownHooksInOrder().size());
-    Assert.assertEquals(hook1, mgr.getShutdownHooksInOrder().get(0).getHook());
-    mgr.removeShutdownHook(hook1);
-    Assert.assertFalse(mgr.hasShutdownHook(hook1));
+    assertTrue(mgr.hasShutdownHook(hook1));
+    assertEquals(1, mgr.getShutdownHooksInOrder().size());
+    assertEquals(hook1, mgr.getShutdownHooksInOrder().get(0).getHook());
+    assertTrue(mgr.removeShutdownHook(hook1));
+    assertFalse(mgr.hasShutdownHook(hook1));
+    assertFalse(mgr.removeShutdownHook(hook1));
 
     mgr.addShutdownHook(hook1, 0);
-    Assert.assertTrue(mgr.hasShutdownHook(hook1));
-    Assert.assertEquals(1, mgr.getShutdownHooksInOrder().size());
-    Assert.assertTrue(mgr.hasShutdownHook(hook1));
-    Assert.assertEquals(1, mgr.getShutdownHooksInOrder().size());
+    assertTrue(mgr.hasShutdownHook(hook1));
+    assertEquals(1, mgr.getShutdownHooksInOrder().size());
+    assertEquals(SERVICE_SHUTDOWN_TIMEOUT_DEFAULT,
+        mgr.getShutdownHooksInOrder().get(0).getTimeout());
 
     mgr.addShutdownHook(hook2, 1);
-    Assert.assertTrue(mgr.hasShutdownHook(hook1));
-    Assert.assertTrue(mgr.hasShutdownHook(hook2));
-    Assert.assertEquals(2, mgr.getShutdownHooksInOrder().size());
-    Assert.assertEquals(hook2, mgr.getShutdownHooksInOrder().get(0).getHook());
-    Assert.assertEquals(hook1, mgr.getShutdownHooksInOrder().get(1).getHook());
+    assertTrue(mgr.hasShutdownHook(hook1));
+    assertTrue(mgr.hasShutdownHook(hook2));
+    assertEquals(2, mgr.getShutdownHooksInOrder().size());
+    assertEquals(hook2, mgr.getShutdownHooksInOrder().get(0).getHook());
+    assertEquals(hook1, mgr.getShutdownHooksInOrder().get(1).getHook());
 
     // Test hook finish without timeout
     mgr.addShutdownHook(hook3, 2, 4, TimeUnit.SECONDS);
-    Assert.assertTrue(mgr.hasShutdownHook(hook3));
-    Assert.assertEquals(hook3, mgr.getShutdownHooksInOrder().get(0).getHook());
-    Assert.assertEquals(4, mgr.getShutdownHooksInOrder().get(0).getTimeout());
-
-    // Test hook finish with timeout
-    mgr.addShutdownHook(hook4, 3, 2, TimeUnit.SECONDS);
-    Assert.assertTrue(mgr.hasShutdownHook(hook4));
-    Assert.assertEquals(hook4, mgr.getShutdownHooksInOrder().get(0).getHook());
-    Assert.assertEquals(2, mgr.getShutdownHooksInOrder().get(0).getTimeout());
-    LOG.info("Shutdown starts here");
+    assertTrue(mgr.hasShutdownHook(hook3));
+    assertEquals(hook3, mgr.getShutdownHooksInOrder().get(0).getHook());
+    assertEquals(4, mgr.getShutdownHooksInOrder().get(0).getTimeout());
+
+    // Test hook finish with timeout; highest priority
+    int hook4timeout = 2;
+    mgr.addShutdownHook(hook4, 3, hook4timeout, TimeUnit.SECONDS);
+    assertTrue(mgr.hasShutdownHook(hook4));
+    assertEquals(hook4, mgr.getShutdownHooksInOrder().get(0).getHook());
+    assertEquals(2, mgr.getShutdownHooksInOrder().get(0).getTimeout());
+
+    // a default timeout hook and verify it gets the default timeout
+    mgr.addShutdownHook(hook5, 5);
+    ShutdownHookManager.HookEntry hookEntry5 = mgr.getShutdownHooksInOrder()
+        .get(0);
+    assertEquals(hook5, hookEntry5.getHook());
+    assertEquals("default timeout not used",
+        ShutdownHookManager.getShutdownTimeout(new Configuration()),
+        hookEntry5.getTimeout());
+    assertEquals("hook priority", 5, hookEntry5.getPriority());
+    // remove this to avoid a longer sleep in the test run
+    assertTrue("failed to remove " + hook5,
+        mgr.removeShutdownHook(hook5));
+
+
+    // now execute the hook shutdown sequence
+    INVOCATION_COUNT.set(0);
+    LOG.info("invoking executeShutdown()");
+    int timeouts = ShutdownHookManager.executeShutdown();
+    LOG.info("Shutdown completed");
+    assertEquals("Number of timed out hooks", 1, timeouts);
+
+    List<ShutdownHookManager.HookEntry> hooks
+        = mgr.getShutdownHooksInOrder();
+
+    // analyze the hooks
+    for (ShutdownHookManager.HookEntry entry : hooks) {
+      Hook hook = (Hook) entry.getHook();
+      assertTrue("Was not invoked " + hook, hook.invoked);
+      // did any hook raise an exception?
+      hook.maybeThrowAssertion();
+    }
+
+    // check the state of some of the invoked hooks
+    // hook4 was invoked first, but it timed out.
+    assertEquals("Expected to be invoked first " + hook4,
+        1, hook4.invokedOrder);
+    assertFalse("Expected to time out " + hook4, hook4.completed);
+
+
+    // hook1 completed, but in order after the others, so its start time
+    // is the longest.
+    assertTrue("Expected to complete " + hook1, hook1.completed);
+    long invocationInterval = hook1.startTime - hook4.startTime;
+    assertTrue("invocation difference too short " + invocationInterval,
+        invocationInterval >= hook4timeout * 1000);
+    assertTrue("sleeping hook4 blocked other threads for " + invocationInterval,
+        invocationInterval < hook4.sleepTime);
+
+    // finally, clear the hooks
+    mgr.clearShutdownHooks();
+    // and verify that the hooks are empty
+    assertFalse(mgr.hasShutdownHook(hook1));
+    assertEquals("shutdown hook list is not empty",
+        0,
+        mgr.getShutdownHooksInOrder().size());
+  }
+
+  @Test
+  public void testShutdownTimeoutConfiguration() throws Throwable {
+    // set the shutdown timeout and verify it can be read back.
+    Configuration conf = new Configuration();
+    long shutdownTimeout = 5;
+    conf.setTimeDuration(SERVICE_SHUTDOWN_TIMEOUT,
+        shutdownTimeout, TimeUnit.SECONDS);
+    assertEquals(SERVICE_SHUTDOWN_TIMEOUT,
+        shutdownTimeout,
+        ShutdownHookManager.getShutdownTimeout(conf));
+  }
+
+  /**
+   * Verify that low timeouts simply fall back to
+   * {@link ShutdownHookManager#TIMEOUT_MINIMUM}.
+   */
+  @Test
+  public void testShutdownTimeoutBadConfiguration() throws Throwable {
+    // set the shutdown timeout and verify it can be read back.
+    Configuration conf = new Configuration();
+    long shutdownTimeout = 50;
+    conf.setTimeDuration(SERVICE_SHUTDOWN_TIMEOUT,
+        shutdownTimeout, TimeUnit.NANOSECONDS);
+    assertEquals(SERVICE_SHUTDOWN_TIMEOUT,
+        ShutdownHookManager.TIMEOUT_MINIMUM,
+        ShutdownHookManager.getShutdownTimeout(conf));
+  }
+
+  /**
+   * Verifies that a hook cannot be re-registered: an attempt to do so
+   * will simply be ignored.
+   */
+  @Test
+  public void testDuplicateRegistration() throws Throwable {
+    ShutdownHookManager mgr = ShutdownHookManager.get();
+    Hook hook = new Hook("hook1", 0, false);
+
+    // add the hook
+    mgr.addShutdownHook(hook, 2, 1, TimeUnit.SECONDS);
+
+    // add it at a higher priority. This will be ignored.
+    mgr.addShutdownHook(hook, 5);
+    List<ShutdownHookManager.HookEntry> hookList
+        = mgr.getShutdownHooksInOrder();
+    assertEquals("Hook added twice", 1, hookList.size());
+    ShutdownHookManager.HookEntry entry = hookList.get(0);
+    assertEquals("priority of hook", 2, entry.getPriority());
+    assertEquals("timeout of hook", 1, entry.getTimeout());
+
+    // remove the hook
+    assertTrue("failed to remove hook " + hook, mgr.removeShutdownHook(hook));
+    // which will fail a second time
+    assertFalse("expected hook removal to fail", mgr.removeShutdownHook(hook));
+
+    // now register it
+    mgr.addShutdownHook(hook, 5);
+    hookList = mgr.getShutdownHooksInOrder();
+    entry = hookList.get(0);
+    assertEquals("priority of hook", 5, entry.getPriority());
+    assertNotEquals("timeout of hook", 1, entry.getTimeout());
+
+  }
+
+  private static final AtomicInteger INVOCATION_COUNT = new AtomicInteger();
+
+  /**
+   * Hooks for testing; save state for ease of asserting on
+   * invocation.
+   */
+  private class Hook implements Runnable {
+
+    private final String name;
+    private final long sleepTime;
+    private final boolean expectFailure;
+    private AssertionError assertion;
+    private boolean invoked;
+    private int invokedOrder;
+    private boolean completed;
+    private boolean interrupted;
+    private long startTime;
+
+    Hook(final String name,
+        final long sleepTime,
+        final boolean expectFailure) {
+      this.name = name;
+      this.sleepTime = sleepTime;
+      this.expectFailure = expectFailure;
+    }
+
+    @Override
+    public void run() {
+      try {
+        invoked = true;
+        invokedOrder = INVOCATION_COUNT.incrementAndGet();
+        startTime = System.currentTimeMillis();
+        LOG.info("Starting shutdown of {} with sleep time of {}",
+            name, sleepTime);
+        if (sleepTime > 0) {
+          sleep(sleepTime);
+        }
+        LOG.info("Completed shutdown of {}", name);
+        completed = true;
+        if (expectFailure) {
+          assertion = new AssertionError("Expected a failure of " + name);
+        }
+      } catch (InterruptedException ex) {
+        LOG.info("Shutdown {} interrupted exception", name, ex);
+        interrupted = true;
+        if (!expectFailure) {
+          assertion = new AssertionError("Timeout of " + name, ex);
+        }
+      }
+      maybeThrowAssertion();
+    }
+
+    /**
+     * Raise any exception generated during the shutdown process.
+     * @throws AssertionError any assertion from the shutdown.
+     */
+    void maybeThrowAssertion() throws AssertionError {
+      if (assertion != null) {
+        throw assertion;
+      }
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder("Hook{");
+      sb.append("name='").append(name).append('\'');
+      sb.append(", sleepTime=").append(sleepTime);
+      sb.append(", expectFailure=").append(expectFailure);
+      sb.append(", invoked=").append(invoked);
+      sb.append(", invokedOrder=").append(invokedOrder);
+      sb.append(", completed=").append(completed);
+      sb.append(", interrupted=").append(interrupted);
+      sb.append('}');
+      return sb.toString();
+    }
   }
 }


---------------------------------------------------------------------
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: HDFS-13821. RBF: Add dfs.federation.router.mount-table.cache.enable so that users can disable cache. Contributed by Fei Hui.

Posted by su...@apache.org.
HDFS-13821. RBF: Add dfs.federation.router.mount-table.cache.enable so that users can disable cache. Contributed by Fei Hui.


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

Branch: refs/heads/YARN-3409
Commit: 81847392badcd58d934333e7c3b5bf14b4fa1f3f
Parents: e557c6b
Author: Yiqun Lin <yq...@apache.org>
Authored: Wed Aug 22 11:43:40 2018 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Wed Aug 22 11:43:40 2018 +0800

----------------------------------------------------------------------
 .../federation/resolver/MountTableResolver.java | 37 ++++++++++++++------
 .../server/federation/router/RBFConfigKeys.java |  4 +++
 .../src/main/resources/hdfs-rbf-default.xml     |  9 +++++
 .../resolver/TestMountTableResolver.java        | 31 ++++++++++++++++
 4 files changed, 71 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/81847392/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
index c264de3..d45441f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
@@ -22,6 +22,8 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DeprecatedKeys.
 import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE;
 import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE;
 import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.FEDERATION_MOUNT_TABLE_CACHE_ENABLE;
+import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.FEDERATION_MOUNT_TABLE_CACHE_ENABLE_DEFAULT;
 import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.isParentEntry;
 
 import java.io.IOException;
@@ -124,12 +126,19 @@ public class MountTableResolver
       this.stateStore = null;
     }
 
-    int maxCacheSize = conf.getInt(
-        FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE,
-        FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE_DEFAULT);
-    this.locationCache = CacheBuilder.newBuilder()
-        .maximumSize(maxCacheSize)
-        .build();
+    boolean mountTableCacheEnable = conf.getBoolean(
+        FEDERATION_MOUNT_TABLE_CACHE_ENABLE,
+        FEDERATION_MOUNT_TABLE_CACHE_ENABLE_DEFAULT);
+    if (mountTableCacheEnable) {
+      int maxCacheSize = conf.getInt(
+          FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE,
+          FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE_DEFAULT);
+      this.locationCache = CacheBuilder.newBuilder()
+          .maximumSize(maxCacheSize)
+          .build();
+    } else {
+      this.locationCache = null;
+    }
 
     registerCacheExternal();
     initDefaultNameService(conf);
@@ -239,7 +248,7 @@ public class MountTableResolver
    */
   private void invalidateLocationCache(final String path) {
     LOG.debug("Invalidating {} from {}", path, locationCache);
-    if (locationCache.size() == 0) {
+    if (locationCache == null || locationCache.size() == 0) {
       return;
     }
 
@@ -359,7 +368,9 @@ public class MountTableResolver
     LOG.info("Clearing all mount location caches");
     writeLock.lock();
     try {
-      this.locationCache.invalidateAll();
+      if (this.locationCache != null) {
+        this.locationCache.invalidateAll();
+      }
       this.tree.clear();
     } finally {
       writeLock.unlock();
@@ -372,6 +383,9 @@ public class MountTableResolver
     verifyMountTable();
     readLock.lock();
     try {
+      if (this.locationCache == null) {
+        return lookupLocation(path);
+      }
       Callable<? extends PathLocation> meh = new Callable<PathLocation>() {
         @Override
         public PathLocation call() throws Exception {
@@ -603,7 +617,10 @@ public class MountTableResolver
    * Get the size of the cache.
    * @return Size of the cache.
    */
-  protected long getCacheSize() {
-    return this.locationCache.size();
+  protected long getCacheSize() throws IOException{
+    if (this.locationCache != null) {
+      return this.locationCache.size();
+    }
+    throw new IOException("localCache is null");
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81847392/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
index 363db20..87df5d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
@@ -186,6 +186,10 @@ public class RBFConfigKeys extends CommonConfigurationKeysPublic {
       FEDERATION_ROUTER_PREFIX + "mount-table.max-cache-size";
   /** Remove cache entries if we have more than 10k. */
   public static final int FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE_DEFAULT = 10000;
+  public static final String FEDERATION_MOUNT_TABLE_CACHE_ENABLE =
+      FEDERATION_ROUTER_PREFIX + "mount-table.cache.enable";
+  public static final boolean FEDERATION_MOUNT_TABLE_CACHE_ENABLE_DEFAULT =
+      true;
 
   // HDFS Router-based federation admin
   public static final String DFS_ROUTER_ADMIN_HANDLER_COUNT_KEY =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81847392/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
index 8806cb2..8be5b8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
@@ -394,6 +394,15 @@
   </property>
 
   <property>
+    <name>dfs.federation.router.mount-table.cache.enable</name>
+    <value>true</value>
+    <description>
+      Set to true to enable mount table cache (Path to Remote Location cache).
+      Disabling the cache is recommended when a large amount of unique paths are queried.
+    </description>
+  </property>
+
+  <property>
     <name>dfs.federation.router.quota.enable</name>
     <value>false</value>
     <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81847392/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java
index cb3b472..b19a973 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.federation.resolver;
 
+import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.FEDERATION_MOUNT_TABLE_CACHE_ENABLE;
 import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE;
 import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE;
 import static org.junit.Assert.assertEquals;
@@ -37,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.federation.router.Router;
 import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -473,6 +475,35 @@ public class TestMountTableResolver {
   }
 
   @Test
+  public void testDisableLocalCache() throws IOException {
+    Configuration conf = new Configuration();
+    // Disable mount table cache
+    conf.setBoolean(FEDERATION_MOUNT_TABLE_CACHE_ENABLE, false);
+    conf.setStrings(DFS_ROUTER_DEFAULT_NAMESERVICE, "0");
+    MountTableResolver tmpMountTable = new MountTableResolver(conf);
+
+    // Root mount point
+    Map<String, String> map = getMountTableEntry("1", "/");
+    tmpMountTable.addEntry(MountTable.newInstance("/", map));
+
+    // /tmp
+    map = getMountTableEntry("2", "/tmp");
+    tmpMountTable.addEntry(MountTable.newInstance("/tmp", map));
+
+    // Check localCache is null
+    try {
+      tmpMountTable.getCacheSize();
+      fail("getCacheSize call should fail.");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("localCache is null", e);
+    }
+
+    // Check resolve path without cache
+    assertEquals("2->/tmp/tesfile1.txt",
+        tmpMountTable.getDestinationForPath("/tmp/tesfile1.txt").toString());
+  }
+
+  @Test
   public void testCacheCleaning() throws Exception {
     for (int i = 0; i < 1000; i++) {
       String filename = String.format("/user/a/file-%04d.txt", i);


---------------------------------------------------------------------
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-342. Add example byteman script to print out hadoop rpc traffic. Contributed by Elek, Marton.

Posted by su...@apache.org.
HDDS-342. Add example byteman script to print out hadoop rpc traffic.
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/af4b705b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/af4b705b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/af4b705b

Branch: refs/heads/YARN-3409
Commit: af4b705b5f73b177be24292d8dda3a150aa12596
Parents: 4c25f37
Author: Anu Engineer <ae...@apache.org>
Authored: Wed Aug 22 14:48:22 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Wed Aug 22 14:48:22 2018 -0700

----------------------------------------------------------------------
 dev-support/byteman/README.md                   | 31 ++++++++++++++
 dev-support/byteman/hadooprpc.btm               | 44 ++++++++++++++++++++
 .../src/main/compose/ozone/docker-config        |  2 +
 3 files changed, 77 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/af4b705b/dev-support/byteman/README.md
----------------------------------------------------------------------
diff --git a/dev-support/byteman/README.md b/dev-support/byteman/README.md
new file mode 100644
index 0000000..9a17fc5
--- /dev/null
+++ b/dev-support/byteman/README.md
@@ -0,0 +1,31 @@
+<!--
+  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.
+-->
+
+This folder contains example byteman scripts (http://byteman.jboss.org/) to help 
+Hadoop debuging.
+
+As the startup script of the hadoop-runner docker image supports byteman 
+instrumentation it's enough to set the URL of a script to a specific environment
+variable to activate it with the docker runs:
+
+
+```
+BYTEMAN_SCRIPT_URL=https://raw.githubusercontent.com/apache/hadoop/trunk/dev-support/byteman/hadooprpc.btm
+```
+
+For more info see HADOOP-15656 and HDDS-342
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af4b705b/dev-support/byteman/hadooprpc.btm
----------------------------------------------------------------------
diff --git a/dev-support/byteman/hadooprpc.btm b/dev-support/byteman/hadooprpc.btm
new file mode 100644
index 0000000..13894fe
--- /dev/null
+++ b/dev-support/byteman/hadooprpc.btm
@@ -0,0 +1,44 @@
+# 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.
+
+#
+# This script instruments hadoop rpc layer to print out all the request/response messages to the standard output.
+#
+
+RULE Hadoop RPC request
+INTERFACE ^com.google.protobuf.BlockingService
+METHOD callBlockingMethod
+IF true
+DO traceln("--> RPC message request: " + $3.getClass().getSimpleName() + " from " + linked(Thread.currentThread(), "source")); 
+   traceln($3.toString())
+ENDRULE
+
+
+RULE Hadoop RPC response
+INTERFACE ^com.google.protobuf.BlockingService
+METHOD callBlockingMethod
+AT EXIT
+IF true
+DO traceln("--> RPC message response: " + $3.getClass().getSimpleName() + " to " + unlink(Thread.currentThread(), "source")); 
+   traceln($!.toString())
+ENDRULE
+
+
+RULE Hadoop RPC source IP
+CLASS org.apache.hadoop.ipc.Server$RpcCall
+METHOD run
+IF true
+DO link(Thread.currentThread(), "source", $0.connection.toString())
+ENDRULE

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af4b705b/hadoop-dist/src/main/compose/ozone/docker-config
----------------------------------------------------------------------
diff --git a/hadoop-dist/src/main/compose/ozone/docker-config b/hadoop-dist/src/main/compose/ozone/docker-config
index 1b75c01..a1828a3 100644
--- a/hadoop-dist/src/main/compose/ozone/docker-config
+++ b/hadoop-dist/src/main/compose/ozone/docker-config
@@ -29,3 +29,5 @@ LOG4J.PROPERTIES_log4j.rootLogger=INFO, stdout
 LOG4J.PROPERTIES_log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 LOG4J.PROPERTIES_log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 LOG4J.PROPERTIES_log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n
+#Enable this variable to print out all hadoop rpc traffic to the stdout. See http://byteman.jboss.org/ to define your own instrumentation.
+#BYTEMAN_SCRIPT_URL=https://raw.githubusercontent.com/apache/hadoop/trunk/dev-support/byteman/hadooprpc.btm


---------------------------------------------------------------------
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: HADOOP-14314. The OpenSolaris taxonomy link is dead in InterfaceClassification.md. Contributed by Rui Gao.

Posted by su...@apache.org.
HADOOP-14314. The OpenSolaris taxonomy link is dead in InterfaceClassification.md. Contributed by Rui Gao.


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

Branch: refs/heads/YARN-3409
Commit: bed8cb6979e0460141ed77e3b15d4f18db098a8e
Parents: 1ac0144
Author: Takanobu Asanuma <ta...@apache.org>
Authored: Thu Aug 23 21:54:38 2018 +0900
Committer: Takanobu Asanuma <ta...@apache.org>
Committed: Thu Aug 23 21:54:38 2018 +0900

----------------------------------------------------------------------
 .../src/site/markdown/InterfaceClassification.md              | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bed8cb69/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md b/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
index 451f9be..a21e28b 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
@@ -35,10 +35,9 @@ Interface Classification
 ------------------------
 
 Hadoop adopts the following interface classification,
-this classification was derived from the
-[OpenSolaris taxonomy](http://www.opensolaris.org/os/community/arc/policies/interface-taxonomy/#Advice)
-and, to some extent, from taxonomy used inside Yahoo.
-Interfaces have two main attributes: Audience and Stability
+this classification was derived from the OpenSolaris taxonomy and, to some extent,
+from taxonomy used inside Yahoo.
+Interfaces have two main attributes: Audience and Stability.
 
 ### Audience
 


---------------------------------------------------------------------
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-8015. Support all types of placement constraint support for Capacity Scheduler. Contributed by Weiwei Yang.

Posted by su...@apache.org.
YARN-8015. Support all types of placement constraint support for Capacity Scheduler. 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/1ac01444
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1ac01444
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1ac01444

Branch: refs/heads/YARN-3409
Commit: 1ac01444a24faee6f74f2e83d9521eb4e0be651b
Parents: b021249
Author: Sunil G <su...@apache.org>
Authored: Thu Aug 23 10:05:43 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Thu Aug 23 10:05:43 2018 +0530

----------------------------------------------------------------------
 .../SingleConstraintAppPlacementAllocator.java  | 175 ++------
 .../yarn/server/resourcemanager/MockRM.java     |  35 +-
 ...estSchedulingRequestContainerAllocation.java | 438 ++++++++++++++++++-
 ...stSingleConstraintAppPlacementAllocator.java |  78 ----
 4 files changed, 509 insertions(+), 217 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ac01444/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
index 914f35d..54e4666 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
@@ -19,18 +19,15 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableSet;
 import org.apache.commons.collections.IteratorUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceSizing;
 import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.impl.pb.SchedulingRequestPBImpl;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
-import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
 import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
@@ -48,12 +45,12 @@ import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression.TargetType.NODE_ATTRIBUTE;
 import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE_PARTITION;
 
 /**
@@ -70,7 +67,6 @@ public class SingleConstraintAppPlacementAllocator<N extends SchedulerNode>
 
   private SchedulingRequest schedulingRequest = null;
   private String targetNodePartition;
-  private Set<String> targetAllocationTags;
   private AllocationTagsManager allocationTagsManager;
   private PlacementConstraintManager placementConstraintManager;
 
@@ -239,135 +235,55 @@ public class SingleConstraintAppPlacementAllocator<N extends SchedulerNode>
           "Only GUARANTEED execution type is supported.");
     }
 
-    // Node partition
-    String nodePartition = null;
-    // Target allocation tags
-    Set<String> targetAllocationTags = null;
-
-    PlacementConstraint constraint =
-        newSchedulingRequest.getPlacementConstraint();
-
-    if (constraint != null) {
-      // We only accept SingleConstraint
-      PlacementConstraint.AbstractConstraint ac = constraint
-          .getConstraintExpr();
-      if (!(ac instanceof PlacementConstraint.SingleConstraint)) {
-        throwExceptionWithMetaInfo("Only accepts "
-            + PlacementConstraint.SingleConstraint.class.getName()
-                + " as constraint-expression. Rejecting the new added "
-            + "constraint-expression.class=" + ac.getClass().getName());
-      }
-
-      PlacementConstraint.SingleConstraint singleConstraint =
-          (PlacementConstraint.SingleConstraint) ac;
-
-      // Make sure it is an anti-affinity request (actually this implementation
-      // should be able to support both affinity / anti-affinity without much
-      // effort. Considering potential test effort required. Limit to
-      // anti-affinity to intra-app and scope is node.
-      if (!singleConstraint.getScope().equals(PlacementConstraints.NODE)) {
-        throwExceptionWithMetaInfo(
-            "Only support scope=" + PlacementConstraints.NODE
-                + "now. PlacementConstraint=" + singleConstraint);
-      }
-
-      if (singleConstraint.getMinCardinality() != 0
-          || singleConstraint.getMaxCardinality() != 0) {
-        throwExceptionWithMetaInfo(
-            "Only support anti-affinity, which is: minCardinality=0, "
-                + "maxCardinality=1");
-      }
-
-      Set<PlacementConstraint.TargetExpression> targetExpressionSet =
-          singleConstraint.getTargetExpressions();
-      if (targetExpressionSet == null || targetExpressionSet.isEmpty()) {
-        throwExceptionWithMetaInfo(
-            "TargetExpression should not be null or empty");
-      }
-
-      for (PlacementConstraint.TargetExpression targetExpression :
-          targetExpressionSet) {
-        // Handle node partition
-        if (targetExpression.getTargetType().equals(
-            PlacementConstraint.TargetExpression.TargetType.NODE_ATTRIBUTE)) {
-          // For node attribute target, we only support Partition now. And once
-          // YARN-3409 is merged, we will support node attribute.
-          if (!targetExpression.getTargetKey().equals(NODE_PARTITION)) {
-            throwExceptionWithMetaInfo("When TargetType="
-                + PlacementConstraint.TargetExpression.TargetType.NODE_ATTRIBUTE
-                + " only " + NODE_PARTITION + " is accepted as TargetKey.");
-          }
-
-          if (nodePartition != null) {
-            // This means we have duplicated node partition entry
-            // inside placement constraint, which might be set by mistake.
-            throwExceptionWithMetaInfo(
-                "Only one node partition targetExpression is allowed");
-          }
-
-          Set<String> values = targetExpression.getTargetValues();
-          if (values == null || values.isEmpty()) {
-            nodePartition = RMNodeLabelsManager.NO_LABEL;
-            continue;
-          }
+    this.targetNodePartition = validateAndGetTargetNodePartition(
+        newSchedulingRequest.getPlacementConstraint());
+    this.schedulingRequest = new SchedulingRequestPBImpl(
+        ((SchedulingRequestPBImpl) newSchedulingRequest).getProto());
 
-          if (values.size() > 1) {
-            throwExceptionWithMetaInfo("Inside one targetExpression, we only "
-                + "support affinity to at most one node partition now");
-          }
 
-          nodePartition = values.iterator().next();
-        } else if (targetExpression.getTargetType().equals(
-            PlacementConstraint.TargetExpression.TargetType.ALLOCATION_TAG)) {
-          // Handle allocation tags
-          if (targetAllocationTags != null) {
-            // This means we have duplicated AllocationTag expressions entries
-            // inside placement constraint, which might be set by mistake.
-            throwExceptionWithMetaInfo(
-                "Only one AllocationTag targetExpression is allowed");
-          }
+    LOG.info("Successfully added SchedulingRequest to app="
+        + appSchedulingInfo.getApplicationAttemptId()
+        + " placementConstraint=["
+        + schedulingRequest.getPlacementConstraint()
+        + "]. nodePartition=" + targetNodePartition);
+  }
 
-          if (targetExpression.getTargetValues() == null ||
-              targetExpression.getTargetValues().isEmpty()) {
-            throwExceptionWithMetaInfo("Failed to find allocation tags from "
-                + "TargetExpressions or couldn't find self-app target.");
+  // Tentatively find out potential exist node-partition in the placement
+  // constraint and set as the app's primary node-partition.
+  // Currently only single constraint is handled.
+  private String validateAndGetTargetNodePartition(
+      PlacementConstraint placementConstraint) {
+    String nodePartition = RMNodeLabelsManager.NO_LABEL;
+    if (placementConstraint != null &&
+        placementConstraint.getConstraintExpr() != null) {
+      PlacementConstraint.AbstractConstraint ac =
+          placementConstraint.getConstraintExpr();
+      if (ac != null && ac instanceof PlacementConstraint.SingleConstraint) {
+        PlacementConstraint.SingleConstraint singleConstraint =
+            (PlacementConstraint.SingleConstraint) ac;
+        for (PlacementConstraint.TargetExpression targetExpression :
+            singleConstraint.getTargetExpressions()) {
+          // Handle node partition
+          if (targetExpression.getTargetType().equals(NODE_ATTRIBUTE) &&
+              targetExpression.getTargetKey().equals(NODE_PARTITION)) {
+            Set<String> values = targetExpression.getTargetValues();
+            if (values == null || values.isEmpty()) {
+              continue;
+            }
+            if (values.size() > 1) {
+              throwExceptionWithMetaInfo(
+                  "Inside one targetExpression, we only support"
+                      + " affinity to at most one node partition now");
+            }
+            nodePartition = values.iterator().next();
+            if (nodePartition != null) {
+              break;
+            }
           }
-
-          targetAllocationTags = new HashSet<>(
-              targetExpression.getTargetValues());
         }
       }
-
-      if (targetAllocationTags == null) {
-        // That means we don't have ALLOCATION_TAG specified
-        throwExceptionWithMetaInfo(
-            "Couldn't find target expression with type == ALLOCATION_TAG,"
-                + " it is required to include one and only one target"
-                + " expression with type == ALLOCATION_TAG");
-      }
-    }
-
-    // If this scheduling request doesn't contain a placement constraint,
-    // we set allocation tags an empty set.
-    if (targetAllocationTags == null) {
-      targetAllocationTags = ImmutableSet.of();
-    }
-
-    if (nodePartition == null) {
-      nodePartition = RMNodeLabelsManager.NO_LABEL;
     }
-
-    // Validation is done. set local results:
-    this.targetNodePartition = nodePartition;
-    this.targetAllocationTags = targetAllocationTags;
-
-    this.schedulingRequest = new SchedulingRequestPBImpl(
-        ((SchedulingRequestPBImpl) newSchedulingRequest).getProto());
-
-    LOG.info("Successfully added SchedulingRequest to app=" + appSchedulingInfo
-        .getApplicationAttemptId() + " targetAllocationTags=[" + StringUtils
-        .join(",", targetAllocationTags) + "]. nodePartition="
-        + targetNodePartition);
+    return nodePartition;
   }
 
   @Override
@@ -515,11 +431,6 @@ public class SingleConstraintAppPlacementAllocator<N extends SchedulerNode>
     return targetNodePartition;
   }
 
-  @VisibleForTesting
-  Set<String> getTargetAllocationTags() {
-    return targetAllocationTags;
-  }
-
   @Override
   public void initialize(AppSchedulingInfo appSchedulingInfo,
       SchedulerRequestKey schedulerRequestKey, RMContext rmContext) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ac01444/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.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/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index eb4c626..2ad4391 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -513,6 +513,19 @@ public class MockRM extends ResourceManager {
     return submitApp(masterMemory, false);
   }
 
+  public RMApp submitApp(int masterMemory, Set<String> appTags)
+      throws Exception {
+    Resource resource = Resource.newInstance(masterMemory, 0);
+    ResourceRequest amResourceRequest = ResourceRequest.newInstance(
+        Priority.newInstance(0), ResourceRequest.ANY, resource, 1);
+    return submitApp(Collections.singletonList(amResourceRequest), "",
+        UserGroupInformation.getCurrentUser().getShortUserName(), null, false,
+        null, super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+        YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true,
+        false, false, null, 0, null, true, Priority.newInstance(0), null,
+        null, null, appTags);
+  }
+
   public RMApp submitApp(int masterMemory, Priority priority) throws Exception {
     Resource resource = Resource.newInstance(masterMemory, 0);
     return submitApp(resource, "", UserGroupInformation.getCurrentUser()
@@ -732,8 +745,23 @@ public class MockRM extends ResourceManager {
       LogAggregationContext logAggregationContext,
       boolean cancelTokensWhenComplete, Priority priority, String amLabel,
       Map<ApplicationTimeoutType, Long> applicationTimeouts,
-      ByteBuffer tokensConf)
-      throws Exception {
+      ByteBuffer tokensConf) throws Exception {
+    return submitApp(amResourceRequests, name, user, acls, unmanaged, queue,
+        maxAppAttempts, ts, appType, waitForAccepted, keepContainers,
+        isAppIdProvided, applicationId, attemptFailuresValidityInterval,
+        logAggregationContext, cancelTokensWhenComplete, priority, amLabel,
+        applicationTimeouts, tokensConf, null);
+  }
+
+  public RMApp submitApp(List<ResourceRequest> amResourceRequests, String name,
+      String user, Map<ApplicationAccessType, String> acls, boolean unmanaged,
+      String queue, int maxAppAttempts, Credentials ts, String appType,
+      boolean waitForAccepted, boolean keepContainers, boolean isAppIdProvided,
+      ApplicationId applicationId, long attemptFailuresValidityInterval,
+      LogAggregationContext logAggregationContext,
+      boolean cancelTokensWhenComplete, Priority priority, String amLabel,
+      Map<ApplicationTimeoutType, Long> applicationTimeouts,
+      ByteBuffer tokensConf, Set<String> applicationTags) throws Exception {
     ApplicationId appId = isAppIdProvided ? applicationId : null;
     ApplicationClientProtocol client = getClientRMService();
     if (! isAppIdProvided) {
@@ -749,6 +777,9 @@ public class MockRM extends ResourceManager {
     sub.setApplicationId(appId);
     sub.setApplicationName(name);
     sub.setMaxAppAttempts(maxAppAttempts);
+    if (applicationTags != null) {
+      sub.setApplicationTags(applicationTags);
+    }
     if (applicationTimeouts != null && applicationTimeouts.size() > 0) {
       sub.setApplicationTimeouts(applicationTimeouts);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ac01444/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestSchedulingRequestContainerAllocation.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/TestSchedulingRequestContainerAllocation.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/TestSchedulingRequestContainerAllocation.java
index f23fd8f..26c709f 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/TestSchedulingRequestContainerAllocation.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/TestSchedulingRequestContainerAllocation.java
@@ -26,12 +26,18 @@ import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
-import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.TargetApplicationsNamespace;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceSizing;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.AllocationTagNamespaceType;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
@@ -46,10 +52,24 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.concurrent.ConcurrentMap;
+
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTagWithNamespace;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.and;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.cardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetNotIn;
 
 public class TestSchedulingRequestContainerAllocation {
-  private final int GB = 1024;
+  private static final int GB = 1024;
 
   private YarnConfiguration conf;
 
@@ -435,8 +455,7 @@ public class TestSchedulingRequestContainerAllocation {
 
     CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
 
-    PlacementConstraint constraint = PlacementConstraints
-        .targetNotIn("node", allocationTag("t1"))
+    PlacementConstraint constraint = targetNotIn("node", allocationTag("t1"))
         .build();
     SchedulingRequest sc = SchedulingRequest
         .newInstance(0, Priority.newInstance(1),
@@ -477,4 +496,413 @@ public class TestSchedulingRequestContainerAllocation {
 
     rm1.close();
   }
+
+  private void doNodeHeartbeat(MockNM... nms) throws Exception {
+    for (MockNM nm : nms) {
+      nm.nodeHeartbeat(true);
+    }
+  }
+
+  private List<Container> waitForAllocation(int allocNum, int timeout,
+      MockAM am, MockNM... nms) throws Exception {
+    final List<Container> result = new ArrayList<>();
+    GenericTestUtils.waitFor(() -> {
+      try {
+        AllocateResponse response = am.schedule();
+        List<Container> allocated = response.getAllocatedContainers();
+        System.out.println("Expecting allocation: " + allocNum
+            + ", actual allocation: " + allocated.size());
+        for (Container c : allocated) {
+          System.out.println("Container " + c.getId().toString()
+              + " is allocated on node: " + c.getNodeId().toString()
+              + ", allocation tags: "
+              + String.join(",", c.getAllocationTags()));
+        }
+        result.addAll(allocated);
+        if (result.size() == allocNum) {
+          return true;
+        }
+        doNodeHeartbeat(nms);
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+      return false;
+    }, 500, timeout);
+    return result;
+  }
+
+  private static SchedulingRequest schedulingRequest(int requestId,
+      int containers, int cores, int mem, PlacementConstraint constraint,
+      String... tags) {
+    return schedulingRequest(1, requestId, containers, cores, mem,
+        ExecutionType.GUARANTEED, constraint, tags);
+  }
+
+  private static SchedulingRequest schedulingRequest(
+      int priority, long allocReqId, int containers, int cores, int mem,
+      ExecutionType execType, PlacementConstraint constraint, String... tags) {
+    return SchedulingRequest.newBuilder()
+        .priority(Priority.newInstance(priority))
+        .allocationRequestId(allocReqId)
+        .allocationTags(new HashSet<>(Arrays.asList(tags)))
+        .executionType(ExecutionTypeRequest.newInstance(execType, true))
+        .resourceSizing(
+            ResourceSizing.newInstance(containers,
+                Resource.newInstance(mem, cores)))
+        .placementConstraintExpression(constraint)
+        .build();
+  }
+
+  private int getContainerNodesNum(List<Container> containers) {
+    Set<NodeId> nodes = new HashSet<>();
+    if (containers != null) {
+      containers.forEach(c -> nodes.add(c.getNodeId()));
+    }
+    return nodes.size();
+  }
+
+  @Test(timeout = 30000L)
+  public void testInterAppCompositeConstraints() throws Exception {
+    // This test both intra and inter app constraints.
+    // Including simple affinity, anti-affinity, cardinality constraints,
+    // and simple AND composite constraints.
+    YarnConfiguration config = new YarnConfiguration();
+    config.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER,
+        YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER);
+    config.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    MockRM rm = new MockRM(config);
+    try {
+      rm.start();
+
+      MockNM nm1 = rm.registerNode("192.168.0.1:1234", 100*GB, 100);
+      MockNM nm2 = rm.registerNode("192.168.0.2:1234", 100*GB, 100);
+      MockNM nm3 = rm.registerNode("192.168.0.3:1234", 100*GB, 100);
+      MockNM nm4 = rm.registerNode("192.168.0.4:1234", 100*GB, 100);
+      MockNM nm5 = rm.registerNode("192.168.0.5:1234", 100*GB, 100);
+
+      RMApp app1 = rm.submitApp(1*GB, ImmutableSet.of("hbase"));
+      MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
+
+      // App1 (hbase)
+      // h1: hbase-master(1)
+      // h2: hbase-master(1)
+      // h3:
+      // h4:
+      // h5:
+      PlacementConstraint pc = targetNotIn("node",
+          allocationTag("hbase-master")).build();
+      am1.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 2, 1, 2048, pc, "hbase-master")));
+      List<Container> allocated = waitForAllocation(2, 3000, am1, nm1, nm2);
+
+      // 2 containers allocated
+      Assert.assertEquals(2, allocated.size());
+      // containers should be distributed on 2 different nodes
+      Assert.assertEquals(2, getContainerNodesNum(allocated));
+
+      // App1 (hbase)
+      // h1: hbase-rs(1), hbase-master(1)
+      // h2: hbase-rs(1), hbase-master(1)
+      // h3: hbase-rs(1)
+      // h4: hbase-rs(1)
+      // h5:
+      pc = targetNotIn("node", allocationTag("hbase-rs")).build();
+      am1.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(2, 4, 1, 1024, pc, "hbase-rs")));
+      allocated = waitForAllocation(4, 3000, am1, nm1, nm2, nm3, nm4, nm5);
+
+      Assert.assertEquals(4, allocated.size());
+      Assert.assertEquals(4, getContainerNodesNum(allocated));
+
+      // App2 (web-server)
+      // Web server instance has 2 instance and non of them can be co-allocated
+      // with hbase-master.
+      RMApp app2 = rm.submitApp(1*GB, ImmutableSet.of("web-server"));
+      MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
+
+      // App2 (web-server)
+      // h1: hbase-rs(1), hbase-master(1)
+      // h2: hbase-rs(1), hbase-master(1)
+      // h3: hbase-rs(1), ws-inst(1)
+      // h4: hbase-rs(1), ws-inst(1)
+      // h5:
+      pc = and(
+          targetIn("node", allocationTagWithNamespace(
+              new TargetApplicationsNamespace.All().toString(),
+              "hbase-master")),
+          targetNotIn("node", allocationTag("ws-inst"))).build();
+      am2.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 2, 1, 2048, pc, "ws-inst")));
+      allocated = waitForAllocation(2, 3000, am2, nm1, nm2, nm3, nm4, nm5);
+      Assert.assertEquals(2, allocated.size());
+      Assert.assertEquals(2, getContainerNodesNum(allocated));
+
+      ConcurrentMap<NodeId, RMNode> rmNodes = rm.getRMContext().getRMNodes();
+      for (Container c : allocated) {
+        RMNode rmNode = rmNodes.get(c.getNodeId());
+        Assert.assertNotNull(rmNode);
+        Assert.assertTrue("If ws-inst is allocated to a node,"
+                + " this node should have inherited the ws-inst tag ",
+            rmNode.getAllocationTagsWithCount().get("ws-inst") == 1);
+        Assert.assertTrue("ws-inst should be co-allocated to "
+                + "hbase-master nodes",
+            rmNode.getAllocationTagsWithCount().get("hbase-master") == 1);
+      }
+
+      // App3 (ws-servant)
+      // App3 has multiple instances that must be co-allocated
+      // with app2 server instance, and each node cannot have more than
+      // 3 instances.
+      RMApp app3 = rm.submitApp(1*GB, ImmutableSet.of("ws-servants"));
+      MockAM am3 = MockRM.launchAndRegisterAM(app3, rm, nm3);
+
+
+      // App3 (ws-servant)
+      // h1: hbase-rs(1), hbase-master(1)
+      // h2: hbase-rs(1), hbase-master(1)
+      // h3: hbase-rs(1), ws-inst(1), ws-servant(3)
+      // h4: hbase-rs(1), ws-inst(1), ws-servant(3)
+      // h5:
+      pc = and(
+          targetIn("node", allocationTagWithNamespace(
+              new TargetApplicationsNamespace.AppTag("web-server").toString(),
+              "ws-inst")),
+          cardinality("node", 0, 2, "ws-servant")).build();
+      am3.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 10, 1, 512, pc, "ws-servant")));
+      // total 6 containers can be allocated due to cardinality constraint
+      // each round, 2 containers can be allocated
+      allocated = waitForAllocation(6, 10000, am3, nm1, nm2, nm3, nm4, nm5);
+      Assert.assertEquals(6, allocated.size());
+      Assert.assertEquals(2, getContainerNodesNum(allocated));
+
+      for (Container c : allocated) {
+        RMNode rmNode = rmNodes.get(c.getNodeId());
+        Assert.assertNotNull(rmNode);
+        Assert.assertTrue("Node has ws-servant allocated must have 3 instances",
+            rmNode.getAllocationTagsWithCount().get("ws-servant") == 3);
+        Assert.assertTrue("Every ws-servant container should be co-allocated"
+                + " with ws-inst",
+            rmNode.getAllocationTagsWithCount().get("ws-inst") == 1);
+      }
+    } finally {
+      rm.stop();
+    }
+  }
+
+  @Test(timeout = 30000L)
+  public void testMultiAllocationTagsConstraints() throws Exception {
+    // This test simulates to use PC to avoid port conflicts
+    YarnConfiguration config = new YarnConfiguration();
+    config.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER,
+        YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER);
+    config.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    MockRM rm = new MockRM(config);
+    try {
+      rm.start();
+
+      MockNM nm1 = rm.registerNode("192.168.0.1:1234", 10*GB, 10);
+      MockNM nm2 = rm.registerNode("192.168.0.2:1234", 10*GB, 10);
+      MockNM nm3 = rm.registerNode("192.168.0.3:1234", 10*GB, 10);
+      MockNM nm4 = rm.registerNode("192.168.0.4:1234", 10*GB, 10);
+      MockNM nm5 = rm.registerNode("192.168.0.5:1234", 10*GB, 10);
+
+      RMApp app1 = rm.submitApp(1*GB, ImmutableSet.of("server1"));
+      // Allocate AM container on nm1
+      doNodeHeartbeat(nm1);
+      RMAppAttempt attempt1 = app1.getCurrentAppAttempt();
+      MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
+      am1.registerAppAttempt();
+
+      // App1 uses ports: 7000, 8000 and 9000
+      String[] server1Ports =
+          new String[] {"port_6000", "port_7000", "port_8000"};
+      PlacementConstraint pc = targetNotIn("node",
+          allocationTagWithNamespace(AllocationTagNamespaceType.ALL.toString(),
+              server1Ports))
+          .build();
+      am1.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 2, 1, 1024, pc, server1Ports)));
+      List<Container> allocated = waitForAllocation(2, 3000,
+          am1, nm1, nm2, nm3, nm4, nm5);
+
+      // 2 containers allocated
+      Assert.assertEquals(2, allocated.size());
+      // containers should be distributed on 2 different nodes
+      Assert.assertEquals(2, getContainerNodesNum(allocated));
+
+      // App1 uses ports: 6000
+      String[] server2Ports = new String[] {"port_6000"};
+      RMApp app2 = rm.submitApp(1*GB, ImmutableSet.of("server2"));
+      // Allocate AM container on nm1
+      doNodeHeartbeat(nm2);
+      RMAppAttempt app2attempt1 = app2.getCurrentAppAttempt();
+      MockAM am2 = rm.sendAMLaunched(app2attempt1.getAppAttemptId());
+      am2.registerAppAttempt();
+
+      pc = targetNotIn("node",
+          allocationTagWithNamespace(AllocationTagNamespaceType.ALL.toString(),
+              server2Ports))
+          .build();
+      am2.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 3, 1, 1024, pc, server2Ports)));
+      allocated = waitForAllocation(3, 3000, am2, nm1, nm2, nm3, nm4, nm5);
+      Assert.assertEquals(3, allocated.size());
+      Assert.assertEquals(3, getContainerNodesNum(allocated));
+
+      ConcurrentMap<NodeId, RMNode> rmNodes = rm.getRMContext().getRMNodes();
+      for (Container c : allocated) {
+        RMNode rmNode = rmNodes.get(c.getNodeId());
+        Assert.assertNotNull(rmNode);
+        Assert.assertTrue("server2 should not co-allocate to server1 as"
+                + " they both need to use port 6000",
+            rmNode.getAllocationTagsWithCount().get("port_6000") == 1);
+        Assert.assertFalse(rmNode.getAllocationTagsWithCount()
+            .containsKey("port_7000"));
+        Assert.assertFalse(rmNode.getAllocationTagsWithCount()
+            .containsKey("port_8000"));
+      }
+    } finally {
+      rm.stop();
+    }
+  }
+
+  @Test(timeout = 30000L)
+  public void testInterAppConstraintsWithNamespaces() throws Exception {
+    // This test verifies inter-app constraints with namespaces
+    // not-self/app-id/app-tag
+    YarnConfiguration config = new YarnConfiguration();
+    config.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER,
+        YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER);
+    config.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    MockRM rm = new MockRM(config);
+    try {
+      rm.start();
+
+      MockNM nm1 = rm.registerNode("192.168.0.1:1234:", 100*GB, 100);
+      MockNM nm2 = rm.registerNode("192.168.0.2:1234", 100*GB, 100);
+      MockNM nm3 = rm.registerNode("192.168.0.3:1234", 100*GB, 100);
+      MockNM nm4 = rm.registerNode("192.168.0.4:1234", 100*GB, 100);
+      MockNM nm5 = rm.registerNode("192.168.0.5:1234", 100*GB, 100);
+
+      ApplicationId app5Id = null;
+      Map<ApplicationId, List<Container>> allocMap = new HashMap<>();
+      // 10 apps and all containers are attached with foo tag
+      for (int i = 0; i<10; i++) {
+        // App1 ~ app5 tag "former5"
+        // App6 ~ app10 tag "latter5"
+        String applicationTag = i<5 ? "former5" : "latter5";
+        RMApp app = rm.submitApp(1*GB, ImmutableSet.of(applicationTag));
+        // Allocate AM container on nm1
+        doNodeHeartbeat(nm1, nm2, nm3, nm4, nm5);
+        RMAppAttempt attempt = app.getCurrentAppAttempt();
+        MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());
+        am.registerAppAttempt();
+
+        PlacementConstraint pc = targetNotIn("node", allocationTag("foo"))
+            .build();
+        am.addSchedulingRequest(
+            ImmutableList.of(
+                schedulingRequest(1, 3, 1, 1024, pc, "foo")));
+        List<Container> allocated = waitForAllocation(3, 3000,
+            am, nm1, nm2, nm3, nm4, nm5);
+        // Memorize containers that has app5 foo
+        if (i == 5) {
+          app5Id = am.getApplicationAttemptId().getApplicationId();
+        }
+        allocMap.put(am.getApplicationAttemptId().getApplicationId(),
+            allocated);
+      }
+
+      Assert.assertNotNull(app5Id);
+      Assert.assertEquals(3, getContainerNodesNum(allocMap.get(app5Id)));
+
+      // *** app-id
+      // Submit another app, use app-id constraint against app5
+      RMApp app1 = rm.submitApp(1*GB, ImmutableSet.of("xyz"));
+      // Allocate AM container on nm1
+      doNodeHeartbeat(nm1);
+      RMAppAttempt attempt1 = app1.getCurrentAppAttempt();
+      MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
+      am1.registerAppAttempt();
+
+      PlacementConstraint pc = targetIn("node",
+          allocationTagWithNamespace(
+              new TargetApplicationsNamespace.AppID(app5Id).toString(),
+              "foo"))
+          .build();
+      am1.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 3, 1, 1024, pc, "foo")));
+      List<Container> allocated = waitForAllocation(3, 3000,
+          am1, nm1, nm2, nm3, nm4, nm5);
+
+      ConcurrentMap<NodeId, RMNode> rmNodes = rm.getRMContext().getRMNodes();
+      List<Container> app5Alloc = allocMap.get(app5Id);
+      for (Container c : allocated) {
+        RMNode rmNode = rmNodes.get(c.getNodeId());
+        Assert.assertNotNull(rmNode);
+        Assert.assertTrue("This app is affinity with app-id/app5/foo "
+                + "containers",
+            app5Alloc.stream().anyMatch(
+                c5 -> c5.getNodeId() == c.getNodeId()));
+      }
+
+      // *** app-tag
+      RMApp app2 = rm.submitApp(1*GB);
+      // Allocate AM container on nm1
+      doNodeHeartbeat(nm2);
+      RMAppAttempt app2attempt1 = app2.getCurrentAppAttempt();
+      MockAM am2 = rm.sendAMLaunched(app2attempt1.getAppAttemptId());
+      am2.registerAppAttempt();
+
+      pc = targetNotIn("node",
+          allocationTagWithNamespace(
+              new TargetApplicationsNamespace.AppTag("xyz").toString(),
+              "foo"))
+          .build();
+      am2.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 2, 1, 1024, pc, "foo")));
+      allocated = waitForAllocation(2, 3000, am2, nm1, nm2, nm3, nm4, nm5);
+      Assert.assertEquals(2, allocated.size());
+
+      // none of them can be allocated to nodes that has app5 foo containers
+      for (Container c : app5Alloc) {
+        Assert.assertNotEquals(c.getNodeId(),
+            allocated.iterator().next().getNodeId());
+      }
+
+      // *** not-self
+      RMApp app3 = rm.submitApp(1*GB);
+      // Allocate AM container on nm1
+      doNodeHeartbeat(nm3);
+      RMAppAttempt app3attempt1 = app3.getCurrentAppAttempt();
+      MockAM am3 = rm.sendAMLaunched(app3attempt1.getAppAttemptId());
+      am3.registerAppAttempt();
+
+      pc = cardinality("node",
+          new TargetApplicationsNamespace.NotSelf().toString(),
+          1, 1, "foo").build();
+      am3.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 1, 1, 1024, pc, "foo")));
+      allocated = waitForAllocation(1, 3000, am3, nm1, nm2, nm3, nm4, nm5);
+      Assert.assertEquals(1, allocated.size());
+      // All 5 containers should be allocated
+      Assert.assertTrue(rmNodes.get(allocated.iterator().next().getNodeId())
+          .getAllocationTagsWithCount().get("foo") == 2);
+    } finally {
+      rm.stop();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ac01444/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java
index ccf4281..902c6d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
 
-import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTags;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
@@ -131,8 +130,6 @@ public class TestSingleConstraintAppPlacementAllocator {
             .build()).resourceSizing(
             ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
         .build());
-    Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
-        allocator.getTargetAllocationTags());
     Assert.assertEquals("", allocator.getTargetNodePartition());
 
     // Valid (with partition)
@@ -147,8 +144,6 @@ public class TestSingleConstraintAppPlacementAllocator {
             .build()).resourceSizing(
             ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
         .build());
-    Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
-        allocator.getTargetAllocationTags());
     Assert.assertEquals("x", allocator.getTargetNodePartition());
 
     // Valid (without specifying node partition)
@@ -162,8 +157,6 @@ public class TestSingleConstraintAppPlacementAllocator {
         .resourceSizing(
             ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
         .build());
-    Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
-        allocator.getTargetAllocationTags());
     Assert.assertEquals("", allocator.getTargetNodePartition());
 
     // Valid (with application Id target)
@@ -178,8 +171,6 @@ public class TestSingleConstraintAppPlacementAllocator {
             ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
         .build());
     // Allocation tags should not include application Id
-    Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
-        allocator.getTargetAllocationTags());
     Assert.assertEquals("", allocator.getTargetNodePartition());
 
     // Invalid (without sizing)
@@ -200,75 +191,6 @@ public class TestSingleConstraintAppPlacementAllocator {
             .targetNotIn(PlacementConstraints.NODE).build())
         .build(), true);
 
-    // Invalid (with multiple allocation tags expression specified)
-    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
-        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
-        .allocationRequestId(10L).priority(Priority.newInstance(1))
-        .placementConstraintExpression(PlacementConstraints
-            .targetNotIn(PlacementConstraints.NODE,
-                PlacementConstraints.PlacementTargets
-                    .allocationTag("mapper"),
-                PlacementConstraints.PlacementTargets
-                    .allocationTag("reducer"),
-                PlacementConstraints.PlacementTargets.nodePartition(""))
-            .build()).resourceSizing(
-            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
-        .build(), true);
-
-    // Invalid (with multiple node partition target expression specified)
-    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
-        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
-        .allocationRequestId(10L).priority(Priority.newInstance(1))
-        .placementConstraintExpression(PlacementConstraints
-            .targetNotIn(PlacementConstraints.NODE,
-                PlacementConstraints.PlacementTargets
-                    .allocationTag("mapper"),
-                PlacementConstraints.PlacementTargets
-                    .allocationTag(""),
-                PlacementConstraints.PlacementTargets.nodePartition("x"))
-            .build()).resourceSizing(
-            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
-        .build(), true);
-
-    // Invalid (not anti-affinity cardinality)
-    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
-        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
-        .allocationRequestId(10L).priority(Priority.newInstance(1))
-        .placementConstraintExpression(PlacementConstraints
-            .targetCardinality(PlacementConstraints.NODE, 1, 2,
-                PlacementConstraints.PlacementTargets
-                    .allocationTag("mapper"),
-                PlacementConstraints.PlacementTargets.nodePartition(""))
-            .build()).resourceSizing(
-            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
-        .build(), true);
-
-    // Invalid (not anti-affinity cardinality)
-    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
-        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
-        .allocationRequestId(10L).priority(Priority.newInstance(1))
-        .placementConstraintExpression(PlacementConstraints
-            .targetCardinality(PlacementConstraints.NODE, 0, 2,
-                PlacementConstraints.PlacementTargets
-                    .allocationTag("mapper"),
-                PlacementConstraints.PlacementTargets.nodePartition(""))
-            .build()).resourceSizing(
-            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
-        .build(), true);
-
-    // Invalid (not NODE scope)
-    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
-        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
-        .allocationRequestId(10L).priority(Priority.newInstance(1))
-        .placementConstraintExpression(PlacementConstraints
-            .targetNotIn(PlacementConstraints.RACK,
-                PlacementConstraints.PlacementTargets
-                    .allocationTag("mapper", "reducer"),
-                PlacementConstraints.PlacementTargets.nodePartition(""))
-            .build()).resourceSizing(
-            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
-        .build(), true);
-
     // Invalid (not GUARANTEED)
     assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
         ExecutionTypeRequest.newInstance(ExecutionType.OPPORTUNISTIC))


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