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 ae...@apache.org on 2017/08/22 01:58:04 UTC

[01/50] [abbrv] hadoop git commit: YARN-6687. Validate that the duration of the periodic reservation is less than the periodicity. (subru via curino)

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 7d132596d -> d0bd0f623


YARN-6687. Validate that the duration of the periodic reservation is less than the periodicity. (subru via curino)


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

Branch: refs/heads/HDFS-7240
Commit: 28d97b79b69bb2be02d9320105e155eeed6f9e78
Parents: cc59b5f
Author: Carlo Curino <cu...@apache.org>
Authored: Fri Aug 11 16:58:04 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Fri Aug 11 16:58:04 2017 -0700

----------------------------------------------------------------------
 .../reservation/ReservationInputValidator.java  | 18 ++--
 .../TestReservationInputValidator.java          | 93 ++++++++++++++++++++
 2 files changed, 106 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/28d97b79/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.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/reservation/ReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
index 0e9a825..027d066 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
@@ -129,11 +129,12 @@ public class ReservationInputValidator {
               Resources.multiply(rr.getCapability(), rr.getConcurrency()));
     }
     // verify the allocation is possible (skip for ANY)
-    if (contract.getDeadline() - contract.getArrival() < minDuration
+    long duration = contract.getDeadline() - contract.getArrival();
+    if (duration < minDuration
         && type != ReservationRequestInterpreter.R_ANY) {
       message =
           "The time difference ("
-              + (contract.getDeadline() - contract.getArrival())
+              + (duration)
               + ") between arrival (" + contract.getArrival() + ") "
               + "and deadline (" + contract.getDeadline() + ") must "
               + " be greater or equal to the minimum resource duration ("
@@ -158,15 +159,22 @@ public class ReservationInputValidator {
     // check that the recurrence is a positive long value.
     String recurrenceExpression = contract.getRecurrenceExpression();
     try {
-      Long recurrence = Long.parseLong(recurrenceExpression);
+      long recurrence = Long.parseLong(recurrenceExpression);
       if (recurrence < 0) {
         message = "Negative Period : " + recurrenceExpression + ". Please try"
-            + " again with a non-negative long value as period";
+            + " again with a non-negative long value as period.";
+        throw RPCUtil.getRemoteException(message);
+      }
+      // verify duration is less than recurrence for periodic reservations
+      if (recurrence > 0 && duration > recurrence) {
+        message = "Duration of the requested reservation: " + duration
+            + " is greater than the recurrence: " + recurrence
+            + ". Please try again with a smaller duration.";
         throw RPCUtil.getRemoteException(message);
       }
     } catch (NumberFormatException e) {
       message = "Invalid period " + recurrenceExpression + ". Please try"
-          + " again with a non-negative long value as period";
+          + " again with a non-negative long value as period.";
       throw RPCUtil.getRemoteException(message);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28d97b79/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.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/TestReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
index 2917cd9..90a681d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
@@ -303,6 +303,7 @@ public class TestReservationInputValidator {
 
   @Test
   public void testSubmitReservationInvalidRecurrenceExpression() {
+    // first check recurrence expression
     ReservationSubmissionRequest request =
         createSimpleReservationSubmissionRequest(1, 1, 1, 5, 3, "123abc");
     plan = null;
@@ -318,6 +319,23 @@ public class TestReservationInputValidator {
           .startsWith("Invalid period "));
       LOG.info(message);
     }
+
+    // now check duration
+    request =
+        createSimpleReservationSubmissionRequest(1, 1, 1, 50, 3, "10");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationSubmissionRequest(rSystem, request,
+              ReservationSystemTestUtil.getNewReservationId());
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("Duration of the requested reservation:"));
+      LOG.info(message);
+    }
   }
 
   @Test
@@ -500,6 +518,73 @@ public class TestReservationInputValidator {
   }
 
   @Test
+  public void testUpdateReservationValidRecurrenceExpression() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 5, 3, "600000");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationUpdateRequest(rSystem, request);
+    } catch (YarnException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNotNull(plan);
+  }
+
+  @Test
+  public void testUpdateReservationNegativeRecurrenceExpression() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 5, 3, "-1234");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("Negative Period : "));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testUpdateReservationInvalidRecurrenceExpression() {
+    // first check recurrence expression
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 5, 3, "123abc");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("Invalid period "));
+      LOG.info(message);
+    }
+
+    // now check duration
+    request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 50, 3, "10");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("Duration of the requested reservation:"));
+      LOG.info(message);
+    }
+  }
+
+  @Test
   public void testDeleteReservationNormal() {
     ReservationDeleteRequest request = new ReservationDeleteRequestPBImpl();
     ReservationId reservationID =
@@ -710,11 +795,19 @@ public class TestReservationInputValidator {
   private ReservationUpdateRequest createSimpleReservationUpdateRequest(
       int numRequests, int numContainers, long arrival, long deadline,
       long duration) {
+    return createSimpleReservationUpdateRequest(numRequests, numContainers,
+        arrival, deadline, duration, "0");
+  }
+
+  private ReservationUpdateRequest createSimpleReservationUpdateRequest(
+      int numRequests, int numContainers, long arrival, long deadline,
+      long duration, String recurrence) {
     // create a request with a single atomic ask
     ReservationUpdateRequest request = new ReservationUpdateRequestPBImpl();
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
+    rDef.setRecurrenceExpression(recurrence);
     if (numRequests > 0) {
       ReservationRequests reqs = new ReservationRequestsPBImpl();
       rDef.setReservationRequests(reqs);


---------------------------------------------------------------------
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-3254. HealthReport should include disk full information. Contributed by Suma Shivaprasad.

Posted by ae...@apache.org.
YARN-3254. HealthReport should include disk full information. Contributed by Suma Shivaprasad.


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

Branch: refs/heads/HDFS-7240
Commit: f9a0e2338150f1bd3ba2c29f76979183fd3ed80c
Parents: 1f04cb4
Author: Sunil G <su...@apache.org>
Authored: Thu Aug 17 15:07:15 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Thu Aug 17 15:07:15 2017 +0530

----------------------------------------------------------------------
 .../server/nodemanager/DirectoryCollection.java | 61 +++++++++++++++++++-
 .../nodemanager/LocalDirsHandlerService.java    | 59 +++++++++++++++----
 .../nodemanager/TestDirectoryCollection.java    | 23 ++++++++
 3 files changed, 130 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9a0e233/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DirectoryCollection.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/DirectoryCollection.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DirectoryCollection.java
index ae2a4ef..502485f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DirectoryCollection.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DirectoryCollection.java
@@ -38,6 +38,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
@@ -99,6 +100,7 @@ public class DirectoryCollection {
   private List<String> localDirs;
   private List<String> errorDirs;
   private List<String> fullDirs;
+  private Map<String, DiskErrorInformation> directoryErrorInfo;
 
   // read/write lock for accessing above directories.
   private final ReadLock readLock;
@@ -192,6 +194,7 @@ public class DirectoryCollection {
     localDirs = new CopyOnWriteArrayList<>(dirs);
     errorDirs = new CopyOnWriteArrayList<>();
     fullDirs = new CopyOnWriteArrayList<>();
+    directoryErrorInfo = new ConcurrentHashMap<>();
 
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     this.readLock = lock.readLock();
@@ -248,11 +251,25 @@ public class DirectoryCollection {
   /**
    * @return the directories that have used all disk space
    */
-
   List<String> getFullDirs() {
     this.readLock.lock();
     try {
-      return fullDirs;
+      return Collections.unmodifiableList(fullDirs);
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+
+  /**
+   * @return the directories that have errors - many not have appropriate permissions
+   * or other disk validation checks might have failed in {@link DiskValidator}
+   *
+   */
+  @InterfaceStability.Evolving
+  List<String> getErroredDirs() {
+    this.readLock.lock();
+    try {
+      return Collections.unmodifiableList(errorDirs);
     } finally {
       this.readLock.unlock();
     }
@@ -271,6 +288,39 @@ public class DirectoryCollection {
   }
 
   /**
+   *
+   * @param dirName Absolute path of Directory for which error diagnostics are needed
+   * @return DiskErrorInformation - disk error diagnostics for the specified directory
+   *         null - the disk associated with the directory has passed disk utilization checks
+   *         /error validations in {@link DiskValidator}
+   *
+   */
+  @InterfaceStability.Evolving
+  DiskErrorInformation getDirectoryErrorInfo(String dirName) {
+    this.readLock.lock();
+    try {
+      return directoryErrorInfo.get(dirName);
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+
+  /**
+   *
+   * @param dirName Absolute path of Directory for which the disk has been marked as unhealthy
+   * @return Check if disk associated with the directory is unhealthy
+   */
+  @InterfaceStability.Evolving
+  boolean isDiskUnHealthy(String dirName) {
+    this.readLock.lock();
+    try {
+      return directoryErrorInfo.containsKey(dirName);
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+
+  /**
    * Create any non-existent directories and parent directories, updating the
    * list of valid directories if necessary.
    * @param localFs local file system to use
@@ -297,6 +347,9 @@ public class DirectoryCollection {
         try {
           localDirs.remove(dir);
           errorDirs.add(dir);
+          directoryErrorInfo.put(dir,
+              new DiskErrorInformation(DiskErrorCause.OTHER,
+                  "Cannot create directory : " + dir + ", error " + e.getMessage()));
           numFailures++;
         } finally {
           this.writeLock.unlock();
@@ -343,11 +396,13 @@ public class DirectoryCollection {
       localDirs.clear();
       errorDirs.clear();
       fullDirs.clear();
+      directoryErrorInfo.clear();
 
       for (Map.Entry<String, DiskErrorInformation> entry : dirsFailedCheck
           .entrySet()) {
         String dir = entry.getKey();
         DiskErrorInformation errorInformation = entry.getValue();
+
         switch (entry.getValue().cause) {
         case DISK_FULL:
           fullDirs.add(entry.getKey());
@@ -359,6 +414,8 @@ public class DirectoryCollection {
           LOG.warn(entry.getValue().cause + " is unknown for disk error.");
           break;
         }
+        directoryErrorInfo.put(entry.getKey(), errorInformation);
+
         if (preCheckGoodDirs.contains(dir)) {
           LOG.warn("Directory " + dir + " error, " + errorInformation.message
               + ", removing from list of valid directories");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9a0e233/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.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/LocalDirsHandlerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
index f8cb4ee..6e00808 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
@@ -53,6 +53,8 @@ public class LocalDirsHandlerService extends AbstractService {
 
   private static Log LOG = LogFactory.getLog(LocalDirsHandlerService.class);
 
+  private static final String diskCapacityExceededErrorMsg =  "usable space is below configured utilization percentage/no more usable space";
+
   /**
    * Good local directories, use internally,
    * initial value is the same as NM_LOCAL_DIRS.
@@ -344,21 +346,36 @@ public class LocalDirsHandlerService extends AbstractService {
     }
 
     StringBuilder report = new StringBuilder();
-    List<String> failedLocalDirsList = localDirs.getFailedDirs();
-    List<String> failedLogDirsList = logDirs.getFailedDirs();
+    List<String> erroredLocalDirsList = localDirs.getErroredDirs();
+    List<String> erroredLogDirsList = logDirs.getErroredDirs();
+    List<String> diskFullLocalDirsList = localDirs.getFullDirs();
+    List<String> diskFullLogDirsList = logDirs.getFullDirs();
     List<String> goodLocalDirsList = localDirs.getGoodDirs();
     List<String> goodLogDirsList = logDirs.getGoodDirs();
-    int numLocalDirs = goodLocalDirsList.size() + failedLocalDirsList.size();
-    int numLogDirs = goodLogDirsList.size() + failedLogDirsList.size();
+
+    int numLocalDirs = goodLocalDirsList.size() + erroredLocalDirsList.size() + diskFullLocalDirsList.size();
+    int numLogDirs = goodLogDirsList.size() + erroredLogDirsList.size() + diskFullLogDirsList.size();
     if (!listGoodDirs) {
-      if (!failedLocalDirsList.isEmpty()) {
-        report.append(failedLocalDirsList.size() + "/" + numLocalDirs
-            + " local-dirs are bad: "
-            + StringUtils.join(",", failedLocalDirsList) + "; ");
+      if (!erroredLocalDirsList.isEmpty()) {
+        report.append(erroredLocalDirsList.size() + "/" + numLocalDirs
+            + " local-dirs have errors: "
+            + buildDiskErrorReport(erroredLocalDirsList, localDirs));
+      }
+      if (!diskFullLocalDirsList.isEmpty()) {
+        report.append(diskFullLocalDirsList.size() + "/" + numLocalDirs
+            + " local-dirs " + diskCapacityExceededErrorMsg
+            + buildDiskErrorReport(diskFullLocalDirsList, localDirs) + "; ");
       }
-      if (!failedLogDirsList.isEmpty()) {
-        report.append(failedLogDirsList.size() + "/" + numLogDirs
-            + " log-dirs are bad: " + StringUtils.join(",", failedLogDirsList));
+
+      if (!erroredLogDirsList.isEmpty()) {
+        report.append(erroredLogDirsList.size() + "/" + numLogDirs
+            + " log-dirs have errors: "
+            + buildDiskErrorReport(erroredLogDirsList, logDirs));
+      }
+      if (!diskFullLogDirsList.isEmpty()) {
+        report.append(diskFullLogDirsList.size() + "/" + numLogDirs
+            + " log-dirs " + diskCapacityExceededErrorMsg
+            + buildDiskErrorReport(diskFullLogDirsList, logDirs));
       }
     } else {
       report.append(goodLocalDirsList.size() + "/" + numLocalDirs
@@ -620,4 +637,24 @@ public class LocalDirsHandlerService extends AbstractService {
           logDirs.getGoodDirsDiskUtilizationPercentage());
     }
   }
+
+  private String buildDiskErrorReport(List<String> dirs, DirectoryCollection directoryCollection) {
+    StringBuilder sb = new StringBuilder();
+
+    sb.append(" [ ");
+    for (int i = 0; i < dirs.size(); i++) {
+      final String dirName = dirs.get(i);
+      if ( directoryCollection.isDiskUnHealthy(dirName)) {
+        sb.append(dirName + " : " + directoryCollection.getDirectoryErrorInfo(dirName).message);
+      } else {
+        sb.append(dirName + " : " + "Unknown cause for disk error");
+      }
+
+      if ( i != (dirs.size() - 1)) {
+        sb.append(" , ");
+      }
+    }
+    sb.append(" ] ");
+    return sb.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9a0e233/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDirectoryCollection.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/TestDirectoryCollection.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDirectoryCollection.java
index e529628..095f21a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDirectoryCollection.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDirectoryCollection.java
@@ -128,8 +128,12 @@ public class TestDirectoryCollection {
     DirectoryCollection dc = new DirectoryCollection(dirs, 0.0F);
     dc.checkDirs();
     Assert.assertEquals(0, dc.getGoodDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
     Assert.assertEquals(1, dc.getFailedDirs().size());
     Assert.assertEquals(1, dc.getFullDirs().size());
+    Assert.assertNotNull(dc.getDirectoryErrorInfo(dirA));
+    Assert.assertEquals(DirectoryCollection.DiskErrorCause.DISK_FULL, dc.getDirectoryErrorInfo(dirA).cause);
+
     // no good dirs
     Assert.assertEquals(0, dc.getGoodDirsDiskUtilizationPercentage());
 
@@ -139,16 +143,21 @@ public class TestDirectoryCollection {
             testDir.getTotalSpace());
     dc.checkDirs();
     Assert.assertEquals(1, dc.getGoodDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
     Assert.assertEquals(0, dc.getFailedDirs().size());
     Assert.assertEquals(0, dc.getFullDirs().size());
+    Assert.assertNull(dc.getDirectoryErrorInfo(dirA));
+
     Assert.assertEquals(utilizedSpacePerc,
       dc.getGoodDirsDiskUtilizationPercentage());
 
     dc = new DirectoryCollection(dirs, testDir.getTotalSpace() / (1024 * 1024));
     dc.checkDirs();
     Assert.assertEquals(0, dc.getGoodDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
     Assert.assertEquals(1, dc.getFailedDirs().size());
     Assert.assertEquals(1, dc.getFullDirs().size());
+    Assert.assertNotNull(dc.getDirectoryErrorInfo(dirA));
     // no good dirs
     Assert.assertEquals(0, dc.getGoodDirsDiskUtilizationPercentage());
 
@@ -158,8 +167,11 @@ public class TestDirectoryCollection {
             testDir.getTotalSpace());
     dc.checkDirs();
     Assert.assertEquals(1, dc.getGoodDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
     Assert.assertEquals(0, dc.getFailedDirs().size());
     Assert.assertEquals(0, dc.getFullDirs().size());
+    Assert.assertNull(dc.getDirectoryErrorInfo(dirA));
+
     Assert.assertEquals(utilizedSpacePerc,
       dc.getGoodDirsDiskUtilizationPercentage());
   }
@@ -209,12 +221,17 @@ public class TestDirectoryCollection {
     Assert.assertEquals(0, dc.getGoodDirs().size());
     Assert.assertEquals(1, dc.getFailedDirs().size());
     Assert.assertEquals(1, dc.getFullDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
+    Assert.assertNotNull(dc.getDirectoryErrorInfo(dirA));
+    Assert.assertEquals(DirectoryCollection.DiskErrorCause.DISK_FULL, dc.getDirectoryErrorInfo(dirA).cause);
 
     dc.setDiskUtilizationPercentageCutoff(100.0F, 100.0F);
     dc.checkDirs();
     Assert.assertEquals(1, dc.getGoodDirs().size());
     Assert.assertEquals(0, dc.getFailedDirs().size());
     Assert.assertEquals(0, dc.getFullDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
+    Assert.assertNull(dc.getDirectoryErrorInfo(dirA));
 
     conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
 
@@ -232,12 +249,18 @@ public class TestDirectoryCollection {
     Assert.assertEquals(0, dc.getGoodDirs().size());
     Assert.assertEquals(1, dc.getFailedDirs().size());
     Assert.assertEquals(0, dc.getFullDirs().size());
+    Assert.assertEquals(1, dc.getErroredDirs().size());
+    Assert.assertNotNull(dc.getDirectoryErrorInfo(dirB));
+    Assert.assertEquals(DirectoryCollection.DiskErrorCause.OTHER, dc.getDirectoryErrorInfo(dirB).cause);
+
     permDirB = new FsPermission((short) 0700);
     localFs.setPermission(pathB, permDirB);
     dc.checkDirs();
     Assert.assertEquals(1, dc.getGoodDirs().size());
     Assert.assertEquals(0, dc.getFailedDirs().size());
     Assert.assertEquals(0, dc.getFullDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
+    Assert.assertNull(dc.getDirectoryErrorInfo(dirA));
   }
 
   @Test


---------------------------------------------------------------------
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: HDFS-11988. Verify HDFS Snapshots with open files captured are consistent across truncates and appends to current version file.

Posted by ae...@apache.org.
HDFS-11988. Verify HDFS Snapshots with open files captured are consistent across truncates and appends to current version file.


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

Branch: refs/heads/HDFS-7240
Commit: 913760cb4fe7123e55004800f75dc00540a79f69
Parents: 267e19a
Author: Manoj Govindassamy <ma...@apache.org>
Authored: Mon Aug 21 11:08:38 2017 -0700
Committer: Manoj Govindassamy <ma...@apache.org>
Committed: Mon Aug 21 11:08:38 2017 -0700

----------------------------------------------------------------------
 .../snapshot/TestOpenFilesWithSnapshot.java     | 112 +++++++++++++++++++
 1 file changed, 112 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/913760cb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
index bf27f2c..537612c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
@@ -30,6 +30,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSOutputStream;
@@ -731,6 +732,117 @@ public class TestOpenFilesWithSnapshot {
     cluster.waitActive();
   }
 
+  /**
+   * Verify snapshots with open files captured are safe even when the
+   * 'current' version of the file is truncated and appended later.
+   */
+  @Test (timeout = 120000)
+  public void testOpenFilesSnapChecksumWithTrunkAndAppend() throws Exception {
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES,
+        true);
+    // Construct the directory tree
+    final Path dir = new Path("/A/B/C");
+    fs.mkdirs(dir);
+
+    // String constants
+    final Path hbaseSnapRootDir = dir;
+    final String hbaseFileName = "hbase.wal";
+    final String hbaseSnap1Name = "hbase_snap_s1";
+    final String hbaseSnap2Name = "hbase_snap_s2";
+    final String hbaseSnap3Name = "hbase_snap_s3";
+    final String hbaseSnap4Name = "hbase_snap_s4";
+
+    // Create files and open a stream
+    final Path hbaseFile = new Path(dir, hbaseFileName);
+    createFile(hbaseFile);
+    final FileChecksum hbaseWALFileCksum0 =
+        fs.getFileChecksum(hbaseFile);
+    FSDataOutputStream hbaseOutputStream = fs.append(hbaseFile);
+
+    // Create Snapshot S1
+    final Path hbaseS1Dir = SnapshotTestHelper.createSnapshot(
+        fs, hbaseSnapRootDir, hbaseSnap1Name);
+    final Path hbaseS1Path = new Path(hbaseS1Dir, hbaseFileName);
+    final FileChecksum hbaseFileCksumS1 = fs.getFileChecksum(hbaseS1Path);
+
+    // Verify if Snap S1 checksum is same as the current version one
+    Assert.assertEquals("Live and snap1 file checksum doesn't match!",
+        hbaseWALFileCksum0, fs.getFileChecksum(hbaseS1Path));
+
+    int newWriteLength = (int) (BLOCKSIZE * 1.5);
+    byte[] buf = new byte[newWriteLength];
+    Random random = new Random();
+    random.nextBytes(buf);
+    writeToStream(hbaseOutputStream, buf);
+
+    // Create Snapshot S2
+    final Path hbaseS2Dir = SnapshotTestHelper.createSnapshot(
+        fs, hbaseSnapRootDir, hbaseSnap2Name);
+    final Path hbaseS2Path = new Path(hbaseS2Dir, hbaseFileName);
+    final FileChecksum hbaseFileCksumS2 = fs.getFileChecksum(hbaseS2Path);
+
+    // Verify if the s1 checksum is still the same
+    Assert.assertEquals("Snap file checksum has changed!",
+        hbaseFileCksumS1, fs.getFileChecksum(hbaseS1Path));
+    // Verify if the s2 checksum is different from the s1 checksum
+    Assert.assertNotEquals("Snap1 and snap2 file checksum should differ!",
+        hbaseFileCksumS1, hbaseFileCksumS2);
+
+    newWriteLength = (int) (BLOCKSIZE * 2.5);
+    buf = new byte[newWriteLength];
+    random.nextBytes(buf);
+    writeToStream(hbaseOutputStream, buf);
+
+    // Create Snapshot S3
+    final Path hbaseS3Dir = SnapshotTestHelper.createSnapshot(
+        fs, hbaseSnapRootDir, hbaseSnap3Name);
+    final Path hbaseS3Path = new Path(hbaseS3Dir, hbaseFileName);
+    FileChecksum hbaseFileCksumS3 = fs.getFileChecksum(hbaseS3Path);
+
+    // Record the checksum for the before truncate current file
+    hbaseOutputStream.close();
+    final FileChecksum hbaseFileCksumBeforeTruncate =
+        fs.getFileChecksum(hbaseFile);
+    Assert.assertEquals("Snap3 and before truncate file checksum should match!",
+        hbaseFileCksumBeforeTruncate, hbaseFileCksumS3);
+
+    // Truncate the current file and record the after truncate checksum
+    long currentFileLen = fs.getFileStatus(hbaseFile).getLen();
+    boolean fileTruncated = fs.truncate(hbaseFile, currentFileLen / 2);
+    Assert.assertTrue("File truncation failed!", fileTruncated);
+    final FileChecksum hbaseFileCksumAfterTruncate =
+        fs.getFileChecksum(hbaseFile);
+
+    Assert.assertNotEquals("Snap3 and after truncate checksum shouldn't match!",
+        hbaseFileCksumS3, hbaseFileCksumAfterTruncate);
+
+    // Append more data to the current file
+    hbaseOutputStream = fs.append(hbaseFile);
+    newWriteLength = (int) (BLOCKSIZE * 5.5);
+    buf = new byte[newWriteLength];
+    random.nextBytes(buf);
+    writeToStream(hbaseOutputStream, buf);
+
+    // Create Snapshot S4
+    final Path hbaseS4Dir = SnapshotTestHelper.createSnapshot(
+        fs, hbaseSnapRootDir, hbaseSnap4Name);
+    final Path hbaseS4Path = new Path(hbaseS4Dir, hbaseFileName);
+    final FileChecksum hbaseFileCksumS4 = fs.getFileChecksum(hbaseS4Path);
+
+    // Record the checksum for the current file after append
+    hbaseOutputStream.close();
+    final FileChecksum hbaseFileCksumAfterAppend =
+        fs.getFileChecksum(hbaseFile);
+
+    Assert.assertEquals("Snap4 and after append file checksum should match!",
+        hbaseFileCksumAfterAppend, hbaseFileCksumS4);
+
+    // Recompute checksum for S3 path and verify it has not changed
+    hbaseFileCksumS3 = fs.getFileChecksum(hbaseS3Path);
+    Assert.assertEquals("Snap3 and before truncate file checksum should match!",
+        hbaseFileCksumBeforeTruncate, hbaseFileCksumS3);
+  }
+
   private void restartNameNode() throws Exception {
     cluster.triggerBlockReports();
     NameNode nameNode = cluster.getNameNode();


---------------------------------------------------------------------
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: Merge branch 'trunk' into HDFS-7240

Posted by ae...@apache.org.
Merge branch 'trunk' into HDFS-7240

 Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
	hadoop-hdfs-project/hadoop-hdfs/pom.xml
	hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
	hadoop-project/pom.xml
	hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java


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

Branch: refs/heads/HDFS-7240
Commit: d0bd0f623338dbb558d0dee5e747001d825d92c5
Parents: 7d13259 b6bfb2f
Author: Anu Engineer <ae...@apache.org>
Authored: Mon Aug 21 18:57:15 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Mon Aug 21 18:57:15 2017 -0700

----------------------------------------------------------------------
 LICENSE.txt                                     |     1 +
 dev-support/docker/hadoop_env_checks.sh         |     2 +-
 dev-support/findHangingTest.sh                  |     2 +-
 dev-support/verify-xml.sh                       |     2 +-
 .../resources/assemblies/hadoop-yarn-dist.xml   |     1 +
 .../hadoop-client-check-invariants/pom.xml      |     1 -
 .../hadoop-client-check-test-invariants/pom.xml |     1 -
 .../hadoop-client-minicluster/pom.xml           |    33 +
 .../hadoop-client-runtime/pom.xml               |     7 -
 .../dev-support/findbugsExcludeFile.xml         |     4 +
 hadoop-common-project/hadoop-common/pom.xml     |     3 +-
 .../hadoop-common/src/main/bin/hadoop           |    28 +-
 .../src/main/bin/hadoop-functions.sh            |   184 +-
 .../org/apache/hadoop/conf/Configuration.java   |    51 +-
 .../key/kms/LoadBalancingKMSClientProvider.java |     4 +-
 .../hadoop/fs/CommonConfigurationKeys.java      |    21 +
 .../fs/CommonConfigurationKeysPublic.java       |     2 +
 .../src/main/java/org/apache/hadoop/fs/DF.java  |     9 +-
 .../hadoop/fs/FSDataOutputStreamBuilder.java    |   203 +-
 .../apache/hadoop/fs/FileEncryptionInfo.java    |     6 +-
 .../java/org/apache/hadoop/fs/FileStatus.java   |   189 +-
 .../java/org/apache/hadoop/fs/FileSystem.java   |    24 +-
 .../java/org/apache/hadoop/fs/FileUtil.java     |     4 +-
 .../org/apache/hadoop/fs/FsUrlConnection.java   |    10 +
 .../hadoop/fs/FsUrlStreamHandlerFactory.java    |    26 +-
 .../org/apache/hadoop/fs/LocatedFileStatus.java |    64 +-
 .../org/apache/hadoop/fs/ftp/FTPFileSystem.java |     4 +-
 .../hadoop/fs/permission/FsPermission.java      |    14 +-
 .../apache/hadoop/fs/protocolPB/PBHelper.java   |   131 +
 .../hadoop/fs/protocolPB/package-info.java      |    18 +
 .../apache/hadoop/fs/sftp/SFTPFileSystem.java   |     6 +
 .../org/apache/hadoop/fs/shell/AclCommands.java |     6 +-
 .../hadoop/fs/shell/CommandWithDestination.java |     4 +-
 .../java/org/apache/hadoop/fs/shell/Ls.java     |     4 +-
 .../java/org/apache/hadoop/fs/shell/Stat.java   |    19 +-
 .../hadoop/fs/viewfs/ViewFsFileStatus.java      |     8 +-
 .../fs/viewfs/ViewFsLocatedFileStatus.java      |     6 -
 .../org/apache/hadoop/http/HttpServer2.java     |     9 +-
 .../main/java/org/apache/hadoop/io/IOUtils.java |     2 +-
 .../java/org/apache/hadoop/io/SequenceFile.java |     2 +-
 .../hadoop/io/erasurecode/CodecRegistry.java    |     2 +-
 .../apache/hadoop/io/erasurecode/ECSchema.java  |     6 +-
 .../io/erasurecode/ErasureCodeConstants.java    |     8 +
 .../hadoop/io/retry/RetryInvocationHandler.java |    11 +
 .../hadoop/security/CompositeGroupsMapping.java |     4 +-
 .../org/apache/hadoop/util/GenericsUtil.java    |    15 +
 .../hadoop/util/curator/ZKCuratorManager.java   |   340 +
 .../hadoop/util/curator/package-info.java       |    27 +
 .../hadoop-common/src/main/proto/FSProtos.proto |    69 +
 .../src/main/resources/core-default.xml         |    95 +-
 .../src/site/markdown/FileSystemShell.md        |     4 +-
 .../hadoop-common/src/site/markdown/Metrics.md  |    20 +-
 .../src/site/markdown/SecureMode.md             |     9 +-
 .../src/site/markdown/ServiceLevelAuth.md       |    32 +-
 .../src/site/markdown/UnixShellGuide.md         |     4 +-
 .../src/site/markdown/filesystem/filesystem.md  |    57 +-
 .../filesystem/fsdataoutputstreambuilder.md     |   182 +
 .../src/site/markdown/filesystem/index.md       |     1 +
 .../conf/TestCommonConfigurationFields.java     |     6 +
 .../apache/hadoop/conf/TestConfiguration.java   |   229 +-
 .../org/apache/hadoop/fs/TestFileStatus.java    |     1 +
 .../apache/hadoop/fs/TestLocalFileSystem.java   |    78 +-
 .../fs/contract/AbstractContractAppendTest.java |    33 +-
 .../fs/contract/AbstractContractCreateTest.java |    90 +-
 .../hadoop/fs/contract/ContractTestUtils.java   |    88 +-
 .../fs/protocolPB/TestFSSerialization.java      |    85 +
 .../org/apache/hadoop/http/TestHttpServer.java  |    13 +
 .../apache/hadoop/util/TestGenericsUtil.java    |     5 +
 .../util/curator/TestZKCuratorManager.java      |    95 +
 .../src/test/resources/testConf.xml             |    10 +-
 .../scripts/hadoop-functions_test_helper.bash   |     2 +-
 .../test/scripts/hadoop_add_array_param.bats    |    37 +
 .../src/test/scripts/hadoop_array_contains.bats |    47 +
 .../src/test/scripts/hadoop_escape_chars.bats   |    32 -
 .../src/test/scripts/hadoop_sort_array.bats     |    37 +
 .../main/libexec/shellprofile.d/hadoop-kms.sh   |     4 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |    31 +-
 .../hadoop/hdfs/DFSClientFaultInjector.java     |     2 +
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |   156 +-
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |    23 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |    14 +-
 .../hadoop/hdfs/protocol/ClientProtocol.java    |    10 +-
 .../hdfs/protocol/ErasureCodingPolicy.java      |    13 +-
 .../hdfs/protocol/FsPermissionExtension.java    |     5 +
 .../hadoop/hdfs/protocol/HdfsConstants.java     |     4 +-
 .../hadoop/hdfs/protocol/HdfsFileStatus.java    |   228 +-
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |    42 +-
 .../protocol/SnapshottableDirectoryStatus.java  |    10 +-
 .../protocol/SystemErasureCodingPolicies.java   |    14 +
 .../ClientNamenodeProtocolTranslatorPB.java     |     9 +-
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |    67 +-
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |    50 +-
 .../hadoop/hdfs/web/WebHdfsConstants.java       |     8 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |    16 +-
 .../hadoop-hdfs-client/src/main/proto/acl.proto |     7 +-
 .../src/main/proto/erasurecoding.proto          |     2 +-
 .../src/main/proto/hdfs.proto                   |    16 +-
 .../hadoop/fs/http/client/HttpFSFileSystem.java |    92 +-
 .../hadoop/fs/http/server/FSOperations.java     |   107 +-
 .../http/server/HttpFSParametersProvider.java   |    45 +
 .../hadoop/fs/http/server/HttpFSServer.java     |    36 +
 .../libexec/shellprofile.d/hadoop-httpfs.sh     |     2 +-
 .../fs/http/client/BaseTestHttpFSWith.java      |   130 +-
 .../hadoop/fs/http/server/TestHttpFSServer.java |   140 +-
 .../apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java  |     6 +-
 .../hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java    |     8 +-
 .../dev-support/findbugsExcludeFile.xml         |    33 +
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |     4 +-
 .../hadoop-hdfs/src/main/bin/hdfs               |    82 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |     8 +-
 .../hadoop/hdfs/protocol/SnapshotException.java |     4 +
 .../hadoop/hdfs/protocol/SnapshotInfo.java      |     2 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java |     7 +-
 .../hadoop/hdfs/protocolPB/package-info.java    |    18 +
 .../hadoop/hdfs/qjournal/server/Journal.java    |     3 +-
 .../hdfs/qjournal/server/JournalMetrics.java    |    11 +
 .../hdfs/qjournal/server/JournalNodeSyncer.java |     4 +
 .../server/blockmanagement/BlockManager.java    |    49 +-
 .../blockmanagement/DatanodeAdminManager.java   |   756 +
 .../blockmanagement/DatanodeDescriptor.java     |     6 +-
 .../server/blockmanagement/DatanodeManager.java |    97 +-
 .../blockmanagement/DecommissionManager.java    |   741 -
 .../hadoop/hdfs/server/datanode/DataNode.java   |    21 +
 .../hdfs/server/datanode/DataXceiver.java       |     9 +-
 .../erasurecode/ErasureCodingWorker.java        |    15 +-
 .../erasurecode/StripedBlockReconstructor.java  |     3 +-
 .../datanode/erasurecode/StripedReader.java     |    20 +
 .../erasurecode/StripedReconstructionInfo.java  |    15 +
 .../erasurecode/StripedReconstructor.java       |     8 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |    48 +-
 .../apache/hadoop/hdfs/server/mover/Mover.java  |     2 +-
 .../hadoop/hdfs/server/namenode/BackupNode.java |     2 +-
 .../namenode/ErasureCodingPolicyManager.java    |    25 +-
 .../server/namenode/FSDirErasureCodingOp.java   |    17 +-
 .../server/namenode/FSDirStatAndListingOp.java  |    58 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |     2 +-
 .../hdfs/server/namenode/FSDirectory.java       |    20 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   176 +-
 .../hdfs/server/namenode/LeaseManager.java      |    41 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |    24 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |     4 +-
 .../namenode/metrics/ECBlockGroupsMBean.java    |    59 +
 .../metrics/ECBlockGroupsStatsMBean.java        |    59 -
 .../namenode/metrics/ReplicatedBlocksMBean.java |    63 +
 .../metrics/ReplicatedBlocksStatsMBean.java     |    63 -
 .../snapshot/DirectorySnapshottableFeature.java |    16 +-
 .../namenode/snapshot/SnapshotManager.java      |     7 +-
 .../org/apache/hadoop/hdfs/tools/ECAdmin.java   |    41 +-
 .../offlineEditsViewer/XmlEditsVisitor.java     |    41 +-
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |    28 +-
 .../src/main/resources/hdfs-default.xml         |    40 +-
 .../src/main/webapps/hdfs/explorer.js           |     6 +
 .../src/site/markdown/HDFSErasureCoding.md      |    43 +-
 .../src/site/markdown/HdfsPermissionsGuide.md   |     2 +-
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    |    39 +
 .../org/apache/hadoop/TestGenericRefresh.java   |    28 +-
 .../java/org/apache/hadoop/cli/TestAclCLI.java  |     2 +
 .../apache/hadoop/fs/TestUrlStreamHandler.java  |    48 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |     8 +-
 .../hadoop/hdfs/TestDFSClientRetries.java       |     6 +-
 .../hadoop/hdfs/TestDFSClientSocketSize.java    |    20 +-
 .../org/apache/hadoop/hdfs/TestDFSShell.java    |    36 +-
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    |     3 +-
 .../apache/hadoop/hdfs/TestDecommission.java    |    44 +-
 .../hadoop/hdfs/TestDistributedFileSystem.java  |    50 +
 .../apache/hadoop/hdfs/TestEncryptionZones.java |     3 +-
 .../hadoop/hdfs/TestErasureCodingPolicies.java  |   129 +-
 .../hdfs/TestFileStatusSerialization.java       |   153 +
 .../java/org/apache/hadoop/hdfs/TestLease.java  |     7 +-
 .../java/org/apache/hadoop/hdfs/TestPread.java  |    89 +-
 .../hadoop/hdfs/TestReconstructStripedFile.java |    74 +-
 .../org/apache/hadoop/hdfs/TestSafeMode.java    |    25 +
 .../hdfs/qjournal/TestJournalNodeSync.java      |   265 -
 .../hdfs/qjournal/server/TestJournalNode.java   |     6 +-
 .../qjournal/server/TestJournalNodeSync.java    |   439 +
 .../blockmanagement/BlockManagerTestUtil.java   |     2 +-
 .../blockmanagement/TestBlockManager.java       |    54 +
 .../blockmanagement/TestDatanodeManager.java    |    96 +-
 ...constructStripedBlocksWithRackAwareness.java |     5 +-
 .../TestReplicationPolicyConsiderLoad.java      |     2 +-
 .../TestDataXceiverBackwardsCompat.java         |   212 +
 .../hdfs/server/mover/TestStorageMover.java     |     2 +-
 .../hdfs/server/namenode/AclTestHelpers.java    |    13 +-
 .../hdfs/server/namenode/FSAclBaseTest.java     |    14 +-
 .../server/namenode/TestAddStripedBlocks.java   |     4 +-
 .../namenode/TestDecommissioningStatus.java     |     6 +-
 .../TestDefaultBlockPlacementPolicy.java        |     4 +-
 .../server/namenode/TestEnabledECPolicies.java  |    10 +-
 .../hdfs/server/namenode/TestFSImage.java       |    87 +
 .../server/namenode/TestFSImageWithAcl.java     |    14 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |    17 +-
 .../hdfs/server/namenode/TestMetaSave.java      |     2 +
 .../server/namenode/TestNameNodeMXBean.java     |    14 +-
 .../namenode/TestNameNodeOptionParsing.java     |    27 +-
 .../namenode/TestNameNodeReconfigure.java       |    36 +
 .../namenode/TestNamenodeCapacityReport.java    |     8 +-
 .../namenode/TestReconstructStripedBlocks.java  |     4 +-
 .../hdfs/server/namenode/TestStartup.java       |     4 +-
 .../server/namenode/ha/TestEditLogTailer.java   |     4 +-
 .../namenode/ha/TestFailureToReadEdits.java     |     6 +-
 .../namenode/ha/TestInitializeSharedEdits.java  |     2 +-
 .../namenode/metrics/TestNameNodeMetrics.java   |   134 +-
 .../snapshot/TestOpenFilesWithSnapshot.java     |   347 +
 .../apache/hadoop/hdfs/web/TestJsonUtil.java    |    11 +-
 .../hadoop-hdfs/src/test/resources/editsStored  |   Bin 5850 -> 5850 bytes
 .../src/test/resources/editsStored.xml          |    62 +-
 .../test/resources/testErasureCodingConf.xml    |   113 +-
 hadoop-mapreduce-project/bin/mapred             |    18 +-
 .../jobhistory/JobHistoryEventHandler.java      |    27 +-
 .../hadoop/mapreduce/v2/app/AppContext.java     |     4 +
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |    14 +-
 .../mapreduce/v2/app/job/impl/JobImpl.java      |    35 +-
 .../mapreduce/v2/app/rm/RMCommunicator.java     |     4 +-
 .../mapreduce/v2/app/webapp/AppController.java  |    10 +-
 .../hadoop/mapreduce/v2/app/webapp/AppView.java |     4 +-
 .../mapreduce/v2/app/webapp/ConfBlock.java      |    36 +-
 .../mapreduce/v2/app/webapp/CountersBlock.java  |    38 +-
 .../mapreduce/v2/app/webapp/CountersPage.java   |     4 +-
 .../mapreduce/v2/app/webapp/InfoPage.java       |     2 +-
 .../mapreduce/v2/app/webapp/JobBlock.java       |    95 +-
 .../mapreduce/v2/app/webapp/JobConfPage.java    |     3 +-
 .../hadoop/mapreduce/v2/app/webapp/JobPage.java |     2 +-
 .../mapreduce/v2/app/webapp/JobsBlock.java      |    24 +-
 .../mapreduce/v2/app/webapp/NavBlock.java       |    38 +-
 .../v2/app/webapp/SingleCounterBlock.java       |    28 +-
 .../v2/app/webapp/SingleCounterPage.java        |     3 +-
 .../mapreduce/v2/app/webapp/TaskPage.java       |    20 +-
 .../mapreduce/v2/app/webapp/TasksBlock.java     |    12 +-
 .../mapreduce/v2/app/webapp/TasksPage.java      |     2 +-
 .../jobhistory/TestJobHistoryEventHandler.java  |   102 +
 .../hadoop/mapreduce/v2/app/MockAppContext.java |    10 +
 .../mapreduce/v2/app/TestJobEndNotifier.java    |    16 +-
 .../mapreduce/v2/app/TestRuntimeEstimators.java |    10 +
 .../mapreduce/v2/app/job/impl/TestJobImpl.java  |   139 +-
 .../v2/app/launcher/TestContainerLauncher.java  |     9 +
 .../app/launcher/TestContainerLauncherImpl.java |     9 +
 .../v2/app/rm/TestRMContainerAllocator.java     |     3 +-
 .../hadoop-mapreduce-client-common/pom.xml      |     4 -
 .../org/apache/hadoop/mapreduce/Cluster.java    |    16 +-
 .../apache/hadoop/mapreduce/JobSubmitter.java   |     2 -
 .../apache/hadoop/mapreduce/MRJobConfig.java    |     6 +-
 .../mapreduce/counters/AbstractCounters.java    |     4 -
 .../hadoop/mapreduce/counters/Limits.java       |    12 -
 .../ClientDistributedCacheManager.java          |     6 +-
 .../mapreduce/jobhistory/HistoryViewer.java     |    16 -
 .../TaskAttemptUnsuccessfulCompletionEvent.java |    28 +-
 .../src/main/resources/mapred-default.xml       |     8 +
 .../org/apache/hadoop/mapred/TestQueue.java     |     8 +-
 .../hadoop/mapreduce/v2/hs/CompletedJob.java    |    15 -
 .../hadoop/mapreduce/v2/hs/JobHistory.java      |    10 +
 .../mapreduce/v2/hs/webapp/HsAboutPage.java     |     7 +-
 .../mapreduce/v2/hs/webapp/HsConfPage.java      |     2 +-
 .../mapreduce/v2/hs/webapp/HsCountersPage.java  |     4 +-
 .../mapreduce/v2/hs/webapp/HsJobBlock.java      |    98 +-
 .../mapreduce/v2/hs/webapp/HsJobPage.java       |     2 +-
 .../mapreduce/v2/hs/webapp/HsJobsBlock.java     |    46 +-
 .../mapreduce/v2/hs/webapp/HsLogsPage.java      |     2 +-
 .../mapreduce/v2/hs/webapp/HsNavBlock.java      |    30 +-
 .../v2/hs/webapp/HsSingleCounterPage.java       |     2 +-
 .../mapreduce/v2/hs/webapp/HsTaskPage.java      |    53 +-
 .../mapreduce/v2/hs/webapp/HsTasksBlock.java    |    57 +-
 .../mapreduce/v2/hs/webapp/HsTasksPage.java     |     2 +-
 .../hadoop/mapreduce/v2/hs/webapp/HsView.java   |     4 +-
 .../java/org/apache/hadoop/io/FileBench.java    |     2 +-
 .../mapred/MiniMRClientClusterFactory.java      |     4 +-
 .../mapred/TestCombineFileInputFormat.java      |     6 +-
 .../TestCombineSequenceFileInputFormat.java     |     7 +-
 .../mapred/TestCombineTextInputFormat.java      |     7 +-
 .../mapred/TestConcatenatedCompressedInput.java |     6 +-
 .../org/apache/hadoop/mapred/TestMapRed.java    |     4 +-
 .../hadoop/mapred/TestMiniMRChildTask.java      |     4 +-
 .../hadoop/mapred/TestTextInputFormat.java      |     8 +-
 .../apache/hadoop/mapred/TestYARNRunner.java    |     8 +-
 .../TestWrappedRecordReaderClassloader.java     |     4 +-
 .../apache/hadoop/mapreduce/TestCounters.java   |    39 +-
 .../lib/input/TestCombineFileInputFormat.java   |     2 +-
 .../mapreduce/lib/input/TestMultipleInputs.java |    10 +-
 .../lib/join/TestWrappedRRClassloader.java      |     4 +-
 .../TestUmbilicalProtocolWithJobToken.java      |    22 +-
 .../mapreduce/util/MRAsyncDiskService.java      |     2 +-
 .../apache/hadoop/mapreduce/v2/TestMRJobs.java  |    91 +-
 .../v2/TestMRJobsWithHistoryService.java        |     4 +-
 .../hadoop/mapred/FadvisedFileRegion.java       |     5 +-
 .../hadoop-mapreduce-client/pom.xml             |    10 +-
 hadoop-maven-plugins/pom.xml                    |     5 +-
 hadoop-project-dist/pom.xml                     |    12 +-
 hadoop-project/pom.xml                          |    49 +-
 hadoop-project/src/site/site.xml                |     1 +
 .../fs/aliyun/oss/AliyunOSSFileSystemStore.java |     4 +
 .../main/shellprofile.d/hadoop-archive-logs.sh  |     2 +-
 .../org/apache/hadoop/tools/HadoopArchives.java |     2 +-
 .../src/main/shellprofile.d/hadoop-archives.sh  |     4 +-
 hadoop-tools/hadoop-aws/pom.xml                 |    22 +-
 hadoop-tools/hadoop-azure-datalake/pom.xml      |     2 +-
 .../org/apache/hadoop/fs/adl/AdlConfKeys.java   |     8 +
 .../org/apache/hadoop/fs/adl/AdlFileStatus.java |    69 +
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java |    50 +-
 .../apache/hadoop/fs/adl/TokenProviderType.java |     2 +
 .../src/site/markdown/index.md                  |    98 +-
 .../hadoop/fs/adl/TestAzureADTokenProvider.java |    40 +
 .../apache/hadoop/fs/adl/TestGetFileStatus.java |    58 +-
 .../apache/hadoop/fs/adl/TestListStatus.java    |     8 +-
 .../fs/azure/AzureNativeFileSystemStore.java    |   123 +-
 .../fs/azure/BlobOperationDescriptor.java       |   222 +
 .../hadoop/fs/azure/BlockBlobInputStream.java   |    91 +-
 .../fs/azure/ClientThrottlingAnalyzer.java      |   284 +
 .../fs/azure/ClientThrottlingIntercept.java     |   221 +
 .../hadoop/fs/azure/NativeAzureFileSystem.java  |    47 +-
 .../services/org.apache.hadoop.fs.FileSystem    |    17 -
 .../hadoop/fs/azure/AbstractWasbTestBase.java   |     4 +
 .../fs/azure/TestBlobOperationDescriptor.java   |   305 +
 .../fs/azure/TestBlockBlobInputStream.java      |    85 +-
 .../fs/azure/TestClientThrottlingAnalyzer.java  |   177 +
 .../TestFileSystemOperationsWithThreads.java    |    61 +-
 .../azure/TestOutOfBandAzureBlobOperations.java |     8 +-
 .../fs/azure/TestWasbRemoteCallHelper.java      |     7 +-
 .../src/test/resources/azure-test.xml           |    11 +-
 .../hadoop/tools/CopyListingFileStatus.java     |     4 +-
 .../apache/hadoop/tools/util/DistCpUtils.java   |     4 +-
 .../src/main/shellprofile.d/hadoop-distcp.sh    |     4 +-
 .../src/main/shellprofile.d/hadoop-extras.sh    |     2 +-
 .../apache/hadoop/mapred/gridmix/Gridmix.java   |     2 +-
 .../hadoop/mapred/gridmix/PseudoLocalFs.java    |     8 +-
 .../src/main/shellprofile.d/hadoop-gridmix.sh   |     2 +-
 .../hadoop/mapred/gridmix/TestFilePool.java     |     4 +-
 .../hadoop/mapred/gridmix/TestFileQueue.java    |     8 +-
 .../mapred/gridmix/TestPseudoLocalFs.java       |     2 +-
 .../hadoop/mapred/gridmix/TestUserResolve.java  |     4 +-
 .../fs/swift/snative/SwiftFileStatus.java       |    16 +-
 .../snative/SwiftNativeFileSystemStore.java     |     4 +-
 .../hadoop/fs/swift/util/SwiftTestUtils.java    |     2 +-
 .../fs/swift/SwiftFileSystemBaseTest.java       |     2 +-
 .../swift/TestSwiftFileSystemDirectories.java   |     4 +-
 .../TestSwiftFileSystemPartitionedUploads.java  |     6 +-
 .../src/main/shellprofile.d/hadoop-rumen.sh     |     4 +-
 .../hadoop/tools/rumen/TestHistograms.java      |     6 +-
 .../hadoop/yarn/sls/nodemanager/NodeInfo.java   |     2 +-
 .../yarn/sls/scheduler/RMNodeWrapper.java       |     2 +-
 .../org/apache/hadoop/streaming/StreamJob.java  |    27 +-
 .../src/main/shellprofile.d/hadoop-streaming.sh |     2 +-
 .../MySQL/FederationStateStoreDatabase.sql      |    21 +
 .../MySQL/FederationStateStoreStoredProcs.sql   |   162 +
 .../MySQL/FederationStateStoreTables.sql        |    47 +
 .../MySQL/FederationStateStoreUser.sql          |    25 +
 .../FederationStateStore/MySQL/dropDatabase.sql |    21 +
 .../MySQL/dropStoreProcedures.sql               |    47 +
 .../FederationStateStore/MySQL/dropTables.sql   |    27 +
 .../bin/FederationStateStore/MySQL/dropUser.sql |    21 +
 .../FederationStateStoreStoreProcs.sql          |   511 +
 .../SQLServer/FederationStateStoreTables.sql    |   122 +
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |    47 +-
 hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd    |    13 +-
 .../hadoop-yarn/conf/yarn-env.sh                |    12 +
 .../dev-support/findbugs-exclude.xml            |    22 +-
 .../yarn/api/ContainerManagementProtocol.java   |    23 +
 .../protocolrecords/ContainerUpdateRequest.java |    77 +
 .../ContainerUpdateResponse.java                |    95 +
 .../org/apache/hadoop/yarn/conf/HAUtil.java     |    30 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |   190 +-
 .../proto/containermanagement_protocol.proto    |     1 +
 .../src/main/proto/yarn_service_protos.proto    |     9 +
 .../yarn/conf/TestYarnConfigurationFields.java  |    60 +
 .../yarn/client/api/impl/NMClientImpl.java      |    18 +-
 .../TestFederationRMFailoverProxyProvider.java  |   223 +
 .../yarn/client/api/impl/TestAMRMClient.java    |   401 +-
 .../yarn/client/api/impl/TestAMRMProxy.java     |    10 +-
 .../yarn/client/api/impl/TestNMClient.java      |    44 +-
 .../yarn/client/api/impl/TestYarnClient.java    |     9 +-
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |    22 +
 ...ContainerManagementProtocolPBClientImpl.java |    40 +-
 ...ontainerManagementProtocolPBServiceImpl.java |    28 +-
 .../impl/pb/ContainerUpdateRequestPBImpl.java   |   171 +
 .../impl/pb/ContainerUpdateResponsePBImpl.java  |   241 +
 .../hadoop/yarn/client/ClientRMProxy.java       |     4 +-
 .../org/apache/hadoop/yarn/client/RMProxy.java  |    11 +-
 .../api/impl/FileSystemTimelineWriter.java      |    40 +-
 .../apache/hadoop/yarn/util/AsyncCallback.java  |    35 +
 .../hadoop/yarn/util/LRUCacheHashMap.java       |    49 +
 .../apache/hadoop/yarn/webapp/ResponseInfo.java |     4 +-
 .../hadoop/yarn/webapp/example/HelloWorld.java  |     4 +-
 .../hadoop/yarn/webapp/example/MyApp.java       |     6 +-
 .../hadoop/yarn/webapp/hamlet/Hamlet.java       |     4 +
 .../hadoop/yarn/webapp/hamlet/HamletGen.java    |     2 +
 .../hadoop/yarn/webapp/hamlet/HamletImpl.java   |     2 +
 .../hadoop/yarn/webapp/hamlet/HamletSpec.java   |     2 +
 .../hadoop/yarn/webapp/hamlet/package-info.java |     6 +
 .../hadoop/yarn/webapp/hamlet2/Hamlet.java      | 30557 +++++++++++++++++
 .../hadoop/yarn/webapp/hamlet2/HamletGen.java   |   449 +
 .../hadoop/yarn/webapp/hamlet2/HamletImpl.java  |   385 +
 .../hadoop/yarn/webapp/hamlet2/HamletSpec.java  |  3101 ++
 .../yarn/webapp/hamlet2/package-info.java       |    27 +
 .../yarn/webapp/log/AggregatedLogsBlock.java    |    66 +-
 .../yarn/webapp/log/AggregatedLogsNavBlock.java |     4 +-
 .../yarn/webapp/log/AggregatedLogsPage.java     |     2 +-
 .../hadoop/yarn/webapp/util/WebAppUtils.java    |    14 +
 .../hadoop/yarn/webapp/view/ErrorPage.java      |    12 +-
 .../hadoop/yarn/webapp/view/FooterBlock.java    |     2 +-
 .../hadoop/yarn/webapp/view/HeaderBlock.java    |     6 +-
 .../hadoop/yarn/webapp/view/HtmlBlock.java      |     2 +-
 .../hadoop/yarn/webapp/view/HtmlPage.java       |    12 +-
 .../hadoop/yarn/webapp/view/InfoBlock.java      |    28 +-
 .../hadoop/yarn/webapp/view/JQueryUI.java       |    14 +-
 .../hadoop/yarn/webapp/view/LipsumBlock.java    |     4 +-
 .../hadoop/yarn/webapp/view/NavBlock.java       |    10 +-
 .../yarn/webapp/view/TwoColumnCssLayout.java    |    20 +-
 .../yarn/webapp/view/TwoColumnLayout.java       |    20 +-
 .../src/main/resources/yarn-default.xml         |   224 +-
 .../hadoop/yarn/TestContainerLaunchRPC.java     |     9 +
 .../yarn/TestContainerResourceIncreaseRPC.java  |    20 +-
 .../hadoop/yarn/api/TestPBImplRecords.java      |     4 +
 .../api/impl/TestTimelineClientForATS1_5.java   |    81 +-
 .../hadoop/yarn/conf/TestYarnConfiguration.java |     1 -
 .../hadoop/yarn/util/TestLRUCacheHashMap.java   |    74 +
 .../yarn/util/TestProcfsBasedProcessTree.java   |     2 +-
 .../hadoop/yarn/util/TestYarnVersionInfo.java   |    11 +-
 .../apache/hadoop/yarn/webapp/TestSubViews.java |    10 +-
 .../apache/hadoop/yarn/webapp/TestWebApp.java   |    28 +-
 .../hadoop/yarn/webapp/view/TestHtmlBlock.java  |    10 +-
 .../hadoop/yarn/webapp/view/TestHtmlPage.java   |     9 +-
 .../hadoop/yarn/webapp/view/TestInfoBlock.java  |     6 +-
 .../yarn/webapp/view/TestTwoColumnCssPage.java  |     6 +-
 .../pom.xml                                     |     4 -
 .../ApplicationHistoryClientService.java        |     8 +-
 .../ApplicationHistoryManagerImpl.java          |     8 +-
 ...pplicationHistoryManagerOnTimelineStore.java |     8 +-
 .../ApplicationHistoryServer.java               |    10 +-
 .../FileSystemApplicationHistoryStore.java      |    22 +-
 .../webapp/AHSErrorsAndWarningsPage.java        |     2 +-
 .../webapp/AHSLogsPage.java                     |     2 +-
 .../webapp/AHSView.java                         |     4 +-
 .../webapp/AHSWebServices.java                  |     7 +-
 .../webapp/AboutBlock.java                      |     8 +-
 .../webapp/AboutPage.java                       |     5 +-
 .../webapp/AppAttemptPage.java                  |     2 +-
 .../webapp/AppPage.java                         |     2 +-
 .../webapp/ContainerPage.java                   |     2 +-
 .../webapp/NavBlock.java                        |    36 +-
 .../timeline/KeyValueBasedTimelineStore.java    |     8 +-
 .../server/timeline/LeveldbTimelineStore.java   |    35 +-
 .../yarn/server/timeline/RollingLevelDB.java    |    15 +-
 .../timeline/RollingLevelDBTimelineStore.java   |    22 +-
 .../server/timeline/TimelineDataManager.java    |     7 +-
 .../recovery/LeveldbTimelineStateStore.java     |    30 +-
 .../timeline/security/TimelineACLsManager.java  |     7 +-
 ...lineDelegationTokenSecretManagerService.java |     8 +-
 .../timeline/webapp/TimelineWebServices.java    |     7 +-
 .../TestFileSystemApplicationHistoryStore.java  |     8 +-
 .../timeline/TestLeveldbTimelineStore.java      |     2 +-
 .../hadoop-yarn-server-common/pom.xml           |    62 +
 .../hadoop/yarn/server/api/ServerRMProxy.java   |     4 +-
 .../protocolrecords/NodeHeartbeatResponse.java  |     6 +-
 .../impl/pb/NodeHeartbeatResponsePBImpl.java    |    42 +-
 .../failover/FederationProxyProviderUtil.java   |   133 +
 .../FederationRMFailoverProxyProvider.java      |   221 +
 .../federation/failover/package-info.java       |    17 +
 .../AbstractConfigurableFederationPolicy.java   |   155 +
 .../policies/ConfigurableFederationPolicy.java  |    41 +
 .../FederationPolicyInitializationContext.java  |   130 +
 ...ionPolicyInitializationContextValidator.java |    76 +
 .../policies/FederationPolicyUtils.java         |   203 +
 .../federation/policies/RouterPolicyFacade.java |   265 +
 .../amrmproxy/AbstractAMRMProxyPolicy.java      |    47 +
 .../amrmproxy/BroadcastAMRMProxyPolicy.java     |    85 +
 .../amrmproxy/FederationAMRMProxyPolicy.java    |    65 +
 .../LocalityMulticastAMRMProxyPolicy.java       |   608 +
 .../amrmproxy/RejectAMRMProxyPolicy.java        |    67 +
 .../policies/amrmproxy/package-info.java        |    19 +
 .../policies/dao/WeightedPolicyInfo.java        |   251 +
 .../federation/policies/dao/package-info.java   |    19 +
 .../exceptions/FederationPolicyException.java   |    33 +
 ...FederationPolicyInitializationException.java |    33 +
 .../NoActiveSubclustersException.java           |    27 +
 .../exceptions/UnknownSubclusterException.java  |    28 +
 .../policies/exceptions/package-info.java       |    19 +
 .../policies/manager/AbstractPolicyManager.java |   190 +
 .../manager/FederationPolicyManager.java        |   118 +
 .../manager/HashBroadcastPolicyManager.java     |    38 +
 .../manager/PriorityBroadcastPolicyManager.java |    66 +
 .../manager/RejectAllPolicyManager.java         |    40 +
 .../manager/UniformBroadcastPolicyManager.java  |    44 +
 .../manager/WeightedLocalityPolicyManager.java  |    67 +
 .../policies/manager/package-info.java          |    19 +
 .../federation/policies/package-info.java       |    19 +
 .../policies/router/AbstractRouterPolicy.java   |    66 +
 .../policies/router/FederationRouterPolicy.java |    52 +
 .../policies/router/HashBasedRouterPolicy.java  |   102 +
 .../policies/router/LoadBasedRouterPolicy.java  |   112 +
 .../policies/router/PriorityRouterPolicy.java   |    72 +
 .../policies/router/RejectRouterPolicy.java     |    76 +
 .../router/UniformRandomRouterPolicy.java       |   104 +
 .../router/WeightedRandomRouterPolicy.java      |    94 +
 .../policies/router/package-info.java           |    19 +
 .../resolver/AbstractSubClusterResolver.java    |    67 +
 .../resolver/DefaultSubClusterResolverImpl.java |   164 +
 .../federation/resolver/SubClusterResolver.java |    58 +
 .../federation/resolver/package-info.java       |    17 +
 ...ederationApplicationHomeSubClusterStore.java |   121 +
 .../store/FederationMembershipStateStore.java   |   117 +
 .../federation/store/FederationPolicyStore.java |    77 +
 .../federation/store/FederationStateStore.java  |    64 +
 .../FederationStateStoreException.java          |    50 +
 ...derationStateStoreInvalidInputException.java |    48 +
 .../FederationStateStoreRetriableException.java |    44 +
 .../store/exception/package-info.java           |    17 +
 .../store/impl/MemoryFederationStateStore.java  |   315 +
 .../store/impl/SQLFederationStateStore.java     |   942 +
 .../impl/ZookeeperFederationStateStore.java     |   634 +
 .../federation/store/impl/package-info.java     |    17 +
 .../server/federation/store/package-info.java   |    17 +
 .../AddApplicationHomeSubClusterRequest.java    |    72 +
 .../AddApplicationHomeSubClusterResponse.java   |    65 +
 .../records/ApplicationHomeSubCluster.java      |   124 +
 .../DeleteApplicationHomeSubClusterRequest.java |    65 +
 ...DeleteApplicationHomeSubClusterResponse.java |    43 +
 .../GetApplicationHomeSubClusterRequest.java    |    64 +
 .../GetApplicationHomeSubClusterResponse.java   |    73 +
 .../GetApplicationsHomeSubClusterRequest.java   |    40 +
 .../GetApplicationsHomeSubClusterResponse.java  |    75 +
 .../store/records/GetSubClusterInfoRequest.java |    62 +
 .../records/GetSubClusterInfoResponse.java      |    62 +
 ...SubClusterPoliciesConfigurationsRequest.java |    35 +
 ...ubClusterPoliciesConfigurationsResponse.java |    66 +
 ...GetSubClusterPolicyConfigurationRequest.java |    63 +
 ...etSubClusterPolicyConfigurationResponse.java |    65 +
 .../records/GetSubClustersInfoRequest.java      |    70 +
 .../records/GetSubClustersInfoResponse.java     |    66 +
 ...SetSubClusterPolicyConfigurationRequest.java |    61 +
 ...etSubClusterPolicyConfigurationResponse.java |    36 +
 .../records/SubClusterDeregisterRequest.java    |    89 +
 .../records/SubClusterDeregisterResponse.java   |    42 +
 .../records/SubClusterHeartbeatRequest.java     |   149 +
 .../records/SubClusterHeartbeatResponse.java    |    45 +
 .../federation/store/records/SubClusterId.java  |   100 +
 .../store/records/SubClusterIdInfo.java         |    75 +
 .../store/records/SubClusterInfo.java           |   325 +
 .../records/SubClusterPolicyConfiguration.java  |   162 +
 .../records/SubClusterRegisterRequest.java      |    74 +
 .../records/SubClusterRegisterResponse.java     |    44 +
 .../store/records/SubClusterState.java          |    85 +
 .../UpdateApplicationHomeSubClusterRequest.java |    74 +
 ...UpdateApplicationHomeSubClusterResponse.java |    43 +
 ...dApplicationHomeSubClusterRequestPBImpl.java |   132 +
 ...ApplicationHomeSubClusterResponsePBImpl.java |   117 +
 .../pb/ApplicationHomeSubClusterPBImpl.java     |   167 +
 ...eApplicationHomeSubClusterRequestPBImpl.java |   130 +
 ...ApplicationHomeSubClusterResponsePBImpl.java |    78 +
 ...tApplicationHomeSubClusterRequestPBImpl.java |   139 +
 ...ApplicationHomeSubClusterResponsePBImpl.java |   132 +
 ...ApplicationsHomeSubClusterRequestPBImpl.java |    78 +
 ...pplicationsHomeSubClusterResponsePBImpl.java |   190 +
 .../impl/pb/GetSubClusterInfoRequestPBImpl.java |   125 +
 .../pb/GetSubClusterInfoResponsePBImpl.java     |   134 +
 ...sterPoliciesConfigurationsRequestPBImpl.java |    95 +
 ...terPoliciesConfigurationsResponsePBImpl.java |   191 +
 ...ClusterPolicyConfigurationRequestPBImpl.java |   103 +
 ...lusterPolicyConfigurationResponsePBImpl.java |   143 +
 .../pb/GetSubClustersInfoRequestPBImpl.java     |   108 +
 .../pb/GetSubClustersInfoResponsePBImpl.java    |   184 +
 ...ClusterPolicyConfigurationRequestPBImpl.java |   142 +
 ...lusterPolicyConfigurationResponsePBImpl.java |    93 +
 .../pb/SubClusterDeregisterRequestPBImpl.java   |   156 +
 .../pb/SubClusterDeregisterResponsePBImpl.java  |    77 +
 .../pb/SubClusterHeartbeatRequestPBImpl.java    |   192 +
 .../pb/SubClusterHeartbeatResponsePBImpl.java   |    77 +
 .../records/impl/pb/SubClusterIdPBImpl.java     |    75 +
 .../records/impl/pb/SubClusterInfoPBImpl.java   |   251 +
 .../pb/SubClusterPolicyConfigurationPBImpl.java |   138 +
 .../pb/SubClusterRegisterRequestPBImpl.java     |   134 +
 .../pb/SubClusterRegisterResponsePBImpl.java    |    77 +
 ...eApplicationHomeSubClusterRequestPBImpl.java |   132 +
 ...ApplicationHomeSubClusterResponsePBImpl.java |    78 +
 .../store/records/impl/pb/package-info.java     |    17 +
 .../federation/store/records/package-info.java  |    17 +
 ...cationHomeSubClusterStoreInputValidator.java |   180 +
 ...ationMembershipStateStoreInputValidator.java |   315 +
 .../FederationPolicyStoreInputValidator.java    |   143 +
 .../store/utils/FederationStateStoreUtils.java  |   214 +
 .../federation/store/utils/package-info.java    |    17 +
 .../utils/FederationStateStoreFacade.java       |   602 +
 .../server/federation/utils/package-info.java   |    17 +
 .../yarn/server/uam/UnmanagedAMPoolManager.java |   311 +
 .../server/uam/UnmanagedApplicationManager.java |   607 +
 .../hadoop/yarn/server/uam/package-info.java    |    18 +
 .../yarn/server/utils/AMRMClientUtils.java      |   189 +
 .../server/utils/YarnServerSecurityUtils.java   |    41 +-
 .../yarn/server/webapp/AppAttemptBlock.java     |    31 +-
 .../hadoop/yarn/server/webapp/AppBlock.java     |    80 +-
 .../hadoop/yarn/server/webapp/AppsBlock.java    |    14 +-
 .../yarn/server/webapp/ContainerBlock.java      |    23 +-
 .../server/webapp/ErrorsAndWarningsBlock.java   |    50 +-
 .../yarn_server_common_service_protos.proto     |     3 +
 .../proto/yarn_server_federation_protos.proto   |   165 +
 .../java/org/apache/hadoop/yarn/TestRPC.java    |     9 +
 .../hadoop/yarn/TestYarnServerApiClasses.java   |     6 +-
 .../yarn/server/MockResourceManagerFacade.java  |   855 +
 .../policies/BaseFederationPoliciesTest.java    |   186 +
 ...ionPolicyInitializationContextValidator.java |   138 +
 .../policies/TestRouterPolicyFacade.java        |   222 +
 .../TestBroadcastAMRMProxyFederationPolicy.java |   112 +
 .../TestLocalityMulticastAMRMProxyPolicy.java   |   602 +
 .../amrmproxy/TestRejectAMRMProxyPolicy.java    |    78 +
 .../policies/manager/BasePolicyManagerTest.java |   101 +
 .../TestHashBasedBroadcastPolicyManager.java    |    40 +
 .../TestPriorityBroadcastPolicyManager.java     |    72 +
 .../manager/TestRejectAllPolicyManager.java     |    40 +
 .../TestUniformBroadcastPolicyManager.java      |    40 +
 .../TestWeightedLocalityPolicyManager.java      |    79 +
 .../policies/router/BaseRouterPoliciesTest.java |   118 +
 .../router/TestHashBasedRouterPolicy.java       |    83 +
 .../router/TestLoadBasedRouterPolicy.java       |   106 +
 .../router/TestPriorityRouterPolicy.java        |    85 +
 .../policies/router/TestRejectRouterPolicy.java |    63 +
 .../router/TestUniformRandomRouterPolicy.java   |    64 +
 .../router/TestWeightedRandomRouterPolicy.java  |   137 +
 .../resolver/TestDefaultSubClusterResolver.java |   189 +
 .../impl/FederationStateStoreBaseTest.java      |   578 +
 .../store/impl/HSQLDBFederationStateStore.java  |   252 +
 .../impl/TestMemoryFederationStateStore.java    |    35 +
 .../store/impl/TestSQLFederationStateStore.java |    49 +
 .../impl/TestZookeeperFederationStateStore.java |    89 +
 .../records/TestFederationProtocolRecords.java  |   265 +
 .../TestFederationStateStoreInputValidator.java |  1260 +
 .../utils/FederationPoliciesTestUtil.java       |   227 +
 .../utils/FederationStateStoreTestUtil.java     |   181 +
 .../utils/TestFederationStateStoreFacade.java   |   192 +
 .../TestFederationStateStoreFacadeRetry.java    |   140 +
 .../uam/TestUnmanagedApplicationManager.java    |   335 +
 .../src/test/resources/nodes                    |     8 +
 .../src/test/resources/nodes-malformed          |     3 +
 .../hadoop-yarn-server-nodemanager/pom.xml      |    45 +
 .../src/CMakeLists.txt                          |    33 +
 .../CMgrDecreaseContainersResourceEvent.java    |    37 -
 .../nodemanager/CMgrUpdateContainersEvent.java  |    48 +
 .../nodemanager/ContainerManagerEventType.java  |     2 +-
 .../server/nodemanager/DirectoryCollection.java |    61 +-
 .../nodemanager/LinuxContainerExecutor.java     |     2 +-
 .../nodemanager/LocalDirsHandlerService.java    |    59 +-
 .../nodemanager/NodeStatusUpdaterImpl.java      |    19 +-
 .../amrmproxy/DefaultRequestInterceptor.java    |    30 +-
 .../amrmproxy/FederationInterceptor.java        |  1150 +
 .../containermanager/ContainerManagerImpl.java  |   180 +-
 .../containermanager/container/Container.java   |     4 +-
 .../container/ContainerImpl.java                |    41 +-
 .../CGroupsCpuResourceHandlerImpl.java          |    20 +-
 .../linux/resources/CGroupsHandler.java         |    15 +
 .../linux/resources/CGroupsHandlerImpl.java     |    26 +-
 .../CGroupsMemoryResourceHandlerImpl.java       |    35 +-
 .../linux/resources/ResourceHandlerModule.java  |    58 +-
 .../localizer/ContainerLocalizer.java           |    15 +-
 .../monitor/ContainerMetrics.java               |     2 +-
 .../monitor/ContainersMonitorImpl.java          |    15 -
 ...locationBasedResourceUtilizationTracker.java |    31 +-
 .../scheduler/ContainerScheduler.java           |    83 +-
 .../scheduler/ContainerSchedulerEventType.java  |     1 +
 .../UpdateContainerSchedulerEvent.java          |    85 +
 .../util/CgroupsLCEResourcesHandler.java        |    53 +-
 .../nodemanager/webapp/AllApplicationsPage.java |    28 +-
 .../nodemanager/webapp/AllContainersPage.java   |    30 +-
 .../nodemanager/webapp/ApplicationPage.java     |    25 +-
 .../nodemanager/webapp/ContainerLogsPage.java   |    17 +-
 .../nodemanager/webapp/ContainerPage.java       |    31 +-
 .../webapp/NMErrorsAndWarningsPage.java         |     2 +-
 .../yarn/server/nodemanager/webapp/NMView.java  |     4 +-
 .../server/nodemanager/webapp/NavBlock.java     |    28 +-
 .../server/nodemanager/webapp/NodePage.java     |    28 +-
 .../container-executor/impl/configuration.c     |   672 +-
 .../container-executor/impl/configuration.h     |   182 +-
 .../impl/container-executor.c                   |   204 +-
 .../impl/container-executor.h                   |    63 +-
 .../container-executor/impl/get_executable.c    |     1 +
 .../main/native/container-executor/impl/main.c  |    30 +-
 .../impl/modules/cgroups/cgroups-operations.c   |   161 +
 .../impl/modules/cgroups/cgroups-operations.h   |    55 +
 .../impl/modules/common/constants.h             |    29 +
 .../impl/modules/common/module-configs.c        |    41 +
 .../impl/modules/common/module-configs.h        |    33 +
 .../impl/modules/gpu/gpu-module.c               |   229 +
 .../impl/modules/gpu/gpu-module.h               |    45 +
 .../main/native/container-executor/impl/util.c  |   134 +
 .../main/native/container-executor/impl/util.h  |   122 +
 .../container-executor/impl/utils/path-utils.c  |    52 +
 .../container-executor/impl/utils/path-utils.h  |    35 +
 .../impl/utils/string-utils.c                   |   159 +
 .../impl/utils/string-utils.h                   |    37 +
 .../test-configurations/configuration-1.cfg     |    31 +
 .../test-configurations/configuration-2.cfg     |    28 +
 .../test/test-configurations/old-config.cfg     |    25 +
 .../test/modules/cgroups/test-cgroups-module.cc |   121 +
 .../test/modules/gpu/test-gpu-module.cc         |   203 +
 .../test/test-container-executor-common.h       |    36 +
 .../test/test-container-executor.c              |   246 +-
 .../test/test_configuration.cc                  |   432 +
 .../native/container-executor/test/test_main.cc |    32 +
 .../native/container-executor/test/test_util.cc |   138 +
 .../test/utils/test-path-utils.cc               |    67 +
 .../test/utils/test-string-utils.cc             |    93 +
 .../nodemanager/TestDirectoryCollection.java    |    23 +
 .../nodemanager/TestNodeManagerResync.java      |    41 +-
 .../amrmproxy/MockRequestInterceptor.java       |    14 +-
 .../amrmproxy/MockResourceManagerFacade.java    |   514 -
 .../amrmproxy/TestAMRMProxyService.java         |     1 +
 .../amrmproxy/TestFederationInterceptor.java    |   496 +
 .../TestableFederationInterceptor.java          |   139 +
 .../BaseContainerManagerTest.java               |    43 +-
 .../containermanager/TestAuxServices.java       |     2 +-
 .../containermanager/TestContainerManager.java  |   308 +-
 .../TestContainerManagerRecovery.java           |    24 +-
 .../TestCGroupsCpuResourceHandlerImpl.java      |    23 +
 .../linux/resources/TestCGroupsHandlerImpl.java |    25 +
 .../TestCGroupsMemoryResourceHandlerImpl.java   |    45 +-
 ...locationBasedResourceUtilizationTracker.java |    93 +
 .../TestContainerSchedulerQueuing.java          |    96 +
 .../util/TestCgroupsLCEResourcesHandler.java    |    33 +-
 .../nodemanager/webapp/MockContainer.java       |     2 +-
 ...ActiveStandbyElectorBasedElectorService.java |     5 +-
 .../ApplicationMasterService.java               |    13 +-
 .../server/resourcemanager/RMAppManager.java    |    56 +-
 .../yarn/server/resourcemanager/RMZKUtils.java  |    81 -
 .../server/resourcemanager/ResourceManager.java |   125 +-
 .../resourcemanager/ResourceTrackerService.java |     2 +-
 .../FederationStateStoreHeartbeat.java          |   108 +
 .../federation/FederationStateStoreService.java |   304 +
 .../federation/package-info.java                |    17 +
 .../monitor/SchedulingEditPolicy.java           |     4 +-
 .../monitor/SchedulingMonitor.java              |     4 +-
 .../ProportionalCapacityPreemptionPolicy.java   |     4 +-
 .../monitor/invariants/InvariantsChecker.java   |    10 +-
 .../invariants/MetricsInvariantChecker.java     |     7 +-
 .../recovery/FileSystemRMStateStore.java        |     2 +-
 .../recovery/LeveldbRMStateStore.java           |     2 +-
 .../recovery/MemoryRMStateStore.java            |     1 +
 .../resourcemanager/recovery/RMStateStore.java  |     5 +
 .../recovery/ZKRMStateStore.java                |    59 +-
 .../reservation/ReservationInputValidator.java  |    18 +-
 .../server/resourcemanager/rmapp/RMAppImpl.java |     3 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |    19 +-
 .../rmapp/attempt/RMAppAttemptMetrics.java      |    19 +-
 .../rmcontainer/RMContainerImpl.java            |     8 +-
 .../server/resourcemanager/rmnode/RMNode.java   |     6 +-
 .../rmnode/RMNodeDecreaseContainerEvent.java    |    39 -
 .../resourcemanager/rmnode/RMNodeEventType.java |     2 +-
 .../resourcemanager/rmnode/RMNodeImpl.java      |    29 +-
 .../rmnode/RMNodeUpdateContainerEvent.java      |    44 +
 .../scheduler/AbstractYarnScheduler.java        |    23 +-
 .../scheduler/AppSchedulingInfo.java            |     6 +-
 .../scheduler/SchedulerApplicationAttempt.java  |    43 +-
 .../scheduler/SchedulerUtils.java               |    17 +-
 .../scheduler/capacity/AbstractCSQueue.java     |     8 +
 .../scheduler/capacity/CapacityScheduler.java   |    66 +-
 .../CapacitySchedulerConfiguration.java         |    23 +
 .../capacity/CapacitySchedulerQueueManager.java |     4 +
 .../scheduler/capacity/LeafQueue.java           |     5 -
 .../scheduler/capacity/ParentQueue.java         |    45 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |    13 +
 .../scheduler/fair/AllocationConfiguration.java |     3 -
 .../fair/AllocationFileLoaderService.java       |    10 +-
 .../scheduler/fair/FSAppAttempt.java            |    17 +-
 .../scheduler/fair/FSPreemptionThread.java      |    28 +-
 .../scheduler/fair/FSSchedulerNode.java         |    68 +-
 .../scheduler/fair/FairScheduler.java           |    24 +-
 .../scheduler/fifo/FifoScheduler.java           |    15 +-
 .../security/RMContainerTokenSecretManager.java |    30 +-
 .../resourcemanager/webapp/AboutBlock.java      |    20 +-
 .../resourcemanager/webapp/AboutPage.java       |     2 +-
 .../resourcemanager/webapp/AppAttemptPage.java  |     2 +-
 .../webapp/AppLogAggregationStatusPage.java     |     2 +-
 .../server/resourcemanager/webapp/AppPage.java  |     2 +-
 .../webapp/AppsBlockWithMetrics.java            |     5 +-
 .../webapp/CapacitySchedulerPage.java           |   220 +-
 .../resourcemanager/webapp/ContainerPage.java   |     2 +-
 .../webapp/DefaultSchedulerPage.java            |    58 +-
 .../resourcemanager/webapp/ErrorBlock.java      |     2 +-
 .../webapp/FairSchedulerAppsBlock.java          |    12 +-
 .../webapp/FairSchedulerPage.java               |   118 +-
 .../webapp/MetricsOverviewTable.java            |   124 +-
 .../server/resourcemanager/webapp/NavBlock.java |    34 +-
 .../resourcemanager/webapp/NodeLabelsPage.java  |    18 +-
 .../resourcemanager/webapp/NodesPage.java       |    16 +-
 .../webapp/RMAppAttemptBlock.java               |    48 +-
 .../resourcemanager/webapp/RMAppBlock.java      |    28 +-
 .../webapp/RMAppLogAggregationStatusBlock.java  |    34 +-
 .../resourcemanager/webapp/RMAppsBlock.java     |    15 +-
 .../webapp/RMErrorsAndWarningsPage.java         |     2 +-
 .../resourcemanager/webapp/RMWSConsts.java      |    15 +
 .../resourcemanager/webapp/RMWebAppUtil.java    |   224 +
 .../webapp/RMWebServiceProtocol.java            |   133 +-
 .../resourcemanager/webapp/RMWebServices.java   |   183 +-
 .../webapp/RedirectionErrorPage.java            |     2 +-
 .../server/resourcemanager/webapp/RmView.java   |     4 +-
 .../webapp/SchedulerPageUtil.java               |     8 +-
 .../webapp/dao/AppAttemptInfo.java              |     5 +-
 .../webapp/dao/ClusterMetricsInfo.java          |     5 +-
 .../webapp/dao/FairSchedulerQueueInfo.java      |    41 +-
 .../yarn/server/resourcemanager/MockNodes.java  |     2 +-
 .../server/resourcemanager/NodeManager.java     |     9 +
 .../server/resourcemanager/RMHATestBase.java    |     5 +-
 .../resourcemanager/TestAMAuthorization.java    |     9 +
 .../TestApplicationMasterLauncher.java          |    15 +-
 .../resourcemanager/TestClientRMService.java    |     4 +-
 ...pportunisticContainerAllocatorAMService.java |   168 +
 .../server/resourcemanager/TestRMRestart.java   |     8 +-
 .../TestWorkPreservingRMRestart.java            |    10 +-
 .../TestFederationRMStateStoreService.java      |   176 +
 .../recovery/RMStateStoreTestBase.java          |     8 +-
 .../recovery/TestFSRMStateStore.java            |     1 +
 .../recovery/TestLeveldbRMStateStore.java       |     1 +
 .../recovery/TestZKRMStateStore.java            |     1 +
 .../TestReservationInputValidator.java          |   103 +-
 .../scheduler/TestSchedulerUtils.java           |     4 +-
 .../capacity/TestCapacityScheduler.java         |   360 +-
 .../TestCapacitySchedulerAsyncScheduling.java   |   147 +
 .../TestCapacitySchedulerDynamicBehavior.java   |     2 +-
 .../capacity/TestContainerResizing.java         |     7 +-
 .../capacity/TestIncreaseAllocationExpirer.java |     4 +-
 .../scheduler/capacity/TestLeafQueue.java       |     2 -
 .../scheduler/fair/TestFSSchedulerNode.java     |    52 +
 .../scheduler/fair/TestFairScheduler.java       |    46 +-
 .../hadoop/yarn/webapp/TestRMWithXFSFilter.java |    14 +-
 .../hadoop-yarn-server-router/pom.xml           |   110 +
 .../hadoop/yarn/server/router/Router.java       |   179 +
 .../yarn/server/router/RouterServerUtil.java    |    63 +
 .../AbstractClientRequestInterceptor.java       |   127 +
 .../clientrm/ClientRequestInterceptor.java      |    65 +
 .../DefaultClientRequestInterceptor.java        |   311 +
 .../clientrm/FederationClientInterceptor.java   |   677 +
 .../router/clientrm/RouterClientRMService.java  |   546 +
 .../server/router/clientrm/package-info.java    |    20 +
 .../hadoop/yarn/server/router/package-info.java |    20 +
 .../AbstractRMAdminRequestInterceptor.java      |    90 +
 .../DefaultRMAdminRequestInterceptor.java       |   215 +
 .../rmadmin/RMAdminRequestInterceptor.java      |    65 +
 .../router/rmadmin/RouterRMAdminService.java    |   423 +
 .../server/router/rmadmin/package-info.java     |    20 +
 .../webapp/AbstractRESTRequestInterceptor.java  |    89 +
 .../webapp/DefaultRequestInterceptorREST.java   |   510 +
 .../webapp/FederationInterceptorREST.java       |   750 +
 .../yarn/server/router/webapp/HTTPMethods.java  |    34 +
 .../router/webapp/RESTRequestInterceptor.java   |   125 +
 .../yarn/server/router/webapp/RouterWebApp.java |    48 +
 .../router/webapp/RouterWebServiceUtil.java     |   227 +
 .../server/router/webapp/RouterWebServices.java |   876 +
 .../yarn/server/router/webapp/package-info.java |    20 +
 .../router/clientrm/BaseRouterClientRMTest.java |   586 +
 .../clientrm/MockClientRequestInterceptor.java  |    36 +
 .../PassThroughClientRequestInterceptor.java    |   267 +
 .../TestFederationClientInterceptor.java        |   403 +
 .../TestFederationClientInterceptorRetry.java   |   295 +
 .../clientrm/TestRouterClientRMService.java     |   210 +
 .../TestableFederationClientInterceptor.java    |    75 +
 .../router/rmadmin/BaseRouterRMAdminTest.java   |   346 +
 .../rmadmin/MockRMAdminRequestInterceptor.java  |    36 +
 .../PassThroughRMAdminRequestInterceptor.java   |   148 +
 .../rmadmin/TestRouterRMAdminService.java       |   219 +
 .../webapp/BaseRouterWebServicesTest.java       |   614 +
 .../yarn/server/router/webapp/JavaProcess.java  |    52 +
 .../MockDefaultRequestInterceptorREST.java      |   136 +
 .../webapp/MockRESTRequestInterceptor.java      |   340 +
 .../PassThroughRESTRequestInterceptor.java      |   339 +
 .../webapp/TestFederationInterceptorREST.java   |   379 +
 .../TestFederationInterceptorRESTRetry.java     |   274 +
 .../router/webapp/TestRouterWebServices.java    |   269 +
 .../webapp/TestRouterWebServicesREST.java       |  1298 +
 .../TestableFederationInterceptorREST.java      |    54 +
 .../src/test/resources/capacity-scheduler.xml   |   111 +
 .../src/test/resources/log4j.properties         |    19 +
 .../src/test/resources/yarn-site.xml            |    30 +
 .../sharedcachemanager/CleanerService.java      |     7 +-
 .../server/sharedcachemanager/CleanerTask.java  |     7 +-
 .../ClientProtocolService.java                  |     7 +-
 .../SCMAdminProtocolService.java                |     8 +-
 .../sharedcachemanager/SharedCacheManager.java  |     9 +-
 .../metrics/CleanerMetrics.java                 |     7 +-
 .../metrics/ClientSCMMetrics.java               |     7 +-
 .../metrics/SharedCacheUploaderMetrics.java     |     8 +-
 .../store/InMemorySCMStore.java                 |     7 +-
 .../webapp/SCMOverviewPage.java                 |    26 +-
 .../sharedcachemanager/webapp/SCMWebServer.java |     7 +-
 .../hadoop/yarn/server/MiniYARNCluster.java     |    13 +-
 .../timeline/EntityGroupFSTimelineStore.java    |    23 +-
 .../timeline/LevelDBCacheTimelineStore.java     |    14 +-
 .../TestEntityGroupFSTimelineStore.java         |    99 +-
 ...stTimelineReaderWebServicesHBaseStorage.java |    30 +-
 .../storage/TestHBaseTimelineStorageApps.java   |     4 +-
 .../TestHBaseTimelineStorageEntities.java       |    14 +-
 .../reader/filter/TimelineFilterUtils.java      |     7 +-
 .../storage/HBaseTimelineReaderImpl.java        |     8 +-
 .../storage/HBaseTimelineWriterImpl.java        |     8 +-
 .../storage/TimelineSchemaCreator.java          |     7 +-
 .../storage/application/ApplicationTable.java   |     7 +-
 .../storage/apptoflow/AppToFlowTable.java       |     7 +-
 .../storage/common/AppIdKeyConverter.java       |     3 +-
 .../storage/common/ColumnHelper.java            |     8 +-
 .../common/HBaseTimelineStorageUtils.java       |    41 +-
 .../storage/entity/EntityTable.java             |     7 +-
 .../storage/flow/FlowActivityTable.java         |     7 +-
 .../storage/flow/FlowRunCoprocessor.java        |     7 +-
 .../storage/flow/FlowRunTable.java              |     7 +-
 .../storage/flow/FlowScanner.java               |     7 +-
 .../storage/reader/TimelineEntityReader.java    |     7 +-
 .../TestCustomApplicationIdConversion.java      |    39 +
 .../collector/AppLevelTimelineCollector.java    |     7 +-
 .../collector/NodeTimelineCollectorManager.java |     8 +-
 .../PerNodeTimelineCollectorsAuxService.java    |    10 +-
 .../collector/TimelineCollector.java            |     7 +-
 .../collector/TimelineCollectorManager.java     |     8 +-
 .../collector/TimelineCollectorWebService.java  |     8 +-
 .../reader/TimelineReaderServer.java            |     9 +-
 .../reader/TimelineReaderWebServices.java       |     8 +-
 .../storage/FileSystemTimelineReaderImpl.java   |     8 +-
 .../storage/common/TimelineStorageUtils.java    |     4 -
 .../hadoop/yarn/server/webproxy/ProxyUtils.java |    16 +-
 .../server/webproxy/WebAppProxyServlet.java     |    14 +-
 .../hadoop-yarn/hadoop-yarn-server/pom.xml      |     1 +
 .../src/site/markdown/Federation.md             |   309 +
 .../src/site/markdown/GracefulDecommission.md   |    12 +-
 .../src/site/markdown/NodeManagerCgroups.md     |    17 +-
 .../site/markdown/WritingYarnApplications.md    |     4 +-
 .../hadoop-yarn-site/src/site/markdown/YARN.md  |     5 +-
 .../src/site/markdown/registry/yarn-registry.md |    14 +-
 .../resources/images/amrmproxy_architecture.png |   Bin 0 -> 35010 bytes
 .../images/federation_architecture.png          |   Bin 0 -> 46964 bytes
 .../images/federation_sequence_diagram.png      |   Bin 0 -> 47910 bytes
 .../src/main/webapp/app/adapters/yarn-queue.js  |    30 -
 .../app/adapters/yarn-queue/capacity-queue.js   |    23 +
 .../app/adapters/yarn-queue/fair-queue.js       |    23 +
 .../app/adapters/yarn-queue/fifo-queue.js       |    23 +
 .../app/adapters/yarn-queue/yarn-queue.js       |    30 +
 .../main/webapp/app/components/tree-selector.js |    19 +-
 .../src/main/webapp/app/initializers/loader.js  |    42 +-
 .../src/main/webapp/app/models/yarn-queue.js    |    94 -
 .../app/models/yarn-queue/capacity-queue.js     |    95 +
 .../webapp/app/models/yarn-queue/fair-queue.js  |    79 +
 .../webapp/app/models/yarn-queue/fifo-queue.js  |    52 +
 .../webapp/app/models/yarn-queue/yarn-queue.js  |    23 +
 .../main/webapp/app/routes/cluster-overview.js  |     4 +-
 .../src/main/webapp/app/routes/yarn-queue.js    |    26 +-
 .../src/main/webapp/app/routes/yarn-queues.js   |    12 +-
 .../main/webapp/app/routes/yarn-queues/index.js |    25 -
 .../app/routes/yarn-queues/queues-selector.js   |    25 -
 .../main/webapp/app/serializers/yarn-queue.js   |   129 -
 .../serializers/yarn-queue/capacity-queue.js    |   128 +
 .../app/serializers/yarn-queue/fair-queue.js    |    92 +
 .../app/serializers/yarn-queue/fifo-queue.js    |    59 +
 .../app/serializers/yarn-queue/yarn-queue.js    |    47 +
 .../components/queue-configuration-table.hbs    |    54 -
 .../templates/components/queue-navigator.hbs    |     7 +-
 .../yarn-queue/capacity-queue-conf-table.hbs    |    54 +
 .../yarn-queue/capacity-queue-info.hbs          |    84 +
 .../components/yarn-queue/capacity-queue.hbs    |    63 +
 .../yarn-queue/fair-queue-conf-table.hbs        |    52 +
 .../components/yarn-queue/fair-queue-info.hbs   |    66 +
 .../components/yarn-queue/fair-queue.hbs        |    63 +
 .../yarn-queue/fifo-queue-conf-table.hbs        |    56 +
 .../components/yarn-queue/fifo-queue-info.hbs   |    47 +
 .../components/yarn-queue/fifo-queue.hbs        |    48 +
 .../webapp/app/templates/yarn-queue/info.hbs    |    73 +-
 .../main/webapp/app/templates/yarn-queues.hbs   |    54 +-
 .../src/main/webapp/app/utils/color-utils.js    |     1 -
 hadoop-yarn-project/hadoop-yarn/pom.xml         |    12 +-
 hadoop-yarn-project/pom.xml                     |     4 +
 pom.xml                                         |     5 +-
 start-build-env.sh                              |     2 +-
 962 files changed, 94710 insertions(+), 7270 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0bd0f62/LICENSE.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0bd0f62/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0bd0f62/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0bd0f62/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0bd0f62/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0bd0f62/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0bd0f62/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index 914635e,e6405b5..61e4808
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@@ -1,4 -1,4 +1,4 @@@
--#!/usr/bin/env bash
++  #!/usr/bin/env bash
  
  # Licensed to the Apache Software Foundation (ASF) under one or more
  # contributor license agreements.  See the NOTICE file distributed with
@@@ -31,45 -31,37 +31,47 @@@ function hadoop_usag
    hadoop_add_option "--hosts filename" "list of hosts to use in worker mode"
    hadoop_add_option "--workers" "turn on worker mode"
  
-   hadoop_add_subcommand "balancer" "run a cluster balancing utility"
-   hadoop_add_subcommand "cacheadmin" "configure the HDFS cache"
-   hadoop_add_subcommand "cblock" "cblock CLI"
-   hadoop_add_subcommand "cblockserver" "run cblock server"
-   hadoop_add_subcommand "classpath" "prints the class path needed to get the hadoop jar and the required libraries"
-   hadoop_add_subcommand "corona" "run an ozone data generator"
-   hadoop_add_subcommand "crypto" "configure HDFS encryption zones"
-   hadoop_add_subcommand "datanode" "run a DFS datanode"
-   hadoop_add_subcommand "debug" "run a Debug Admin to execute HDFS debug commands"
-   hadoop_add_subcommand "dfs" "run a filesystem command on the file system"
-   hadoop_add_subcommand "dfsadmin" "run a DFS admin client"
-   hadoop_add_subcommand "diskbalancer" "Distributes data evenly among disks on a given node"
-   hadoop_add_subcommand "envvars" "display computed Hadoop environment variables"
-   hadoop_add_subcommand "ec" "run a HDFS ErasureCoding CLI"
-   hadoop_add_subcommand "fetchdt" "fetch a delegation token from the NameNode"
-   hadoop_add_subcommand "fsck" "run a DFS filesystem checking utility"
-   hadoop_add_subcommand "getconf" "get config values from configuration"
-   hadoop_add_subcommand "groups" "get the groups which users belong to"
-   hadoop_add_subcommand "haadmin" "run a DFS HA admin client"
-   hadoop_add_subcommand "jmxget" "get JMX exported values from NameNode or DataNode."
-   hadoop_add_subcommand "journalnode" "run the DFS journalnode"
-   hadoop_add_subcommand "jscsi" "run cblock jscsi server"
-   hadoop_add_subcommand "ksm" "Ozone keyspace manager"
-   hadoop_add_subcommand "lsSnapshottableDir" "list all snapshottable dirs owned by the current user"
-   hadoop_add_subcommand "mover" "run a utility to move block replicas across storage types"
-   hadoop_add_subcommand "namenode" "run the DFS namenode"
-   hadoop_add_subcommand "nfs3" "run an NFS version 3 gateway"
-   hadoop_add_subcommand "oev" "apply the offline edits viewer to an edits file"
-   hadoop_add_subcommand "oiv" "apply the offline fsimage viewer to an fsimage"
-   hadoop_add_subcommand "oiv_legacy" "apply the offline fsimage viewer to a legacy fsimage"
-   hadoop_add_subcommand "oz" "command line interface for ozone"
-   hadoop_add_subcommand "oz_debug" "ozone debug tool, convert ozone meta data db into sqlite db"
-   hadoop_add_subcommand "portmap" "run a portmap service"
-   hadoop_add_subcommand "scm" "run the Storage Container Manager service"
-   hadoop_add_subcommand "secondarynamenode" "run the DFS secondary namenode"
-   hadoop_add_subcommand "snapshotDiff" "diff two snapshots of a directory or diff the current directory contents with a snapshot"
-   hadoop_add_subcommand "storagepolicies" "list/get/set block storage policies"
-   hadoop_add_subcommand "version" "print the version"
-   hadoop_add_subcommand "zkfc" "run the ZK Failover Controller daemon"
++
+   hadoop_add_subcommand "balancer" daemon "run a cluster balancing utility"
+   hadoop_add_subcommand "cacheadmin" admin "configure the HDFS cache"
++  hadoop_add_subcommand "cblock" admin "cblock CLI"
++   hadoop_add_subcommand "cblockserver" daemon "run cblock server"
+   hadoop_add_subcommand "classpath" client "prints the class path needed to get the hadoop jar and the required libraries"
++  hadoop_add_subcommand "corona" client "run an ozone data generator"
+   hadoop_add_subcommand "crypto" admin "configure HDFS encryption zones"
+   hadoop_add_subcommand "datanode" daemon "run a DFS datanode"
+   hadoop_add_subcommand "debug" admin "run a Debug Admin to execute HDFS debug commands"
+   hadoop_add_subcommand "dfs" client "run a filesystem command on the file system"
+   hadoop_add_subcommand "dfsadmin" admin "run a DFS admin client"
+   hadoop_add_subcommand "diskbalancer" daemon "Distributes data evenly among disks on a given node"
+   hadoop_add_subcommand "envvars" client "display computed Hadoop environment variables"
+   hadoop_add_subcommand "ec" admin "run a HDFS ErasureCoding CLI"
+   hadoop_add_subcommand "fetchdt" client "fetch a delegation token from the NameNode"
+   hadoop_add_subcommand "fsck" admin "run a DFS filesystem checking utility"
+   hadoop_add_subcommand "getconf" client "get config values from configuration"
+   hadoop_add_subcommand "groups" client "get the groups which users belong to"
+   hadoop_add_subcommand "haadmin" admin "run a DFS HA admin client"
+   hadoop_add_subcommand "jmxget" admin "get JMX exported values from NameNode or DataNode."
+   hadoop_add_subcommand "journalnode" daemon "run the DFS journalnode"
++  hadoop_add_subcommand "jscsi" daemon "run cblock jscsi server"
++  hadoop_add_subcommand "ksm" daemon "Ozone keyspace manager"
+   hadoop_add_subcommand "lsSnapshottableDir" client "list all snapshottable dirs owned by the current user"
+   hadoop_add_subcommand "mover" daemon "run a utility to move block replicas across storage types"
+   hadoop_add_subcommand "namenode" daemon "run the DFS namenode"
+   hadoop_add_subcommand "nfs3" daemon "run an NFS version 3 gateway"
+   hadoop_add_subcommand "oev" admin "apply the offline edits viewer to an edits file"
+   hadoop_add_subcommand "oiv" admin "apply the offline fsimage viewer to an fsimage"
+   hadoop_add_subcommand "oiv_legacy" admin "apply the offline fsimage viewer to a legacy fsimage"
++  hadoop_add_subcommand "oz" client "command line interface for ozone"
++  hadoop_add_subcommand "oz_debug" client "ozone debug tool, convert ozone metadata into relational data"
+   hadoop_add_subcommand "portmap" daemon "run a portmap service"
++  hadoop_add_subcommand "scm" daemon "run the Storage Container Manager service"
+   hadoop_add_subcommand "secondarynamenode" daemon "run the DFS secondary namenode"
+   hadoop_add_subcommand "snapshotDiff" client "diff two snapshots of a directory or diff the current directory contents with a snapshot"
+   hadoop_add_subcommand "storagepolicies" admin "list/get/set block storage policies"
+   hadoop_add_subcommand "version" client "print the version"
+   hadoop_add_subcommand "zkfc" daemon "run the ZK Failover Controller daemon"
++
    hadoop_generate_usage "${HADOOP_SHELL_EXECNAME}" false
  }
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0bd0f62/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0bd0f62/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0bd0f62/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --cc hadoop-project/pom.xml
index 8b99379,8c1d374..e1d22b4
mode 100644,100755..100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@@ -98,8 -96,10 +96,12 @@@
      <apacheds.version>2.0.0-M21</apacheds.version>
      <ldap-api.version>1.0.0-M33</ldap-api.version>
  
 +    <!-- Apache Ratis version -->
 +    <ratis.version>0.1.1-alpha-SNAPSHOT</ratis.version>
+     <jcache.version>1.0-alpha-1</jcache.version>
+     <ehcache.version>3.3.1</ehcache.version>
+     <hikari.version>2.4.12</hikari.version>
+     <mssql.version>6.2.1.jre7</mssql.version>
  
      <!-- define the Java language version used by the compiler -->
      <javac.version>1.8</javac.version>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0bd0f62/hadoop-project/src/site/site.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0bd0f62/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
----------------------------------------------------------------------
diff --cc hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
index fbb56b0,09b12f2..1c39fa7
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
@@@ -815,8 -821,8 +821,8 @@@ public class TestAMRMClient 
        amClient.updateBlacklist(nodeList02, null);
        assertEquals(3, amClient.blacklistAdditions.size());
        assertEquals(0, amClient.blacklistRemovals.size());
 -      
 -      // Add nodes[1] and nodes[2] to removal list, 
 +
-       // Add nodes[1] and nodes[2] to removal list, 
++      // Add nodes[1] and nodes[2] to removal list,
        // Verify addition list remove these two nodes.
        List<String> nodeList12 = new ArrayList<String>();
        nodeList12.add(nodes[1]);
@@@ -824,8 -830,8 +830,8 @@@
        amClient.updateBlacklist(null, nodeList12);
        assertEquals(1, amClient.blacklistAdditions.size());
        assertEquals(2, amClient.blacklistRemovals.size());
 -      
 -      // Add nodes[1] again to addition list, 
 +
-       // Add nodes[1] again to addition list, 
++      // Add nodes[1] again to addition list,
        // Verify removal list will remove this node.
        List<String> nodeList1 = new ArrayList<String>();
        nodeList1.add(nodes[1]);
@@@ -933,10 -939,10 +939,10 @@@
          1), null, null, Priority.UNDEFINED, true, "x"));
      client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
          1), null, null, Priority.UNDEFINED, true, "a"));
-     Assert.assertEquals(1, client.ask.size());
-     Assert.assertEquals("a", client.ask.iterator().next()
+     assertEquals(1, client.ask.size());
+     assertEquals("a", client.ask.iterator().next()
          .getNodeLabelExpression());
 -    
 +
      // add exp=x to ANY, rack and node, only resource request has ANY resource
      // name will be assigned the label expression
      // add exp=x then add exp=a to ANY in same priority, only exp=a should kept
@@@ -971,9 -977,9 +977,9 @@@
      } catch (InvalidContainerRequestException e) {
        return;
      }
-     Assert.fail();
+     fail();
    }
 -  
 +
    @Test(timeout=30000)
    public void testAskWithInvalidNodeLabels() {
      AMRMClientImpl<ContainerRequest> client =
@@@ -1170,11 -1507,11 +1507,11 @@@
      int allocatedContainerCount = 0;
      int iterationsLeft = 3;
      Set<ContainerId> releases = new TreeSet<ContainerId>();
 -    
 +
      amClient.getNMTokenCache().clearCache();
-     Assert.assertEquals(0, amClient.getNMTokenCache().numberOfTokensInCache());
+     assertEquals(0, amClient.getNMTokenCache().numberOfTokensInCache());
      HashMap<String, Token> receivedNMTokens = new HashMap<String, Token>();
 -    
 +
      while (allocatedContainerCount < containersRequestedAny
          && iterationsLeft-- > 0) {
        AllocateResponse allocResponse = amClient.allocate(0.1f);
@@@ -1202,11 -1539,11 +1539,11 @@@
          triggerSchedulingWithNMHeartBeat();
        }
      }
 -    
 +
      // Should receive atleast 1 token
-     Assert.assertTrue(receivedNMTokens.size() > 0
+     assertTrue(receivedNMTokens.size() > 0
          && receivedNMTokens.size() <= nodeCount);
 -    
 +
      assertEquals(allocatedContainerCount, containersRequestedAny);
      assertEquals(2, releases.size());
      assertEquals(0, amClient.ask.size());
@@@ -1254,7 -1591,7 +1591,7 @@@
      assertEquals(2, amClient.release.size());
      assertEquals(3, amClient.ask.size());
      snoopRequest = amClient.ask.iterator().next();
--    // verify that the remove request made in between makeRequest and allocate 
++    // verify that the remove request made in between makeRequest and allocate
      // has not been lost
      assertEquals(0, snoopRequest.getNumContainers());
  
@@@ -1444,8 -1781,8 +1781,8 @@@
        org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> amrmToken_1 =
            getAMRMToken();
        Assert.assertNotNull(amrmToken_1);
-       Assert.assertEquals(amrmToken_1.decodeIdentifier().getKeyId(),
+       assertEquals(amrmToken_1.decodeIdentifier().getKeyId(),
 -        amrmTokenSecretManager.getMasterKey().getMasterKey().getKeyId());
 +          amrmTokenSecretManager.getMasterKey().getMasterKey().getKeyId());
  
        // Wait for enough time and make sure the roll_over happens
        // At mean time, the old AMRMToken should continue to work
@@@ -1459,8 -1796,8 +1796,8 @@@
        org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> amrmToken_2 =
            getAMRMToken();
        Assert.assertNotNull(amrmToken_2);
-       Assert.assertEquals(amrmToken_2.decodeIdentifier().getKeyId(),
+       assertEquals(amrmToken_2.decodeIdentifier().getKeyId(),
 -        amrmTokenSecretManager.getMasterKey().getMasterKey().getKeyId());
 +          amrmTokenSecretManager.getMasterKey().getMasterKey().getKeyId());
  
        Assert.assertNotEquals(amrmToken_1, amrmToken_2);
  
@@@ -1470,22 -1807,22 +1807,22 @@@
        // Verify latest AMRMToken can be used to send allocation request.
        UserGroupInformation testUser1 =
            UserGroupInformation.createRemoteUser("testUser1");
 -      
 -      AMRMTokenIdentifierForTest newVersionTokenIdentifier = 
 +
 +      AMRMTokenIdentifierForTest newVersionTokenIdentifier =
            new AMRMTokenIdentifierForTest(amrmToken_2.decodeIdentifier(), "message");
 -      
 +
-       Assert.assertEquals("Message is changed after set to newVersionTokenIdentifier",
+       assertEquals("Message is changed after set to newVersionTokenIdentifier",
            "message", newVersionTokenIdentifier.getMessage());
 -      org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> newVersionToken = 
 +      org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> newVersionToken =
            new org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> (
 -              newVersionTokenIdentifier.getBytes(), 
 +              newVersionTokenIdentifier.getBytes(),
                amrmTokenSecretManager.retrievePassword(newVersionTokenIdentifier),
                newVersionTokenIdentifier.getKind(), new Text());
 -      
 +
        SecurityUtil.setTokenService(newVersionToken, yarnCluster
 -        .getResourceManager().getApplicationMasterService().getBindAddress());
 +          .getResourceManager().getApplicationMasterService().getBindAddress());
        testUser1.addToken(newVersionToken);
 -      
 +
        AllocateRequest request = Records.newRecord(AllocateRequest.class);
        request.setResponseId(response.getResponseId());
        testUser1.doAs(new PrivilegedAction<ApplicationMasterProtocol>() {
@@@ -1525,17 -1862,17 +1862,17 @@@
            @Override
            public ApplicationMasterProtocol run() {
              return (ApplicationMasterProtocol) YarnRPC.create(conf).getProxy(
 -              ApplicationMasterProtocol.class,
 -              yarnCluster.getResourceManager().getApplicationMasterService()
 -                .getBindAddress(), conf);
 +                ApplicationMasterProtocol.class,
 +                yarnCluster.getResourceManager().getApplicationMasterService()
 +                    .getBindAddress(), conf);
            }
          }).allocate(Records.newRecord(AllocateRequest.class));
-         Assert.fail("The old Token should not work");
+         fail("The old Token should not work");
        } catch (Exception ex) {
-         Assert.assertTrue(ex instanceof InvalidToken);
-         Assert.assertTrue(ex.getMessage().contains(
+         assertTrue(ex instanceof InvalidToken);
+         assertTrue(ex.getMessage().contains(
 -          "Invalid AMRMToken from "
 -              + amrmToken_2.decodeIdentifier().getApplicationAttemptId()));
 +            "Invalid AMRMToken from "
 +                + amrmToken_2.decodeIdentifier().getApplicationAttemptId()));
        }
  
        amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,


---------------------------------------------------------------------
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: HADOOP-14398. Modify documents for the FileSystem Builder API. (Lei (Eddy) Xu)

Posted by ae...@apache.org.
HADOOP-14398. Modify documents for the FileSystem Builder API. (Lei (Eddy) Xu)


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

Branch: refs/heads/HDFS-7240
Commit: 99e558b13ba4d5832aea97374e1d07b4e78e5e39
Parents: 4230872
Author: Lei Xu <le...@apache.org>
Authored: Thu Aug 17 18:06:23 2017 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Thu Aug 17 18:06:23 2017 -0700

----------------------------------------------------------------------
 .../hadoop/fs/FSDataOutputStreamBuilder.java    |  74 ++++++--
 .../src/site/markdown/filesystem/filesystem.md  |  33 +++-
 .../filesystem/fsdataoutputstreambuilder.md     | 182 +++++++++++++++++++
 .../src/site/markdown/filesystem/index.md       |   1 +
 4 files changed, 272 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/99e558b1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java
index 1f668eb..86c284a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java
@@ -54,16 +54,29 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_
  * options accordingly, for example:
  *
  * <code>
- * FSDataOutputStreamBuilder builder = fs.createFile(path);
- * builder.permission(perm)
+ *
+ * // Don't
+ * if (fs instanceof FooFileSystem) {
+ *   FooFileSystem fs = (FooFileSystem) fs;
+ *   OutputStream out = dfs.createFile(path)
+ *     .optionA()
+ *     .optionB("value")
+ *     .cache()
+ *   .build()
+ * } else if (fs instanceof BarFileSystem) {
+ *   ...
+ * }
+ *
+ * // Do
+ * OutputStream out = fs.createFile(path)
+ *   .permission(perm)
  *   .bufferSize(bufSize)
- *   .opt("dfs.outputstream.builder.lazy-persist", true)
- *   .opt("dfs.outputstream.builder.ec.policy-name", "rs-3-2-64k")
- *   .opt("fs.local.o-direct", true)
- *   .must("fs.s3a.fast-upload", true)
- *   .must("fs.azure.buffer-size", 256 * 1024 * 1024);
- * FSDataOutputStream out = builder.build();
- * ...
+ *   .opt("foofs:option.a", true)
+ *   .opt("foofs:option.b", "value")
+ *   .opt("barfs:cache", true)
+ *   .must("foofs:cache", true)
+ *   .must("barfs:cache-size", 256 * 1024 * 1024)
+ *   .build();
  * </code>
  *
  * If the option is not related to the file system, the option will be ignored.
@@ -263,6 +276,8 @@ public abstract class FSDataOutputStreamBuilder
 
   /**
    * Set optional boolean parameter for the Builder.
+   *
+   * @see #opt(String, String)
    */
   public B opt(@Nonnull final String key, boolean value) {
     mandatoryKeys.remove(key);
@@ -272,6 +287,8 @@ public abstract class FSDataOutputStreamBuilder
 
   /**
    * Set optional int parameter for the Builder.
+   *
+   * @see #opt(String, String)
    */
   public B opt(@Nonnull final String key, int value) {
     mandatoryKeys.remove(key);
@@ -281,6 +298,8 @@ public abstract class FSDataOutputStreamBuilder
 
   /**
    * Set optional float parameter for the Builder.
+   *
+   * @see #opt(String, String)
    */
   public B opt(@Nonnull final String key, float value) {
     mandatoryKeys.remove(key);
@@ -290,6 +309,8 @@ public abstract class FSDataOutputStreamBuilder
 
   /**
    * Set optional double parameter for the Builder.
+   *
+   * @see #opt(String, String)
    */
   public B opt(@Nonnull final String key, double value) {
     mandatoryKeys.remove(key);
@@ -299,6 +320,8 @@ public abstract class FSDataOutputStreamBuilder
 
   /**
    * Set an array of string values as optional parameter for the Builder.
+   *
+   * @see #opt(String, String)
    */
   public B opt(@Nonnull final String key, @Nonnull final String... values) {
     mandatoryKeys.remove(key);
@@ -310,8 +333,7 @@ public abstract class FSDataOutputStreamBuilder
    * Set mandatory option to the Builder.
    *
    * If the option is not supported or unavailable on the {@link FileSystem},
-   * the client should expect {@link #build()} throws
-   * {@link IllegalArgumentException}.
+   * the client should expect {@link #build()} throws IllegalArgumentException.
    */
   public B must(@Nonnull final String key, @Nonnull final String value) {
     mandatoryKeys.add(key);
@@ -319,35 +341,55 @@ public abstract class FSDataOutputStreamBuilder
     return getThisBuilder();
   }
 
-  /** Set mandatory boolean option. */
+  /**
+   * Set mandatory boolean option.
+   *
+   * @see #must(String, String)
+   */
   public B must(@Nonnull final String key, boolean value) {
     mandatoryKeys.add(key);
     options.setBoolean(key, value);
     return getThisBuilder();
   }
 
-  /** Set mandatory int option. */
+  /**
+   * Set mandatory int option.
+   *
+   * @see #must(String, String)
+   */
   public B must(@Nonnull final String key, int value) {
     mandatoryKeys.add(key);
     options.setInt(key, value);
     return getThisBuilder();
   }
 
-  /** Set mandatory float option. */
+  /**
+   * Set mandatory float option.
+   *
+   * @see #must(String, String)
+   */
   public B must(@Nonnull final String key, float value) {
     mandatoryKeys.add(key);
     options.setFloat(key, value);
     return getThisBuilder();
   }
 
-  /** Set mandatory double option. */
+  /**
+   * Set mandatory double option.
+   *
+   * @see #must(String, String)
+   */
   public B must(@Nonnull final String key, double value) {
     mandatoryKeys.add(key);
     options.setDouble(key, value);
     return getThisBuilder();
   }
 
-  /** Set a string array as mandatory option. */
+  /**
+   * Set a string array as mandatory option.
+   *
+   * @see #must(String, String)
+   */
   public B must(@Nonnull final String key, @Nonnull final String... values) {
     mandatoryKeys.add(key);
     options.setStrings(key, values);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99e558b1/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
index d7e57ce..1e522c7 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
@@ -553,7 +553,7 @@ on a path that exists and is a file. Instead the operation returns false.
        FS' = FS
        result = False
 
-### `FSDataOutputStream create(Path, ...)`
+### <a name='FileSystem.create'></a> `FSDataOutputStream create(Path, ...)`
 
 
     FSDataOutputStream create(Path p,
@@ -616,7 +616,24 @@ this precondition fails.
 
 * Not covered: symlinks. The resolved path of the symlink is used as the final path argument to the `create()` operation
 
-### `FSDataOutputStream append(Path p, int bufferSize, Progressable progress)`
+### `FSDataOutputStreamBuilder createFile(Path p)`
+
+Make a `FSDataOutputStreamBuilder` to specify the parameters to create a file.
+
+#### Implementation Notes
+
+`createFile(p)` returns a `FSDataOutputStreamBuilder` only and does not make
+change on filesystem immediately. When `build()` is invoked on the `FSDataOutputStreamBuilder`,
+the builder parameters are verified and [`create(Path p)`](#FileSystem.create)
+is invoked on the underlying filesystem. `build()` has the same preconditions
+and postconditions as [`create(Path p)`](#FileSystem.create).
+
+* Similar to [`create(Path p)`](#FileSystem.create), files are overwritten
+by default, unless specify `builder.overwrite(false)`.
+* Unlike [`create(Path p)`](#FileSystem.create), missing parent directories are
+not created by default, unless specify `builder.recursive()`.
+
+### <a name='FileSystem.append'></a> `FSDataOutputStream append(Path p, int bufferSize, Progressable progress)`
 
 Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
@@ -634,6 +651,18 @@ Implementations without a compliant call SHOULD throw `UnsupportedOperationExcep
 Return: `FSDataOutputStream`, which can update the entry `FS.Files[p]`
 by appending data to the existing list.
 
+### `FSDataOutputStreamBuilder appendFile(Path p)`
+
+Make a `FSDataOutputStreamBuilder` to specify the parameters to append to an
+existing file.
+
+#### Implementation Notes
+
+`appendFile(p)` returns a `FSDataOutputStreamBuilder` only and does not make
+change on filesystem immediately. When `build()` is invoked on the `FSDataOutputStreamBuilder`,
+the builder parameters are verified and [`append()`](#FileSystem.append) is
+invoked on the underlying filesystem. `build()` has the same preconditions and
+postconditions as [`append()`](#FileSystem.append).
 
 ### `FSDataInputStream open(Path f, int bufferSize)`
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99e558b1/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md
new file mode 100644
index 0000000..4ea1fd1
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md
@@ -0,0 +1,182 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+<!--  ============================================================= -->
+<!--  CLASS: FSDataOutputStreamBuilder -->
+<!--  ============================================================= -->
+
+# class `org.apache.hadoop.fs.FSDataOutputStreamBuilder`
+
+<!-- MACRO{toc|fromDepth=1|toDepth=2} -->
+
+Builder pattern for `FSDataOutputStream` and its subclasses. It is used to
+create a new file or open an existing file on `FileSystem` for write.
+
+## Invariants
+
+The `FSDataOutputStreamBuilder` interface does not validate parameters
+and modify the state of `FileSystem` until [`build()`](#Builder.build) is
+invoked.
+
+## Implementation-agnostic parameters.
+
+### <a name="Builder.create"></a> `FSDataOutputStreamBuilder create()`
+
+Specify `FSDataOutputStreamBuilder` to create a file on `FileSystem`, equivalent
+to `CreateFlag#CREATE`.
+
+### <a name="Builder.append"></a> `FSDataOutputStreamBuilder append()`
+
+Specify `FSDataOutputStreamBuilder` to append to an existing file on
+`FileSystem`, equivalent to `CreateFlag#APPEND`.
+
+### <a name="Builder.overwrite"></a> `FSDataOutputStreamBuilder overwrite(boolean overwrite)`
+
+Specify `FSDataOutputStreamBuilder` to overwrite an existing file or not. If
+giving `overwrite==true`, it truncates an existing file, equivalent to
+`CreateFlag#OVERWITE`.
+
+### <a name="Builder.permission"></a> `FSDataOutputStreamBuilder permission(FsPermission permission)`
+
+Set permission for the file.
+
+### <a name="Builder.bufferSize"></a> `FSDataOutputStreamBuilder bufferSize(int bufSize)`
+
+Set the size of the buffer to be used.
+
+### <a name="Builder.replication"></a> `FSDataOutputStreamBuilder replication(short replica)`
+
+Set the replication factor.
+
+### <a name="Builder.blockSize"></a> `FSDataOutputStreamBuilder blockSize(long size)`
+
+Set block size in bytes.
+
+### <a name="Builder.recursive"></a> `FSDataOutputStreamBuilder recursive()`
+
+Create parent directories if they do not exist.
+
+### <a name="Builder.progress"></a> `FSDataOutputStreamBuilder progress(Progresable prog)`
+
+Set the facility of reporting progress.
+
+### <a name="Builder.checksumOpt"></a> `FSDataOutputStreamBuilder checksumOpt(ChecksumOpt chksumOpt)`
+
+Set checksum opt.
+
+### Set optional or mandatory parameters
+
+    FSDataOutputStreamBuilder opt(String key, ...)
+    FSDataOutputStreamBuilder must(String key, ...)
+
+Set optional or mandatory parameters to the builder. Using `opt()` or `must()`,
+client can specify FS-specific parameters without inspecting the concrete type
+of `FileSystem`.
+
+    // Don't
+    if (fs instanceof FooFileSystem) {
+        FooFileSystem fs = (FooFileSystem) fs;
+        out = dfs.createFile(path)
+            .optionA()
+            .optionB("value")
+            .cache()
+            .build()
+    } else if (fs instanceof BarFileSystem) {
+        ...
+    }
+
+    // Do
+    out = fs.createFile(path)
+        .permission(perm)
+        .bufferSize(bufSize)
+        .opt("foofs:option.a", true)
+        .opt("foofs:option.b", "value")
+        .opt("barfs:cache", true)
+        .must("foofs:cache", true)
+        .must("barfs:cache-size", 256 * 1024 * 1024)
+        .build();
+
+#### Implementation Notes
+
+The concrete `FileSystem` and/or `FSDataOutputStreamBuilder` implementation
+MUST verify that implementation-agnostic parameters (i.e., "syncable") or
+implementation-specific parameters (i.e., "foofs:cache")
+are supported. `FileSystem` will satisfy optional parameters (via `opt(key, ...)`)
+on best effort. If the mandatory parameters (via `must(key, ...)`) can not be satisfied
+in the `FileSystem`, `IllegalArgumentException` should be thrown in `build()`.
+
+The behavior of resolving the conflicts between the parameters set by
+builder methods (i.e., `bufferSize()`) and `opt()`/`must()` is undefined.
+
+## HDFS-specific parameters.
+
+`HdfsDataOutputStreamBuilder extends FSDataOutputStreamBuilder` provides additional
+HDFS-specific parameters, for further customize file creation / append behavior.
+
+### `FSDataOutpuStreamBuilder favoredNodes(InetSocketAddress[] nodes)`
+
+Set favored DataNodes for new blocks.
+
+### `FSDataOutputStreamBuilder syncBlock()`
+
+Force closed blocks to the disk device. See `CreateFlag#SYNC_BLOCK`
+
+### `FSDataOutputStreamBuilder lazyPersist()`
+
+Create the block on transient storage if possible.
+
+### `FSDataOutputStreamBuilder newBlock()`
+
+Append data to a new block instead of the end of the last partial block.
+
+### `FSDataOutputStreamBuilder noLocalWrite()`
+
+Advise that a block replica NOT be written to the local DataNode.
+
+### `FSDataOutputStreamBuilder ecPolicyName()`
+
+Enforce the file to be a striped file with erasure coding policy 'policyName',
+no matter what its parent directory's replication or erasure coding policy is.
+
+### `FSDataOutputStreamBuilder replicate()`
+
+Enforce the file to be a replicated file, no matter what its parent directory's
+replication or erasure coding policy is.
+
+## Builder interface
+
+### <a name="Builder.build"></a> `FSDataOutputStream build()`
+
+Create a new file or append an existing file on the underlying `FileSystem`,
+and return `FSDataOutputStream` for write.
+
+#### Preconditions
+
+The following combinations of parameters are not supported:
+
+    if APPEND|OVERWRITE: raise HadoopIllegalArgumentException
+    if CREATE|APPEND|OVERWRITE: raise HadoopIllegalArgumentExdeption
+
+`FileSystem` may reject the request for other reasons and throw `IOException`,
+see `FileSystem#create(path, ...)` and `FileSystem#append()`.
+
+#### Postconditions
+
+    FS' where :
+       FS'.Files'[p] == []
+       ancestors(p) is-subset-of FS'.Directories'
+
+    result = FSDataOutputStream
+
+The result is `FSDataOutputStream` to be used to write data to filesystem.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99e558b1/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
index 66a7eb3..532b6c7 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
@@ -33,5 +33,6 @@ HDFS as these are commonly expected by Hadoop client applications.
 1. [Model](model.html)
 1. [FileSystem class](filesystem.html)
 1. [FSDataInputStream class](fsdatainputstream.html)
+1. [FSDataOutputStreamBuilder class](fsdataoutputstreambuilder.html)
 2. [Testing with the Filesystem specification](testing.html)
 2. [Extending the specification and its tests](extending.html)


---------------------------------------------------------------------
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-6979. [Addendum patch] Fixed classname and added javadocs. (Kartheek Muthyala via asuresh)

Posted by ae...@apache.org.
YARN-6979. [Addendum patch] Fixed classname and added javadocs. (Kartheek Muthyala via asuresh)


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

Branch: refs/heads/HDFS-7240
Commit: 7a82d7bcea8124e1b65c275fac15bf2047d17471
Parents: 8410d86
Author: Arun Suresh <as...@apache.org>
Authored: Sun Aug 20 08:55:13 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Sun Aug 20 10:24:05 2017 -0700

----------------------------------------------------------------------
 .../CMgrDecreaseContainersResourceEvent.java    | 37 ---------------
 .../nodemanager/CMgrUpdateContainersEvent.java  | 48 ++++++++++++++++++++
 .../nodemanager/ContainerManagerEventType.java  |  2 +-
 .../nodemanager/NodeStatusUpdaterImpl.java      |  8 ++--
 .../containermanager/ContainerManagerImpl.java  | 10 ++--
 5 files changed, 57 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a82d7bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrDecreaseContainersResourceEvent.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/CMgrDecreaseContainersResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrDecreaseContainersResourceEvent.java
deleted file mode 100644
index 9479d0b..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrDecreaseContainersResourceEvent.java
+++ /dev/null
@@ -1,37 +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;
-
-import org.apache.hadoop.yarn.api.records.Container;
-import java.util.List;
-
-public class CMgrDecreaseContainersResourceEvent extends ContainerManagerEvent {
-
-  private final List<Container> containersToDecrease;
-
-  public CMgrDecreaseContainersResourceEvent(List<Container>
-      containersToDecrease) {
-    super(ContainerManagerEventType.DECREASE_CONTAINERS_RESOURCE);
-    this.containersToDecrease = containersToDecrease;
-  }
-
-  public List<Container> getContainersToDecrease() {
-    return this.containersToDecrease;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a82d7bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrUpdateContainersEvent.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/CMgrUpdateContainersEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrUpdateContainersEvent.java
new file mode 100644
index 0000000..5e41701
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrUpdateContainersEvent.java
@@ -0,0 +1,48 @@
+/**
+ * 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;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import java.util.List;
+
+/**
+ * Event used by the NodeStatusUpdater to notify the ContainerManager of
+ * container update commands it received from the RM.
+ */
+public class CMgrUpdateContainersEvent extends ContainerManagerEvent {
+
+  private final List<Container> containersToUpdate;
+
+  /**
+   * Create event.
+   * @param containersToUpdate Container to update.
+   */
+  public CMgrUpdateContainersEvent(List<Container> containersToUpdate) {
+    super(ContainerManagerEventType.UPDATE_CONTAINERS);
+    this.containersToUpdate = containersToUpdate;
+  }
+
+  /**
+   * Get containers to update.
+   * @return List of containers to update.
+   */
+  public List<Container> getContainersToUpdate() {
+    return this.containersToUpdate;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a82d7bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.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/ContainerManagerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java
index 8861bc7..8c5f7e2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java
@@ -21,6 +21,6 @@ package org.apache.hadoop.yarn.server.nodemanager;
 public enum ContainerManagerEventType {
   FINISH_APPS,
   FINISH_CONTAINERS,
-  DECREASE_CONTAINERS_RESOURCE,
+  UPDATE_CONTAINERS,
   SIGNAL_CONTAINERS
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a82d7bc/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 1d9256f..ade42e3 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
@@ -1099,12 +1099,10 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
                   parseCredentials(systemCredentials));
             }
             List<org.apache.hadoop.yarn.api.records.Container>
-                containersToDecrease = response.getContainersToUpdate();
-            if (!containersToDecrease.isEmpty()) {
+                containersToUpdate = response.getContainersToUpdate();
+            if (!containersToUpdate.isEmpty()) {
               dispatcher.getEventHandler().handle(
-                  new CMgrDecreaseContainersResourceEvent(
-                      containersToDecrease)
-              );
+                  new CMgrUpdateContainersEvent(containersToUpdate));
             }
 
             // SignalContainer request originally comes from end users via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a82d7bc/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 12931bc..22484b7 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
@@ -96,7 +96,7 @@ import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
 import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedContainersEvent;
-import org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent;
+import org.apache.hadoop.yarn.server.nodemanager.CMgrUpdateContainersEvent;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrSignalContainersEvent;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerManagerEvent;
@@ -1599,11 +1599,11 @@ public class ContainerManagerImpl extends CompositeService implements
                   "Container Killed by ResourceManager"));
       }
       break;
-    case DECREASE_CONTAINERS_RESOURCE:
-      CMgrDecreaseContainersResourceEvent containersDecreasedEvent =
-          (CMgrDecreaseContainersResourceEvent) event;
+    case UPDATE_CONTAINERS:
+      CMgrUpdateContainersEvent containersDecreasedEvent =
+          (CMgrUpdateContainersEvent) event;
       for (org.apache.hadoop.yarn.api.records.Container container
-          : containersDecreasedEvent.getContainersToDecrease()) {
+          : containersDecreasedEvent.getContainersToUpdate()) {
         try {
           ContainerTokenIdentifier containerTokenIdentifier =
               BuilderUtils.newContainerTokenIdentifier(


---------------------------------------------------------------------
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-6969. Clean up unused code in class FairSchedulerQueueInfo. (Larry Lo via Yufei Gu)

Posted by ae...@apache.org.
YARN-6969. Clean up unused code in class FairSchedulerQueueInfo. (Larry Lo via Yufei Gu)


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

Branch: refs/heads/HDFS-7240
Commit: 8991f0baec62625c45144e2544066195800ab95b
Parents: 2d105a2
Author: Yufei Gu <yu...@apache.org>
Authored: Fri Aug 18 14:38:44 2017 -0700
Committer: Yufei Gu <yu...@apache.org>
Committed: Fri Aug 18 14:38:44 2017 -0700

----------------------------------------------------------------------
 .../webapp/dao/FairSchedulerQueueInfo.java         | 17 -----------------
 1 file changed, 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8991f0ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.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/FairSchedulerQueueInfo.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/FairSchedulerQueueInfo.java
index 79339c7..913513c 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/FairSchedulerQueueInfo.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/FairSchedulerQueueInfo.java
@@ -48,8 +48,6 @@ public class FairSchedulerQueueInfo {
   @XmlTransient
   private float fractionMemFairShare;
   @XmlTransient
-  private float fractionMemMinShare;
-  @XmlTransient
   private float fractionMemMaxShare;
   
   private ResourceInfo minResources;
@@ -63,7 +61,6 @@ public class FairSchedulerQueueInfo {
   private ResourceInfo clusterResources;
   private ResourceInfo reservedResources;
 
-  private long pendingContainers;
   private long allocatedContainers;
   private long reservedContainers;
 
@@ -108,12 +105,10 @@ public class FairSchedulerQueueInfo {
         (float)steadyFairResources.getMemorySize() / clusterResources.getMemorySize();
     fractionMemFairShare = (float) fairResources.getMemorySize()
         / clusterResources.getMemorySize();
-    fractionMemMinShare = (float)minResources.getMemorySize() / clusterResources.getMemorySize();
     fractionMemMaxShare = (float)maxResources.getMemorySize() / clusterResources.getMemorySize();
     
     maxApps = queue.getMaxRunningApps();
 
-    pendingContainers = queue.getMetrics().getPendingContainers();
     allocatedContainers = queue.getMetrics().getAllocatedContainers();
     reservedContainers = queue.getMetrics().getReservedContainers();
 
@@ -126,10 +121,6 @@ public class FairSchedulerQueueInfo {
     childQueues = getChildQueues(queue, scheduler);
   }
 
-  public long getPendingContainers() {
-    return pendingContainers;
-  }
-
   public long getAllocatedContainers() {
     return allocatedContainers;
   }
@@ -234,14 +225,6 @@ public class FairSchedulerQueueInfo {
   }
 
   /**
-   * Returns the queue's min share in as a fraction of the entire
-   * cluster capacity.
-   */
-  public float getMinShareMemoryFraction() {
-    return fractionMemMinShare;
-  }
-  
-  /**
    * Returns the memory used by this queue as a fraction of the entire 
    * cluster capacity.
    */


---------------------------------------------------------------------
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-6979. Add flag to notify all types of container updates to NM via NodeHeartbeatResponse. (Kartheek Muthyala via asuresh)

Posted by ae...@apache.org.
YARN-6979. Add flag to notify all types of container updates to NM via NodeHeartbeatResponse. (Kartheek Muthyala via asuresh)


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

Branch: refs/heads/HDFS-7240
Commit: 8410d862d3a72740f461ef91dddb5325955e1ca5
Parents: 436c263
Author: Arun Suresh <as...@apache.org>
Authored: Sun Aug 20 07:54:09 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Sun Aug 20 07:54:09 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/sls/nodemanager/NodeInfo.java   |   2 +-
 .../yarn/sls/scheduler/RMNodeWrapper.java       |   2 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |   4 +
 .../src/main/resources/yarn-default.xml         |   8 +
 .../protocolrecords/NodeHeartbeatResponse.java  |   6 +-
 .../impl/pb/NodeHeartbeatResponsePBImpl.java    |  42 ++---
 .../yarn_server_common_service_protos.proto     |   3 +
 .../hadoop/yarn/TestYarnServerApiClasses.java   |   6 +-
 .../nodemanager/NodeStatusUpdaterImpl.java      |   2 +-
 .../containermanager/ContainerManagerImpl.java  |  35 ++--
 .../scheduler/ContainerScheduler.java           |   1 +
 .../resourcemanager/ResourceTrackerService.java |   2 +-
 .../rmcontainer/RMContainerImpl.java            |   8 +-
 .../server/resourcemanager/rmnode/RMNode.java   |   6 +-
 .../rmnode/RMNodeDecreaseContainerEvent.java    |  39 -----
 .../resourcemanager/rmnode/RMNodeEventType.java |   2 +-
 .../resourcemanager/rmnode/RMNodeImpl.java      |  29 ++--
 .../rmnode/RMNodeUpdateContainerEvent.java      |  44 +++++
 .../scheduler/AbstractYarnScheduler.java        |  11 ++
 .../scheduler/SchedulerApplicationAttempt.java  |  39 +++--
 .../yarn/server/resourcemanager/MockNodes.java  |   2 +-
 ...pportunisticContainerAllocatorAMService.java | 168 +++++++++++++++++++
 .../capacity/TestContainerResizing.java         |   7 +-
 .../capacity/TestIncreaseAllocationExpirer.java |   4 +-
 24 files changed, 346 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/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 8962aba..e71ddff 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
@@ -179,7 +179,7 @@ public class NodeInfo {
     }
 
     @Override
-    public void updateNodeHeartbeatResponseForContainersDecreasing(
+    public void updateNodeHeartbeatResponseForUpdatedContainers(
         NodeHeartbeatResponse response) {
       // TODO Auto-generated method stub
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/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 d7b159c..6b7ac3c 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
@@ -168,7 +168,7 @@ public class RMNodeWrapper implements RMNode {
   }
 
   @Override
-  public void updateNodeHeartbeatResponseForContainersDecreasing(
+  public void updateNodeHeartbeatResponseForUpdatedContainers(
       NodeHeartbeatResponse response) {
     // TODO Auto-generated method stub
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/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 8515e0a..86f45b8 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
@@ -167,6 +167,10 @@ public class YarnConfiguration extends Configuration {
   public static final String RM_APPLICATION_MASTER_SERVICE_PROCESSORS =
       RM_PREFIX + "application-master-service.processors";
 
+  public static final String RM_AUTO_UPDATE_CONTAINERS =
+      RM_PREFIX + "auto-update.containers";
+  public static final boolean DEFAULT_RM_AUTO_UPDATE_CONTAINERS = false;
+
   /** The actual bind address for the RM.*/
   public static final String RM_BIND_HOST =
     RM_PREFIX + "bind-host";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/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 dbf115b..f93de44 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
@@ -73,6 +73,14 @@
   </property>
 
   <property>
+    <description>
+      If set to true, then ALL container updates will be automatically sent to
+      the NM in the next heartbeat</description>
+    <name>yarn.resourcemanager.auto-update.containers</name>
+    <value>false</value>
+  </property>
+
+  <property>
     <description>The number of threads used to handle applications manager requests.</description>
     <name>yarn.resourcemanager.client.thread-count</name>
     <value>50</value>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.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/NodeHeartbeatResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
index 7568bbb..3b0ec10 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
@@ -104,10 +104,10 @@ public abstract class NodeHeartbeatResponse {
 
   public abstract void setResource(Resource resource);
 
-  public abstract List<Container> getContainersToDecrease();
+  public abstract List<Container> getContainersToUpdate();
 
-  public abstract void addAllContainersToDecrease(
-      Collection<Container> containersToDecrease);
+  public abstract void addAllContainersToUpdate(
+      Collection<Container> containersToUpdate);
 
   public abstract ContainerQueuingLimit getContainerQueuingLimit();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.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/NodeHeartbeatResponsePBImpl.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/NodeHeartbeatResponsePBImpl.java
index 51c1a78..46c2b0b 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/NodeHeartbeatResponsePBImpl.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/NodeHeartbeatResponsePBImpl.java
@@ -75,7 +75,7 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
   private MasterKey containerTokenMasterKey = null;
   private MasterKey nmTokenMasterKey = null;
   private ContainerQueuingLimit containerQueuingLimit = null;
-  private List<Container> containersToDecrease = null;
+  private List<Container> containersToUpdate = null;
   private List<SignalContainerRequest> containersToSignal = null;
 
   public NodeHeartbeatResponsePBImpl() {
@@ -119,8 +119,8 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
     if (this.systemCredentials != null) {
       addSystemCredentialsToProto();
     }
-    if (this.containersToDecrease != null) {
-      addContainersToDecreaseToProto();
+    if (this.containersToUpdate != null) {
+      addContainersToUpdateToProto();
     }
     if (this.containersToSignal != null) {
       addContainersToSignalToProto();
@@ -499,39 +499,39 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
     builder.addAllApplicationsToCleanup(iterable);
   }
 
-  private void initContainersToDecrease() {
-    if (this.containersToDecrease != null) {
+  private void initContainersToUpdate() {
+    if (this.containersToUpdate != null) {
       return;
     }
     NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
-    List<ContainerProto> list = p.getContainersToDecreaseList();
-    this.containersToDecrease = new ArrayList<>();
+    List<ContainerProto> list = p.getContainersToUpdateList();
+    this.containersToUpdate = new ArrayList<>();
 
     for (ContainerProto c : list) {
-      this.containersToDecrease.add(convertFromProtoFormat(c));
+      this.containersToUpdate.add(convertFromProtoFormat(c));
     }
   }
 
   @Override
-  public List<Container> getContainersToDecrease() {
-    initContainersToDecrease();
-    return this.containersToDecrease;
+  public List<Container> getContainersToUpdate() {
+    initContainersToUpdate();
+    return this.containersToUpdate;
   }
 
   @Override
-  public void addAllContainersToDecrease(
-      final Collection<Container> containersToDecrease) {
-    if (containersToDecrease == null) {
+  public void addAllContainersToUpdate(
+      final Collection<Container> containersToBeUpdated) {
+    if (containersToBeUpdated == null) {
       return;
     }
-    initContainersToDecrease();
-    this.containersToDecrease.addAll(containersToDecrease);
+    initContainersToUpdate();
+    this.containersToUpdate.addAll(containersToBeUpdated);
   }
 
-  private void addContainersToDecreaseToProto() {
+  private void addContainersToUpdateToProto() {
     maybeInitBuilder();
-    builder.clearContainersToDecrease();
-    if (this.containersToDecrease == null) {
+    builder.clearContainersToUpdate();
+    if (this.containersToUpdate == null) {
       return;
     }
     Iterable<ContainerProto> iterable = new
@@ -539,7 +539,7 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
       @Override
       public Iterator<ContainerProto> iterator() {
         return new Iterator<ContainerProto>() {
-          private Iterator<Container> iter = containersToDecrease.iterator();
+          private Iterator<Container> iter = containersToUpdate.iterator();
           @Override
           public boolean hasNext() {
             return iter.hasNext();
@@ -555,7 +555,7 @@ public class NodeHeartbeatResponsePBImpl extends NodeHeartbeatResponse {
         };
       }
     };
-    builder.addAllContainersToDecrease(iterable);
+    builder.addAllContainersToUpdate(iterable);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/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 edb2d9c..4e05fba 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
@@ -111,11 +111,14 @@ message NodeHeartbeatResponseProto {
   repeated ContainerIdProto containers_to_be_removed_from_nm = 9;
   repeated SystemCredentialsForAppsProto system_credentials_for_apps = 10;
   optional bool areNodeLabelsAcceptedByRM = 11 [default = false];
+  // to be deprecated in favour of containers_to_update
   repeated ContainerProto containers_to_decrease = 12;
   repeated SignalContainerRequestProto containers_to_signal = 13;
   optional ResourceProto resource = 14;
   optional ContainerQueuingLimitProto container_queuing_limit = 15;
   repeated AppCollectorsMapProto app_collectors_map = 16;
+  // to be used in place of containers_to_decrease
+  repeated ContainerProto containers_to_update = 17;
 }
 
 message ContainerQueuingLimitProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
index b670c36..8c0c73a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
@@ -180,14 +180,14 @@ public class TestYarnServerApiClasses {
   @Test
   public void testNodeHeartbeatResponsePBImplWithDecreasedContainers() {
     NodeHeartbeatResponsePBImpl original = new NodeHeartbeatResponsePBImpl();
-    original.addAllContainersToDecrease(
+    original.addAllContainersToUpdate(
         Arrays.asList(getDecreasedContainer(1, 2, 2048, 2),
             getDecreasedContainer(2, 3, 1024, 1)));
     NodeHeartbeatResponsePBImpl copy =
         new NodeHeartbeatResponsePBImpl(original.getProto());
-    assertEquals(1, copy.getContainersToDecrease().get(0)
+    assertEquals(1, copy.getContainersToUpdate().get(0)
         .getId().getContainerId());
-    assertEquals(1024, copy.getContainersToDecrease().get(1)
+    assertEquals(1024, copy.getContainersToUpdate().get(1)
         .getResource().getMemorySize());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/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 b5ec383..1d9256f 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
@@ -1099,7 +1099,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
                   parseCredentials(systemCredentials));
             }
             List<org.apache.hadoop.yarn.api.records.Container>
-                containersToDecrease = response.getContainersToDecrease();
+                containersToDecrease = response.getContainersToUpdate();
             if (!containersToDecrease.isEmpty()) {
               dispatcher.getEventHandler().handle(
                   new CMgrDecreaseContainersResourceEvent(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/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 a1e8ca0..12931bc 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
@@ -166,6 +166,7 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -1241,10 +1242,19 @@ public class ContainerManagerImpl extends CompositeService implements
     org.apache.hadoop.yarn.server.nodemanager.
         containermanager.container.ContainerState currentState =
         container.getContainerState();
-    if (currentState != org.apache.hadoop.yarn.server.
-            nodemanager.containermanager.container.ContainerState.RUNNING &&
-        currentState != org.apache.hadoop.yarn.server.
-            nodemanager.containermanager.container.ContainerState.SCHEDULED) {
+    EnumSet<org.apache.hadoop.yarn.server.nodemanager.containermanager
+        .container.ContainerState> allowedStates = EnumSet.of(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
+            .ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
+            .ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
+            .ContainerState.LOCALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
+            .ContainerState.REINITIALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
+            .ContainerState.RELAUNCHING);
+    if (!allowedStates.contains(currentState)) {
       throw RPCUtil.getRemoteException("Container " + containerId.toString()
           + " is in " + currentState.name() + " state."
           + " Resource can only be changed when a container is in"
@@ -1279,17 +1289,12 @@ public class ContainerManagerImpl extends CompositeService implements
             org.apache.hadoop.yarn.api.records.Container.newInstance(
                 containerId, null, null, targetResource, null, null,
                 currentExecType);
-      } else {
-        increasedContainer =
-            org.apache.hadoop.yarn.api.records.Container.newInstance(
-                containerId, null, null, currentResource, null, null,
-                targetExecType);
-      }
-      if (context.getIncreasedContainers().putIfAbsent(containerId,
-          increasedContainer) != null){
-        throw RPCUtil.getRemoteException("Container " + containerId.toString()
-            + " resource is being increased -or- " +
-            "is undergoing ExecutionType promoted.");
+        if (context.getIncreasedContainers().putIfAbsent(containerId,
+            increasedContainer) != null){
+          throw RPCUtil.getRemoteException("Container " + containerId.toString()
+              + " resource is being increased -or- " +
+              "is undergoing ExecutionType promoted.");
+        }
       }
     }
     this.readLock.lock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.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/scheduler/ContainerScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
index 19b4505..644bdae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
@@ -173,6 +173,7 @@ public class ContainerScheduler extends AbstractService implements
             new ChangeMonitoringContainerResourceEvent(containerId,
                 updateEvent.getUpdatedToken().getResource()));
       } else {
+        // Is Queued or localizing..
         updateEvent.getContainer().setContainerTokenIdentifier(
             updateEvent.getUpdatedToken());
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/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 aa7f524..e6f2bb2 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
@@ -551,7 +551,7 @@ public class ResourceTrackerService extends AbstractService implements
             getResponseId() + 1, NodeAction.NORMAL, null, null, null, null,
             nextHeartBeatInterval);
     rmNode.updateNodeHeartbeatResponseForCleanup(nodeHeartBeatResponse);
-    rmNode.updateNodeHeartbeatResponseForContainersDecreasing(
+    rmNode.updateNodeHeartbeatResponseForUpdatedContainers(
         nodeHeartBeatResponse);
 
     populateKeys(request, nodeHeartBeatResponse);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index 1e9463a..f49db7e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -51,8 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode
-    .RMNodeDecreaseContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeUpdateContainerEvent;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
@@ -284,7 +283,6 @@ public class RMContainerImpl implements RMContainer {
   @Override
   public RMContainerState getState() {
     this.readLock.lock();
-
     try {
       return this.stateMachine.getCurrentState();
     } finally {
@@ -598,7 +596,7 @@ public class RMContainerImpl implements RMContainer {
       RMContainerUpdatesAcquiredEvent acquiredEvent =
           (RMContainerUpdatesAcquiredEvent) event;
       if (acquiredEvent.isIncreasedContainer()) {
-        // If container is increased but not acquired by AM, we will start
+        // If container is increased but not started by AM, we will start
         // containerAllocationExpirer for this container in this transition. 
         container.containerAllocationExpirer.register(
             new AllocationExpirationInfo(event.getContainerId(), true));
@@ -641,7 +639,7 @@ public class RMContainerImpl implements RMContainer {
         container.lastConfirmedResource = rmContainerResource;
         container.containerAllocationExpirer.unregister(
             new AllocationExpirationInfo(event.getContainerId()));
-        container.eventHandler.handle(new RMNodeDecreaseContainerEvent(
+        container.eventHandler.handle(new RMNodeUpdateContainerEvent(
             container.nodeId,
             Collections.singletonList(container.getContainer())));
       } else if (Resources.fitsIn(nmContainerResource, rmContainerResource)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/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 86f8679..ab15c95 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
@@ -144,7 +144,7 @@ public interface RMNode {
    * applications to clean up for this node.
    * @param response the {@link NodeHeartbeatResponse} to update
    */
-  public void updateNodeHeartbeatResponseForCleanup(NodeHeartbeatResponse response);
+  void updateNodeHeartbeatResponseForCleanup(NodeHeartbeatResponse response);
 
   public NodeHeartbeatResponse getLastNodeHeartBeatResponse();
 
@@ -169,9 +169,9 @@ public interface RMNode {
   public Set<String> getNodeLabels();
   
   /**
-   * Update containers to be decreased
+   * Update containers to be updated
    */
-  public void updateNodeHeartbeatResponseForContainersDecreasing(
+  void updateNodeHeartbeatResponseForUpdatedContainers(
       NodeHeartbeatResponse response);
   
   public List<Container> pullNewlyIncreasedContainers();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeDecreaseContainerEvent.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/RMNodeDecreaseContainerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeDecreaseContainerEvent.java
deleted file mode 100644
index 62925ad..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeDecreaseContainerEvent.java
+++ /dev/null
@@ -1,39 +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.resourcemanager.rmnode;
-
-import java.util.List;
-
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.NodeId;
-
-public class RMNodeDecreaseContainerEvent extends RMNodeEvent {
-  final List<Container> toBeDecreasedContainers;
-
-  public RMNodeDecreaseContainerEvent(NodeId nodeId,
-      List<Container> toBeDecreasedContainers) {
-    super(nodeId, RMNodeEventType.DECREASE_CONTAINER);
-    
-    this.toBeDecreasedContainers = toBeDecreasedContainers;
-  }
-  
-  public List<Container> getToBeDecreasedContainers() {
-    return toBeDecreasedContainers;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.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/RMNodeEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
index b28fef3..a3b2ed7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
@@ -42,7 +42,7 @@ public enum RMNodeEventType {
   // Source: Container
   CONTAINER_ALLOCATED,
   CLEANUP_CONTAINER,
-  DECREASE_CONTAINER,
+  UPDATE_CONTAINER,
 
   // Source: ClientRMService
   SIGNAL_CONTAINER,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/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 1f121f8..1bdaa98 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
@@ -171,7 +171,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   private final List<ApplicationId> runningApplications =
       new ArrayList<ApplicationId>();
   
-  private final Map<ContainerId, Container> toBeDecreasedContainers =
+  private final Map<ContainerId, Container> toBeUpdatedContainers =
       new HashMap<>();
   
   private final Map<ContainerId, Container> nmReportedIncreasedContainers =
@@ -228,8 +228,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       .addTransition(NodeState.RUNNING, NodeState.RUNNING,
           RMNodeEventType.RESOURCE_UPDATE, new UpdateNodeResourceWhenRunningTransition())
       .addTransition(NodeState.RUNNING, NodeState.RUNNING,
-          RMNodeEventType.DECREASE_CONTAINER,
-          new DecreaseContainersTransition())
+          RMNodeEventType.UPDATE_CONTAINER,
+          new UpdateContainersTransition())
       .addTransition(NodeState.RUNNING, NodeState.RUNNING,
           RMNodeEventType.SIGNAL_CONTAINER, new SignalContainerTransition())
       .addTransition(NodeState.RUNNING, NodeState.SHUTDOWN,
@@ -614,18 +614,18 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   };
   
   @VisibleForTesting
-  public Collection<Container> getToBeDecreasedContainers() {
-    return toBeDecreasedContainers.values(); 
+  public Collection<Container> getToBeUpdatedContainers() {
+    return toBeUpdatedContainers.values();
   }
   
   @Override
-  public void updateNodeHeartbeatResponseForContainersDecreasing(
+  public void updateNodeHeartbeatResponseForUpdatedContainers(
       NodeHeartbeatResponse response) {
     this.writeLock.lock();
     
     try {
-      response.addAllContainersToDecrease(toBeDecreasedContainers.values());
-      toBeDecreasedContainers.clear();
+      response.addAllContainersToUpdate(toBeUpdatedContainers.values());
+      toBeUpdatedContainers.clear();
     } finally {
       this.writeLock.unlock();
     }
@@ -1031,16 +1031,19 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
           RMNodeFinishedContainersPulledByAMEvent) event).getContainers());
     }
   }
-  
-  public static class DecreaseContainersTransition
+
+  /**
+   * Transition to Update a container.
+   */
+  public static class UpdateContainersTransition
       implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
  
     @Override
     public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
-      RMNodeDecreaseContainerEvent de = (RMNodeDecreaseContainerEvent) event;
+      RMNodeUpdateContainerEvent de = (RMNodeUpdateContainerEvent) event;
 
-      for (Container c : de.getToBeDecreasedContainers()) {
-        rmNode.toBeDecreasedContainers.put(c.getId(), c);
+      for (Container c : de.getToBeUpdatedContainers()) {
+        rmNode.toBeUpdatedContainers.put(c.getId(), c);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeUpdateContainerEvent.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/RMNodeUpdateContainerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeUpdateContainerEvent.java
new file mode 100644
index 0000000..73af563
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeUpdateContainerEvent.java
@@ -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.
+*/
+
+package org.apache.hadoop.yarn.server.resourcemanager.rmnode;
+
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NodeId;
+
+/**
+ * This class is used to create update container event
+ * for the containers running on a node.
+ *
+ */
+public class RMNodeUpdateContainerEvent extends RMNodeEvent {
+  private List<Container> toBeUpdatedContainers;
+
+  public RMNodeUpdateContainerEvent(NodeId nodeId,
+      List<Container> toBeUpdatedContainers) {
+    super(nodeId, RMNodeEventType.UPDATE_CONTAINER);
+
+    this.toBeUpdatedContainers = toBeUpdatedContainers;
+  }
+
+  public List<Container> getToBeUpdatedContainers() {
+    return toBeUpdatedContainers;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/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 79caab0..c3879dd 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
@@ -150,6 +150,10 @@ public abstract class AbstractYarnScheduler
    */
   protected final ReentrantReadWriteLock.WriteLock writeLock;
 
+  // If set to true, then ALL container updates will be automatically sent to
+  // the NM in the next heartbeat.
+  private boolean autoUpdateContainers = false;
+
   /**
    * Construct the service.
    *
@@ -178,6 +182,9 @@ public abstract class AbstractYarnScheduler
         configuredMaximumAllocationWaitTime);
     maxClusterLevelAppPriority = getMaxPriorityFromConf(conf);
     createReleaseCache();
+    autoUpdateContainers =
+        conf.getBoolean(YarnConfiguration.RM_AUTO_UPDATE_CONTAINERS,
+            YarnConfiguration.DEFAULT_RM_AUTO_UPDATE_CONTAINERS);
     super.serviceInit(conf);
   }
 
@@ -235,6 +242,10 @@ public abstract class AbstractYarnScheduler
     return nodeTracker.getNodes(nodeFilter);
   }
 
+  public boolean shouldContainersBeAutoUpdated() {
+    return this.autoUpdateContainers;
+  }
+
   @Override
   public Resource getClusterResource() {
     return nodeTracker.getClusterCapacity();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.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/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index 397d507..cc14a1e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -74,8 +74,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerUpdatesAcquiredEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode
-    .RMNodeDecreaseContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeUpdateContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity;
@@ -663,20 +662,38 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
           + " an updated container " + container.getId(), e);
       return null;
     }
-    
-    if (updateType == null ||
-        ContainerUpdateType.PROMOTE_EXECUTION_TYPE == updateType ||
-        ContainerUpdateType.DEMOTE_EXECUTION_TYPE == updateType) {
+
+    if (updateType == null) {
+      // This is a newly allocated container
       rmContainer.handle(new RMContainerEvent(
           rmContainer.getContainerId(), RMContainerEventType.ACQUIRED));
     } else {
-      rmContainer.handle(new RMContainerUpdatesAcquiredEvent(
-          rmContainer.getContainerId(),
-          ContainerUpdateType.INCREASE_RESOURCE == updateType));
-      if (ContainerUpdateType.DECREASE_RESOURCE == updateType) {
+      // Resource increase is handled as follows:
+      // If the AM does not use the updated token to increase the container
+      // for a configured period of time, the RM will automatically rollback
+      // the update by performing a container decrease. This rollback (which
+      // essentially is another resource decrease update) is notified to the
+      // NM heartbeat response. If autoUpdate flag is set, then AM does not
+      // need to do anything - same code path as resource decrease.
+      //
+      // Resource Decrease is always automatic: the AM never has to do
+      // anything. It is always via NM heartbeat response.
+      //
+      // ExecutionType updates (both Promotion and Demotion) are either
+      // always automatic (if the flag is set) or the AM has to explicitly
+      // call updateContainer() on the NM. There is no expiry
+      boolean autoUpdate =
+          ContainerUpdateType.DECREASE_RESOURCE == updateType ||
+              ((AbstractYarnScheduler)rmContext.getScheduler())
+                  .shouldContainersBeAutoUpdated();
+      if (autoUpdate) {
         this.rmContext.getDispatcher().getEventHandler().handle(
-            new RMNodeDecreaseContainerEvent(rmContainer.getNodeId(),
+            new RMNodeUpdateContainerEvent(rmContainer.getNodeId(),
                 Collections.singletonList(rmContainer.getContainer())));
+      } else {
+        rmContainer.handle(new RMContainerUpdatesAcquiredEvent(
+            rmContainer.getContainerId(),
+            ContainerUpdateType.INCREASE_RESOURCE == updateType));
       }
     }
     return container;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/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 91170d1..7f58711 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
@@ -242,7 +242,7 @@ public class MockNodes {
     }
 
     @Override
-    public void updateNodeHeartbeatResponseForContainersDecreasing(
+    public void updateNodeHeartbeatResponseForUpdatedContainers(
         NodeHeartbeatResponse response) {
       
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.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/TestOpportunisticContainerAllocatorAMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
index 6819395..b885118 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
@@ -43,11 +43,13 @@ 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.api.records.UpdateContainerRequest;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocolPB;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
@@ -122,6 +124,21 @@ public class TestOpportunisticContainerAllocatorAMService {
     rm.start();
   }
 
+  public void createAndStartRMWithAutoUpdateContainer() {
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration();
+    YarnConfiguration conf = new YarnConfiguration(csConf);
+    conf.setBoolean(YarnConfiguration.RM_AUTO_UPDATE_CONTAINERS, true);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.setBoolean(
+        YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
+    conf.setInt(
+        YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS, 100);
+    rm = new MockRM(conf);
+    rm.start();
+  }
+
   @After
   public void stopRM() {
     if (rm != null) {
@@ -548,6 +565,157 @@ public class TestOpportunisticContainerAllocatorAMService {
     verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
   }
 
+  @Test(timeout = 600000)
+  public void testContainerAutoUpdateContainer() throws Exception {
+    rm.stop();
+    createAndStartRMWithAutoUpdateContainer();
+    MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
+    nm1.registerNode();
+
+    OpportunisticContainerAllocatorAMService amservice =
+        (OpportunisticContainerAllocatorAMService) rm
+            .getApplicationMasterService();
+    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
+    ApplicationAttemptId attemptId =
+        app1.getCurrentAppAttempt().getAppAttemptId();
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
+    ResourceScheduler scheduler = rm.getResourceScheduler();
+    RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    nm1.nodeHeartbeat(true);
+
+    ((RMNodeImpl) rmNode1)
+        .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
+
+    OpportunisticContainerContext ctxt =
+        ((CapacityScheduler) scheduler).getApplicationAttempt(attemptId)
+            .getOpportunisticContainerContext();
+    // Send add and update node events to AM Service.
+    amservice.handle(new NodeAddedSchedulerEvent(rmNode1));
+    amservice.handle(new NodeUpdateSchedulerEvent(rmNode1));
+
+    nm1.nodeHeartbeat(true);
+    Thread.sleep(1000);
+
+    AllocateResponse allocateResponse = am1.allocate(Arrays.asList(
+        ResourceRequest.newInstance(Priority.newInstance(1), "*",
+            Resources.createResource(1 * GB), 2, true, null,
+            ExecutionTypeRequest
+                .newInstance(ExecutionType.OPPORTUNISTIC, true))), null);
+    List<Container> allocatedContainers =
+        allocateResponse.getAllocatedContainers();
+    Assert.assertEquals(2, allocatedContainers.size());
+    Container container = allocatedContainers.get(0);
+    // Start Container in NM
+    nm1.nodeHeartbeat(Arrays.asList(ContainerStatus
+        .newInstance(container.getId(), ExecutionType.OPPORTUNISTIC,
+            ContainerState.RUNNING, "", 0)), true);
+    Thread.sleep(200);
+
+    // Verify that container is actually running wrt the RM..
+    RMContainer rmContainer = ((CapacityScheduler) scheduler)
+        .getApplicationAttempt(container.getId().getApplicationAttemptId())
+        .getRMContainer(container.getId());
+    Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+
+    // Send Promotion req... this should result in update error
+    // Since the container doesn't exist anymore..
+    allocateResponse = am1.sendContainerUpdateRequest(Arrays.asList(
+        UpdateContainerRequest.newInstance(0, container.getId(),
+            ContainerUpdateType.PROMOTE_EXECUTION_TYPE, null,
+            ExecutionType.GUARANTEED)));
+
+    nm1.nodeHeartbeat(Arrays.asList(ContainerStatus
+        .newInstance(container.getId(), ExecutionType.OPPORTUNISTIC,
+            ContainerState.RUNNING, "", 0)), true);
+    Thread.sleep(200);
+    // Get the update response on next allocate
+    allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
+    // Check the update response from YARNRM
+    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+    UpdatedContainer uc = allocateResponse.getUpdatedContainers().get(0);
+    Assert.assertEquals(container.getId(), uc.getContainer().getId());
+    Assert.assertEquals(ExecutionType.GUARANTEED,
+        uc.getContainer().getExecutionType());
+    // Check that the container is updated in NM through NM heartbeat response
+    NodeHeartbeatResponse response = nm1.nodeHeartbeat(true);
+    Assert.assertEquals(1, response.getContainersToUpdate().size());
+    Container containersFromNM = response.getContainersToUpdate().get(0);
+    Assert.assertEquals(container.getId(), containersFromNM.getId());
+    Assert.assertEquals(ExecutionType.GUARANTEED,
+        containersFromNM.getExecutionType());
+
+    //Increase resources
+    allocateResponse = am1.sendContainerUpdateRequest(Arrays.asList(
+        UpdateContainerRequest.newInstance(1, container.getId(),
+            ContainerUpdateType.INCREASE_RESOURCE,
+            Resources.createResource(2 * GB, 1), null)));
+    response = nm1.nodeHeartbeat(Arrays.asList(ContainerStatus
+        .newInstance(container.getId(), ExecutionType.GUARANTEED,
+            ContainerState.RUNNING, "", 0)), true);
+
+    Thread.sleep(200);
+    if (allocateResponse.getUpdatedContainers().size() == 0) {
+      allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
+    }
+    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+    uc = allocateResponse.getUpdatedContainers().get(0);
+    Assert.assertEquals(container.getId(), uc.getContainer().getId());
+    Assert.assertEquals(Resource.newInstance(2 * GB, 1),
+        uc.getContainer().getResource());
+
+    // Check that the container resources are increased in
+    // NM through NM heartbeat response
+    if (response.getContainersToUpdate().size() == 0) {
+      response = nm1.nodeHeartbeat(true);
+    }
+    Assert.assertEquals(1, response.getContainersToUpdate().size());
+    Assert.assertEquals(Resource.newInstance(2 * GB, 1),
+        response.getContainersToUpdate().get(0).getResource());
+
+    //Decrease resources
+    allocateResponse = am1.sendContainerUpdateRequest(Arrays.asList(
+        UpdateContainerRequest.newInstance(2, container.getId(),
+            ContainerUpdateType.DECREASE_RESOURCE,
+            Resources.createResource(1 * GB, 1), null)));
+    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+
+    // Check that the container resources are decreased
+    // in NM through NM heartbeat response
+    response = nm1.nodeHeartbeat(true);
+    Assert.assertEquals(1, response.getContainersToUpdate().size());
+    Assert.assertEquals(Resource.newInstance(1 * GB, 1),
+        response.getContainersToUpdate().get(0).getResource());
+
+    nm1.nodeHeartbeat(true);
+    // DEMOTE the container
+    allocateResponse = am1.sendContainerUpdateRequest(Arrays.asList(
+        UpdateContainerRequest.newInstance(3, container.getId(),
+            ContainerUpdateType.DEMOTE_EXECUTION_TYPE, null,
+            ExecutionType.OPPORTUNISTIC)));
+
+    response = nm1.nodeHeartbeat(Arrays.asList(ContainerStatus
+        .newInstance(container.getId(), ExecutionType.GUARANTEED,
+            ContainerState.RUNNING, "", 0)), true);
+    Thread.sleep(200);
+    if (allocateResponse.getUpdatedContainers().size() == 0) {
+      // Get the update response on next allocate
+      allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
+    }
+    // Check the update response from YARNRM
+    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+    uc = allocateResponse.getUpdatedContainers().get(0);
+    Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
+        uc.getContainer().getExecutionType());
+    // Check that the container is updated in NM through NM heartbeat response
+    if (response.getContainersToUpdate().size() == 0) {
+      response = nm1.nodeHeartbeat(true);
+    }
+    Assert.assertEquals(1, response.getContainersToUpdate().size());
+    Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
+        response.getContainersToUpdate().get(0).getExecutionType());
+  }
+
   private void verifyMetrics(QueueMetrics metrics, long availableMB,
       int availableVirtualCores, long allocatedMB,
       int allocatedVirtualCores, int allocatedContainers) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java
index b4b05ed..291a74e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java
@@ -51,8 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-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.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica
@@ -60,11 +59,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimplePlacementSet;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 
 public class TestContainerResizing {
@@ -205,7 +202,7 @@ public class TestContainerResizing {
     RMNodeImpl rmNode =
         (RMNodeImpl) rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
     Collection<Container> decreasedContainers =
-        rmNode.getToBeDecreasedContainers();
+        rmNode.getToBeUpdatedContainers();
     boolean rmNodeReceivedDecreaseContainer = false;
     for (Container c : decreasedContainers) {
       if (c.getId().equals(containerId1)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8410d862/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestIncreaseAllocationExpirer.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/TestIncreaseAllocationExpirer.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/TestIncreaseAllocationExpirer.java
index 184e854..a76ed64 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/TestIncreaseAllocationExpirer.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/TestIncreaseAllocationExpirer.java
@@ -319,7 +319,7 @@ public class TestIncreaseAllocationExpirer {
     verifyAvailableResourceOfSchedulerNode(rm1, nm1.getNodeId(), 16 * GB);
     // Verify NM receives the decrease message (3G)
     List<Container> containersToDecrease =
-        nm1.nodeHeartbeat(true).getContainersToDecrease();
+        nm1.nodeHeartbeat(true).getContainersToUpdate();
     Assert.assertEquals(1, containersToDecrease.size());
     Assert.assertEquals(
         3 * GB, containersToDecrease.get(0).getResource().getMemorySize());
@@ -435,7 +435,7 @@ public class TestIncreaseAllocationExpirer {
             .getAllocatedResource().getMemorySize());
     // Verify NM receives 2 decrease message
     List<Container> containersToDecrease =
-        nm1.nodeHeartbeat(true).getContainersToDecrease();
+        nm1.nodeHeartbeat(true).getContainersToUpdate();
     Assert.assertEquals(2, containersToDecrease.size());
     // Sort the list to make sure containerId3 is the first
     Collections.sort(containersToDecrease);


---------------------------------------------------------------------
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: HADOOP-14560. Make HttpServer2 backlog size configurable. Contributed by Alexander Krasheninnikov.

Posted by ae...@apache.org.
HADOOP-14560. Make HttpServer2 backlog size configurable. Contributed by Alexander Krasheninnikov.

This closes #242.


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

Branch: refs/heads/HDFS-7240
Commit: 1f04cb45f70648678840cdafbec68d534b03fe95
Parents: 96b3a6b
Author: Alexandr Krasheninnikov <a....@corp.badoo.com>
Authored: Wed Jun 21 12:57:34 2017 +0300
Committer: John Zhuge <jz...@apache.org>
Committed: Thu Aug 17 01:05:19 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/http/HttpServer2.java  |  9 ++++++++-
 .../java/org/apache/hadoop/http/TestHttpServer.java    | 13 +++++++++++++
 2 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f04cb45/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
index 28b9bb0..a450f66 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
@@ -128,6 +128,10 @@ public final class HttpServer2 implements FilterContainer {
   public static final String HTTP_MAX_RESPONSE_HEADER_SIZE_KEY =
       "hadoop.http.max.response.header.size";
   public static final int HTTP_MAX_RESPONSE_HEADER_SIZE_DEFAULT = 65536;
+
+  public static final String HTTP_SOCKET_BACKLOG_SIZE_KEY =
+      "hadoop.http.socket.backlog.size";
+  public static final int HTTP_SOCKET_BACKLOG_SIZE_DEFAULT = 128;
   public static final String HTTP_MAX_THREADS_KEY = "hadoop.http.max.threads";
   public static final String HTTP_TEMP_DIR_KEY = "hadoop.http.temp.dir";
 
@@ -433,6 +437,9 @@ public final class HttpServer2 implements FilterContainer {
       httpConfig.setResponseHeaderSize(responseHeaderSize);
       httpConfig.setSendServerVersion(false);
 
+      int backlogSize = conf.getInt(HTTP_SOCKET_BACKLOG_SIZE_KEY,
+          HTTP_SOCKET_BACKLOG_SIZE_DEFAULT);
+
       for (URI ep : endpoints) {
         final ServerConnector connector;
         String scheme = ep.getScheme();
@@ -448,6 +455,7 @@ public final class HttpServer2 implements FilterContainer {
         }
         connector.setHost(ep.getHost());
         connector.setPort(ep.getPort() == -1 ? 0 : ep.getPort());
+        connector.setAcceptQueueSize(backlogSize);
         server.addListener(connector);
       }
       server.loadListeners();
@@ -640,7 +648,6 @@ public final class HttpServer2 implements FilterContainer {
 
   private static void configureChannelConnector(ServerConnector c) {
     c.setIdleTimeout(10000);
-    c.setAcceptQueueSize(128);
     if(Shell.WINDOWS) {
       // result of setting the SO_REUSEADDR flag is different on Windows
       // http://msdn.microsoft.com/en-us/library/ms740621(v=vs.85).aspx

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f04cb45/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
index 6ec6e0f..ca7e466 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
@@ -682,4 +682,17 @@ public class TestHttpServer extends HttpServerFunctionalTest {
       stopHttpServer(myServer2);
     }
   }
+
+  @Test
+  public void testBacklogSize() throws Exception
+  {
+    final int backlogSize = 2048;
+    Configuration conf = new Configuration();
+    conf.setInt(HttpServer2.HTTP_SOCKET_BACKLOG_SIZE_KEY, backlogSize);
+    HttpServer2 srv = createServer("test", conf);
+    List<?> listeners = (List<?>) Whitebox.getInternalState(srv,
+            "listeners");
+    ServerConnector listener = (ServerConnector)listeners.get(0);
+    assertEquals(backlogSize, listener.getAcceptQueueSize());
+  }
 }


---------------------------------------------------------------------
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-6881. LOG is unused in AllocationConfiguration (Contributed by weiyuan via Daniel Templeton)

Posted by ae...@apache.org.
YARN-6881. LOG is unused in AllocationConfiguration (Contributed by weiyuan via Daniel Templeton)


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

Branch: refs/heads/HDFS-7240
Commit: 6b09c327057947049ef7984afbb5ed225f15fc2d
Parents: 608a06c
Author: Daniel Templeton <te...@apache.org>
Authored: Mon Aug 14 11:55:33 2017 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Mon Aug 14 11:55:33 2017 -0700

----------------------------------------------------------------------
 .../resourcemanager/scheduler/fair/AllocationConfiguration.java   | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b09c327/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
index f143aa6..71e6f7f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
@@ -23,8 +23,6 @@ import java.util.HashSet;
 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.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -41,7 +39,6 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import com.google.common.annotations.VisibleForTesting;
 
 public class AllocationConfiguration extends ReservationSchedulerConfiguration {
-  private static final Log LOG = LogFactory.getLog(FSQueue.class.getName());
   private static final AccessControlList EVERYBODY_ACL = new AccessControlList("*");
   private static final AccessControlList NOBODY_ACL = new AccessControlList(" ");
   private static final ResourceCalculator RESOURCE_CALCULATOR =


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

[18/50] [abbrv] hadoop git commit: HDFS-11696. Fix warnings from Spotbugs in hadoop-hdfs. Contributed by Yiqun Lin.

Posted by ae...@apache.org.
HDFS-11696. Fix warnings from Spotbugs in hadoop-hdfs. Contributed by Yiqun Lin.


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

Branch: refs/heads/HDFS-7240
Commit: 2e43c28e01fe006210e71aab179527669f6412ed
Parents: 645a8f2
Author: Yiqun Lin <yq...@apache.org>
Authored: Tue Aug 15 16:48:49 2017 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Tue Aug 15 16:48:49 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  7 +++--
 .../hdfs/server/protocol/SlowDiskReports.java   |  5 ++--
 .../dev-support/findbugsExcludeFile.xml         | 26 +++++++++++++++++++
 .../hdfs/qjournal/server/JournalNode.java       | 16 +++++++-----
 .../hdfs/server/datanode/DataStorage.java       | 12 ++++++---
 .../namenode/NNStorageRetentionManager.java     | 27 +++++++++++---------
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |  6 ++---
 .../offlineImageViewer/ImageLoaderCurrent.java  | 10 +++++---
 .../namenode/TestNameNodeOptionParsing.java     | 27 +++++++++++++++++++-
 9 files changed, 103 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 677ea35..88b273a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -2901,9 +2901,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
     synchronized (DFSClient.class) {
       if (STRIPED_READ_THREAD_POOL == null) {
-        STRIPED_READ_THREAD_POOL = DFSUtilClient.getThreadPoolExecutor(1,
+        // Only after thread pool is fully constructed then save it to
+        // volatile field.
+        ThreadPoolExecutor threadPool = DFSUtilClient.getThreadPoolExecutor(1,
             numThreads, 60, "StripedRead-", true);
-        STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
+        threadPool.allowCoreThreadTimeOut(true);
+        STRIPED_READ_THREAD_POOL = threadPool;
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowDiskReports.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowDiskReports.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowDiskReports.java
index 8095c2a..496389a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowDiskReports.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowDiskReports.java
@@ -101,8 +101,9 @@ public final class SlowDiskReports {
     }
 
     boolean areEqual;
-    for (String disk : this.slowDisks.keySet()) {
-      if (!this.slowDisks.get(disk).equals(that.slowDisks.get(disk))) {
+    for (Map.Entry<String, Map<DiskOp, Double>> entry : this.slowDisks
+        .entrySet()) {
+      if (!entry.getValue().equals(that.slowDisks.get(entry.getKey()))) {
         return false;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
index 2a7824a..9582fcb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -264,4 +264,30 @@
         <Field name="locations" />
         <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED" />
     </Match>
+    <Match>
+       <Class name="org.apache.hadoop.hdfs.server.namenode.NNUpgradeUtil$1" />
+       <Method name="visitFile" />
+       <Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
+     </Match>
+     <!-- Ignore warnings for not changing the startup option parsing behavior. -->
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.common.HdfsServerConstants$StartupOption" />
+       <Method name="setClusterId" />
+       <Bug pattern="ME_ENUM_FIELD_SETTER" />
+     </Match>
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.common.HdfsServerConstants$StartupOption" />
+       <Method name="setForce" />
+       <Bug pattern="ME_ENUM_FIELD_SETTER" />
+     </Match>
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.common.HdfsServerConstants$StartupOption" />
+       <Method name="setForceFormat" />
+       <Bug pattern="ME_ENUM_FIELD_SETTER" />
+     </Match>
+     <Match>
+      <Class name="org.apache.hadoop.hdfs.server.common.HdfsServerConstants$StartupOption" />
+      <Method name="setInteractiveFormat" />
+      <Bug pattern="ME_ENUM_FIELD_SETTER" />
+    </Match>
  </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
index af7a84f..6056e34 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
@@ -299,14 +299,18 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
         return file.isDirectory();
       }
     });
-    for (File journalDir : journalDirs) {
-      String jid = journalDir.getName();
-      if (!status.containsKey(jid)) {
-        Map<String, String> jMap = new HashMap<String, String>();
-        jMap.put("Formatted", "true");
-        status.put(jid, jMap);
+
+    if (journalDirs != null) {
+      for (File journalDir : journalDirs) {
+        String jid = journalDir.getName();
+        if (!status.containsKey(jid)) {
+          Map<String, String> jMap = new HashMap<String, String>();
+          jMap.put("Formatted", "true");
+          status.put(jid, jMap);
+        }
       }
     }
+
     return JSON.toString(status);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index 9a71081..6d6e96a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -1336,10 +1336,14 @@ public class DataStorage extends Storage {
           return name.startsWith(BLOCK_SUBDIR_PREFIX);
         }
       });
-    for(int i = 0; i < otherNames.length; i++)
-      linkBlocksHelper(new File(from, otherNames[i]),
-          new File(to, otherNames[i]), oldLV, hl, upgradeToIdBasedLayout,
-          blockRoot, idBasedLayoutSingleLinks);
+
+    if (otherNames != null) {
+      for (int i = 0; i < otherNames.length; i++) {
+        linkBlocksHelper(new File(from, otherNames[i]),
+            new File(to, otherNames[i]), oldLV, hl, upgradeToIdBasedLayout,
+            blockRoot, idBasedLayoutSingleLinks);
+      }
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
index 98b7e9a..2a83541 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
@@ -255,24 +255,27 @@ public class NNStorageRetentionManager {
     });
 
     // Check whether there is any work to do.
-    if (filesInStorage.length <= numCheckpointsToRetain) {
+    if (filesInStorage != null
+        && filesInStorage.length <= numCheckpointsToRetain) {
       return;
     }
 
     // Create a sorted list of txids from the file names.
     TreeSet<Long> sortedTxIds = new TreeSet<Long>();
-    for (String fName : filesInStorage) {
-      // Extract the transaction id from the file name.
-      long fTxId;
-      try {
-        fTxId = Long.parseLong(fName.substring(oivImagePrefix.length() + 1));
-      } catch (NumberFormatException nfe) {
-        // This should not happen since we have already filtered it.
-        // Log and continue.
-        LOG.warn("Invalid file name. Skipping " + fName);
-        continue;
+    if (filesInStorage != null) {
+      for (String fName : filesInStorage) {
+        // Extract the transaction id from the file name.
+        long fTxId;
+        try {
+          fTxId = Long.parseLong(fName.substring(oivImagePrefix.length() + 1));
+        } catch (NumberFormatException nfe) {
+          // This should not happen since we have already filtered it.
+          // Log and continue.
+          LOG.warn("Invalid file name. Skipping " + fName);
+          continue;
+        }
+        sortedTxIds.add(Long.valueOf(fTxId));
       }
-      sortedTxIds.add(Long.valueOf(fTxId));
     }
 
     int numFilesToDelete = sortedTxIds.size() - numCheckpointsToRetain;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index f2233eb..1fb1d5f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -1992,7 +1992,7 @@ public class DFSAdmin extends FsShell {
         return exitCode;
       }
     } else if ("-report".equals(cmd)) {
-      if (argv.length < 1) {
+      if (argv.length > 6) {
         printUsage(cmd);
         return exitCode;
       }
@@ -2022,7 +2022,7 @@ public class DFSAdmin extends FsShell {
         return exitCode;
       }
     } else if (RollingUpgradeCommand.matches(cmd)) {
-      if (argv.length < 1 || argv.length > 2) {
+      if (argv.length > 2) {
         printUsage(cmd);
         return exitCode;
       }
@@ -2097,7 +2097,7 @@ public class DFSAdmin extends FsShell {
         return exitCode;
       }
     } else if ("-triggerBlockReport".equals(cmd)) {
-      if (argv.length < 1) {
+      if ((argv.length != 2) && (argv.length != 3)) {
         printUsage(cmd);
         return exitCode;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
index f2c7427..2e2eaf4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
@@ -722,9 +722,13 @@ class ImageLoaderCurrent implements ImageLoader {
       if (supportSnapshot && supportInodeId) {
         dirNodeMap.put(inodeId, pathName);
       }
-      v.visit(ImageElement.NS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
-      if (NameNodeLayoutVersion.supports(Feature.DISKSPACE_QUOTA, imageVersion))
-        v.visit(ImageElement.DS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
+
+      v.visit(ImageElement.NS_QUOTA, in.readLong());
+      if (NameNodeLayoutVersion.supports(Feature.DISKSPACE_QUOTA,
+          imageVersion)) {
+        v.visit(ImageElement.DS_QUOTA, in.readLong());
+      }
+
       if (supportSnapshot) {
         boolean snapshottable = in.readBoolean();
         if (!snapshottable) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java
index 7ee49a9..92b96a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java
@@ -138,5 +138,30 @@ public class TestNameNodeOptionParsing {
       }
     }
   }
-    
+
+  @Test
+  public void testFormat() {
+    String[] args = new String[] {"-format"};
+    StartupOption opt = NameNode.parseArguments(args);
+    assertEquals(StartupOption.FORMAT, opt);
+    assertEquals(true, opt.getInteractiveFormat());
+    assertEquals(false, opt.getForceFormat());
+
+    args = new String[] {"-format", "-nonInteractive"};
+    opt = NameNode.parseArguments(args);
+    assertEquals(StartupOption.FORMAT, opt);
+    assertEquals(false, opt.getInteractiveFormat());
+    assertEquals(false, opt.getForceFormat());
+
+    args = new String[] {"-format", "-nonInteractive", "-force"};
+    opt = NameNode.parseArguments(args);
+    assertEquals(StartupOption.FORMAT, opt);
+    assertEquals(false, opt.getInteractiveFormat());
+    assertEquals(true, opt.getForceFormat());
+
+    // test error condition
+    args = new String[] {"-nonInteractive"};
+    opt = NameNode.parseArguments(args);
+    assertNull(opt);
+  }
 }


---------------------------------------------------------------------
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: HDFS-11738. Hedged pread takes more time when block moved from initial locations. Contributed by Vinayakumar B.

Posted by ae...@apache.org.
HDFS-11738. Hedged pread takes more time when block moved from initial locations. Contributed by Vinayakumar B.


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

Branch: refs/heads/HDFS-7240
Commit: b6bfb2fcb2391d51b8de97c01c1290880779132e
Parents: 736ceab
Author: John Zhuge <jz...@apache.org>
Authored: Mon Aug 21 13:44:32 2017 -0700
Committer: John Zhuge <jz...@apache.org>
Committed: Mon Aug 21 13:45:30 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/DFSClientFaultInjector.java     |   2 +
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 145 +++++++++++--------
 .../java/org/apache/hadoop/hdfs/TestPread.java  |  26 +++-
 3 files changed, 112 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6bfb2fc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
index 748edcd..b58cf16 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
@@ -61,4 +61,6 @@ public class DFSClientFaultInjector {
   public boolean skipRollingRestartWait() {
     return false;
   }
+
+  public void sleepBeforeHedgedGet() {}
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6bfb2fc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 6bff172..97d3de4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -830,60 +830,85 @@ public class DFSInputStream extends FSInputStream
 
   private DNAddrPair chooseDataNode(LocatedBlock block,
       Collection<DatanodeInfo> ignoredNodes) throws IOException {
+    return chooseDataNode(block, ignoredNodes, true);
+  }
+
+  /**
+   * Choose datanode to read from.
+   *
+   * @param block             Block to choose datanode addr from
+   * @param ignoredNodes      Ignored nodes inside.
+   * @param refetchIfRequired Whether to refetch if no nodes to chose
+   *                          from.
+   * @return Returns chosen DNAddrPair; Can be null if refetchIfRequired is
+   * false.
+   */
+  private DNAddrPair chooseDataNode(LocatedBlock block,
+      Collection<DatanodeInfo> ignoredNodes, boolean refetchIfRequired)
+      throws IOException {
     while (true) {
       DNAddrPair result = getBestNodeDNAddrPair(block, ignoredNodes);
       if (result != null) {
         return result;
+      } else if (refetchIfRequired) {
+        block = refetchLocations(block, ignoredNodes);
       } else {
-        String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
-            deadNodes, ignoredNodes);
-        String blockInfo = block.getBlock() + " file=" + src;
-        if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
-          String description = "Could not obtain block: " + blockInfo;
-          DFSClient.LOG.warn(description + errMsg
-              + ". Throwing a BlockMissingException");
-          throw new BlockMissingException(src, description,
-              block.getStartOffset());
-        }
-
-        DatanodeInfo[] nodes = block.getLocations();
-        if (nodes == null || nodes.length == 0) {
-          DFSClient.LOG.info("No node available for " + blockInfo);
-        }
-        DFSClient.LOG.info("Could not obtain " + block.getBlock()
-            + " from any node: " + errMsg
-            + ". Will get new block locations from namenode and retry...");
-        try {
-          // Introducing a random factor to the wait time before another retry.
-          // The wait time is dependent on # of failures and a random factor.
-          // At the first time of getting a BlockMissingException, the wait time
-          // is a random number between 0..3000 ms. If the first retry
-          // still fails, we will wait 3000 ms grace period before the 2nd retry.
-          // Also at the second retry, the waiting window is expanded to 6000 ms
-          // alleviating the request rate from the server. Similarly the 3rd retry
-          // will wait 6000ms grace period before retry and the waiting window is
-          // expanded to 9000ms.
-          final int timeWindow = dfsClient.getConf().getTimeWindow();
-          double waitTime = timeWindow * failures +       // grace period for the last round of attempt
-              // expanding time window for each failure
-              timeWindow * (failures + 1) *
-              ThreadLocalRandom.current().nextDouble();
-          DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) +
-              " IOException, will wait for " + waitTime + " msec.");
-          Thread.sleep((long)waitTime);
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new InterruptedIOException(
-              "Interrupted while choosing DataNode for read.");
-        }
-        deadNodes.clear(); //2nd option is to remove only nodes[blockId]
-        openInfo(true);
-        block = refreshLocatedBlock(block);
-        failures++;
+        return null;
       }
     }
   }
 
+  private LocatedBlock refetchLocations(LocatedBlock block,
+      Collection<DatanodeInfo> ignoredNodes) throws IOException {
+    String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
+        deadNodes, ignoredNodes);
+    String blockInfo = block.getBlock() + " file=" + src;
+    if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
+      String description = "Could not obtain block: " + blockInfo;
+      DFSClient.LOG.warn(description + errMsg
+          + ". Throwing a BlockMissingException");
+      throw new BlockMissingException(src, description,
+          block.getStartOffset());
+    }
+
+    DatanodeInfo[] nodes = block.getLocations();
+    if (nodes == null || nodes.length == 0) {
+      DFSClient.LOG.info("No node available for " + blockInfo);
+    }
+    DFSClient.LOG.info("Could not obtain " + block.getBlock()
+        + " from any node: " + errMsg
+        + ". Will get new block locations from namenode and retry...");
+    try {
+      // Introducing a random factor to the wait time before another retry.
+      // The wait time is dependent on # of failures and a random factor.
+      // At the first time of getting a BlockMissingException, the wait time
+      // is a random number between 0..3000 ms. If the first retry
+      // still fails, we will wait 3000 ms grace period before the 2nd retry.
+      // Also at the second retry, the waiting window is expanded to 6000 ms
+      // alleviating the request rate from the server. Similarly the 3rd retry
+      // will wait 6000ms grace period before retry and the waiting window is
+      // expanded to 9000ms.
+      final int timeWindow = dfsClient.getConf().getTimeWindow();
+      // grace period for the last round of attempt
+      double waitTime = timeWindow * failures +
+          // expanding time window for each failure
+          timeWindow * (failures + 1) *
+          ThreadLocalRandom.current().nextDouble();
+      DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) +
+          " IOException, will wait for " + waitTime + " msec.");
+      Thread.sleep((long)waitTime);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new InterruptedIOException(
+          "Interrupted while choosing DataNode for read.");
+    }
+    deadNodes.clear(); //2nd option is to remove only nodes[blockId]
+    openInfo(true);
+    block = refreshLocatedBlock(block);
+    failures++;
+    return block;
+  }
+
   /**
    * Get the best node from which to stream the data.
    * @param block LocatedBlock, containing nodes in priority order.
@@ -985,6 +1010,7 @@ public class DFSInputStream extends FSInputStream
     return new Callable<ByteBuffer>() {
       @Override
       public ByteBuffer call() throws Exception {
+        DFSClientFaultInjector.get().sleepBeforeHedgedGet();
         try (TraceScope ignored = dfsClient.getTracer().
             newScope("hedgedRead" + hedgedReadId, parentSpanId)) {
           actualGetFromOneDataNode(datanode, start, end, bb, corruptedBlocks);
@@ -1159,20 +1185,22 @@ public class DFSInputStream extends FSInputStream
         // We are starting up a 'hedged' read. We have a read already
         // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
         // If no nodes to do hedged reads against, pass.
+        boolean refetch = false;
         try {
-          chosenNode = getBestNodeDNAddrPair(block, ignored);
-          if (chosenNode == null) {
-            chosenNode = chooseDataNode(block, ignored);
+          chosenNode = chooseDataNode(block, ignored, false);
+          if (chosenNode != null) {
+            // Latest block, if refreshed internally
+            block = chosenNode.block;
+            bb = ByteBuffer.allocate(len);
+            Callable<ByteBuffer> getFromDataNodeCallable =
+                getFromOneDataNode(chosenNode, block, start, end, bb,
+                    corruptedBlocks, hedgedReadId++);
+            Future<ByteBuffer> oneMoreRequest =
+                hedgedService.submit(getFromDataNodeCallable);
+            futures.add(oneMoreRequest);
+          } else {
+            refetch = true;
           }
-          // Latest block, if refreshed internally
-          block = chosenNode.block;
-          bb = ByteBuffer.allocate(len);
-          Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
-              chosenNode, block, start, end, bb,
-              corruptedBlocks, hedgedReadId++);
-          Future<ByteBuffer> oneMoreRequest = hedgedService
-              .submit(getFromDataNodeCallable);
-          futures.add(oneMoreRequest);
         } catch (IOException ioe) {
           DFSClient.LOG.debug("Failed getting node for hedged read: {}",
               ioe.getMessage());
@@ -1190,6 +1218,9 @@ public class DFSInputStream extends FSInputStream
         } catch (InterruptedException ie) {
           // Ignore and retry
         }
+        if (refetch) {
+          refetchLocations(block, ignored);
+        }
         // We got here if exception. Ignore this node on next go around IFF
         // we found a chosenNode to hedge read against.
         if (chosenNode != null && chosenNode.info != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6bfb2fc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
index bcb02b3..0834d30 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
@@ -626,7 +626,7 @@ public class TestPread {
    */
   @Test
   public void testPreadFailureWithChangedBlockLocations() throws Exception {
-    doPreadTestWithChangedLocations();
+    doPreadTestWithChangedLocations(1);
   }
 
   /**
@@ -639,21 +639,36 @@ public class TestPread {
    * 7. Consider next calls to getBlockLocations() always returns DN3 as last
    * location.<br>
    */
-  @Test
+  @Test(timeout = 60000)
   public void testPreadHedgedFailureWithChangedBlockLocations()
       throws Exception {
     isHedgedRead = true;
-    doPreadTestWithChangedLocations();
+    DFSClientFaultInjector old = DFSClientFaultInjector.get();
+    try {
+      DFSClientFaultInjector.set(new DFSClientFaultInjector() {
+        public void sleepBeforeHedgedGet() {
+          try {
+            Thread.sleep(500);
+          } catch (InterruptedException e) {
+          }
+        }
+      });
+      doPreadTestWithChangedLocations(2);
+    } finally {
+      DFSClientFaultInjector.set(old);
+    }
   }
 
-  private void doPreadTestWithChangedLocations()
+  private void doPreadTestWithChangedLocations(int maxFailures)
       throws IOException, TimeoutException, InterruptedException {
     GenericTestUtils.setLogLevel(DFSClient.LOG, Level.DEBUG);
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 2);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     if (isHedgedRead) {
+      conf.setInt(HdfsClientConfigKeys.HedgedRead.THRESHOLD_MILLIS_KEY, 100);
       conf.setInt(HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY, 2);
+      conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 1000);
     }
     try (MiniDFSCluster cluster =
         new MiniDFSCluster.Builder(conf).numDataNodes(3).build()) {
@@ -747,6 +762,9 @@ public class TestPread {
       int n = din.read(0, buf, 0, data.length());
       assertEquals(data.length(), n);
       assertEquals("Data should be read", data, new String(buf, 0, n));
+      assertTrue("Read should complete with maximum " + maxFailures
+              + " failures, but completed with " + din.failures,
+          din.failures <= maxFailures);
       DFSClient.LOG.info("Read completed");
     }
   }


---------------------------------------------------------------------
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: HADOOP-14769. WASB: delete recursive should not fail if a file is deleted. Contributed by Thomas Marquardt

Posted by ae...@apache.org.
HADOOP-14769. WASB: delete recursive should not fail if a file is deleted.
Contributed by Thomas Marquardt


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

Branch: refs/heads/HDFS-7240
Commit: c6b4e656b76b68cc1d0dbcc15a5aa5ea23335b7b
Parents: 99e558b
Author: Steve Loughran <st...@apache.org>
Authored: Fri Aug 18 14:13:40 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Aug 18 14:13:40 2017 +0100

----------------------------------------------------------------------
 .../fs/azure/AzureNativeFileSystemStore.java    | 21 ++++---
 .../hadoop/fs/azure/NativeAzureFileSystem.java  | 47 ++++++++-------
 .../TestFileSystemOperationsWithThreads.java    | 61 ++++++++++++++++----
 3 files changed, 86 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6b4e656/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
index 554027b..b0cd701 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
@@ -2459,8 +2459,11 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     try {
       blob.delete(operationContext, lease);
     } catch (StorageException e) {
-      LOG.error("Encountered Storage Exception for delete on Blob: {}, Exception Details: {} Error Code: {}",
-          blob.getUri(), e.getMessage(), e.getErrorCode());
+      if (!NativeAzureFileSystemHelper.isFileNotFoundException(e)) {
+        LOG.error("Encountered Storage Exception for delete on Blob: {}"
+            + ", Exception Details: {} Error Code: {}",
+            blob.getUri(), e.getMessage(), e.getErrorCode());
+      }
       // On exception, check that if:
       // 1. It's a BlobNotFound exception AND
       // 2. It got there after one-or-more retries THEN
@@ -2491,17 +2494,17 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
         // Container doesn't exist, no need to do anything
         return true;
       }
-
       // Get the blob reference and delete it.
       CloudBlobWrapper blob = getBlobReference(key);
-      if (blob.exists(getInstrumentedContext())) {
-        safeDelete(blob, lease);
-        return true;
-      } else {
+      safeDelete(blob, lease);
+      return true;
+    } catch (Exception e) {
+      if (e instanceof StorageException
+          && NativeAzureFileSystemHelper.isFileNotFoundException(
+              (StorageException) e)) {
+        // the file or directory does not exist
         return false;
       }
-    } catch (Exception e) {
-      // Re-throw as an Azure storage exception.
       throw new AzureException(e);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6b4e656/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
index a7558a3..2abc6c6 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
@@ -2043,7 +2043,12 @@ public class NativeAzureFileSystem extends FileSystem {
       AzureFileSystemThreadTask task = new AzureFileSystemThreadTask() {
         @Override
         public boolean execute(FileMetadata file) throws IOException{
-          return deleteFile(file.getKey(), file.isDir());
+          if (!deleteFile(file.getKey(), file.isDir())) {
+            LOG.warn("Attempt to delete non-existent {} {}",
+                file.isDir() ? "directory" : "file",
+                file.getKey());
+          }
+          return true;
         }
       };
 
@@ -2080,30 +2085,28 @@ public class NativeAzureFileSystem extends FileSystem {
     return new AzureFileSystemThreadPoolExecutor(threadCount, threadNamePrefix, operation, key, config);
   }
 
-  // Delete single file / directory from key.
+  /**
+   * Delete the specified file or directory and increment metrics.
+   * If the file or directory does not exist, the operation returns false.
+   * @param path the path to a file or directory.
+   * @param isDir true if the path is a directory; otherwise false.
+   * @return true if delete is successful; otherwise false.
+   * @throws IOException if an IO error occurs while attempting to delete the
+   * path.
+   *
+   */
   @VisibleForTesting
-  boolean deleteFile(String key, boolean isDir) throws IOException {
-    try {
-      if (store.delete(key)) {
-        if (isDir) {
-          instrumentation.directoryDeleted();
-        } else {
-          instrumentation.fileDeleted();
-        }
-        return true;
-      } else {
-        return false;
-      }
-    } catch(IOException e) {
-      Throwable innerException = NativeAzureFileSystemHelper.checkForAzureStorageException(e);
-
-      if (innerException instanceof StorageException
-          && NativeAzureFileSystemHelper.isFileNotFoundException((StorageException) innerException)) {
-        return false;
-      }
+  boolean deleteFile(String path, boolean isDir) throws IOException {
+    if (!store.delete(path)) {
+      return false;
+    }
 
-      throw e;
+    if (isDir) {
+      instrumentation.directoryDeleted();
+    } else {
+      instrumentation.fileDeleted();
     }
+    return true;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6b4e656/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsWithThreads.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsWithThreads.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsWithThreads.java
index ce3cdee..fd3690c 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsWithThreads.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsWithThreads.java
@@ -39,6 +39,8 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 /**
  * Tests the Native Azure file system (WASB) using parallel threads for rename and delete operations.
@@ -529,30 +531,65 @@ public class TestFileSystemOperationsWithThreads extends AbstractWasbTestBase {
   }
 
   /*
-   * Test case for delete operation with multiple threads and flat listing enabled.
+   * Validate that when a directory is deleted recursively, the operation succeeds
+   * even if a child directory delete fails because the directory does not exist.
+   * This can happen if a child directory is deleted by an external agent while
+   * the parent is in progress of being deleted recursively.
+   */
+  @Test
+  public void testRecursiveDirectoryDeleteWhenChildDirectoryDeleted()
+      throws Exception {
+    testRecusiveDirectoryDelete(true);
+  }
+
+  /*
+   * Validate that when a directory is deleted recursively, the operation succeeds
+   * even if a file delete fails because it does not exist.
+   * This can happen if a file is deleted by an external agent while
+   * the parent directory is in progress of being deleted.
    */
   @Test
-  public void testDeleteSingleDeleteFailure() throws Exception {
+  public void testRecursiveDirectoryDeleteWhenDeletingChildFileReturnsFalse()
+      throws Exception {
+    testRecusiveDirectoryDelete(false);
+  }
 
+  private void testRecusiveDirectoryDelete(boolean useDir) throws Exception {
+    String childPathToBeDeletedByExternalAgent = (useDir)
+        ? "root/0"
+        : "root/0/fileToRename";
     // Spy azure file system object and return false for deleting one file
-    LOG.info("testDeleteSingleDeleteFailure");
     NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
-    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root/0")));
-    Mockito.when(mockFs.deleteFile(path, true)).thenReturn(false);
+    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path(
+        childPathToBeDeletedByExternalAgent)));
+
+    Answer<Boolean> answer = new Answer<Boolean>() {
+      public Boolean answer(InvocationOnMock invocation) throws Throwable {
+        String path = (String) invocation.getArguments()[0];
+        boolean isDir = (boolean) invocation.getArguments()[1];
+        boolean realResult = fs.deleteFile(path, isDir);
+        assertTrue(realResult);
+        boolean fakeResult = false;
+        return fakeResult;
+      }
+    };
+
+    Mockito.when(mockFs.deleteFile(path, useDir)).thenAnswer(answer);
 
     createFolder(mockFs, "root");
     Path sourceFolder = new Path("root");
-    assertFalse(mockFs.delete(sourceFolder, true));
-    assertTrue(mockFs.exists(sourceFolder));
 
-    // Validate from logs that threads are enabled and delete operation failed.
+    assertTrue(mockFs.delete(sourceFolder, true));
+    assertFalse(mockFs.exists(sourceFolder));
+
+    // Validate from logs that threads are enabled, that a child directory was
+    // deleted by an external caller, and the parent delete operation still
+    // succeeds.
     String content = logs.getOutput();
     assertInLog(content,
         "Using thread pool for Delete operation with threads");
-    assertInLog(content, "Delete operation failed for file " + path);
-    assertInLog(content,
-        "Terminating execution of Delete operation now as some other thread already got exception or operation failed");
-    assertInLog(content, "Failed to delete files / subfolders in blob");
+    assertInLog(content, String.format("Attempt to delete non-existent %s %s",
+        useDir ? "directory" : "file", path));
   }
 
   /*


---------------------------------------------------------------------
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-6900. ZooKeeper based implementation of the FederationStateStore. (Íñigo Goiri via Subru).

Posted by ae...@apache.org.
YARN-6900. ZooKeeper based implementation of the FederationStateStore. (Íñigo Goiri via Subru).


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

Branch: refs/heads/HDFS-7240
Commit: de462da04e167a04b89ecf0f40d464cf39dc6549
Parents: 1455306
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Aug 16 11:43:24 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Wed Aug 16 11:43:24 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   8 +
 .../yarn/conf/TestYarnConfigurationFields.java  |   4 +
 .../hadoop-yarn-server-common/pom.xml           |   5 +
 .../impl/ZookeeperFederationStateStore.java     | 634 +++++++++++++++++++
 .../impl/TestZookeeperFederationStateStore.java |  89 +++
 .../TestFederationStateStoreFacadeRetry.java    |  20 +-
 .../src/site/markdown/Federation.md             |  56 +-
 7 files changed, 785 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/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 8acaef8..8515e0a 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
@@ -2629,6 +2629,14 @@ public class YarnConfiguration extends Configuration {
 
   public static final String DEFAULT_FEDERATION_POLICY_MANAGER_PARAMS = "";
 
+  public static final String FEDERATION_STATESTORE_ZK_PREFIX =
+      FEDERATION_PREFIX + "zk-state-store.";
+  /** Parent znode path under which ZKRMStateStore will create znodes. */
+  public static final String FEDERATION_STATESTORE_ZK_PARENT_PATH =
+      FEDERATION_STATESTORE_ZK_PREFIX + "parent-path";
+  public static final String DEFAULT_FEDERATION_STATESTORE_ZK_PARENT_PATH =
+      "/federationstore";
+
   private static final String FEDERATION_STATESTORE_SQL_PREFIX =
       FEDERATION_PREFIX + "state-store.sql.";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/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 91a8b0a..c40c2c5 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
@@ -96,6 +96,10 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare
         .add(YarnConfiguration.DEFAULT_FEDERATION_POLICY_MANAGER_PARAMS);
 
+    // Federation StateStore ZK implementation configs to be ignored
+    configurationPropsToSkipCompare.add(
+        YarnConfiguration.FEDERATION_STATESTORE_ZK_PARENT_PATH);
+
     // Federation StateStore SQL implementation configs to be ignored
     configurationPropsToSkipCompare
         .add(YarnConfiguration.FEDERATION_STATESTORE_SQL_JDBC_CLASS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
index 441a574..e8d3880 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
@@ -130,6 +130,11 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-test</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.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/store/impl/ZookeeperFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java
new file mode 100644
index 0000000..6ae7d3c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java
@@ -0,0 +1,634 @@
+/**
+ * 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
+ * <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. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.impl;
+
+import static org.apache.hadoop.util.curator.ZKCuratorManager.getNodePath;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.List;
+import java.util.TimeZone;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.curator.ZKCuratorManager;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterIdProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterInfoProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterPolicyConfigurationProto;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationsHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationsHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterIdPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterInfoPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterPolicyConfigurationPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationApplicationHomeSubClusterStoreInputValidator;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationMembershipStateStoreInputValidator;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationPolicyStoreInputValidator;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils;
+import org.apache.hadoop.yarn.server.records.Version;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+
+/**
+ * ZooKeeper implementation of {@link FederationStateStore}.
+ *
+ * The znode structure is as follows:
+ * ROOT_DIR_PATH
+ * |--- MEMBERSHIP
+ * |     |----- SC1
+ * |     |----- SC2
+ * |--- APPLICATION
+ * |     |----- APP1
+ * |     |----- APP2
+ * |--- POLICY
+ *       |----- QUEUE1
+ *       |----- QUEUE1
+ */
+public class ZookeeperFederationStateStore implements FederationStateStore {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ZookeeperFederationStateStore.class);
+
+  private final static String ROOT_ZNODE_NAME_MEMBERSHIP = "memberships";
+  private final static String ROOT_ZNODE_NAME_APPLICATION = "applications";
+  private final static String ROOT_ZNODE_NAME_POLICY = "policies";
+
+  /** Interface to Zookeeper. */
+  private ZKCuratorManager zkManager;
+
+  /** Directory to store the state store data. */
+  private String baseZNode;
+
+  private String appsZNode;
+  private String membershipZNode;
+  private String policiesZNode;
+
+  @Override
+  public void init(Configuration conf) throws YarnException {
+    LOG.info("Initializing ZooKeeper connection");
+
+    baseZNode = conf.get(
+        YarnConfiguration.FEDERATION_STATESTORE_ZK_PARENT_PATH,
+        YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_ZK_PARENT_PATH);
+    try {
+      this.zkManager = new ZKCuratorManager(conf);
+      this.zkManager.start();
+    } catch (IOException e) {
+      LOG.error("Cannot initialize the ZK connection", e);
+    }
+
+    // Base znodes
+    membershipZNode = getNodePath(baseZNode, ROOT_ZNODE_NAME_MEMBERSHIP);
+    appsZNode = getNodePath(baseZNode, ROOT_ZNODE_NAME_APPLICATION);
+    policiesZNode = getNodePath(baseZNode, ROOT_ZNODE_NAME_POLICY);
+
+    // Create base znode for each entity
+    try {
+      zkManager.createRootDirRecursively(membershipZNode);
+      zkManager.createRootDirRecursively(appsZNode);
+      zkManager.createRootDirRecursively(policiesZNode);
+    } catch (Exception e) {
+      String errMsg = "Cannot create base directories: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (zkManager != null) {
+      zkManager.close();
+    }
+  }
+
+  @Override
+  public AddApplicationHomeSubClusterResponse addApplicationHomeSubCluster(
+      AddApplicationHomeSubClusterRequest request) throws YarnException {
+
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
+    ApplicationHomeSubCluster app = request.getApplicationHomeSubCluster();
+    ApplicationId appId = app.getApplicationId();
+
+    // Try to write the subcluster
+    SubClusterId homeSubCluster = app.getHomeSubCluster();
+    try {
+      putApp(appId, homeSubCluster, false);
+    } catch (Exception e) {
+      String errMsg = "Cannot add application home subcluster for " + appId;
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    // Check for the actual subcluster
+    try {
+      homeSubCluster = getApp(appId);
+    } catch (Exception e) {
+      String errMsg = "Cannot check app home subcluster for " + appId;
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    return AddApplicationHomeSubClusterResponse
+        .newInstance(homeSubCluster);
+  }
+
+  @Override
+  public UpdateApplicationHomeSubClusterResponse
+      updateApplicationHomeSubCluster(
+          UpdateApplicationHomeSubClusterRequest request)
+              throws YarnException {
+
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
+    ApplicationHomeSubCluster app = request.getApplicationHomeSubCluster();
+    ApplicationId appId = app.getApplicationId();
+    SubClusterId homeSubCluster = getApp(appId);
+    if (homeSubCluster == null) {
+      String errMsg = "Application " + appId + " does not exist";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    SubClusterId newSubClusterId =
+        request.getApplicationHomeSubCluster().getHomeSubCluster();
+    putApp(appId, newSubClusterId, true);
+    return UpdateApplicationHomeSubClusterResponse.newInstance();
+  }
+
+  @Override
+  public GetApplicationHomeSubClusterResponse getApplicationHomeSubCluster(
+      GetApplicationHomeSubClusterRequest request) throws YarnException {
+
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
+    ApplicationId appId = request.getApplicationId();
+    SubClusterId homeSubCluster = getApp(appId);
+    if (homeSubCluster == null) {
+      String errMsg = "Application " + appId + " does not exist";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return GetApplicationHomeSubClusterResponse.newInstance(
+        ApplicationHomeSubCluster.newInstance(appId, homeSubCluster));
+  }
+
+  @Override
+  public GetApplicationsHomeSubClusterResponse getApplicationsHomeSubCluster(
+      GetApplicationsHomeSubClusterRequest request) throws YarnException {
+    List<ApplicationHomeSubCluster> result = new ArrayList<>();
+
+    try {
+      for (String child : zkManager.getChildren(appsZNode)) {
+        ApplicationId appId = ApplicationId.fromString(child);
+        SubClusterId homeSubCluster = getApp(appId);
+        ApplicationHomeSubCluster app =
+            ApplicationHomeSubCluster.newInstance(appId, homeSubCluster);
+        result.add(app);
+      }
+    } catch (Exception e) {
+      String errMsg = "Cannot get apps: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    return GetApplicationsHomeSubClusterResponse.newInstance(result);
+  }
+
+  @Override
+  public DeleteApplicationHomeSubClusterResponse
+      deleteApplicationHomeSubCluster(
+          DeleteApplicationHomeSubClusterRequest request)
+              throws YarnException {
+
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
+    ApplicationId appId = request.getApplicationId();
+    String appZNode = getNodePath(appsZNode, appId.toString());
+
+    boolean exists = false;
+    try {
+      exists = zkManager.exists(appZNode);
+    } catch (Exception e) {
+      String errMsg = "Cannot check app: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    if (!exists) {
+      String errMsg = "Application " + appId + " does not exist";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    try {
+      zkManager.delete(appZNode);
+    } catch (Exception e) {
+      String errMsg = "Cannot delete app: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    return DeleteApplicationHomeSubClusterResponse.newInstance();
+  }
+
+  @Override
+  public SubClusterRegisterResponse registerSubCluster(
+      SubClusterRegisterRequest request) throws YarnException {
+    FederationMembershipStateStoreInputValidator.validate(request);
+    SubClusterInfo subClusterInfo = request.getSubClusterInfo();
+    SubClusterId subclusterId = subClusterInfo.getSubClusterId();
+
+    // Update the heartbeat time
+    long currentTime = getCurrentTime();
+    subClusterInfo.setLastHeartBeat(currentTime);
+
+    try {
+      putSubclusterInfo(subclusterId, subClusterInfo, true);
+    } catch (Exception e) {
+      String errMsg = "Cannot register subcluster: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return SubClusterRegisterResponse.newInstance();
+  }
+
+  @Override
+  public SubClusterDeregisterResponse deregisterSubCluster(
+      SubClusterDeregisterRequest request) throws YarnException {
+    FederationMembershipStateStoreInputValidator.validate(request);
+    SubClusterId subClusterId = request.getSubClusterId();
+    SubClusterState state = request.getState();
+
+    // Get the current information and update it
+    SubClusterInfo subClusterInfo = getSubclusterInfo(subClusterId);
+    if (subClusterInfo == null) {
+      String errMsg = "SubCluster " + subClusterId + " not found";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    } else {
+      subClusterInfo.setState(state);
+      putSubclusterInfo(subClusterId, subClusterInfo, true);
+    }
+
+    return SubClusterDeregisterResponse.newInstance();
+  }
+
+  @Override
+  public SubClusterHeartbeatResponse subClusterHeartbeat(
+      SubClusterHeartbeatRequest request) throws YarnException {
+
+    FederationMembershipStateStoreInputValidator.validate(request);
+    SubClusterId subClusterId = request.getSubClusterId();
+
+    SubClusterInfo subClusterInfo = getSubclusterInfo(subClusterId);
+    if (subClusterInfo == null) {
+      String errMsg = "SubCluster " + subClusterId
+          + " does not exist; cannot heartbeat";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    long currentTime = getCurrentTime();
+    subClusterInfo.setLastHeartBeat(currentTime);
+    subClusterInfo.setState(request.getState());
+    subClusterInfo.setCapability(request.getCapability());
+
+    putSubclusterInfo(subClusterId, subClusterInfo, true);
+
+    return SubClusterHeartbeatResponse.newInstance();
+  }
+
+  @Override
+  public GetSubClusterInfoResponse getSubCluster(
+      GetSubClusterInfoRequest request) throws YarnException {
+
+    FederationMembershipStateStoreInputValidator.validate(request);
+    SubClusterId subClusterId = request.getSubClusterId();
+    SubClusterInfo subClusterInfo = null;
+    try {
+      subClusterInfo = getSubclusterInfo(subClusterId);
+      if (subClusterInfo == null) {
+        LOG.warn("The queried SubCluster: {} does not exist.", subClusterId);
+        return null;
+      }
+    } catch (Exception e) {
+      String errMsg = "Cannot get subcluster: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return GetSubClusterInfoResponse.newInstance(subClusterInfo);
+  }
+
+  @Override
+  public GetSubClustersInfoResponse getSubClusters(
+      GetSubClustersInfoRequest request) throws YarnException {
+    List<SubClusterInfo> result = new ArrayList<>();
+
+    try {
+      for (String child : zkManager.getChildren(membershipZNode)) {
+        SubClusterId subClusterId = SubClusterId.newInstance(child);
+        SubClusterInfo info = getSubclusterInfo(subClusterId);
+        if (!request.getFilterInactiveSubClusters() ||
+            info.getState().isActive()) {
+          result.add(info);
+        }
+      }
+    } catch (Exception e) {
+      String errMsg = "Cannot get subclusters: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return GetSubClustersInfoResponse.newInstance(result);
+  }
+
+
+  @Override
+  public GetSubClusterPolicyConfigurationResponse getPolicyConfiguration(
+      GetSubClusterPolicyConfigurationRequest request) throws YarnException {
+
+    FederationPolicyStoreInputValidator.validate(request);
+    String queue = request.getQueue();
+    SubClusterPolicyConfiguration policy = null;
+    try {
+      policy = getPolicy(queue);
+    } catch (Exception e) {
+      String errMsg = "Cannot get policy: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    if (policy == null) {
+      LOG.warn("Policy for queue: {} does not exist.", queue);
+      return null;
+    }
+    return GetSubClusterPolicyConfigurationResponse
+        .newInstance(policy);
+  }
+
+  @Override
+  public SetSubClusterPolicyConfigurationResponse setPolicyConfiguration(
+      SetSubClusterPolicyConfigurationRequest request) throws YarnException {
+
+    FederationPolicyStoreInputValidator.validate(request);
+    SubClusterPolicyConfiguration policy =
+        request.getPolicyConfiguration();
+    try {
+      String queue = policy.getQueue();
+      putPolicy(queue, policy, true);
+    } catch (Exception e) {
+      String errMsg = "Cannot set policy: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return SetSubClusterPolicyConfigurationResponse.newInstance();
+  }
+
+  @Override
+  public GetSubClusterPoliciesConfigurationsResponse getPoliciesConfigurations(
+      GetSubClusterPoliciesConfigurationsRequest request) throws YarnException {
+    List<SubClusterPolicyConfiguration> result = new ArrayList<>();
+
+    try {
+      for (String child : zkManager.getChildren(policiesZNode)) {
+        SubClusterPolicyConfiguration policy = getPolicy(child);
+        result.add(policy);
+      }
+    } catch (Exception e) {
+      String errMsg = "Cannot get policies: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return GetSubClusterPoliciesConfigurationsResponse.newInstance(result);
+  }
+
+  @Override
+  public Version getCurrentVersion() {
+    return null;
+  }
+
+  @Override
+  public Version loadVersion() {
+    return null;
+  }
+
+  /**
+   * Get the subcluster for an application.
+   * @param appId Application identifier.
+   * @return Subcluster identifier.
+   * @throws Exception If it cannot contact ZooKeeper.
+   */
+  private SubClusterId getApp(final ApplicationId appId) throws YarnException {
+    String appZNode = getNodePath(appsZNode, appId.toString());
+
+    SubClusterId subClusterId = null;
+    byte[] data = get(appZNode);
+    if (data != null) {
+      try {
+        subClusterId = new SubClusterIdPBImpl(
+            SubClusterIdProto.parseFrom(data));
+      } catch (InvalidProtocolBufferException e) {
+        String errMsg = "Cannot parse application at " + appZNode;
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+    }
+    return subClusterId;
+  }
+
+  /**
+   * Put an application.
+   * @param appId Application identifier.
+   * @param subClusterId Subcluster identifier.
+   * @throws Exception If it cannot contact ZooKeeper.
+   */
+  private void putApp(final ApplicationId appId,
+      final SubClusterId subClusterId, boolean update)
+          throws YarnException {
+    String appZNode = getNodePath(appsZNode, appId.toString());
+    SubClusterIdProto proto =
+        ((SubClusterIdPBImpl)subClusterId).getProto();
+    byte[] data = proto.toByteArray();
+    put(appZNode, data, update);
+  }
+
+  /**
+   * Get the current information for a subcluster from Zookeeper.
+   * @param subclusterId Subcluster identifier.
+   * @return Subcluster information or null if it doesn't exist.
+   * @throws Exception If it cannot contact ZooKeeper.
+   */
+  private SubClusterInfo getSubclusterInfo(final SubClusterId subclusterId)
+      throws YarnException {
+    String memberZNode = getNodePath(membershipZNode, subclusterId.toString());
+
+    SubClusterInfo policy = null;
+    byte[] data = get(memberZNode);
+    if (data != null) {
+      try {
+        policy = new SubClusterInfoPBImpl(
+            SubClusterInfoProto.parseFrom(data));
+      } catch (InvalidProtocolBufferException e) {
+        String errMsg = "Cannot parse subcluster info at " + memberZNode;
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+    }
+    return policy;
+  }
+
+  /**
+   * Put the subcluster information in Zookeeper.
+   * @param subclusterId Subcluster identifier.
+   * @param subClusterInfo Subcluster information.
+   * @throws Exception If it cannot contact ZooKeeper.
+   */
+  private void putSubclusterInfo(final SubClusterId subclusterId,
+      final SubClusterInfo subClusterInfo, final boolean update)
+          throws YarnException {
+    String memberZNode = getNodePath(membershipZNode, subclusterId.toString());
+    SubClusterInfoProto proto =
+        ((SubClusterInfoPBImpl)subClusterInfo).getProto();
+    byte[] data = proto.toByteArray();
+    put(memberZNode, data, update);
+  }
+
+  /**
+   * Get the queue policy from Zookeeper.
+   * @param queue Name of the queue.
+   * @return Subcluster policy configuration.
+   * @throws YarnException If it cannot contact ZooKeeper.
+   */
+  private SubClusterPolicyConfiguration getPolicy(final String queue)
+      throws YarnException {
+    String policyZNode = getNodePath(policiesZNode, queue);
+
+    SubClusterPolicyConfiguration policy = null;
+    byte[] data = get(policyZNode);
+    if (data != null) {
+      try {
+        policy = new SubClusterPolicyConfigurationPBImpl(
+            SubClusterPolicyConfigurationProto.parseFrom(data));
+      } catch (InvalidProtocolBufferException e) {
+        String errMsg = "Cannot parse policy at " + policyZNode;
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+    }
+    return policy;
+  }
+
+  /**
+   * Put the subcluster information in Zookeeper.
+   * @param queue Name of the queue.
+   * @param policy Subcluster policy configuration.
+   * @throws YarnException If it cannot contact ZooKeeper.
+   */
+  private void putPolicy(final String queue,
+      final SubClusterPolicyConfiguration policy, boolean update)
+          throws YarnException {
+    String policyZNode = getNodePath(policiesZNode, queue);
+
+    SubClusterPolicyConfigurationProto proto =
+        ((SubClusterPolicyConfigurationPBImpl)policy).getProto();
+    byte[] data = proto.toByteArray();
+    put(policyZNode, data, update);
+  }
+
+  /**
+   * Get data from a znode in Zookeeper.
+   * @param znode Path of the znode.
+   * @return Data in the znode.
+   * @throws YarnException If it cannot contact ZooKeeper.
+   */
+  private byte[] get(String znode) throws YarnException {
+    boolean exists = false;
+    try {
+      exists = zkManager.exists(znode);
+    } catch (Exception e) {
+      String errMsg = "Cannot find znode " + znode;
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    if (!exists) {
+      LOG.error("{} does not exist", znode);
+      return null;
+    }
+
+    byte[] data = null;
+    try {
+      data = zkManager.getData(znode);
+    } catch (Exception e) {
+      String errMsg = "Cannot get data from znode " + znode
+          + ": " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return data;
+  }
+
+  /**
+   * Put data into a znode in Zookeeper.
+   * @param znode Path of the znode.
+   * @param data Data to write.
+   * @throws YarnException If it cannot contact ZooKeeper.
+   */
+  private void put(String znode, byte[] data, boolean update)
+      throws YarnException {
+    // Create the znode
+    boolean created = false;
+    try {
+      created = zkManager.create(znode);
+    } catch (Exception e) {
+      String errMsg = "Cannot create znode " + znode + ": " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    if (!created) {
+      LOG.debug("{} not created", znode);
+      if (!update) {
+        LOG.info("{} already existed and we are not updating", znode);
+        return;
+      }
+    }
+
+    // Write the data into the znode
+    try {
+      zkManager.setData(znode, data, -1);
+    } catch (Exception e) {
+      String errMsg = "Cannot write data into znode " + znode
+          + ": " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+  }
+
+  /**
+   * Get the current time.
+   * @return Current time in milliseconds.
+   */
+  private static long getCurrentTime() {
+    Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
+    return cal.getTimeInMillis();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestZookeeperFederationStateStore.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/store/impl/TestZookeeperFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestZookeeperFederationStateStore.java
new file mode 100644
index 0000000..390b803
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestZookeeperFederationStateStore.java
@@ -0,0 +1,89 @@
+/**
+ * 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
+ * <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. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.impl;
+
+import java.io.IOException;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryNTimes;
+import org.apache.curator.test.TestingServer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.junit.After;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Unit tests for ZookeeperFederationStateStore.
+ */
+public class TestZookeeperFederationStateStore
+    extends FederationStateStoreBaseTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestZookeeperFederationStateStore.class);
+
+  /** Zookeeper test server. */
+  private static TestingServer curatorTestingServer;
+  private static CuratorFramework curatorFramework;
+
+  @Before
+  public void before() throws IOException, YarnException {
+    try {
+      curatorTestingServer = new TestingServer();
+      curatorTestingServer.start();
+      String connectString = curatorTestingServer.getConnectString();
+      curatorFramework = CuratorFrameworkFactory.builder()
+          .connectString(connectString)
+          .retryPolicy(new RetryNTimes(100, 100))
+          .build();
+      curatorFramework.start();
+
+      Configuration conf = new YarnConfiguration();
+      conf.set(CommonConfigurationKeys.ZK_ADDRESS, connectString);
+      setConf(conf);
+    } catch (Exception e) {
+      LOG.error("Cannot initialize ZooKeeper store", e);
+      throw new IOException(e);
+    }
+
+    super.before();
+  }
+
+  @After
+  public void after() throws Exception {
+    super.after();
+
+    curatorFramework.close();
+    try {
+      curatorTestingServer.stop();
+    } catch (IOException e) {
+    }
+  }
+
+  @Override
+  protected FederationStateStore createStateStore() {
+    Configuration conf = new Configuration();
+    super.setConf(conf);
+    return new ZookeeperFederationStateStore();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.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/TestFederationStateStoreFacadeRetry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
index ea43268..868e771 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateS
 import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException;
 import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreRetriableException;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 
 import com.zaxxer.hikari.pool.HikariPool.PoolInitializationException;
@@ -40,14 +41,18 @@ public class TestFederationStateStoreFacadeRetry {
   private int maxRetries = 4;
   private Configuration conf;
 
+  @Before
+  public void setup() {
+    conf = new Configuration();
+    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
+  }
+
   /*
    * Test to validate that FederationStateStoreRetriableException is a retriable
    * exception.
    */
   @Test
   public void testFacadeRetriableException() throws Exception {
-    conf = new Configuration();
-    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
     RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
     RetryAction action = policy.shouldRetry(
         new FederationStateStoreRetriableException(""), 0, 0, false);
@@ -66,9 +71,6 @@ public class TestFederationStateStoreFacadeRetry {
    */
   @Test
   public void testFacadeYarnException() throws Exception {
-
-    conf = new Configuration();
-    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
     RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
     RetryAction action = policy.shouldRetry(new YarnException(), 0, 0, false);
     Assert.assertEquals(RetryAction.FAIL.action, action.action);
@@ -80,8 +82,6 @@ public class TestFederationStateStoreFacadeRetry {
    */
   @Test
   public void testFacadeStateStoreException() throws Exception {
-    conf = new Configuration();
-    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
     RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
     RetryAction action = policy
         .shouldRetry(new FederationStateStoreException("Error"), 0, 0, false);
@@ -94,8 +94,6 @@ public class TestFederationStateStoreFacadeRetry {
    */
   @Test
   public void testFacadeInvalidInputException() throws Exception {
-    conf = new Configuration();
-    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
     RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
     RetryAction action = policy.shouldRetry(
         new FederationStateStoreInvalidInputException(""), 0, 0, false);
@@ -107,8 +105,6 @@ public class TestFederationStateStoreFacadeRetry {
    */
   @Test
   public void testFacadeCacheRetriableException() throws Exception {
-    conf = new Configuration();
-    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
     RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
     RetryAction action =
         policy.shouldRetry(new CacheLoaderException(""), 0, 0, false);
@@ -128,8 +124,6 @@ public class TestFederationStateStoreFacadeRetry {
   @Test
   public void testFacadePoolInitRetriableException() throws Exception {
     // PoolInitializationException is a retriable exception
-    conf = new Configuration();
-    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
     RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
     RetryAction action = policy.shouldRetry(
         new PoolInitializationException(new YarnException()), 0, 0, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
index 3e3580c..8a6c137 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
@@ -129,6 +129,7 @@ AMRMProxy, Global Policy Generator (GPG) and Router work together to make this h
 
 
 The figure shows a sequence diagram for the following job execution flow:
+
 1. The Router receives an application submission request that is complaint to the YARN Application Client Protocol.
 2. The router interrogates a routing table / policy to choose the “home RM” for the job (the policy configuration is received from the state-store on heartbeat).
 3. The router queries the membership state to determine the endpoint of the home RM.
@@ -160,15 +161,50 @@ These are common configurations that should appear in the **conf/yarn-site.xml**
 | Property | Example | Description |
 |:---- |:---- |
 |`yarn.federation.enabled` | `true` | Whether federation is enabled or not |
+|`yarn.resourcemanager.cluster-id` | `<unique-subcluster-id>` | The unique subcluster identifier for this RM (same as the one used for HA). |
+
+####State-Store:
+
+Currently, we support ZooKeeper and SQL based implementations of the state-store.
+
+**Note:** The State-Store implementation must always be overwritten with one of the below.
+
+ZooKeeper: one must set the ZooKeeper settings for Hadoop:
+
+| Property | Example | Description |
+|:---- |:---- |
+|`yarn.federation.state-store.class` | `org.apache.hadoop.yarn.server.federation.store.impl.ZookeeperFederationStateStore` | The type of state-store to use. |
+|`hadoop.zk.address` | `host:port` | The address for the ZooKeeper ensemble. |
+
+SQL: one must setup the following parameters:
+
+| Property | Example | Description |
+|:---- |:---- |
 |`yarn.federation.state-store.class` | `org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore` | The type of state-store to use. |
 |`yarn.federation.state-store.sql.url` | `jdbc:mysql://<host>:<port>/FederationStateStore` | For SQLFederationStateStore the name of the DB where the state is stored. |
 |`yarn.federation.state-store.sql.jdbc-class` | `com.mysql.jdbc.jdbc2.optional.MysqlDataSource` | For SQLFederationStateStore the jdbc class to use. |
 |`yarn.federation.state-store.sql.username` | `<dbuser>` | For SQLFederationStateStore the username for the DB connection. |
 |`yarn.federation.state-store.sql.password` | `<dbpass>` | For SQLFederationStateStore the password for the DB connection. |
-|`yarn.resourcemanager.cluster-id` | `<unique-subcluster-id>` | The unique subcluster identifier for this RM (same as the one used for HA). |
 
+We provide scripts for MySQL and Microsoft SQL Server.
 
-Optional:
+For MySQL, one must download the latest jar version 5.x from [MVN Repository](https://mvnrepository.com/artifact/mysql/mysql-connector-java) and add it to the CLASSPATH.
+Then the DB schema is created by executing the following SQL scripts in the database:
+
+1. **sbin/FederationStateStore/MySQL/FederationStateStoreDatabase.sql**.
+2. **sbin/FederationStateStore/MySQL/FederationStateStoreUser.sql**.
+3. **sbin/FederationStateStore/MySQL/FederationStateStoreTables.sql**.
+4. **sbin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql**.
+
+In the same directory we provide scripts to drop the Stored Procedures, the Tables, the User and the Database.
+
+**Note:** the FederationStateStoreUser.sql defines a default user/password for the DB that you are **highly encouraged** to set this to a proper strong password.
+
+For SQL-Server, the process is similar, but the jdbc driver is already included.
+SQL-Server scripts are located in **sbin/FederationStateStore/SQLServer/**.
+
+
+####Optional:
 
 | Property | Example | Description |
 |:---- |:---- |
@@ -236,22 +272,6 @@ Optional:
 |`yarn.federation.statestore.max-connections` | `1` | The maximum number of parallel connections from each AMRMProxy to the state-store. This value is typically lower than the router one, since we have many AMRMProxy that could burn-through many DB connections quickly. |
 |`yarn.federation.cache-ttl.secs` | `300` | The time to leave for the AMRMProxy cache. Typically larger than at the router, as the number of AMRMProxy is large, and we want to limit the load to the centralized state-store. |
 
-###State-Store:
-
-Currently, we support only SQL based implementation of state-store (ZooKeeper is in the works), i.e. either MySQL or Microsoft SQL Server.
-
-For MySQL, one must download the latest jar version 5.x from [MVN Repository](https://mvnrepository.com/artifact/mysql/mysql-connector-java) and add it to the CLASSPATH.
-Then the DB schema is created by executing the following SQL scripts in the database:
-1. **sbin/FederationStateStore/MySQL/FederationStateStoreDatabase.sql**.
-2. **sbin/FederationStateStore/MySQL/FederationStateStoreUser.sql**.
-3. **sbin/FederationStateStore/MySQL/FederationStateStoreTables.sql**.
-4. **sbin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql**.
-In the same directory we provide scripts to drop the Stored Procedures, the Tables, the User and the Database.
-**Note:** the FederationStateStoreUser.sql defines a default user/password for the DB that you are **highly encouraged** to set this to a proper strong password.
-
-For SQL-Server, the process is similar, but the jdbc driver is already included in the pom (license allows it).
-SQL-Server scripts are located in **sbin/FederationStateStore/SQLServer/**.
-
 Running a Sample Job
 --------------------
 In order to submit jobs to a Federation cluster one must create a seperate set of configs for the client from which jobs will be submitted. In these, the **conf/yarn-site.xml** should have the following additional configurations:


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


[04/50] [abbrv] hadoop git commit: YARN-6741. Deleting all children of a Parent Queue on refresh throws exception. Contributed by Naganarasimha G R.

Posted by ae...@apache.org.
YARN-6741. Deleting all children of a Parent Queue on refresh throws exception. 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/d8f74c39
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d8f74c39
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d8f74c39

Branch: refs/heads/HDFS-7240
Commit: d8f74c3964fa429a4a53c3651d175792cf00ac81
Parents: 7769e96
Author: bibinchundatt <bi...@apache.org>
Authored: Mon Aug 14 09:39:00 2017 +0530
Committer: bibinchundatt <bi...@apache.org>
Committed: Mon Aug 14 09:39:00 2017 +0530

----------------------------------------------------------------------
 .../capacity/CapacitySchedulerQueueManager.java |   4 +
 .../scheduler/capacity/ParentQueue.java         |  39 +++----
 .../capacity/TestCapacityScheduler.java         | 114 ++++++++++++++++++-
 3 files changed, 137 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8f74c39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.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/CapacitySchedulerQueueManager.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/CapacitySchedulerQueueManager.java
index e33fbb3..1ceb6fb 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/CapacitySchedulerQueueManager.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/CapacitySchedulerQueueManager.java
@@ -327,6 +327,10 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
                 + "it is not yet in stopped state. Current State : "
                 + oldQueue.getState());
           }
+        } else if (oldQueue instanceof ParentQueue
+            && newQueue instanceof LeafQueue) {
+          LOG.info("Converting the parent queue: " + oldQueue.getQueuePath()
+              + " to leaf queue.");
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8f74c39/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 f6ada4f..e0baa07 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
@@ -18,6 +18,14 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -34,7 +42,6 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.AccessType;
@@ -45,7 +52,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger;
@@ -62,14 +68,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.Placeme
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
 @Private
 @Evolving
 public class ParentQueue extends AbstractCSQueue {
@@ -315,18 +313,21 @@ public class ParentQueue extends AbstractCSQueue {
 
         // Check if the child-queue already exists
         if (childQueue != null) {
-          // Check if the child-queue has been converted into parent queue.
-          // The CS has already checked to ensure that this child-queue is in
-          // STOPPED state.
-          if (childQueue instanceof LeafQueue
-              && newChildQueue instanceof ParentQueue) {
-            // We would convert this LeafQueue to ParentQueue, consider this
-            // as the combination of DELETE then ADD.
+          // Check if the child-queue has been converted into parent queue or
+          // parent Queue has been converted to child queue. The CS has already
+          // checked to ensure that this child-queue is in STOPPED state if
+          // Child queue has been converted to ParentQueue.
+          if ((childQueue instanceof LeafQueue
+              && newChildQueue instanceof ParentQueue)
+              || (childQueue instanceof ParentQueue
+                  && newChildQueue instanceof LeafQueue)) {
+            // We would convert this LeafQueue to ParentQueue, or vice versa.
+            // consider this as the combination of DELETE then ADD.
             newChildQueue.setParent(this);
             currentChildQueues.put(newChildQueueName, newChildQueue);
             // inform CapacitySchedulerQueueManager
-            CapacitySchedulerQueueManager queueManager = this.csContext
-                .getCapacitySchedulerQueueManager();
+            CapacitySchedulerQueueManager queueManager =
+                this.csContext.getCapacitySchedulerQueueManager();
             queueManager.addQueue(newChildQueueName, newChildQueue);
             continue;
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8f74c39/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 64e0df4..a526222 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -42,7 +43,6 @@ import java.util.Set;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.CyclicBarrier;
 
-import com.google.common.base.Supplier;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -167,6 +167,7 @@ import org.junit.Before;
 import org.junit.Test;
 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;
@@ -667,6 +668,36 @@ public class TestCapacityScheduler {
 
   /**
    * @param conf, to be modified
+   * @return, CS configuration which has deleted all childred of queue(b)
+   *           root
+   *          /     \
+   *        a        b
+   *       / \
+   *      a1  a2
+   */
+  private CapacitySchedulerConfiguration setupQueueConfWithOutChildrenOfB(
+      CapacitySchedulerConfiguration conf) {
+
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] {"a","b"});
+
+    conf.setCapacity(A, A_CAPACITY);
+    conf.setCapacity(B, B_CAPACITY);
+
+    // Define 2nd-level queues
+    conf.setQueues(A, new String[] {"a1","a2"});
+    conf.setCapacity(A1, A1_CAPACITY);
+    conf.setUserLimitFactor(A1, 100.0f);
+    conf.setCapacity(A2, A2_CAPACITY);
+    conf.setUserLimitFactor(A2, 100.0f);
+
+    LOG.info("Setup top-level queues a and b (without children)");
+    return conf;
+  }
+
+  /**
+   * @param conf, to be modified
    * @return, CS configuration which has deleted a queue(b1)
    *           root
    *          /     \
@@ -4643,6 +4674,10 @@ public class TestCapacityScheduler {
     try {
       cs.reinitialize(conf, mockContext);
     } catch (IOException e) {
+      LOG.error(
+          "Expected to NOT throw exception when refresh queue tries to delete"
+              + " a queue WITHOUT running apps",
+          e);
       fail("Expected to NOT throw exception when refresh queue tries to delete"
           + " a queue WITHOUT running apps");
     }
@@ -4713,6 +4748,83 @@ public class TestCapacityScheduler {
   }
 
   /**
+   * Test for all child queue deletion and thus making parent queue a child.
+   * @throws Exception
+   */
+  @Test
+  public void testRefreshQueuesWithAllChildQueuesDeleted() throws Exception {
+    CapacityScheduler cs = new CapacityScheduler();
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    RMContextImpl rmContext = new RMContextImpl(null, null, null, null, null,
+        null, new RMContainerTokenSecretManager(conf),
+        new NMTokenSecretManagerInRM(conf),
+        new ClientToAMTokenSecretManagerInRM(), null);
+    setupQueueConfiguration(conf);
+    cs.setConf(new YarnConfiguration());
+    cs.setRMContext(resourceManager.getRMContext());
+    cs.init(conf);
+    cs.start();
+    cs.reinitialize(conf, rmContext);
+    checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
+
+    // test delete all leaf queues when there is no application running.
+    Map<String, CSQueue> queues =
+        cs.getCapacitySchedulerQueueManager().getQueues();
+
+    CSQueue bQueue = Mockito.spy((LeafQueue) queues.get("b1"));
+    when(bQueue.getState()).thenReturn(QueueState.RUNNING)
+        .thenReturn(QueueState.STOPPED);
+    queues.put("b1", bQueue);
+
+    bQueue = Mockito.spy((LeafQueue) queues.get("b2"));
+    when(bQueue.getState()).thenReturn(QueueState.STOPPED);
+    queues.put("b2", bQueue);
+
+    bQueue = Mockito.spy((LeafQueue) queues.get("b3"));
+    when(bQueue.getState()).thenReturn(QueueState.STOPPED);
+    queues.put("b3", bQueue);
+
+    conf = new CapacitySchedulerConfiguration();
+    setupQueueConfWithOutChildrenOfB(conf);
+
+    // test convert parent queue to leaf queue(root.b) when there is no
+    // application running.
+    try {
+      cs.reinitialize(conf, mockContext);
+      fail("Expected to throw exception when refresh queue tries to make parent"
+          + " queue a child queue when one of its children is still running.");
+    } catch (IOException e) {
+      //do not do anything, expected exception
+    }
+
+    // test delete leaf queues(root.b.b1,b2,b3) when there is no application
+    // running.
+    try {
+      cs.reinitialize(conf, mockContext);
+    } catch (IOException e) {
+      e.printStackTrace();
+      fail("Expected to NOT throw exception when refresh queue tries to delete"
+          + " all children of a parent queue(without running apps).");
+    }
+    CSQueue rootQueue = cs.getRootQueue();
+    CSQueue queueB = findQueue(rootQueue, B);
+    assertNotNull("Parent Queue B should not be deleted", queueB);
+    Assert.assertTrue("As Queue'B children are not deleted",
+        queueB instanceof LeafQueue);
+
+    String message =
+        "Refresh needs to support delete of all children of Parent queue.";
+    assertNull(message,
+        cs.getCapacitySchedulerQueueManager().getQueues().get("b3"));
+    assertNull(message,
+        cs.getCapacitySchedulerQueueManager().getQueues().get("b1"));
+    assertNull(message,
+        cs.getCapacitySchedulerQueueManager().getQueues().get("b2"));
+
+    cs.stop();
+  }
+
+  /**
    * Test if we can convert a leaf queue to a parent queue
    * @throws Exception
    */


---------------------------------------------------------------------
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-7007. NPE in RM while using YarnClient.getApplications(). Contributed by Lingfeng Su.

Posted by ae...@apache.org.
YARN-7007. NPE in RM while using YarnClient.getApplications(). Contributed by Lingfeng Su.


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

Branch: refs/heads/HDFS-7240
Commit: e05fa3451db343c0d22496b332910874b6be5b7f
Parents: c6b4e65
Author: bibinchundatt <bi...@apache.org>
Authored: Fri Aug 18 20:28:50 2017 +0530
Committer: bibinchundatt <bi...@apache.org>
Committed: Fri Aug 18 20:28:50 2017 +0530

----------------------------------------------------------------------
 .../rmapp/attempt/RMAppAttemptMetrics.java       | 19 +++++++++++--------
 1 file changed, 11 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e05fa345/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.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/rmapp/attempt/RMAppAttemptMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java
index e089050..0655609 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptMetrics.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -125,14 +126,16 @@ public class RMAppAttemptMetrics {
     long vcoreSeconds = finishedVcoreSeconds.get();
 
     // Only add in the running containers if this is the active attempt.
-    RMAppAttempt currentAttempt = rmContext.getRMApps()
-                   .get(attemptId.getApplicationId()).getCurrentAppAttempt();
-    if (currentAttempt.getAppAttemptId().equals(attemptId)) {
-      ApplicationResourceUsageReport appResUsageReport = rmContext
-            .getScheduler().getAppResourceUsageReport(attemptId);
-      if (appResUsageReport != null) {
-        memorySeconds += appResUsageReport.getMemorySeconds();
-        vcoreSeconds += appResUsageReport.getVcoreSeconds();
+    RMApp rmApp = rmContext.getRMApps().get(attemptId.getApplicationId());
+    if (null != rmApp) {
+      RMAppAttempt currentAttempt = rmApp.getCurrentAppAttempt();
+      if (currentAttempt.getAppAttemptId().equals(attemptId)) {
+        ApplicationResourceUsageReport appResUsageReport = rmContext
+                .getScheduler().getAppResourceUsageReport(attemptId);
+        if (appResUsageReport != null) {
+          memorySeconds += appResUsageReport.getMemorySeconds();
+          vcoreSeconds += appResUsageReport.getVcoreSeconds();
+        }
       }
     }
     return new AggregateAppResourceUsage(memorySeconds, vcoreSeconds);


---------------------------------------------------------------------
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: HDFS-11082. Provide replicated EC policy to replicate files. Contributed by SammiChen.

Posted by ae...@apache.org.
HDFS-11082. Provide replicated EC policy to replicate files. Contributed by SammiChen.


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

Branch: refs/heads/HDFS-7240
Commit: 96b3a6b9721e922d33fadc2459b561a85dbf9b8e
Parents: 08aaa4b
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Aug 16 22:17:06 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Aug 16 22:17:06 2017 -0700

----------------------------------------------------------------------
 .../io/erasurecode/ErasureCodeConstants.java    |  8 ++
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  3 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |  6 +-
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  6 +-
 .../hdfs/protocol/ErasureCodingPolicy.java      |  5 ++
 .../protocol/SystemErasureCodingPolicies.java   | 14 ++++
 .../namenode/ErasureCodingPolicyManager.java    | 13 ++-
 .../server/namenode/FSDirErasureCodingOp.java   | 13 ++-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |  2 +-
 .../org/apache/hadoop/hdfs/tools/ECAdmin.java   | 24 +++++-
 .../src/site/markdown/HDFSErasureCoding.md      | 16 ++--
 .../hadoop/hdfs/TestErasureCodingPolicies.java  | 81 ++++++++++++++++++
 .../hdfs/server/namenode/TestFSImage.java       | 87 ++++++++++++++++++++
 .../test/resources/testErasureCodingConf.xml    | 78 +++++++++++++++++-
 14 files changed, 331 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
index e0d7946..d3c3b6b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
@@ -30,6 +30,7 @@ public final class ErasureCodeConstants {
   public static final String RS_LEGACY_CODEC_NAME = "rs-legacy";
   public static final String XOR_CODEC_NAME = "xor";
   public static final String HHXOR_CODEC_NAME = "hhxor";
+  public static final String REPLICATION_CODEC_NAME = "replication";
 
   public static final ECSchema RS_6_3_SCHEMA = new ECSchema(
       RS_CODEC_NAME, 6, 3);
@@ -45,4 +46,11 @@ public final class ErasureCodeConstants {
 
   public static final ECSchema RS_10_4_SCHEMA = new ECSchema(
       RS_CODEC_NAME, 10, 4);
+
+  public static final ECSchema REPLICATION_1_2_SCHEMA = new ECSchema(
+      REPLICATION_CODEC_NAME, 1, 2);
+
+  public static final byte USER_DEFINED_POLICY_START_ID = (byte) 64;
+  public static final byte REPLICATION_POLICY_ID = (byte) 63;
+  public static final String REPLICATION_POLICY_NAME = REPLICATION_CODEC_NAME;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 969522d..47c14e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -3044,7 +3044,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    *
    * @param src path to get the information for
    * @return Returns the policy information if file or directory on the path is
-   * erasure coded, null otherwise
+   * erasure coded, null otherwise. Null will be returned if directory or file
+   * has REPLICATION policy.
    * @throws IOException
    */
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 8f82d03..ceec2b3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -2540,7 +2540,8 @@ public class DistributedFileSystem extends FileSystem {
    *
    * @param path The path of the file or directory
    * @return Returns the policy information if file or directory on the path
-   * is erasure coded, null otherwise
+   * is erasure coded, null otherwise. Null will be returned if directory or
+   * file has REPLICATION policy.
    * @throws IOException
    */
   public ErasureCodingPolicy getErasureCodingPolicy(final Path path)
@@ -2567,7 +2568,8 @@ public class DistributedFileSystem extends FileSystem {
   }
 
   /**
-   * Retrieve all the erasure coding policies supported by this file system.
+   * Retrieve all the erasure coding policies supported by this file system,
+   * excluding REPLICATION policy.
    *
    * @return all erasure coding policies supported by this file system.
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index eb9380d..b0e85e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1588,7 +1588,8 @@ public interface ClientProtocol {
 
 
   /**
-   * Get the erasure coding policies loaded in Namenode.
+   * Get the erasure coding policies loaded in Namenode, excluding REPLICATION
+   * policy.
    *
    * @throws IOException
    */
@@ -1604,7 +1605,8 @@ public interface ClientProtocol {
   Map<String, String> getErasureCodingCodecs() throws IOException;
 
   /**
-   * Get the information about the EC policy for the path.
+   * Get the information about the EC policy for the path. Null will be returned
+   * if directory or file has REPLICATION policy.
    *
    * @param src path to get the info for
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
index 7afc377..501b67c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
@@ -25,6 +25,7 @@ import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 
 /**
  * A policy about how to write/read/code an erasure coding file.
@@ -107,6 +108,10 @@ public final class ErasureCodingPolicy implements Serializable {
     this.id = id;
   }
 
+  public boolean isReplicationPolicy() {
+    return (id == ErasureCodeConstants.REPLICATION_POLICY_ID);
+  }
+
   @Override
   public boolean equals(Object o) {
     if (o == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SystemErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SystemErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SystemErasureCodingPolicies.java
index 2cd838b..f0efe76 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SystemErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SystemErasureCodingPolicies.java
@@ -68,6 +68,13 @@ public final class SystemErasureCodingPolicies {
       new ErasureCodingPolicy(ErasureCodeConstants.RS_10_4_SCHEMA,
           DEFAULT_CELLSIZE, RS_10_4_POLICY_ID);
 
+  // REPLICATION policy is always enabled.
+  private static final ErasureCodingPolicy REPLICATION_POLICY =
+      new ErasureCodingPolicy(ErasureCodeConstants.REPLICATION_POLICY_NAME,
+          ErasureCodeConstants.REPLICATION_1_2_SCHEMA,
+          DEFAULT_CELLSIZE,
+          ErasureCodeConstants.REPLICATION_POLICY_ID);
+
   private static final List<ErasureCodingPolicy> SYS_POLICIES =
       Collections.unmodifiableList(Arrays.asList(
           SYS_POLICY1, SYS_POLICY2, SYS_POLICY3, SYS_POLICY4,
@@ -118,4 +125,11 @@ public final class SystemErasureCodingPolicies {
   public static ErasureCodingPolicy getByName(String name) {
     return SYSTEM_POLICIES_BY_NAME.get(name);
   }
+
+  /**
+   * Get the special REPLICATION policy.
+   */
+  public static ErasureCodingPolicy  getReplicationPolicy() {
+    return REPLICATION_POLICY;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/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 18b8e8a..404a0aa 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
@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 
 import org.apache.hadoop.io.erasurecode.CodecUtil;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -48,7 +49,6 @@ public final class ErasureCodingPolicyManager {
 
   public static Logger LOG = LoggerFactory.getLogger(
       ErasureCodingPolicyManager.class);
-  private static final byte USER_DEFINED_POLICY_START_ID = (byte) 64;
   private int maxCellSize =
       DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_MAX_CELLSIZE_DEFAULT;
 
@@ -157,7 +157,13 @@ public final class ErasureCodingPolicyManager {
    * Get enabled policy by policy name.
    */
   public ErasureCodingPolicy getEnabledPolicyByName(String name) {
-    return enabledPoliciesByName.get(name);
+    ErasureCodingPolicy ecPolicy = enabledPoliciesByName.get(name);
+    if (ecPolicy == null) {
+      if (name.equalsIgnoreCase(ErasureCodeConstants.REPLICATION_POLICY_NAME)) {
+        ecPolicy = SystemErasureCodingPolicies.getReplicationPolicy();
+      }
+    }
+    return ecPolicy;
   }
 
   /**
@@ -257,7 +263,8 @@ public final class ErasureCodingPolicyManager {
 
   private byte getNextAvailablePolicyID() {
     byte currentId = this.userPoliciesByID.keySet().stream()
-        .max(Byte::compareTo).orElse(USER_DEFINED_POLICY_START_ID);
+        .max(Byte::compareTo).orElse(
+            ErasureCodeConstants.USER_DEFINED_POLICY_START_ID);
     return (byte) (currentId + 1);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/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 7895433..426b42b 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
@@ -62,7 +62,7 @@ final class FSDirErasureCodingOp {
 
   /**
    * Check if the ecPolicyName is valid and enabled, return the corresponding
-   * EC policy if is.
+   * EC policy if is, including the REPLICATION EC policy.
    * @param fsn namespace
    * @param ecPolicyName name of EC policy to be checked
    * @return an erasure coding policy if ecPolicyName is valid and enabled
@@ -295,7 +295,12 @@ final class FSDirErasureCodingOp {
     if (iip.getLastINode() == null) {
       throw new FileNotFoundException("Path not found: " + iip.getPath());
     }
-    return getErasureCodingPolicyForPath(fsd, iip);
+
+    ErasureCodingPolicy ecPolicy = getErasureCodingPolicyForPath(fsd, iip);
+    if (ecPolicy != null && ecPolicy.isReplicationPolicy()) {
+      ecPolicy = null;
+    }
+    return ecPolicy;
   }
 
   /**
@@ -312,7 +317,8 @@ final class FSDirErasureCodingOp {
   }
 
   /**
-   * Get the erasure coding policy. This does not do any permission checking.
+   * Get the erasure coding policy, including the REPLICATION policy. This does
+   * not do any permission checking.
    *
    * @param fsn namespace
    * @param iip inodes in the path containing the file
@@ -350,6 +356,7 @@ final class FSDirErasureCodingOp {
     return CodecRegistry.getInstance().getCodec2CoderCompactMap();
   }
 
+  //return erasure coding policy for path, including REPLICATION policy
   private static ErasureCodingPolicy getErasureCodingPolicyForPath(
       FSDirectory fsd, INodesInPath iip) throws IOException {
     Preconditions.checkNotNull(iip, "INodes cannot be null");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index a62cddd..7ab05d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -541,7 +541,7 @@ class FSDirWriteFileOp {
           ecPolicy = FSDirErasureCodingOp.unprotectedGetErasureCodingPolicy(
               fsd.getFSNamesystem(), existing);
         }
-        if (ecPolicy != null) {
+        if (ecPolicy != null && (!ecPolicy.isReplicationPolicy())) {
           isStriped = true;
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
index 17a84f9..55d85ff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.util.ECPolicyLoader;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
@@ -309,7 +310,8 @@ public class ECAdmin extends Configured implements Tool {
 
     @Override
     public String getShortUsage() {
-      return "[" + getName() + " -path <path> -policy <policy>]\n";
+      return "[" + getName() +
+          " -path <path> [-policy <policy>] [-replicate]]\n";
     }
 
     @Override
@@ -318,9 +320,13 @@ public class ECAdmin extends Configured implements Tool {
       listing.addRow("<path>", "The path of the file/directory to set " +
           "the erasure coding policy");
       listing.addRow("<policy>", "The name of the erasure coding policy");
+      listing.addRow("-replicate",
+          "force 3x replication scheme on the directory");
       return getShortUsage() + "\n" +
           "Set the erasure coding policy for a file/directory.\n\n" +
-          listing.toString();
+          listing.toString() + "\n" +
+          "-replicate and -policy are optional arguments. They cannot been " +
+          "used at the same time";
     }
 
     @Override
@@ -332,14 +338,24 @@ public class ECAdmin extends Configured implements Tool {
         return 1;
       }
 
-      final String ecPolicyName = StringUtils.popOptionWithArgument("-policy",
+      String ecPolicyName = StringUtils.popOptionWithArgument("-policy",
           args);
+      final boolean replicate = StringUtils.popOption("-replicate", args);
 
       if (args.size() > 0) {
         System.err.println(getName() + ": Too many arguments");
         return 1;
       }
 
+      if (replicate) {
+        if (ecPolicyName != null) {
+          System.err.println(getName() +
+              ": -replicate and -policy cannot been used at the same time");
+          return 2;
+        }
+        ecPolicyName = ErasureCodeConstants.REPLICATION_POLICY_NAME;
+      }
+
       final Path p = new Path(path);
       final DistributedFileSystem dfs = AdminHelper.getDFS(p.toUri(), conf);
       try {
@@ -353,7 +369,7 @@ public class ECAdmin extends Configured implements Tool {
         }
       } catch (Exception e) {
         System.err.println(AdminHelper.prettifyException(e));
-        return 2;
+        return 3;
       }
       return 0;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index 4a48c2a..786b512 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -65,9 +65,11 @@ Architecture
 
       2. _The size of a striping cell._ This determines the granularity of striped reads and writes, including buffer sizes and encoding work.
 
-    Policies are named *codec*-*num data blocks*-*num parity blocks*-*cell size*. Currently, five built-in policies are supported: `RS-3-2-64k`, `RS-6-3-64k`, `RS-10-4-64k`, `RS-LEGACY-6-3-64k`, and `XOR-2-1-64k`.
+    Policies are named *codec*-*num data blocks*-*num parity blocks*-*cell size*. Currently, six built-in policies are supported: `RS-3-2-64k`, `RS-6-3-64k`, `RS-10-4-64k`, `RS-LEGACY-6-3-64k`, `XOR-2-1-64k` and `REPLICATION`.
 
-    By default, all built-in erasure coding policies are disabled.
+    `REPLICATION` is a special policy. It can only be set on directory, to force the directory to adopt 3x replication scheme, instead of inheriting its ancestor's erasure coding policy. This policy makes it possible to interleave 3x replication scheme directory with erasure coding directory.
+
+    `REPLICATION` policy is always enabled. For other built-in policies, unless they are configured in `dfs.namenode.ec.policies.enabled` property, otherwise they are disabled by default.
 
     Similar to HDFS storage policies, erasure coding policies are set on a directory. When a file is created, it inherits the EC policy of its nearest ancestor directory.
 
@@ -112,7 +114,7 @@ Deployment
   what EC policies can be set by clients. It does not affect the behavior of already set file or directory-level EC policies.
 
   By default, all built-in erasure coding policies are disabled. Typically, the cluster administrator will enable set of policies by including them
-  in the `dfs .namenode.ec.policies.enabled` configuration based on the size of the cluster and the desired fault-tolerance properties. For instance,
+  in the `dfs.namenode.ec.policies.enabled` configuration based on the size of the cluster and the desired fault-tolerance properties. For instance,
   for a cluster with 9 racks, a policy like `RS-10-4-64k` will not preserve rack-level fault-tolerance, and `RS-6-3-64k` or `RS-3-2-64k` might
   be more appropriate. If the administrator only cares about node-level fault-tolerance, `RS-10-4-64k` would still be appropriate as long as
   there are at least 14 DataNodes in the cluster.
@@ -153,7 +155,7 @@ Deployment
   HDFS provides an `ec` subcommand to perform administrative commands related to erasure coding.
 
        hdfs ec [generic options]
-         [-setPolicy -policy <policyName> -path <path>]
+         [-setPolicy -path <path> [-policy <policyName>] [-replicate]]
          [-getPolicy -path <path>]
          [-unsetPolicy -path <path>]
          [-listPolicies]
@@ -165,7 +167,7 @@ Deployment
 
 Below are the details about each command.
 
- *  `[-setPolicy -policy <policyName> -path <path>]`
+ *  `[-setPolicy -path <path> [-policy <policyName>] [-replicate]]`
 
     Sets an erasure coding policy on a directory at the specified path.
 
@@ -175,6 +177,10 @@ Below are the details about each command.
       This parameter can be omitted if a 'dfs.namenode.ec.system.default.policy' configuration is set.
       The EC policy of the path will be set with the default value in configuration.
 
+      `-replicate` apply the special `REPLICATION` policy on the directory, force the directory to adopt 3x replication scheme.
+
+      `-replicate` and `-policy <policyName>` are optional arguments. They cannot be specified at the same time.
+
 
  *  `[-getPolicy -path <path>]`
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index 22e118f..47cdf23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -732,4 +732,85 @@ public class TestErasureCodingPolicies {
       }
     });
   }
+
+  @Test
+  public void testReplicationPolicy() throws Exception {
+    ErasureCodingPolicy replicaPolicy =
+        SystemErasureCodingPolicies.getReplicationPolicy();
+
+    final Path rootDir = new Path("/striped");
+    final Path replicaDir = new Path(rootDir, "replica");
+    final Path subReplicaDir = new Path(replicaDir, "replica");
+    final Path replicaFile = new Path(replicaDir, "file");
+    final Path subReplicaFile = new Path(subReplicaDir, "file");
+
+    fs.mkdirs(rootDir);
+    fs.setErasureCodingPolicy(rootDir, ecPolicy.getName());
+
+    // 1. At first, child directory will inherit parent's EC policy
+    fs.mkdirs(replicaDir);
+    fs.createFile(replicaFile).build().close();
+    HdfsFileStatus fileStatus = (HdfsFileStatus)fs.getFileStatus(replicaFile);
+    assertEquals("File should inherit EC policy.", ecPolicy, fileStatus
+        .getErasureCodingPolicy());
+    assertEquals("File should be a EC file.", true, fileStatus
+        .isErasureCoded());
+    assertEquals("File should have the same EC policy as its ancestor.",
+        ecPolicy, fs.getErasureCodingPolicy(replicaFile));
+    fs.delete(replicaFile, false);
+
+    // 2. Set replication policy on child directory, then get back the policy
+    fs.setErasureCodingPolicy(replicaDir, replicaPolicy.getName());
+    ErasureCodingPolicy temp = fs.getErasureCodingPolicy(replicaDir);
+    assertEquals("Directory should hide replication EC policy.",
+        null, temp);
+
+    // 3. New file will be replication file. Please be noted that replication
+    //    policy only set on directory, not on file
+    fs.createFile(replicaFile).build().close();
+    assertEquals("Replication file should have default replication factor.",
+        fs.getDefaultReplication(),
+        fs.getFileStatus(replicaFile).getReplication());
+    fs.setReplication(replicaFile, (short) 2);
+    assertEquals("File should have replication factor as expected.",
+        2, fs.getFileStatus(replicaFile).getReplication());
+    fileStatus = (HdfsFileStatus)fs.getFileStatus(replicaFile);
+    assertEquals("File should not have EC policy.", null, fileStatus
+        .getErasureCodingPolicy());
+    assertEquals("File should not be a EC file.", false,
+        fileStatus.isErasureCoded());
+    ErasureCodingPolicy ecPolicyOnFile = fs.getErasureCodingPolicy(replicaFile);
+    assertEquals("File should not have EC policy.", null, ecPolicyOnFile);
+    fs.delete(replicaFile, false);
+
+    // 4. New directory under replication directory, is also replication
+    // directory
+    fs.mkdirs(subReplicaDir);
+    assertEquals("Directory should inherit hiding replication EC policy.",
+        null, fs.getErasureCodingPolicy(subReplicaDir));
+    fs.createFile(subReplicaFile).build().close();
+    assertEquals("File should have default replication factor.",
+        fs.getDefaultReplication(),
+        fs.getFileStatus(subReplicaFile).getReplication());
+    fileStatus = (HdfsFileStatus)fs.getFileStatus(subReplicaFile);
+    assertEquals("File should not have EC policy.", null,
+        fileStatus.getErasureCodingPolicy());
+    assertEquals("File should not be a EC file.", false,
+        fileStatus.isErasureCoded());
+    assertEquals("File should not have EC policy.", null,
+        fs.getErasureCodingPolicy(subReplicaFile));
+    fs.delete(subReplicaFile, false);
+
+    // 5. Unset replication policy on directory, new file will be EC file
+    fs.unsetErasureCodingPolicy(replicaDir);
+    fs.createFile(subReplicaFile).build().close();
+    fileStatus = (HdfsFileStatus)fs.getFileStatus(subReplicaFile);
+    assertEquals("File should inherit EC policy.", ecPolicy,
+        fileStatus.getErasureCodingPolicy());
+    assertEquals("File should be a EC file.", true,
+        fileStatus.isErasureCoded());
+    assertEquals("File should have the same EC policy as its ancestor",
+        ecPolicy, fs.getErasureCodingPolicy(subReplicaFile));
+    fs.delete(subReplicaFile, false);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index 22c40fb..9256056 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -723,4 +723,91 @@ public class TestFSImage {
         .getBlockType());
     assertEquals(defaultBlockType, BlockType.CONTIGUOUS);
   }
+
+  /**
+   * Test if a INodeFile under a replication EC policy directory
+   * can be saved by FSImageSerialization and loaded by FSImageFormat#Loader.
+   */
+  @Test
+  public void testSaveAndLoadFileUnderReplicationPolicyDir()
+      throws IOException {
+    Configuration conf = new Configuration();
+    DFSTestUtil.enableAllECPolicies(conf);
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      FSNamesystem fsn = cluster.getNamesystem();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      ErasureCodingPolicy replicaPolicy =
+          SystemErasureCodingPolicies.getReplicationPolicy();
+      ErasureCodingPolicy defaultEcPolicy =
+          StripedFileTestUtil.getDefaultECPolicy();
+
+      final Path ecDir = new Path("/ec");
+      final Path replicaDir = new Path(ecDir, "replica");
+      final Path replicaFile1 = new Path(replicaDir, "f1");
+      final Path replicaFile2 = new Path(replicaDir, "f2");
+
+      // create root directory
+      fs.mkdir(ecDir, null);
+      fs.setErasureCodingPolicy(ecDir, defaultEcPolicy.getName());
+
+      // create directory, and set replication Policy
+      fs.mkdir(replicaDir, null);
+      fs.setErasureCodingPolicy(replicaDir, replicaPolicy.getName());
+
+      // create an empty file f1
+      fs.create(replicaFile1).close();
+
+      // create an under-construction file f2
+      FSDataOutputStream out = fs.create(replicaFile2, (short) 2);
+      out.writeBytes("hello");
+      ((DFSOutputStream) out.getWrappedStream()).hsync(EnumSet
+          .of(SyncFlag.UPDATE_LENGTH));
+
+      // checkpoint
+      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.saveNamespace();
+      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
+      cluster.restartNameNode();
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+
+      assertTrue(fs.getFileStatus(ecDir).isDirectory());
+      assertTrue(fs.getFileStatus(replicaDir).isDirectory());
+      assertTrue(fs.exists(replicaFile1));
+      assertTrue(fs.exists(replicaFile2));
+
+      // check directories
+      assertEquals("Directory should have default EC policy.",
+          defaultEcPolicy, fs.getErasureCodingPolicy(ecDir));
+      assertEquals("Directory should hide replication EC policy.",
+          null, fs.getErasureCodingPolicy(replicaDir));
+
+      // check file1
+      assertEquals("File should not have EC policy.", null,
+          fs.getErasureCodingPolicy(replicaFile1));
+      // check internals of file2
+      INodeFile file2Node =
+          fsn.dir.getINode4Write(replicaFile2.toString()).asFile();
+      assertEquals("hello".length(), file2Node.computeFileSize());
+      assertTrue(file2Node.isUnderConstruction());
+      BlockInfo[] blks = file2Node.getBlocks();
+      assertEquals(1, blks.length);
+      assertEquals(BlockUCState.UNDER_CONSTRUCTION, blks[0].getBlockUCState());
+      assertEquals("File should return expected replication factor.",
+          2, blks[0].getReplication());
+      assertEquals("File should not have EC policy.", null,
+          fs.getErasureCodingPolicy(replicaFile2));
+      // check lease manager
+      Lease lease = fsn.leaseManager.getLease(file2Node);
+      Assert.assertNotNull(lease);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
index c68c6d6..1baf355 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -101,7 +101,7 @@
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>[-setPolicy -path &lt;path&gt; -policy &lt;policy&gt;]</expected-output>
+          <expected-output>[-setPolicy -path &lt;path&gt; [-policy &lt;policy&gt;] [-replicate]]</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -238,6 +238,29 @@
     </test>
 
     <test>
+      <description>setPolicy : set replication policy on a directory</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy RS-6-3-64k -path /ecdir</ec-admin-command>
+        <command>-fs NAMENODE -mkdir /ecdir/replica</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -replicate -path /ecdir/replica</ec-admin-command>
+        <command>-fs NAMENODE -touchz /ecdir/replica/file</command>
+        <ec-admin-command>-fs NAMENODE -getPolicy -path /ecdir/replica/file</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /ecdir/replica/file</command>
+        <command>-fs NAMENODE -rmdir /ecdir/replica</command>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>is unspecified</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
       <description>unsetPolicy : unset policy and get</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
@@ -453,7 +476,7 @@
 
 <!-- Test illegal parameters -->
     <test>
-      <description>setPolicy : illegal parameters - path is missing</description>
+      <description>setPolicy : illegal parameters - path option is missing</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
         <ec-admin-command>-fs NAMENODE -setPolicy</ec-admin-command>
@@ -470,7 +493,7 @@
     </test>
 
     <test>
-      <description>setPolicy : illegal parameters - policy name is missing</description>
+      <description>setPolicy : illegal parameters - path name is missing</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
         <ec-admin-command>-fs NAMENODE -setPolicy -path</ec-admin-command>
@@ -487,7 +510,7 @@
     </test>
 
     <test>
-      <description>setPolicy : illegal parameters - too many arguments</description>
+      <description>setPolicy : illegal parameters - too many arguments case 1</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
         <ec-admin-command>-fs NAMENODE -setPolicy -path /ecdir1 -policy RS-3-2-64k /ecdir2</ec-admin-command>
@@ -504,6 +527,23 @@
     </test>
 
     <test>
+      <description>setPolicy : illegal parameters - too many arguments case 2</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -path /ecdir1 -policy RS-3-2-64k -replicate /ecdir2</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-setPolicy: Too many arguments</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
       <description>setPolicy : illegal parameters - invalidpolicy</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
@@ -553,6 +593,36 @@
     </test>
 
     <test>
+      <description>setPolicy : illegal parameters - wrong spelling replicate </description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -setPolicy -path /ecdir -replica</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-setPolicy: Too many arguments</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>setPolicy : illegal parameters - replicate and policy coexist</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -setPolicy -path /ecdir -policy RS-3-2-64k -replicate</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-replicate and -policy cannot been used at the same time</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
       <description>setPolicy : set erasure coding policy without given a specific policy name</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>


---------------------------------------------------------------------
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-6959. RM may allocate wrong AM Container for new attempt. Contributed by Yuqi Wang

Posted by ae...@apache.org.
YARN-6959. RM may allocate wrong AM Container for new attempt. Contributed by Yuqi Wang


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

Branch: refs/heads/HDFS-7240
Commit: e2f6299f6f580d7a03f2377d19ac85f55fd4e73b
Parents: ce797a1
Author: Jian He <ji...@apache.org>
Authored: Mon Aug 14 10:51:04 2017 -0700
Committer: Jian He <ji...@apache.org>
Committed: Mon Aug 14 10:51:30 2017 -0700

----------------------------------------------------------------------
 .../scheduler/AbstractYarnScheduler.java        |  1 +
 .../scheduler/capacity/CapacityScheduler.java   | 13 ++++++
 .../scheduler/fair/FairScheduler.java           | 15 ++++++-
 .../scheduler/fifo/FifoScheduler.java           | 15 ++++++-
 .../scheduler/fair/TestFairScheduler.java       | 46 ++++++++++----------
 5 files changed, 63 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f6299f/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 d506f4d..79caab0 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
@@ -323,6 +323,7 @@ public abstract class AbstractYarnScheduler
 
   }
 
+  // TODO: Rename it to getCurrentApplicationAttempt
   public T getApplicationAttempt(ApplicationAttemptId applicationAttemptId) {
     SchedulerApplication<T> app = applications.get(
         applicationAttemptId.getApplicationId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f6299f/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 3286982..e4ca003 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
@@ -903,6 +903,19 @@ public class CapacityScheduler extends
       ContainerUpdates updateRequests) {
     FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {
+      LOG.error("Calling allocate on removed or non existent application " +
+          applicationAttemptId.getApplicationId());
+      return EMPTY_ALLOCATION;
+    }
+
+    // The allocate may be the leftover from previous attempt, and it will
+    // impact current attempt, such as confuse the request and allocation for
+    // current attempt's AM container.
+    // Note outside precondition check for the attempt id may be
+    // outdated here, so double check it here is necessary.
+    if (!application.getApplicationAttemptId().equals(applicationAttemptId)) {
+      LOG.error("Calling allocate on previous or removed " +
+          "or non existent application attempt " + applicationAttemptId);
       return EMPTY_ALLOCATION;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f6299f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index db02bab..0f417c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -835,8 +835,19 @@ public class FairScheduler extends
     // Make sure this application exists
     FSAppAttempt application = getSchedulerApp(appAttemptId);
     if (application == null) {
-      LOG.info("Calling allocate on removed " +
-          "or non existent application " + appAttemptId);
+      LOG.error("Calling allocate on removed or non existent application " +
+          appAttemptId.getApplicationId());
+      return EMPTY_ALLOCATION;
+    }
+
+    // The allocate may be the leftover from previous attempt, and it will
+    // impact current attempt, such as confuse the request and allocation for
+    // current attempt's AM container.
+    // Note outside precondition check for the attempt id may be
+    // outdated here, so double check it here is necessary.
+    if (!application.getApplicationAttemptId().equals(appAttemptId)) {
+      LOG.error("Calling allocate on previous or removed " +
+          "or non existent application attempt " + appAttemptId);
       return EMPTY_ALLOCATION;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f6299f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.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/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
index a8d4f48..92a88b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
@@ -329,8 +329,19 @@ public class FifoScheduler extends
       ContainerUpdates updateRequests) {
     FifoAppAttempt application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {
-      LOG.error("Calling allocate on removed " +
-          "or non-existent application " + applicationAttemptId);
+      LOG.error("Calling allocate on removed or non existent application " +
+          applicationAttemptId.getApplicationId());
+      return EMPTY_ALLOCATION;
+    }
+
+    // The allocate may be the leftover from previous attempt, and it will
+    // impact current attempt, such as confuse the request and allocation for
+    // current attempt's AM container.
+    // Note outside precondition check for the attempt id may be
+    // outdated here, so double check it here is necessary.
+    if (!application.getApplicationAttemptId().equals(applicationAttemptId)) {
+      LOG.error("Calling allocate on previous or removed " +
+          "or non existent application attempt " + applicationAttemptId);
       return EMPTY_ALLOCATION;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f6299f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.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/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index 0d54c33..941c215 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -2107,49 +2107,49 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     scheduler.init(conf);
     scheduler.start();
     scheduler.reinitialize(conf, resourceManager.getRMContext());
+    int minReqSize =
+        FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB;
 
+    // First ask, queue1 requests 1 large (minReqSize * 2).
     ApplicationAttemptId id11 = createAppAttemptId(1, 1);
     createMockRMApp(id11);
-    scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1", false);
+    scheduler.addApplication(id11.getApplicationId(),
+        "root.queue1", "user1", false);
     scheduler.addApplicationAttempt(id11, false, false);
-    ApplicationAttemptId id21 = createAppAttemptId(2, 1);
-    createMockRMApp(id21);
-    scheduler.addApplication(id21.getApplicationId(), "root.queue2", "user1", false);
-    scheduler.addApplicationAttempt(id21, false, false);
-    ApplicationAttemptId id22 = createAppAttemptId(2, 2);
-    createMockRMApp(id22);
-
-    scheduler.addApplication(id22.getApplicationId(), "root.queue2", "user1", false);
-    scheduler.addApplicationAttempt(id22, false, false);
-
-    int minReqSize = 
-        FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB;
-    
-    // First ask, queue1 requests 1 large (minReqSize * 2).
     List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
-    ResourceRequest request1 =
-        createResourceRequest(minReqSize * 2, ResourceRequest.ANY, 1, 1, true);
+    ResourceRequest request1 = createResourceRequest(minReqSize * 2,
+        ResourceRequest.ANY, 1, 1, true);
     ask1.add(request1);
     scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
 
     // Second ask, queue2 requests 1 large.
+    ApplicationAttemptId id21 = createAppAttemptId(2, 1);
+    createMockRMApp(id21);
+    scheduler.addApplication(id21.getApplicationId(),
+        "root.queue2", "user1", false);
+    scheduler.addApplicationAttempt(id21, false, false);
     List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
-    ResourceRequest request2 = createResourceRequest(2 * minReqSize, "foo", 1, 1,
-        false);
+    ResourceRequest request2 = createResourceRequest(2 * minReqSize,
+        "foo", 1, 1, false);
     ResourceRequest request3 = createResourceRequest(2 * minReqSize,
-            ResourceRequest.ANY, 1, 1, false);
+        ResourceRequest.ANY, 1, 1, false);
     ask2.add(request2);
     ask2.add(request3);
     scheduler.allocate(id21, ask2, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
 
     // Third ask, queue2 requests 2 small (minReqSize).
+    ApplicationAttemptId id22 = createAppAttemptId(2, 2);
+    createMockRMApp(id22);
+    scheduler.addApplication(id22.getApplicationId(),
+        "root.queue2", "user1", false);
+    scheduler.addApplicationAttempt(id22, false, false);
     List<ResourceRequest> ask3 = new ArrayList<ResourceRequest>();
-    ResourceRequest request4 = createResourceRequest(minReqSize, "bar", 2, 2,
-            true);
+    ResourceRequest request4 = createResourceRequest(minReqSize,
+        "bar", 2, 2, true);
     ResourceRequest request5 = createResourceRequest(minReqSize,
-            ResourceRequest.ANY, 2, 2, true);
+        ResourceRequest.ANY, 2, 2, true);
     ask3.add(request4);
     ask3.add(request5);
     scheduler.allocate(id22, ask3, new ArrayList<ContainerId>(),


---------------------------------------------------------------------
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: YARN-7020. TestAMRMProxy#testAMRMProxyTokenRenewal is flakey. Contributed by Robert Kanter

Posted by ae...@apache.org.
YARN-7020. TestAMRMProxy#testAMRMProxyTokenRenewal is flakey. Contributed by Robert Kanter


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

Branch: refs/heads/HDFS-7240
Commit: 14553061be0a341df3e628dcaf06717b4630b05e
Parents: 588c190
Author: Jason Lowe <jl...@yahoo-inc.com>
Authored: Wed Aug 16 13:04:36 2017 -0500
Committer: Jason Lowe <jl...@yahoo-inc.com>
Committed: Wed Aug 16 13:04:36 2017 -0500

----------------------------------------------------------------------
 .../apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/14553061/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java
index 14df94a..6a063e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java
@@ -151,13 +151,13 @@ public class TestAMRMProxy extends BaseAMRMProxyE2ETest {
            YarnClient rmClient = YarnClient.createYarnClient()) {
       Configuration conf = new YarnConfiguration();
       conf.setBoolean(YarnConfiguration.AMRM_PROXY_ENABLED, true);
-      conf.setInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS, 1500);
-      conf.setInt(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, 1500);
-      conf.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 1500);
+      conf.setInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS, 4500);
+      conf.setInt(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, 4500);
+      conf.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 4500);
       // RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS should be at least
       // RM_AM_EXPIRY_INTERVAL_MS * 1.5 *3
       conf.setInt(
-          YarnConfiguration.RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS, 6);
+          YarnConfiguration.RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS, 20);
       cluster.init(conf);
       cluster.start();
       final Configuration yarnConf = cluster.getConfig();
@@ -198,7 +198,7 @@ public class TestAMRMProxy extends BaseAMRMProxyE2ETest {
         lastToken = response.getAMRMToken();
 
         // Time slot to be sure the AMRMProxy renew the token
-        Thread.sleep(1500);
+        Thread.sleep(4500);
 
       }
 


---------------------------------------------------------------------
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-6852. Native code changes to support isolate GPU devices by using CGroups. (wangda)

Posted by ae...@apache.org.
YARN-6852. Native code changes to support isolate GPU devices by using CGroups. (wangda)

Change-Id: I4869cc4d8ad539539ccba4bea5a178cacdb741ab


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

Branch: refs/heads/HDFS-7240
Commit: 436c2638f9ca1fb8de6a630cb5e91d956ac75216
Parents: 8991f0b
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Aug 18 18:26:36 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Aug 18 18:26:36 2017 -0700

----------------------------------------------------------------------
 .../src/CMakeLists.txt                          |  12 +-
 .../impl/container-executor.c                   |  10 +-
 .../impl/container-executor.h                   |   2 +
 .../main/native/container-executor/impl/main.c  |  13 +-
 .../impl/modules/cgroups/cgroups-operations.c   | 161 +++++++++++++
 .../impl/modules/cgroups/cgroups-operations.h   |  55 +++++
 .../impl/modules/common/constants.h             |  29 +++
 .../impl/modules/common/module-configs.c        |  41 ++++
 .../impl/modules/common/module-configs.h        |  33 +++
 .../impl/modules/gpu/gpu-module.c               | 229 +++++++++++++++++++
 .../impl/modules/gpu/gpu-module.h               |  45 ++++
 .../container-executor/impl/utils/path-utils.c  |  52 +++++
 .../container-executor/impl/utils/path-utils.h  |  35 +++
 .../impl/utils/string-utils.c                   | 106 +++++++--
 .../impl/utils/string-utils.h                   |   7 +-
 .../test/modules/cgroups/test-cgroups-module.cc | 121 ++++++++++
 .../test/modules/gpu/test-gpu-module.cc         | 203 ++++++++++++++++
 .../test/test-container-executor-common.h       |  36 +++
 .../test/test-container-executor.c              |  23 +-
 .../native/container-executor/test/test_main.cc |  11 +-
 .../test/utils/test-path-utils.cc               |  67 ++++++
 .../test/utils/test-string-utils.cc             |  93 ++++++++
 22 files changed, 1338 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
index 100d7ca..07c29bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
@@ -101,6 +101,10 @@ add_library(container
     main/native/container-executor/impl/container-executor.c
     main/native/container-executor/impl/get_executable.c
     main/native/container-executor/impl/utils/string-utils.c
+    main/native/container-executor/impl/utils/path-utils.c
+    main/native/container-executor/impl/modules/cgroups/cgroups-operations.c
+    main/native/container-executor/impl/modules/common/module-configs.c
+    main/native/container-executor/impl/modules/gpu/gpu-module.c
 )
 
 add_executable(container-executor
@@ -113,12 +117,14 @@ target_link_libraries(container-executor
 
 output_directory(container-executor target/usr/local/bin)
 
+# Test cases
 add_executable(test-container-executor
     main/native/container-executor/test/test-container-executor.c
 )
 target_link_libraries(test-container-executor
     container ${EXTRA_LIBS}
 )
+
 output_directory(test-container-executor target/usr/local/bin)
 
 # unit tests for container executor
@@ -126,6 +132,10 @@ add_executable(cetest
         main/native/container-executor/impl/util.c
         main/native/container-executor/test/test_configuration.cc
         main/native/container-executor/test/test_main.cc
+        main/native/container-executor/test/utils/test-string-utils.cc
+        main/native/container-executor/test/utils/test-path-utils.cc
+        main/native/container-executor/test/modules/cgroups/test-cgroups-module.cc
+        main/native/container-executor/test/modules/gpu/test-gpu-module.cc
         main/native/container-executor/test/test_util.cc)
-target_link_libraries(cetest gtest)
+target_link_libraries(cetest gtest container)
 output_directory(cetest test)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
index 7361808..560ec18 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
@@ -1384,7 +1384,6 @@ char* sanitize_docker_command(const char *line) {
 }
 
 char* parse_docker_command_file(const char* command_file) {
-
   size_t len = 0;
   char *line = NULL;
   ssize_t read;
@@ -2443,3 +2442,12 @@ int traffic_control_read_state(char *command_file) {
 int traffic_control_read_stats(char *command_file) {
   return run_traffic_control(TC_READ_STATS_OPTS, command_file);
 }
+
+/**
+ * FIXME: (wangda) it's better to move executor_cfg out of container-executor.c
+ * Now initialize of executor_cfg and data structures are stored inside
+ * container-executor which is not a good design.
+ */
+struct configuration* get_cfg() {
+  return &CFG;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
index ea8b5e3..aa38abf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
@@ -274,3 +274,5 @@ int execute_regex_match(const char *regex_str, const char *input);
  * Return 0 on success.
  */
 int validate_docker_image_name(const char *image_name);
+
+struct configuration* get_cfg();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
index b2187c9..a05dc78 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
@@ -20,6 +20,8 @@
 #include "configuration.h"
 #include "container-executor.h"
 #include "util.h"
+#include "modules/gpu/gpu-module.h"
+#include "modules/cgroups/cgroups-operations.h"
 
 #include <errno.h>
 #include <grp.h>
@@ -253,6 +255,14 @@ static int validate_arguments(int argc, char **argv , int *operation) {
     return INVALID_ARGUMENT_NUMBER;
   }
 
+  /*
+   * Check if it is a known module, if yes, redirect to module
+   */
+  if (strcmp("--module-gpu", argv[1]) == 0) {
+    return handle_gpu_request(&update_cgroups_parameters, "gpu", argc - 1,
+           &argv[1]);
+  }
+
   if (strcmp("--checksetup", argv[1]) == 0) {
     *operation = CHECK_SETUP;
     return 0;
@@ -332,6 +342,7 @@ static int validate_arguments(int argc, char **argv , int *operation) {
         return FEATURE_DISABLED;
     }
   }
+
   /* Now we have to validate 'run as user' operations that don't use
     a 'long option' - we should fix this at some point. The validation/argument
     parsing here is extensive enough that it done in a separate function */
@@ -522,7 +533,7 @@ int main(int argc, char **argv) {
   open_log_files();
   assert_valid_setup(argv[0]);
 
-  int operation;
+  int operation = -1;
   int ret = validate_arguments(argc, argv, &operation);
 
   if (ret != 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.c
new file mode 100644
index 0000000..b234109
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.c
@@ -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.
+ */
+
+#include "configuration.h"
+#include "container-executor.h"
+#include "utils/string-utils.h"
+#include "utils/path-utils.h"
+#include "modules/common/module-configs.h"
+#include "modules/common/constants.h"
+#include "modules/cgroups/cgroups-operations.h"
+#include "util.h"
+
+#include <string.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <sys/stat.h>
+
+#define MAX_PATH_LEN 4096
+
+static const struct section* cgroup_cfg_section = NULL;
+
+void reload_cgroups_configuration() {
+  cgroup_cfg_section = get_configuration_section(CGROUPS_SECTION_NAME, get_cfg());
+}
+
+char* get_cgroups_path_to_write(
+    const char* hierarchy_name,
+    const char* param_name,
+    const char* group_id) {
+  int failed = 0;
+  char* buffer = NULL;
+  const char* cgroups_root = get_section_value(CGROUPS_ROOT_KEY,
+     cgroup_cfg_section);
+  const char* yarn_hierarchy_name = get_section_value(
+     CGROUPS_YARN_HIERARCHY_KEY, cgroup_cfg_section);
+
+  // Make sure it is defined.
+  if (!cgroups_root || cgroups_root[0] == 0) {
+    fprintf(ERRORFILE, "%s is not defined in container-executor.cfg\n",
+      CGROUPS_ROOT_KEY);
+    failed = 1;
+    goto cleanup;
+  }
+
+  // Make sure it is defined.
+  if (!yarn_hierarchy_name || yarn_hierarchy_name[0] == 0) {
+    fprintf(ERRORFILE, "%s is not defined in container-executor.cfg\n",
+      CGROUPS_YARN_HIERARCHY_KEY);
+    failed = 1;
+    goto cleanup;
+  }
+
+  buffer = malloc(MAX_PATH_LEN + 1);
+  if (!buffer) {
+    fprintf(ERRORFILE, "Failed to allocate memory for output path.\n");
+    failed = 1;
+    goto cleanup;
+  }
+
+  // Make a path.
+  // CGroups path should not be too long.
+  if (snprintf(buffer, MAX_PATH_LEN, "%s/%s/%s/%s/%s.%s",
+    cgroups_root, hierarchy_name, yarn_hierarchy_name,
+    group_id, hierarchy_name, param_name) < 0) {
+    fprintf(ERRORFILE, "Failed to print output path.\n");
+    failed = 1;
+    goto cleanup;
+  }
+
+cleanup:
+  if (failed) {
+    if (buffer) {
+      free(buffer);
+    }
+    return NULL;
+  }
+  return buffer;
+}
+
+int update_cgroups_parameters(
+   const char* hierarchy_name,
+   const char* param_name,
+   const char* group_id,
+   const char* value) {
+#ifndef __linux
+  fprintf(ERRORFILE, "Failed to update cgroups parameters, not supported\n");
+  return -1;
+#endif
+  int failure = 0;
+
+  if (!cgroup_cfg_section) {
+    reload_cgroups_configuration();
+  }
+
+  char* full_path = get_cgroups_path_to_write(hierarchy_name, param_name,
+    group_id);
+
+  if (!full_path) {
+    fprintf(ERRORFILE,
+      "Failed to get cgroups path to write, it should be a configuration issue");
+    failure = 1;
+    goto cleanup;
+  }
+
+  if (!verify_path_safety(full_path)) {
+    failure = 1;
+    goto cleanup;
+  }
+
+  // Make sure file exists
+  struct stat sb;
+  if (stat(full_path, &sb) != 0) {
+    fprintf(ERRORFILE, "CGroups: Could not find file to write, %s", full_path);
+    failure = 1;
+    goto cleanup;
+  }
+
+  fprintf(ERRORFILE, "CGroups: Updating cgroups, path=%s, value=%s",
+    full_path, value);
+
+  // Write values to file
+  FILE *f;
+  f = fopen(full_path, "a");
+  if (!f) {
+    fprintf(ERRORFILE, "CGroups: Failed to open cgroups file, %s", full_path);
+    failure = 1;
+    goto cleanup;
+  }
+  if (fprintf(f, "%s", value) < 0) {
+    fprintf(ERRORFILE, "CGroups: Failed to write cgroups file, %s", full_path);
+    fclose(f);
+    failure = 1;
+    goto cleanup;
+  }
+  if (fclose(f) != 0) {
+    fprintf(ERRORFILE, "CGroups: Failed to close cgroups file, %s", full_path);
+    failure = 1;
+    goto cleanup;
+  }
+
+cleanup:
+  if (full_path) {
+    free(full_path);
+  }
+  return -failure;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.h
new file mode 100644
index 0000000..cf80bcf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.h
@@ -0,0 +1,55 @@
+/**
+ * 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.
+ */
+
+#ifndef _CGROUPS_OPERATIONS_H_
+#define _CGROUPS_OPERATIONS_H_
+
+#define CGROUPS_SECTION_NAME "cgroups"
+#define CGROUPS_ROOT_KEY "root"
+#define CGROUPS_YARN_HIERARCHY_KEY "yarn-hierarchy"
+
+/**
+ * Handle update CGroups parameter update requests:
+ * - hierarchy_name: e.g. devices / cpu,cpuacct
+ * - param_name: e.g. deny
+ * - group_id: e.g. container_x_y
+ * - value: e.g. "a *:* rwm"
+ *
+ * return 0 if succeeded
+ */
+int update_cgroups_parameters(
+   const char* hierarchy_name,
+   const char* param_name,
+   const char* group_id,
+   const char* value);
+
+ /**
+  * Get CGroups path to update. Visible for testing.
+  * Return 0 if succeeded
+  */
+ char* get_cgroups_path_to_write(
+    const char* hierarchy_name,
+    const char* param_name,
+    const char* group_id);
+
+ /**
+  * Reload config from filesystem, visible for testing.
+  */
+ void reload_cgroups_configuration();
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/constants.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/constants.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/constants.h
new file mode 100644
index 0000000..5c8c4e9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/constants.h
@@ -0,0 +1,29 @@
+/**
+ * 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.
+ */
+
+/* FreeBSD protects the getline() prototype. See getline(3) for more */
+#ifdef __FreeBSD__
+#define _WITH_GETLINE
+#endif
+
+#ifndef _MODULES_COMMON_CONSTANTS_H_
+#define _MODULES_COMMON_CONSTANTS_H_
+
+#define CONFIGS_MODULES_PREFIX "yarn.container-executor.modules."
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.c
new file mode 100644
index 0000000..f0c6d16
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.c
@@ -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.
+ */
+
+#include "util.h"
+#include "configuration.h"
+#include "container-executor.h"
+#include "modules/common/constants.h"
+
+#include <string.h>
+#include <stdio.h>
+#include <stdlib.h>
+
+#define ENABLED_CONFIG_KEY "module.enabled"
+
+int module_enabled(const struct section* section_cfg, const char* module_name) {
+  char* enabled_str = get_section_value(ENABLED_CONFIG_KEY, section_cfg);
+  int enabled = 0;
+  if (enabled_str && 0 == strcmp(enabled_str, "true")) {
+    enabled = 1;
+  } else {
+    fprintf(LOGFILE, "Module %s is disabled\n", module_name);
+  }
+
+  free(enabled_str);
+  return enabled;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.h
new file mode 100644
index 0000000..d58c618
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.h
@@ -0,0 +1,33 @@
+/**
+ * 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.
+ */
+
+#ifdef __FreeBSD__
+#define _WITH_GETLINE
+#endif
+
+#ifndef _MODULES_COMMON_MODULE_CONFIGS_H_
+#define _MODULES_COMMON_MODULE_CONFIGS_H_
+
+
+/**
+ * check if module enabled given name of module.
+ * return 0 if disabled
+ */
+int module_enabled(const struct section* section_cfg, const char* module_name);
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.c
new file mode 100644
index 0000000..f96645d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.c
@@ -0,0 +1,229 @@
+/**
+ * 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.
+ */
+
+#include "configuration.h"
+#include "container-executor.h"
+#include "utils/string-utils.h"
+#include "modules/gpu/gpu-module.h"
+#include "modules/cgroups/cgroups-operations.h"
+#include "modules/common/module-configs.h"
+#include "modules/common/constants.h"
+#include "util.h"
+
+#include <stdio.h>
+#include <string.h>
+#include <stdlib.h>
+#include <getopt.h>
+#include <unistd.h>
+
+#define EXCLUDED_GPUS_OPTION "excluded_gpus"
+#define CONTAINER_ID_OPTION "container_id"
+#define DEFAULT_NVIDIA_MAJOR_NUMBER 195
+#define MAX_CONTAINER_ID_LEN 128
+
+static const struct section* cfg_section;
+
+static int internal_handle_gpu_request(
+    update_cgroups_parameters_func update_cgroups_parameters_func_p,
+    size_t n_minor_devices_to_block, int minor_devices[],
+    const char* container_id) {
+  char* allowed_minor_numbers_str = NULL;
+  int* allowed_minor_numbers = NULL;
+  size_t n_allowed_minor_numbers = 0;
+  int return_code = 0;
+
+  if (n_minor_devices_to_block == 0) {
+    // no device to block, just return;
+    return 0;
+  }
+
+  // Get major device number from cfg, if not set, major number of (Nvidia)
+  // will be the default value.
+  int major_device_number;
+  char* major_number_str = get_section_value(GPU_MAJOR_NUMBER_CONFIG_KEY,
+     cfg_section);
+  if (!major_number_str || 0 == major_number_str[0]) {
+    // Default major number of Nvidia devices
+    major_device_number = DEFAULT_NVIDIA_MAJOR_NUMBER;
+  } else {
+    major_device_number = strtol(major_number_str, NULL, 0);
+  }
+
+  // Get allowed minor device numbers from cfg, if not set, means all minor
+  // devices can be used by YARN
+  allowed_minor_numbers_str = get_section_value(
+      GPU_ALLOWED_DEVICES_MINOR_NUMBERS,
+      cfg_section);
+  if (!allowed_minor_numbers_str || 0 == allowed_minor_numbers_str[0]) {
+    allowed_minor_numbers = NULL;
+  } else {
+    int rc = get_numbers_split_by_comma(allowed_minor_numbers_str,
+                                        &allowed_minor_numbers,
+                                        &n_allowed_minor_numbers);
+    if (0 != rc) {
+      fprintf(ERRORFILE,
+          "Failed to get allowed minor device numbers from cfg, value=%s\n",
+          allowed_minor_numbers_str);
+      return_code = -1;
+      goto cleanup;
+    }
+
+    // Make sure we're trying to black devices allowed in config
+    for (int i = 0; i < n_minor_devices_to_block; i++) {
+      int found = 0;
+      for (int j = 0; j < n_allowed_minor_numbers; j++) {
+        if (minor_devices[i] == allowed_minor_numbers[j]) {
+          found = 1;
+          break;
+        }
+      }
+
+      if (!found) {
+        fprintf(ERRORFILE,
+          "Trying to blacklist device with minor-number=%d which is not on allowed list\n",
+          minor_devices[i]);
+        return_code = -1;
+        goto cleanup;
+      }
+    }
+  }
+
+  // Use cgroup helpers to blacklist devices
+  for (int i = 0; i < n_minor_devices_to_block; i++) {
+    char param_value[128];
+    memset(param_value, 0, sizeof(param_value));
+    snprintf(param_value, sizeof(param_value), "c %d:%d rwm",
+             major_device_number, i);
+
+    int rc = update_cgroups_parameters_func_p("devices", "deny",
+      container_id, param_value);
+
+    if (0 != rc) {
+      fprintf(ERRORFILE, "CGroups: Failed to update cgroups\n");
+      return_code = -1;
+      goto cleanup;
+    }
+  }
+
+cleanup:
+  if (major_number_str) {
+    free(major_number_str);
+  }
+  if (allowed_minor_numbers) {
+    free(allowed_minor_numbers);
+  }
+  if (allowed_minor_numbers_str) {
+    free(allowed_minor_numbers_str);
+  }
+
+  return return_code;
+}
+
+void reload_gpu_configuration() {
+  cfg_section = get_configuration_section(GPU_MODULE_SECTION_NAME, get_cfg());
+}
+
+/*
+ * Format of GPU request commandline:
+ *
+ * c-e gpu --excluded_gpus 0,1,3 --container_id container_x_y
+ */
+int handle_gpu_request(update_cgroups_parameters_func func,
+    const char* module_name, int module_argc, char** module_argv) {
+  if (!cfg_section) {
+    reload_gpu_configuration();
+  }
+
+  if (!module_enabled(cfg_section, GPU_MODULE_SECTION_NAME)) {
+    fprintf(ERRORFILE,
+      "Please make sure gpu module is enabled before using it.\n");
+    return -1;
+  }
+
+  static struct option long_options[] = {
+    {EXCLUDED_GPUS_OPTION, required_argument, 0, 'e' },
+    {CONTAINER_ID_OPTION, required_argument, 0, 'c' },
+    {0, 0, 0, 0}
+  };
+
+  int rc = 0;
+  int c = 0;
+  int option_index = 0;
+
+  int* minor_devices = NULL;
+  char container_id[MAX_CONTAINER_ID_LEN];
+  memset(container_id, 0, sizeof(container_id));
+  size_t n_minor_devices_to_block = 0;
+  int failed = 0;
+
+  optind = 1;
+  while((c = getopt_long(module_argc, module_argv, "e:c:",
+                         long_options, &option_index)) != -1) {
+    switch(c) {
+      case 'e':
+        rc = get_numbers_split_by_comma(optarg, &minor_devices,
+          &n_minor_devices_to_block);
+        if (0 != rc) {
+          fprintf(ERRORFILE,
+            "Failed to get minor devices number from command line, value=%s\n",
+            optarg);
+          failed = 1;
+          goto cleanup;
+        }
+        break;
+      case 'c':
+        if (!validate_container_id(optarg)) {
+          fprintf(ERRORFILE,
+            "Specified container_id=%s is invalid\n", optarg);
+          failed = 1;
+          goto cleanup;
+        }
+        strncpy(container_id, optarg, MAX_CONTAINER_ID_LEN);
+        break;
+      default:
+        fprintf(ERRORFILE,
+          "Unknown option in gpu command character %d %c, optionindex = %d\n",
+          c, c, optind);
+        failed = 1;
+        goto cleanup;
+    }
+  }
+
+  if (0 == container_id[0]) {
+    fprintf(ERRORFILE,
+      "[%s] --container_id must be specified.\n", __func__);
+    failed = 1;
+    goto cleanup;
+  }
+
+  if (!minor_devices) {
+     // Minor devices is null, skip following call.
+     fprintf(ERRORFILE, "is not specified, skip cgroups call.\n");
+     goto cleanup;
+  }
+
+  failed = internal_handle_gpu_request(func, n_minor_devices_to_block,
+         minor_devices,
+         container_id);
+
+cleanup:
+  if (minor_devices) {
+    free(minor_devices);
+  }
+  return failed;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.h
new file mode 100644
index 0000000..59d4c7e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.h
@@ -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.
+ */
+
+#ifdef __FreeBSD__
+#define _WITH_GETLINE
+#endif
+
+#ifndef _MODULES_GPU_GPU_MUDULE_H_
+#define _MODULES_GPU_GPU_MUDULE_H_
+
+#define GPU_MAJOR_NUMBER_CONFIG_KEY "gpu.major-device-number"
+#define GPU_ALLOWED_DEVICES_MINOR_NUMBERS "gpu.allowed-device-minor-numbers"
+#define GPU_MODULE_SECTION_NAME "gpu"
+
+// For unit test stubbing
+typedef int (*update_cgroups_parameters_func)(const char*, const char*,
+   const char*, const char*);
+
+/**
+ * Handle gpu requests
+ */
+int handle_gpu_request(update_cgroups_parameters_func func,
+   const char* module_name, int module_argc, char** module_argv);
+
+/**
+ * Reload config from filesystem, visible for testing.
+ */
+void reload_gpu_configuration();
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.c
new file mode 100644
index 0000000..dea656b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.c
@@ -0,0 +1,52 @@
+/**
+ * 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.
+ */
+
+#include "util.h"
+
+#include <strings.h>
+#include <string.h>
+#include <stdio.h>
+#include <stdlib.h>
+
+int verify_path_safety(const char* path) {
+  if (!path || path[0] == 0) {
+    return 1;
+  }
+
+  char* dup = strdup(path);
+  if (!dup) {
+    fprintf(ERRORFILE, "%s: Failed to allocate memory for path.\n", __func__);
+    return 0;
+  }
+
+  char* p = strtok(dup, "/");
+  int succeeded = 1;
+
+  while (p != NULL) {
+    if (0 == strcmp(p, "..")) {
+      fprintf(ERRORFILE, "%s: Path included \"..\", path=%s.\n", __func__, path);
+      succeeded = 0;
+      break;
+    }
+
+    p = strtok(NULL, "/");
+  }
+  free(dup);
+
+  return succeeded;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.h
new file mode 100644
index 0000000..a42f936
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.h
@@ -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.
+ */
+
+#ifdef __FreeBSD__
+#define _WITH_GETLINE
+#endif
+
+#ifndef _UTILS_PATH_UTILS_H_
+#define _UTILS_PATH_UTILS_H_
+
+/*
+ * Verify if a given path is safe or not. For example, we don't want a path
+ * include ".." which can do things like:
+ * - "/cgroups/cpu,cpuacct/container/../../../etc/passwd"
+ *
+ * return false/true
+ */
+int verify_path_safety(const char* path);
+
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
index 063df7e..d19c084 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
@@ -15,7 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "util.h"
 
+#include <limits.h>
+#include <errno.h>
 #include <strings.h>
 #include <string.h>
 #include <stdio.h>
@@ -26,60 +29,131 @@
  * return true/false
  */
 static int all_numbers(char* input) {
-  if (0 == strlen(input)) {
-    return 0;
-  }
-
-  for (int i = 0; i < strlen(input); i++) {
-    if (input[i] < '0' || input[i] > '9') {
+  for (; input[0] != 0; input++) {
+    if (input[0] < '0' || input[0] > '9') {
       return 0;
     }
   }
   return 1;
 }
 
+int get_numbers_split_by_comma(const char* input, int** numbers,
+                               size_t* ret_n_numbers) {
+  size_t allocation_size = 1;
+  int i = 0;
+  while (input[i] != 0) {
+    if (input[i] == ',') {
+      allocation_size++;
+    }
+    i++;
+  }
+
+  (*numbers) = malloc(sizeof(int) * allocation_size);
+  if (!(*numbers)) {
+    fprintf(ERRORFILE, "Failed to allocating memory for *numbers: %s\n",
+            __func__);
+    exit(OUT_OF_MEMORY);
+  }
+  memset(*numbers, 0, sizeof(int) * allocation_size);
+
+  char* input_cpy = strdup(input);
+  if (!input_cpy) {
+    fprintf(ERRORFILE, "Failed to allocating memory for input_cpy: %s\n",
+            __func__);
+    exit(OUT_OF_MEMORY);
+  }
+
+  char* p = strtok(input_cpy, ",");
+  int idx = 0;
+  size_t n_numbers = 0;
+  while (p != NULL) {
+    char *temp;
+    long n = strtol(p, &temp, 0);
+    // According to answer:
+    // https://stackoverflow.com/questions/14176123/correct-usage-of-strtol
+    // We need to properly check errno and overflows
+    if (temp == p || *temp != '\0' ||
+        ((n == LONG_MIN || n == LONG_MAX) && errno == ERANGE)) {
+      fprintf(stderr,
+              "Could not convert '%s' to long and leftover string is: '%s'\n",
+              p, temp);
+      free(input_cpy);
+      return -1;
+    }
+
+    n_numbers++;
+    (*numbers)[idx] = n;
+    p = strtok(NULL, ",");
+    idx++;
+  }
+
+  free(input_cpy);
+  *ret_n_numbers = n_numbers;
+
+  return 0;
+}
+
 int validate_container_id(const char* input) {
+  int is_container_id = 1;
+
   /*
    * Two different forms of container_id
    * container_e17_1410901177871_0001_01_000005
    * container_1410901177871_0001_01_000005
    */
+  if (!input) {
+    return 0;
+  }
+
   char* input_cpy = strdup(input);
+  if (!input_cpy) {
+    return 0;
+  }
+
   char* p = strtok(input_cpy, "_");
   int idx = 0;
   while (p != NULL) {
     if (0 == idx) {
       if (0 != strcmp("container", p)) {
-        return 0;
+        is_container_id = 0;
+        goto cleanup;
       }
     } else if (1 == idx) {
       // this could be e[n][n], or [n][n]...
       if (!all_numbers(p)) {
-        if (strlen(p) == 0) {
-          return 0;
+        if (p[0] == 0) {
+          is_container_id = 0;
+          goto cleanup;
         }
         if (p[0] != 'e') {
-          return 0;
+          is_container_id = 0;
+          goto cleanup;
         }
         if (!all_numbers(p + 1)) {
-          return 0;
+          is_container_id = 0;
+          goto cleanup;
         }
       }
     } else {
       // otherwise, should be all numbers
       if (!all_numbers(p)) {
-        return 0;
+        is_container_id = 0;
+        goto cleanup;
       }
     }
 
     p = strtok(NULL, "_");
     idx++;
   }
-  free(input_cpy);
+
+cleanup:
+  if (input_cpy) {
+    free(input_cpy);
+  }
 
   // We should have [5,6] elements split by '_'
   if (idx > 6 || idx < 5) {
-    return 0;
+    is_container_id = 0;
   }
-  return 1;
-}
\ No newline at end of file
+  return is_container_id;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h
index 0a41ad1..c095eb6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h
@@ -29,4 +29,9 @@
  */
 int validate_container_id(const char* input);
 
-#endif
\ No newline at end of file
+/*
+ * return 0 if succeeded
+ */
+int get_numbers_split_by_comma(const char* input, int** numbers, size_t* n_numbers);
+
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/cgroups/test-cgroups-module.cc
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/cgroups/test-cgroups-module.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/cgroups/test-cgroups-module.cc
new file mode 100644
index 0000000..8ffbe88
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/cgroups/test-cgroups-module.cc
@@ -0,0 +1,121 @@
+/**
+ * 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.
+ */
+
+#include <errno.h>
+#include <fcntl.h>
+#include <inttypes.h>
+#include <signal.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/stat.h>
+#include <sys/wait.h>
+#include <unistd.h>
+
+#include <gtest/gtest.h>
+#include <sstream>
+
+extern "C" {
+#include "configuration.h"
+#include "container-executor.h"
+#include "modules/cgroups/cgroups-operations.h"
+#include "test/test-container-executor-common.h"
+#include "util.h"
+}
+
+namespace ContainerExecutor {
+
+class TestCGroupsModule : public ::testing::Test {
+protected:
+  virtual void SetUp() {
+    if (mkdirs(TEST_ROOT, 0755) != 0) {
+      fprintf(ERRORFILE, "Failed to mkdir TEST_ROOT: %s\n", TEST_ROOT);
+      exit(1);
+    }
+    LOGFILE = stdout;
+    ERRORFILE = stderr;
+  }
+
+  virtual void TearDown() {}
+};
+
+TEST_F(TestCGroupsModule, test_cgroups_get_path_without_define_root) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_cgroups_get_path_without_root.cfg";
+  FILE *file = fopen(filename, "w");
+  if (file == NULL) {
+    printf("FAIL: Could not open configuration file: %s\n", filename);
+    exit(1);
+  }
+  fprintf(file, "[cgroups]\n");
+  fprintf(file, "yarn-hierarchy=yarn\n");
+  fclose(file);
+
+  // Read config file
+  read_executor_config(filename);
+  reload_cgroups_configuration();
+
+  char* path = get_cgroups_path_to_write("devices", "deny", "container_1");
+
+  ASSERT_TRUE(NULL == path) << "Should fail.\n";
+}
+
+TEST_F(TestCGroupsModule, test_cgroups_get_path_without_define_yarn_hierarchy) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_cgroups_get_path_without_root.cfg";
+  FILE *file = fopen(filename, "w");
+
+  ASSERT_TRUE(file) << "FAIL: Could not open configuration file: " << filename
+                    << "\n";
+  fprintf(file, "[cgroups]\n");
+  fprintf(file, "root=/sys/fs/cgroups\n");
+  fclose(file);
+
+  // Read config file
+  read_executor_config(filename);
+  reload_cgroups_configuration();
+  char* path = get_cgroups_path_to_write("devices", "deny", "container_1");
+
+  ASSERT_TRUE(NULL == path) << "Should fail.\n";
+}
+
+TEST_F(TestCGroupsModule, test_cgroups_get_path_succeeded) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_cgroups_get_path.cfg";
+  FILE *file = fopen(filename, "w");
+
+  ASSERT_TRUE(file) << "FAIL: Could not open configuration file\n";
+  fprintf(file, "[cgroups]\n");
+  fprintf(file, "root=/sys/fs/cgroups \n");
+  fprintf(file, "yarn-hierarchy=yarn \n");
+  fclose(file);
+
+  // Read config file
+  read_executor_config(filename);
+  reload_cgroups_configuration();
+
+  char* path = get_cgroups_path_to_write("devices", "deny", "container_1");
+  ASSERT_TRUE(NULL != path) << "Should success.\n";
+
+  const char *EXPECTED =
+      "/sys/fs/cgroups/devices/yarn/container_1/devices.deny";
+
+  ASSERT_STREQ(EXPECTED, path)
+      << "Return cgroup-path-to-write is not expected\n";
+}
+} // namespace ContainerExecutor
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/gpu/test-gpu-module.cc
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/gpu/test-gpu-module.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/gpu/test-gpu-module.cc
new file mode 100644
index 0000000..7e41fb4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/gpu/test-gpu-module.cc
@@ -0,0 +1,203 @@
+/**
+ * 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.
+ */
+
+#include <vector>
+
+#include <errno.h>
+#include <fcntl.h>
+#include <inttypes.h>
+#include <signal.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/stat.h>
+#include <sys/wait.h>
+#include <unistd.h>
+
+#include <gtest/gtest.h>
+#include <sstream>
+
+extern "C" {
+#include "configuration.h"
+#include "container-executor.h"
+#include "modules/cgroups/cgroups-operations.h"
+#include "modules/gpu/gpu-module.h"
+#include "test/test-container-executor-common.h"
+#include "util.h"
+}
+
+namespace ContainerExecutor {
+
+class TestGpuModule : public ::testing::Test {
+protected:
+  virtual void SetUp() {
+    if (mkdirs(TEST_ROOT, 0755) != 0) {
+      fprintf(ERRORFILE, "Failed to mkdir TEST_ROOT: %s\n", TEST_ROOT);
+      exit(1);
+    }
+    LOGFILE = stdout;
+    ERRORFILE = stderr;
+  }
+
+  virtual void TearDown() {
+
+  }
+};
+
+static std::vector<const char*> cgroups_parameters_invoked;
+
+static int mock_update_cgroups_parameters(
+   const char* controller_name,
+   const char* param_name,
+   const char* group_id,
+   const char* value) {
+  char* buf = (char*) malloc(128);
+  strcpy(buf, controller_name);
+  cgroups_parameters_invoked.push_back(buf);
+
+  buf = (char*) malloc(128);
+  strcpy(buf, param_name);
+  cgroups_parameters_invoked.push_back(buf);
+
+  buf = (char*) malloc(128);
+  strcpy(buf, group_id);
+  cgroups_parameters_invoked.push_back(buf);
+
+  buf = (char*) malloc(128);
+  strcpy(buf, value);
+  cgroups_parameters_invoked.push_back(buf);
+  return 0;
+}
+
+static void verify_param_updated_to_cgroups(
+    int argc, const char** argv) {
+  ASSERT_EQ(argc, cgroups_parameters_invoked.size());
+
+  int offset = 0;
+  while (offset < argc) {
+    ASSERT_STREQ(argv[offset], cgroups_parameters_invoked[offset]);
+    offset++;
+  }
+}
+
+static void write_and_load_gpu_module_to_cfg(const char* cfg_filepath, int enabled) {
+  FILE *file = fopen(cfg_filepath, "w");
+  if (file == NULL) {
+    printf("FAIL: Could not open configuration file: %s\n", cfg_filepath);
+    exit(1);
+  }
+  fprintf(file, "[gpu]\n");
+  if (enabled) {
+    fprintf(file, "module.enabled=true\n");
+  } else {
+    fprintf(file, "module.enabled=false\n");
+  }
+  fclose(file);
+
+  // Read config file
+  read_executor_config(cfg_filepath);
+  reload_gpu_configuration();
+}
+
+static void test_gpu_module_enabled_disabled(int enabled) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_cgroups_module_enabled_disabled.cfg";
+  write_and_load_gpu_module_to_cfg(filename, enabled);
+
+  char* argv[] = { (char*) "--module-gpu", (char*) "--excluded_gpus", (char*) "0,1",
+                   (char*) "--container_id",
+                   (char*) "container_1498064906505_0001_01_000001" };
+
+  int rc = handle_gpu_request(&mock_update_cgroups_parameters,
+              "gpu", 5, argv);
+
+  int EXPECTED_RC;
+  if (enabled) {
+    EXPECTED_RC = 0;
+  } else {
+    EXPECTED_RC = -1;
+  }
+  ASSERT_EQ(EXPECTED_RC, rc);
+}
+
+TEST_F(TestGpuModule, test_verify_gpu_module_calls_cgroup_parameter) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_verify_gpu_module_calls_cgroup_parameter.cfg";
+  write_and_load_gpu_module_to_cfg(filename, 1);
+
+  char* container_id = (char*) "container_1498064906505_0001_01_000001";
+  char* argv[] = { (char*) "--module-gpu", (char*) "--excluded_gpus", (char*) "0,1",
+                   (char*) "--container_id",
+                   container_id };
+
+  /* Test case 1: block 2 devices */
+  cgroups_parameters_invoked.clear();
+  int rc = handle_gpu_request(&mock_update_cgroups_parameters,
+     "gpu", 5, argv);
+  ASSERT_EQ(0, rc) << "Should success.\n";
+
+  // Verify cgroups parameters
+  const char* expected_cgroups_argv[] = { "devices", "deny", container_id, "c 195:0 rwm",
+    "devices", "deny", container_id, "c 195:1 rwm"};
+  verify_param_updated_to_cgroups(8, expected_cgroups_argv);
+
+  /* Test case 2: block 0 devices */
+  cgroups_parameters_invoked.clear();
+  char* argv_1[] = { (char*) "--module-gpu", (char*) "--container_id", container_id };
+  rc = handle_gpu_request(&mock_update_cgroups_parameters,
+     "gpu", 3, argv_1);
+  ASSERT_EQ(0, rc) << "Should success.\n";
+
+  // Verify cgroups parameters
+  verify_param_updated_to_cgroups(0, NULL);
+}
+
+TEST_F(TestGpuModule, test_illegal_cli_parameters) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_illegal_cli_parameters.cfg";
+  write_and_load_gpu_module_to_cfg(filename, 1);
+
+  // Illegal container id - 1
+  char* argv[] = { (char*) "--module-gpu", (char*) "--excluded_gpus", (char*) "0,1",
+                   (char*) "--container_id", (char*) "xxxx" };
+  int rc = handle_gpu_request(&mock_update_cgroups_parameters,
+     "gpu", 5, argv);
+  ASSERT_NE(0, rc) << "Should fail.\n";
+
+  // Illegal container id - 2
+  char* argv_1[] = { (char*) "--module-gpu", (char*) "--excluded_gpus", (char*) "0,1",
+                   (char*) "--container_id", (char*) "container_1" };
+  rc = handle_gpu_request(&mock_update_cgroups_parameters,
+     "gpu", 5, argv_1);
+  ASSERT_NE(0, rc) << "Should fail.\n";
+
+  // Illegal container id - 3
+  char* argv_2[] = { (char*) "--module-gpu", (char*) "--excluded_gpus", (char*) "0,1" };
+  rc = handle_gpu_request(&mock_update_cgroups_parameters,
+     "gpu", 3, argv_2);
+  ASSERT_NE(0, rc) << "Should fail.\n";
+}
+
+TEST_F(TestGpuModule, test_gpu_module_disabled) {
+  test_gpu_module_enabled_disabled(0);
+}
+
+TEST_F(TestGpuModule, test_gpu_module_enabled) {
+  test_gpu_module_enabled_disabled(1);
+}
+} // namespace ContainerExecutor
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor-common.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor-common.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor-common.h
new file mode 100644
index 0000000..d353625
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor-common.h
@@ -0,0 +1,36 @@
+/**
+ * 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.
+ */
+
+ #ifdef __APPLE__
+ #include <CoreFoundation/CFString.h>
+ #include <CoreFoundation/CFPreferences.h>
+
+ #define TMPDIR "/private/tmp"
+ #define RELTMPDIR "../.."
+ #else
+ #define RELTMPDIR ".."
+ #define TMPDIR "/tmp"
+ #endif
+
+ #define TEST_ROOT TMPDIR "/test-container-executor"
+
+ #define DONT_TOUCH_FILE "dont-touch-me"
+ #define NM_LOCAL_DIRS       TEST_ROOT "/local-1%" TEST_ROOT "/local-2%" \
+                TEST_ROOT "/local-3%" TEST_ROOT "/local-4%" TEST_ROOT "/local-5"
+ #define NM_LOG_DIRS         TEST_ROOT "/logs/userlogs"
+ #define ARRAY_SIZE 1000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
index 3cfefa0..64ee717 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
@@ -19,6 +19,7 @@
 #include "container-executor.h"
 #include "utils/string-utils.h"
 #include "util.h"
+#include "test/test-container-executor-common.h"
 
 #include <inttypes.h>
 #include <errno.h>
@@ -31,25 +32,6 @@
 #include <sys/stat.h>
 #include <sys/wait.h>
 
-#ifdef __APPLE__
-#include <CoreFoundation/CFString.h>
-#include <CoreFoundation/CFPreferences.h>
-
-#define TMPDIR "/private/tmp"
-#define RELTMPDIR "../.."
-#else
-#define RELTMPDIR ".."
-#define TMPDIR "/tmp"
-#endif
-
-#define TEST_ROOT TMPDIR "/test-container-executor"
-
-#define DONT_TOUCH_FILE "dont-touch-me"
-#define NM_LOCAL_DIRS       TEST_ROOT "/local-1%" TEST_ROOT "/local-2%" \
-               TEST_ROOT "/local-3%" TEST_ROOT "/local-4%" TEST_ROOT "/local-5"
-#define NM_LOG_DIRS         TEST_ROOT "/logs/userlogs"
-#define ARRAY_SIZE 1000
-
 static char* username = NULL;
 static char* yarn_username = NULL;
 static char** local_dirs = NULL;
@@ -1486,10 +1468,7 @@ int main(int argc, char **argv) {
   test_check_user(1);
 #endif
 
-  run("rm -fr " TEST_ROOT);
-
   test_trim_function();
-
   printf("\nFinished tests\n");
 
   free(current_username);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_main.cc
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_main.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_main.cc
index d59a3f2..44c9b1b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_main.cc
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_main.cc
@@ -20,10 +20,13 @@
 #include <main/native/container-executor/impl/util.h>
 #include <cstdio>
 
-FILE* ERRORFILE = stderr;
-FILE* LOGFILE = stdout;
+extern "C" {
+#include "util.h"
+}
 
 int main(int argc, char **argv) {
-    testing::InitGoogleTest(&argc, argv);
-    return RUN_ALL_TESTS();
+  ERRORFILE = stderr;
+  LOGFILE = stdout;
+  testing::InitGoogleTest(&argc, argv);
+  return RUN_ALL_TESTS();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-path-utils.cc
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-path-utils.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-path-utils.cc
new file mode 100644
index 0000000..a24c0c7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-path-utils.cc
@@ -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.
+ */
+
+ #include <errno.h>
+ #include <fcntl.h>
+ #include <inttypes.h>
+ #include <signal.h>
+ #include <stdio.h>
+ #include <stdlib.h>
+ #include <string.h>
+ #include <sys/stat.h>
+ #include <sys/wait.h>
+ #include <unistd.h>
+
+ #include <gtest/gtest.h>
+ #include <sstream>
+
+ extern "C" {
+ #include "utils/path-utils.h"
+ }
+
+ namespace ContainerExecutor {
+
+ class TestPathUtils : public ::testing::Test {
+ protected:
+   virtual void SetUp() {
+
+   }
+
+   virtual void TearDown() {
+
+   }
+ };
+
+ TEST_F(TestPathUtils, test_path_safety) {
+   const char* input = "./../abc/";
+   int flag = verify_path_safety(input);
+   std::cout << "Testing input=" << input << "\n";
+   ASSERT_FALSE(flag) << "Should failed\n";
+
+   input = "abc/./cde";
+   flag = verify_path_safety(input);
+   std::cout << "Testing input=" << input << "\n";
+   ASSERT_TRUE(flag) << "Should succeeded\n";
+
+   input = "/etc/abc/cde/./x/./y";
+   flag = verify_path_safety(input);
+   std::cout << "Testing input=" << input << "\n";
+   ASSERT_TRUE(flag) << "Should succeeded\n";
+}
+
+} // namespace ContainerExecutor
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-string-utils.cc
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-string-utils.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-string-utils.cc
new file mode 100644
index 0000000..037816a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-string-utils.cc
@@ -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.
+ */
+
+ #include <errno.h>
+ #include <fcntl.h>
+ #include <inttypes.h>
+ #include <signal.h>
+ #include <stdio.h>
+ #include <stdlib.h>
+ #include <string.h>
+ #include <sys/stat.h>
+ #include <sys/wait.h>
+ #include <unistd.h>
+
+ #include <gtest/gtest.h>
+ #include <sstream>
+
+ extern "C" {
+ #include "utils/string-utils.h"
+ }
+
+ namespace ContainerExecutor {
+
+ class TestStringUtils : public ::testing::Test {
+ protected:
+   virtual void SetUp() {
+
+   }
+
+   virtual void TearDown() {
+
+   }
+ };
+
+ TEST_F(TestStringUtils, test_get_numbers_split_by_comma) {
+   const char* input = ",1,2,3,-1,,1,,0,";
+   int* numbers;
+   size_t n_numbers;
+   int rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
+
+   std::cout << "Testing input=" << input << "\n";
+   ASSERT_EQ(0, rc) << "Should succeeded\n";
+   ASSERT_EQ(6, n_numbers);
+   ASSERT_EQ(1, numbers[0]);
+   ASSERT_EQ(-1, numbers[3]);
+   ASSERT_EQ(0, numbers[5]);
+
+   input = "3";
+   rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
+   std::cout << "Testing input=" << input << "\n";
+   ASSERT_EQ(0, rc) << "Should succeeded\n";
+   ASSERT_EQ(1, n_numbers);
+   ASSERT_EQ(3, numbers[0]);
+
+   input = "";
+   rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
+   std::cout << "Testing input=" << input << "\n";
+   ASSERT_EQ(0, rc) << "Should succeeded\n";
+   ASSERT_EQ(0, n_numbers);
+
+   input = ",,";
+   rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
+   std::cout << "Testing input=" << input << "\n";
+   ASSERT_EQ(0, rc) << "Should succeeded\n";
+   ASSERT_EQ(0, n_numbers);
+
+   input = "1,2,aa,bb";
+   rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
+   std::cout << "Testing input=" << input << "\n";
+   ASSERT_TRUE(0 != rc) << "Should failed\n";
+
+   input = "1,2,3,-12312312312312312312321311231231231";
+   rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
+   std::cout << "Testing input=" << input << "\n";
+   ASSERT_TRUE(0 != rc) << "Should failed\n";
+}
+
+} // namespace ContainerExecutor
\ 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


[21/50] [abbrv] hadoop git commit: YARN-5146. Support for Fair Scheduler in new YARN UI. Contributed by Abdullah Yousufi.

Posted by ae...@apache.org.
YARN-5146. Support for Fair Scheduler in new YARN UI. Contributed by Abdullah Yousufi.


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

Branch: refs/heads/HDFS-7240
Commit: dadb0c2225adef5cb0126610733c285b51f4f43e
Parents: e3ae3e2
Author: Sunil G <su...@apache.org>
Authored: Tue Aug 15 21:58:44 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Tue Aug 15 21:58:44 2017 +0530

----------------------------------------------------------------------
 .../src/main/webapp/app/adapters/yarn-queue.js  |  30 -----
 .../app/adapters/yarn-queue/capacity-queue.js   |  23 ++++
 .../app/adapters/yarn-queue/fair-queue.js       |  23 ++++
 .../app/adapters/yarn-queue/fifo-queue.js       |  23 ++++
 .../app/adapters/yarn-queue/yarn-queue.js       |  30 +++++
 .../main/webapp/app/components/tree-selector.js |  19 ++-
 .../src/main/webapp/app/models/yarn-queue.js    |  94 --------------
 .../app/models/yarn-queue/capacity-queue.js     |  95 ++++++++++++++
 .../webapp/app/models/yarn-queue/fair-queue.js  |  79 ++++++++++++
 .../webapp/app/models/yarn-queue/fifo-queue.js  |  52 ++++++++
 .../webapp/app/models/yarn-queue/yarn-queue.js  |  23 ++++
 .../main/webapp/app/routes/cluster-overview.js  |   4 +-
 .../src/main/webapp/app/routes/yarn-queue.js    |  26 ++--
 .../src/main/webapp/app/routes/yarn-queues.js   |  12 +-
 .../main/webapp/app/routes/yarn-queues/index.js |  25 ----
 .../app/routes/yarn-queues/queues-selector.js   |  25 ----
 .../main/webapp/app/serializers/yarn-queue.js   | 129 -------------------
 .../serializers/yarn-queue/capacity-queue.js    | 128 ++++++++++++++++++
 .../app/serializers/yarn-queue/fair-queue.js    |  92 +++++++++++++
 .../app/serializers/yarn-queue/fifo-queue.js    |  59 +++++++++
 .../app/serializers/yarn-queue/yarn-queue.js    |  47 +++++++
 .../components/queue-configuration-table.hbs    |  54 --------
 .../templates/components/queue-navigator.hbs    |   7 +-
 .../yarn-queue/capacity-queue-conf-table.hbs    |  54 ++++++++
 .../yarn-queue/capacity-queue-info.hbs          |  84 ++++++++++++
 .../components/yarn-queue/capacity-queue.hbs    |  63 +++++++++
 .../yarn-queue/fair-queue-conf-table.hbs        |  52 ++++++++
 .../components/yarn-queue/fair-queue-info.hbs   |  66 ++++++++++
 .../components/yarn-queue/fair-queue.hbs        |  63 +++++++++
 .../yarn-queue/fifo-queue-conf-table.hbs        |  56 ++++++++
 .../components/yarn-queue/fifo-queue-info.hbs   |  47 +++++++
 .../components/yarn-queue/fifo-queue.hbs        |  48 +++++++
 .../webapp/app/templates/yarn-queue/info.hbs    |  73 +----------
 .../main/webapp/app/templates/yarn-queues.hbs   |  54 +-------
 .../src/main/webapp/app/utils/color-utils.js    |   1 -
 35 files changed, 1266 insertions(+), 494 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue.js
deleted file mode 100644
index f2017df..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue.js
+++ /dev/null
@@ -1,30 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
-  address: "rmWebAddress",
-  restNameSpace: "cluster",
-  serverName: "RM",
-
-  pathForType(/*modelName*/) {
-    return 'scheduler'; // move to some common place, return path by modelname.
-  }
-
-});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/capacity-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/capacity-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/capacity-queue.js
new file mode 100644
index 0000000..7eb9f76
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/capacity-queue.js
@@ -0,0 +1,23 @@
+/**
+ * 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.
+ */
+
+import YarnQueueAdapter from './yarn-queue';
+
+export default YarnQueueAdapter.extend({
+
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fair-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fair-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fair-queue.js
new file mode 100644
index 0000000..7eb9f76
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fair-queue.js
@@ -0,0 +1,23 @@
+/**
+ * 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.
+ */
+
+import YarnQueueAdapter from './yarn-queue';
+
+export default YarnQueueAdapter.extend({
+
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fifo-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fifo-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fifo-queue.js
new file mode 100644
index 0000000..7eb9f76
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fifo-queue.js
@@ -0,0 +1,23 @@
+/**
+ * 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.
+ */
+
+import YarnQueueAdapter from './yarn-queue';
+
+export default YarnQueueAdapter.extend({
+
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/yarn-queue.js
new file mode 100644
index 0000000..8184c39
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/yarn-queue.js
@@ -0,0 +1,30 @@
+/**
+ * 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.
+ */
+
+import AbstractAdapter from '../abstract';
+
+export default AbstractAdapter.extend({
+  address: "rmWebAddress",
+  restNameSpace: "cluster",
+  serverName: "RM",
+
+  pathForType(/*modelName*/) {
+    return 'scheduler'; // move to some common place, return path by modelname.
+  }
+
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
index 3d72b2f..1a81a32 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
@@ -39,6 +39,9 @@ export default Ember.Component.extend({
   // mainSvg
   mainSvg: undefined,
 
+  used: undefined,
+  max: undefined,
+
   // Init data
   initData: function() {
     this.map = { };
@@ -52,7 +55,8 @@ export default Ember.Component.extend({
       }.bind(this));
 
     // var selected = this.get("selected");
-
+    this.used = this.get("used");
+    this.max = this.get("max");
     this.initQueue("root", 1, this.treeData);
   },
 
@@ -81,7 +85,6 @@ export default Ember.Component.extend({
       // Queue is not existed
       return;
     }
-
     if (depth > this.maxDepth) {
       this.maxDepth = this.maxDepth + 1;
     }
@@ -149,7 +152,9 @@ export default Ember.Component.extend({
     nodeEnter.append("circle")
       .attr("r", 1e-6)
       .style("fill", function(d) {
-        var usedCap = d.queueData.get("usedCapacity");
+        var maxCap = d.queueData.get(this.max);
+        maxCap = maxCap == undefined ? 100 : maxCap;
+        var usedCap = d.queueData.get(this.used) / maxCap * 100.0;
         if (usedCap <= 60.0) {
           return "LimeGreen";
         } else if (usedCap <= 100.0) {
@@ -157,7 +162,7 @@ export default Ember.Component.extend({
         } else {
           return "LightCoral";
         }
-      });
+      }.bind(this));
 
     // append percentage
     nodeEnter.append("text")
@@ -166,13 +171,15 @@ export default Ember.Component.extend({
       .attr("fill", "white")
       .attr("text-anchor", function() { return "middle"; })
       .text(function(d) {
-        var usedCap = d.queueData.get("usedCapacity");
+        var maxCap = d.queueData.get(this.max);
+        maxCap = maxCap == undefined ? 100 : maxCap;
+        var usedCap = d.queueData.get(this.used) / maxCap * 100.0;
         if (usedCap >= 100.0) {
           return usedCap.toFixed(0) + "%";
         } else {
           return usedCap.toFixed(1) + "%";
         }
-      })
+      }.bind(this))
       .style("fill-opacity", 1e-6);
 
     // append queue name

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue.js
deleted file mode 100644
index 27c48f7..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue.js
+++ /dev/null
@@ -1,94 +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.
- */
-
-import DS from 'ember-data';
-
-export default DS.Model.extend({
-  name: DS.attr('string'),
-  children: DS.attr('array'),
-  parent: DS.attr('string'),
-  capacity: DS.attr('number'),
-  maxCapacity: DS.attr('number'),
-  usedCapacity: DS.attr('number'),
-  absCapacity: DS.attr('number'),
-  absMaxCapacity: DS.attr('number'),
-  absUsedCapacity: DS.attr('number'),
-  state: DS.attr('string'),
-  userLimit: DS.attr('number'),
-  userLimitFactor: DS.attr('number'),
-  preemptionDisabled: DS.attr('number'),
-  numPendingApplications: DS.attr('number'),
-  numActiveApplications: DS.attr('number'),
-  users: DS.hasMany('YarnUser'),
-
-  isLeafQueue: function() {
-    var len = this.get("children.length");
-    if (!len) {
-      return true;
-    }
-    return len <= 0;
-  }.property("children"),
-
-  capacitiesBarChartData: function() {
-    return [
-      {
-        label: "Absolute Capacity",
-        value: this.get("name") === "root" ? 100 : this.get("absCapacity")
-      },
-      {
-        label: "Absolute Used",
-        value: this.get("name") === "root" ? this.get("usedCapacity") : this.get("absUsedCapacity")
-      },
-      {
-        label: "Absolute Max Capacity",
-        value: this.get("name") === "root" ? 100 : this.get("absMaxCapacity")
-      }
-    ];
-  }.property("absCapacity", "absUsedCapacity", "absMaxCapacity"),
-
-  userUsagesDonutChartData: function() {
-    var data = [];
-    if (this.get("users")) {
-      this.get("users").forEach(function(o) {
-        data.push({
-          label: o.get("name"),
-          value: o.get("usedMemoryMB")
-        });
-      });
-    }
-
-    return data;
-  }.property("users"),
-
-  hasUserUsages: function() {
-    return this.get("userUsagesDonutChartData").length > 0;
-  }.property(),
-
-  numOfApplicationsDonutChartData: function() {
-    return [
-      {
-        label: "Pending Apps",
-        value: this.get("numPendingApplications") || 0 // TODO, fix the REST API so root will return #applications as well.
-      },
-      {
-        label: "Active Apps",
-        value: this.get("numActiveApplications") || 0
-      }
-    ];
-  }.property()
-});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
new file mode 100644
index 0000000..1cb07bb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
@@ -0,0 +1,95 @@
+/**
+ * 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.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  name: DS.attr('string'),
+  children: DS.attr('array'),
+  parent: DS.attr('string'),
+  capacity: DS.attr('number'),
+  maxCapacity: DS.attr('number'),
+  usedCapacity: DS.attr('number'),
+  absCapacity: DS.attr('number'),
+  absMaxCapacity: DS.attr('number'),
+  absUsedCapacity: DS.attr('number'),
+  state: DS.attr('string'),
+  userLimit: DS.attr('number'),
+  userLimitFactor: DS.attr('number'),
+  preemptionDisabled: DS.attr('number'),
+  numPendingApplications: DS.attr('number'),
+  numActiveApplications: DS.attr('number'),
+  users: DS.hasMany('YarnUser'),
+  type: DS.attr('string'),
+
+  isLeafQueue: function() {
+    var len = this.get("children.length");
+    if (!len) {
+      return true;
+    }
+    return len <= 0;
+  }.property("children"),
+
+  capacitiesBarChartData: function() {
+    return [
+      {
+        label: "Absolute Capacity",
+        value: this.get("name") === "root" ? 100 : this.get("absCapacity")
+      },
+      {
+        label: "Absolute Used",
+        value: this.get("name") === "root" ? this.get("usedCapacity") : this.get("absUsedCapacity")
+      },
+      {
+        label: "Absolute Max Capacity",
+        value: this.get("name") === "root" ? 100 : this.get("absMaxCapacity")
+      }
+    ];
+  }.property("absCapacity", "usedCapacity", "absMaxCapacity"),
+
+  userUsagesDonutChartData: function() {
+    var data = [];
+    if (this.get("users")) {
+      this.get("users").forEach(function(o) {
+        data.push({
+          label: o.get("name"),
+          value: o.get("usedMemoryMB")
+        });
+      });
+    }
+
+    return data;
+  }.property("users"),
+
+  hasUserUsages: function() {
+    return this.get("userUsagesDonutChartData").length > 0;
+  }.property(),
+
+  numOfApplicationsDonutChartData: function() {
+    return [
+      {
+        label: "Pending Apps",
+        value: this.get("numPendingApplications") || 0 // TODO, fix the REST API so root will return #applications as well.
+      },
+      {
+        label: "Active Apps",
+        value: this.get("numActiveApplications") || 0
+      }
+    ];
+  }.property("numPendingApplications", "numActiveApplications")
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fair-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fair-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fair-queue.js
new file mode 100644
index 0000000..be71362
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fair-queue.js
@@ -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.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  name: DS.attr('string'),
+  children: DS.attr('array'),
+  parent: DS.attr('string'),
+  maxApps: DS.attr('number'),
+  minResources: DS.attr(),
+  maxResources: DS.attr(),
+  usedResources: DS.attr(),
+  demandResources: DS.attr(),
+  steadyFairResources: DS.attr(),
+  fairResources: DS.attr(),
+  clusterResources: DS.attr(),
+  pendingContainers: DS.attr('number'),
+  allocatedContainers: DS.attr('number'),
+  reservedContainers: DS.attr('number'),
+  schedulingPolicy: DS.attr('string'),
+  preemptable: DS.attr('number'),
+  numPendingApplications: DS.attr('number'),
+  numActiveApplications: DS.attr('number'),
+  type: DS.attr('string'),
+
+  isLeafQueue: function() {
+    var len = this.get("children.length");
+    if (!len) {
+      return true;
+    }
+    return len <= 0;
+  }.property("children"),
+
+  capacitiesBarChartData: function() {
+    return [
+      {
+        label: "Steady Fair Memory",
+        value: this.get("steadyFairResources.memory")
+      },
+      {
+        label: "Used Memory",
+        value: this.get("usedResources.memory")
+      },
+      {
+        label: "Maximum Memory",
+        value: this.get("maxResources.memory")
+      }
+    ];
+  }.property("maxResources.memory", "usedResources.memory", "maxResources.memory"),
+
+  numOfApplicationsDonutChartData: function() {
+    return [
+      {
+        label: "Pending Apps",
+        value: this.get("numPendingApplications") || 0 // TODO, fix the REST API so root will return #applications as well.
+      },
+      {
+        label: "Active Apps",
+        value: this.get("numActiveApplications") || 0
+      }
+    ];
+  }.property()
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fifo-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fifo-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fifo-queue.js
new file mode 100644
index 0000000..2386dc4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fifo-queue.js
@@ -0,0 +1,52 @@
+/**
+ * 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.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  name: DS.attr('string'),
+  capacity: DS.attr('number'),
+  usedCapacity: DS.attr('number'),
+  state: DS.attr('string'),
+  minQueueMemoryCapacity: DS.attr('number'),
+  maxQueueMemoryCapacity: DS.attr('number'),
+  numNodes: DS.attr('number'),
+  usedNodeCapacity: DS.attr('number'),
+  availNodeCapacity: DS.attr('number'),
+  totalNodeCapacity: DS.attr('number'),
+  numContainers: DS.attr('number'),
+  type: DS.attr('string'),
+
+  capacitiesBarChartData: function() {
+    return [
+      {
+        label: "Available Capacity",
+        value: this.get("availNodeCapacity")
+      },
+      {
+        label: "Used Capacity",
+        value: this.get("usedNodeCapacity")
+      },
+      {
+        label: "Total Capacity",
+        value: this.get("totalNodeCapacity")
+      }
+    ];
+  }.property("availNodeCapacity", "usedNodeCapacity", "totalNodeCapacity")
+
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/yarn-queue.js
new file mode 100644
index 0000000..dcf5f48
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/yarn-queue.js
@@ -0,0 +1,23 @@
+/**
+ * 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.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  type: DS.attr('string')
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/cluster-overview.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/cluster-overview.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/cluster-overview.js
index b5db17d..3c6abd4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/cluster-overview.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/cluster-overview.js
@@ -28,7 +28,7 @@ export default AbstractRoute.extend({
         {
           state: "RUNNING"
         }),
-      queues: this.store.query('yarn-queue', {}),
+      queues: this.store.query('yarn-queue.yarn-queue', {}),
     });
   },
 
@@ -39,6 +39,6 @@ export default AbstractRoute.extend({
   unloadAll() {
     this.store.unloadAll('ClusterMetric');
     this.store.unloadAll('yarn-app');
-    this.store.unloadAll('yarn-queue');
+    this.store.unloadAll('yarn-queue.yarn-queue');
   }
 });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queue.js
index 1c4546c..cd4ed09 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queue.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queue.js
@@ -22,22 +22,28 @@ import AbstractRoute from './abstract';
 
 export default AbstractRoute.extend({
   model(param) {
-    return Ember.RSVP.hash({
-      selected : param.queue_name,
-      queues: this.store.query('yarn-queue', {}),
-      selectedQueue : undefined,
-      apps: this.store.query('yarn-app', {
-        queue: param.queue_name
-      })
-    });
+      return Ember.RSVP.hash({
+        selected : param.queue_name,
+        queues: this.store.query("yarn-queue.yarn-queue", {}).then((model) => {
+          let type = model.get('firstObject').get('type');
+          return this.store.query("yarn-queue." + type + "-queue", {});
+        }),
+        selectedQueue : undefined,
+        apps: this.store.query('yarn-app', {
+          queue: param.queue_name
+        })
+      });
   },
 
   afterModel(model) {
-    model.selectedQueue = this.store.peekRecord('yarn-queue', model.selected);
+    var type = model.queues.get('firstObject').constructor.modelName;
+    model.selectedQueue = this.store.peekRecord(type, model.selected);
   },
 
   unloadAll() {
-    this.store.unloadAll('yarn-queue');
+    this.store.unloadAll('yarn-queue.capacity-queue');
+    this.store.unloadAll('yarn-queue.fair-queue');
+    this.store.unloadAll('yarn-queue.fifo-queue');
     this.store.unloadAll('yarn-app');
   }
 });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues.js
index e4f145d..7d8a200 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues.js
@@ -30,17 +30,23 @@ export default AbstractRoute.extend({
     }
     return Ember.RSVP.hash({
       selected : queueName,
-      queues: this.store.query('yarn-queue', {}),
+      queues: this.store.query("yarn-queue.yarn-queue", {}).then((model) => {
+        let type = model.get('firstObject').get('type');
+        return this.store.query("yarn-queue." + type + "-queue", {});
+      }),
       selectedQueue : undefined
     });
   },
 
   afterModel(model) {
-    model.selectedQueue = this.store.peekRecord('yarn-queue', model.selected);
+    var type = model.queues.get('firstObject').constructor.modelName;
+    model.selectedQueue = this.store.peekRecord(type, model.selected);
   },
 
   unloadAll() {
-    this.store.unloadAll('yarn-queue');
+    this.store.unloadAll('yarn-queue.capacity-queue');
+    this.store.unloadAll('yarn-queue.fair-queue');
+    this.store.unloadAll('yarn-queue.fifo-queue');
     this.store.unloadAll('yarn-app');
   },
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/index.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/index.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/index.js
deleted file mode 100644
index 436c6d8..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/index.js
+++ /dev/null
@@ -1,25 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Route.extend({
-  beforeModel() {
-    this.transitionTo('yarn-queues.root');
-  }
-});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/queues-selector.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/queues-selector.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/queues-selector.js
deleted file mode 100644
index 5d14c6f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/queues-selector.js
+++ /dev/null
@@ -1,25 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Route.extend({
-  model() {
-    return this.store.findAll('yarn-queue');
-  },
-});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue.js
deleted file mode 100644
index 4fc1a29..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue.js
+++ /dev/null
@@ -1,129 +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.
- */
-
-import DS from 'ember-data';
-
-export default DS.JSONAPISerializer.extend({
-
-    normalizeSingleResponse(store, primaryModelClass, payload, id,
-      requestType) {
-      var children = [];
-      if (payload.queues) {
-        payload.queues.queue.forEach(function(queue) {
-          children.push(queue.queueName);
-        });
-      }
-
-      var includedData = [];
-      var relationshipUserData = [];
-
-      // update user models
-      if (payload.users && payload.users.user) {
-        payload.users.user.forEach(function(u) {
-          includedData.push({
-            type: "YarnUser",
-            id: u.username + "_" + payload.queueName,
-            attributes: {
-              name: u.username,
-              queueName: payload.queueName,
-              usedMemoryMB: u.resourcesUsed.memory || 0,
-              usedVCore: u.resourcesUsed.vCores || 0,
-            }
-          });
-
-          relationshipUserData.push({
-            type: "YarnUser",
-            id: u.username + "_" + payload.queueName,
-          });
-        });
-      }
-
-
-      var fixedPayload = {
-        id: id,
-        type: primaryModelClass.modelName, // yarn-queue
-        attributes: {
-          name: payload.queueName,
-          parent: payload.myParent,
-          children: children,
-          capacity: payload.capacity,
-          usedCapacity: payload.usedCapacity,
-          maxCapacity: payload.maxCapacity,
-          absCapacity: payload.absoluteCapacity,
-          absMaxCapacity: payload.absoluteMaxCapacity,
-          absUsedCapacity: payload.absoluteUsedCapacity,
-          state: payload.state,
-          userLimit: payload.userLimit,
-          userLimitFactor: payload.userLimitFactor,
-          preemptionDisabled: payload.preemptionDisabled,
-          numPendingApplications: payload.numPendingApplications,
-          numActiveApplications: payload.numActiveApplications,
-        },
-        // Relationships
-        relationships: {
-          users: {
-            data: relationshipUserData
-          }
-        }
-      };
-
-      return {
-        queue: this._super(store, primaryModelClass, fixedPayload, id, requestType),
-        includedData: includedData
-      };
-    },
-
-    handleQueue(store, primaryModelClass, payload, id, requestType) {
-      var data = [];
-      var includedData = [];
-      var result = this.normalizeSingleResponse(store, primaryModelClass,
-        payload, id, requestType);
-
-      data.push(result.queue);
-      includedData = includedData.concat(result.includedData);
-
-      if (payload.queues) {
-        for (var i = 0; i < payload.queues.queue.length; i++) {
-          var queue = payload.queues.queue[i];
-          queue.myParent = payload.queueName;
-          var childResult = this.handleQueue(store, primaryModelClass, queue,
-            queue.queueName,
-            requestType);
-
-          data = data.concat(childResult.data);
-          includedData = includedData.concat(childResult.includedData);
-        }
-      }
-
-      return {
-        data: data,
-        includedData: includedData
-      };
-    },
-
-    normalizeArrayResponse(store, primaryModelClass, payload, id, requestType) {
-      var normalizedArrayResponse = {};
-      var result = this.handleQueue(store, primaryModelClass,
-        payload.scheduler.schedulerInfo, "root", requestType);
-
-      normalizedArrayResponse.data = result.data;
-      normalizedArrayResponse.included = result.includedData;
-
-      return normalizedArrayResponse;
-    }
-});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js
new file mode 100644
index 0000000..c7350ef
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js
@@ -0,0 +1,128 @@
+/**
+ * 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.
+ */
+
+import DS from 'ember-data';
+
+export default DS.JSONAPISerializer.extend({
+
+    normalizeSingleResponse(store, primaryModelClass, payload, id,
+      requestType) {
+      var children = [];
+      if (payload.queues) {
+        payload.queues.queue.forEach(function(queue) {
+          children.push(queue.queueName);
+        });
+      }
+
+      var includedData = [];
+      var relationshipUserData = [];
+
+      // update user models
+      if (payload.users && payload.users.user) {
+        payload.users.user.forEach(function(u) {
+          includedData.push({
+            type: "YarnUser",
+            id: u.username + "_" + payload.queueName,
+            attributes: {
+              name: u.username,
+              queueName: payload.queueName,
+              usedMemoryMB: u.resourcesUsed.memory || 0,
+              usedVCore: u.resourcesUsed.vCores || 0,
+            }
+          });
+
+          relationshipUserData.push({
+            type: "YarnUser",
+            id: u.username + "_" + payload.queueName,
+          });
+        });
+      }
+
+      var fixedPayload = {
+        id: id,
+        type: primaryModelClass.modelName, // yarn-queue
+        attributes: {
+          name: payload.queueName,
+          parent: payload.myParent,
+          children: children,
+          capacity: payload.capacity,
+          usedCapacity: payload.usedCapacity,
+          maxCapacity: payload.maxCapacity,
+          absCapacity: payload.absoluteCapacity,
+          absMaxCapacity: payload.absoluteMaxCapacity,
+          absUsedCapacity: payload.absoluteUsedCapacity,
+          state: payload.state,
+          userLimit: payload.userLimit,
+          userLimitFactor: payload.userLimitFactor,
+          preemptionDisabled: payload.preemptionDisabled,
+          numPendingApplications: payload.numPendingApplications,
+          numActiveApplications: payload.numActiveApplications,
+          type: "capacity",
+        },
+        // Relationships
+        relationships: {
+          users: {
+            data: relationshipUserData
+          }
+        }
+      };
+      return {
+        queue: this._super(store, primaryModelClass, fixedPayload, id, requestType),
+        includedData: includedData
+      };
+    },
+
+    handleQueue(store, primaryModelClass, payload, id, requestType) {
+      var data = [];
+      var includedData = [];
+      var result = this.normalizeSingleResponse(store, primaryModelClass,
+        payload, id, requestType);
+
+      data.push(result.queue);
+      includedData = includedData.concat(result.includedData);
+
+      if (payload.queues) {
+        for (var i = 0; i < payload.queues.queue.length; i++) {
+          var queue = payload.queues.queue[i];
+          queue.myParent = payload.queueName;
+          var childResult = this.handleQueue(store, primaryModelClass, queue,
+            queue.queueName,
+            requestType);
+
+          data = data.concat(childResult.data);
+          includedData = includedData.concat(childResult.includedData);
+        }
+      }
+
+      return {
+        data: data,
+        includedData: includedData
+      };
+    },
+
+    normalizeArrayResponse(store, primaryModelClass, payload, id, requestType) {
+      var normalizedArrayResponse = {};
+      var result = this.handleQueue(store, primaryModelClass,
+        payload.scheduler.schedulerInfo, "root", requestType);
+
+      normalizedArrayResponse.data = result.data;
+      normalizedArrayResponse.included = result.includedData;
+
+      return normalizedArrayResponse;
+    }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fair-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fair-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fair-queue.js
new file mode 100644
index 0000000..2215d2d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fair-queue.js
@@ -0,0 +1,92 @@
+/**
+ * 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.
+ */
+
+import DS from 'ember-data';
+
+export default DS.JSONAPISerializer.extend({
+
+    normalizeSingleResponse(store, primaryModelClass, payload, id,
+      requestType) {
+      var children = [];
+      if (payload.childQueues) {
+        payload.childQueues.queue.forEach(function(queue) {
+          children.push(queue.queueName);
+        });
+      }
+
+      var fixedPayload = {
+        id: id,
+        type: primaryModelClass.modelName,
+        attributes: {
+          name: payload.queueName,
+          parent: payload.myParent,
+          children: children,
+          maxApps: payload.maxApps,
+          minResources: payload.minResources,
+          maxResources: payload.maxResources,
+          usedResources: payload.usedResources,
+          demandResources: payload.demandResources,
+          steadyFairResources: payload.steadyFairResources,
+          fairResources: payload.fairResources,
+          clusterResources: payload.clusterResources,
+          pendingContainers: payload.pendingContainers,
+          allocatedContainers: payload.allocatedContainers,
+          reservedContainers: payload.reservedContainers,
+          schedulingPolicy: payload.schedulingPolicy,
+          preemptable: payload.preemptable,
+          numPendingApplications: payload.numPendingApps,
+          numActiveApplications: payload.numActiveApps,
+          type: "fair",
+        },
+      };
+      return this._super(store, primaryModelClass, fixedPayload, id, requestType);
+    },
+
+    handleQueue(store, primaryModelClass, payload, id, requestType) {
+      var data = [];
+      var includedData = [];
+      if(!payload) return data;
+      var result = this.normalizeSingleResponse(store, primaryModelClass,
+        payload, id, requestType);
+
+      data.push(result);
+
+      if (payload.childQueues) {
+        for (var i = 0; i < payload.childQueues.queue.length; i++) {
+          var queue = payload.childQueues.queue[i];
+          queue.myParent = payload.queueName;
+          var childResult = this.handleQueue(store, primaryModelClass, queue,
+            queue.queueName,
+            requestType);
+
+          data = data.concat(childResult);
+        }
+      }
+
+      return data;
+    },
+
+    normalizeArrayResponse(store, primaryModelClass, payload, id, requestType) {
+      var normalizedArrayResponse = {};
+      var result = this.handleQueue(store, primaryModelClass,
+        payload.scheduler.schedulerInfo.rootQueue, "root", requestType);
+
+      normalizedArrayResponse.data = result;
+      return normalizedArrayResponse;
+    }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fifo-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fifo-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fifo-queue.js
new file mode 100644
index 0000000..297ec18
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fifo-queue.js
@@ -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.
+ */
+
+import DS from 'ember-data';
+
+export default DS.JSONAPISerializer.extend({
+
+    normalizeSingleResponse(store, primaryModelClass, payload, id,
+      requestType) {
+
+      var fixedPayload = {
+        id: id,
+        type: primaryModelClass.modelName,
+        attributes: {
+          name: id,
+          capacity: payload.capacity * 100,
+          usedCapacity: payload.usedCapacity * 100,
+          usedNodeCapacity: payload.usedNodeCapacity,
+          availNodeCapacity: payload.availNodeCapacity,
+          totalNodeCapacity: payload.totalNodeCapacity,
+          numNodes: payload.numNodes,
+          numContainers: payload.numContainers,
+          state: payload.qstate,
+          minQueueMemoryCapacity: payload.minQueueMemoryCapacity,
+          maxQueueMemoryCapacity: payload.maxQueueMemoryCapacity,
+          type: "fifo",
+        },
+
+      };
+
+      return this._super(store, primaryModelClass, fixedPayload, id,
+        requestType);
+    },
+
+    normalizeArrayResponse(store, primaryModelClass, payload, id, requestType) {
+      var normalizedArrayResponse = {};
+      normalizedArrayResponse.data = [
+        this.normalizeSingleResponse(store, primaryModelClass,
+          payload.scheduler.schedulerInfo, "root", requestType)
+      ];
+
+      return normalizedArrayResponse;
+    }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/yarn-queue.js
new file mode 100644
index 0000000..b2e0f2f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/yarn-queue.js
@@ -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.
+ */
+
+import DS from 'ember-data';
+
+export default DS.JSONAPISerializer.extend({
+
+    normalizeSingleResponse(store, primaryModelClass, payload, id,
+      requestType) {
+
+      var fixedPayload = {
+        id: id,
+        type: primaryModelClass.modelName,
+        attributes: {
+          type: payload.type.split(/(?=[A-Z])/)[0]
+        }
+      };
+      return this._super(store, primaryModelClass, fixedPayload, id,
+        requestType);
+    },
+
+    normalizeArrayResponse(store, primaryModelClass, payload, id, requestType) {
+      var normalizedArrayResponse = {};
+
+      normalizedArrayResponse.data = [
+        this.normalizeSingleResponse(store, primaryModelClass,
+          payload.scheduler.schedulerInfo, "root", requestType)
+        ];
+
+      return normalizedArrayResponse;
+    }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-configuration-table.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-configuration-table.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-configuration-table.hbs
deleted file mode 100644
index 17a1e1a..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-configuration-table.hbs
+++ /dev/null
@@ -1,54 +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.
-}}
-
-<table id="queue-configuration-table" class="table table-striped table-bordered" cellspacing="0" width="100%" height="100%">
-  <thead>
-    <tr>
-      <td><b>Configurations</b></td>
-      <td>Value</td>
-    </tr>
-  </thead>
-  <tbody>
-    <tr>
-      <td>Configured Capacity</td>
-      <td>{{queue.capacity}}</td>
-    </tr>
-    <tr>
-      <td>Configured Max Capacity</td>
-      <td>{{queue.maxCapacity}}</td>
-    </tr>
-    <tr>
-      <td>State</td>
-      <td>{{queue.state}}</td>
-    </tr>
-  {{#if queue.isLeafQueue}}
-    <tr>
-      <td>User Limit Percent</td>
-      <td>{{queue.userLimit}}</td>
-    </tr>
-    <tr>
-      <td>User Limit Factor</td>
-      <td>{{queue.userLimitFactor}}</td>
-    </tr>
-    <tr>
-      <td>Preemption Disabled</td>
-      <td>{{queue.preemptionDisabled}}</td>
-    </tr>
-  {{/if}}
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs
index d8dd236..e3b0a90 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs
@@ -20,9 +20,12 @@
 <div class="row">
   <div class="col-md-12 container-fluid">
     <div class="panel panel-default" id="tree-selector-container">
-     {{tree-selector model=model parentId="tree-selector-container" selected=selected}}
+      <div class="panel-heading">
+        Scheduler: {{model.firstObject.type}}
+      </div>
+     {{tree-selector model=model parentId="tree-selector-container" selected=selected used=used max=max}}
     </div>
   </div>
 </div>
 
-{{outlet}}
\ No newline at end of file
+{{outlet}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-conf-table.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-conf-table.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-conf-table.hbs
new file mode 100644
index 0000000..3f6017f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-conf-table.hbs
@@ -0,0 +1,54 @@
+{{!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+}}
+
+<table id="queue-configuration-table" class="table table-striped table-bordered" cellspacing="0" width="100%" height="100%">
+  <thead>
+    <tr>
+      <td><b>Configurations</b></td>
+      <td>Value</td>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>Configured Capacity</td>
+      <td>{{queue.capacity}}</td>
+    </tr>
+    <tr>
+      <td>Configured Max Capacity</td>
+      <td>{{queue.maxCapacity}}</td>
+    </tr>
+    <tr>
+      <td>State</td>
+      <td>{{queue.state}}</td>
+    </tr>
+  {{#if queue.isLeafQueue}}
+    <tr>
+      <td>User Limit Percent</td>
+      <td>{{queue.userLimit}}</td>
+    </tr>
+    <tr>
+      <td>User Limit Factor</td>
+      <td>{{queue.userLimitFactor}}</td>
+    </tr>
+    <tr>
+      <td>Preemption Disabled</td>
+      <td>{{queue.preemptionDisabled}}</td>
+    </tr>
+  {{/if}}
+  </tbody>
+</table>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-info.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-info.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-info.hbs
new file mode 100644
index 0000000..7d44e69
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-info.hbs
@@ -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.
+}}
+
+<div class="row">
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Capacities: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="capacity-bar-chart">
+        <br/>
+        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+        title=""
+        parentId="capacity-bar-chart"
+        textWidth=170
+        ratio=0.55
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Information: {{model.selected}}
+      </div>
+        {{yarn-queue.capacity-queue-conf-table queue=model.selectedQueue}}
+    </div>
+  </div>
+
+</div>
+
+<div class="row">
+
+    <div class="col-lg-6 container-fluid">
+      <div class="panel panel-default">
+        <div class="panel-heading">
+          Running Apps: {{model.selected}}
+        </div>
+        <div class="container-fluid" id="numapplications-donut-chart">
+          {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
+          showLabels=true
+          parentId="numapplications-donut-chart"
+          ratio=0.6
+          maxHeight=350}}
+        </div>
+      </div>
+    </div>
+
+  {{#if model.selectedQueue.hasUserUsages}}
+    <div class="col-lg-6 container-fluid">
+      <div class="panel panel-default">
+        <div class="panel-heading">
+          User Usages: {{model.selected}}
+        </div>
+        <div class="container-fluid"  id="userusage-donut-chart">
+          {{donut-chart data=model.selectedQueue.userUsagesDonutChartData
+          showLabels=true
+          parentId="userusage-donut-chart"
+          type="memory"
+          ratio=0.6
+          maxHeight=350}}
+        </div>
+      </div>
+    </div>
+  {{/if}}
+
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
new file mode 100644
index 0000000..8b63b66
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
@@ -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.
+}}
+
+{{queue-navigator model=model.queues selected=model.selected
+  used="usedCapacity" max="absMaxCapacity"}}
+
+<div class="row">
+  <div class="col-lg-4 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Information: {{model.selected}}
+      </div>
+        {{yarn-queue.capacity-queue-conf-table queue=model.selectedQueue}}
+    </div>
+  </div>
+
+  <div class="col-lg-4 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Capacities: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="capacity-bar-chart">
+        <br/>
+        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+        title=""
+        parentId="capacity-bar-chart"
+        textWidth=175
+        ratio=0.55
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+
+  <div class="col-lg-4 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Running Apps: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="numapplications-donut-chart">
+        {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
+        showLabels=true
+        parentId="numapplications-donut-chart"
+        ratio=0.6
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-conf-table.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-conf-table.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-conf-table.hbs
new file mode 100644
index 0000000..00fabcc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-conf-table.hbs
@@ -0,0 +1,52 @@
+{{!
+ * 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.
+}}
+
+<table id="queue-configuration-table" class="table table-striped table-bordered" cellspacing="0" width="100%" height="100%">
+  <thead>
+    <tr>
+      <td><b>Configurations</b></td>
+      <td>Value</td>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>Fair Memory, VCores</td>
+      <td>{{queue.fairResources.memory}} MB, {{queue.fairResources.vCores}}</td>
+    </tr>
+    <tr>
+      <td>Minimum Memory, VCores</td>
+      <td>{{queue.minResources.memory}} MB, {{queue.minResources.vCores}}</td>
+    </tr>
+    <tr>
+      <td>Cluster Memory, VCores</td>
+      <td>{{queue.clusterResources.memory}} MB, {{queue.clusterResources.vCores}}</td>
+    </tr>
+    <tr>
+      <td>Pending, Allocated, Reserved Containers</td>
+      <td>{{queue.pendingContainers}} , {{queue.allocatedContainers}} , {{queue.reservedContainers}}</td>
+    </tr>
+    <tr>
+      <td>Scheduling Policy</td>
+      <td>{{queue.schedulingPolicy}}</td>
+    </tr>
+    <tr>
+      <td>Preemption Enabled</td>
+      <td>{{queue.preemptable}}</td>
+    </tr>
+  </tbody>
+</table>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-info.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-info.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-info.hbs
new file mode 100644
index 0000000..a770bfe
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-info.hbs
@@ -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.
+}}
+
+<div class="row">
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Capacities: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="capacity-bar-chart">
+        <br/>
+        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+        title=""
+        parentId="capacity-bar-chart"
+        textWidth=170
+        ratio=0.55
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Information: {{model.selected}}
+      </div>
+        {{yarn-queue.fair-queue-conf-table queue=model.selectedQueue}}
+    </div>
+  </div>
+
+</div>
+
+<div class="row">
+
+    <div class="col-lg-6 container-fluid">
+      <div class="panel panel-default">
+        <div class="panel-heading">
+          Running Apps: {{model.selected}}
+        </div>
+        <div class="container-fluid" id="numapplications-donut-chart">
+          {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
+          showLabels=true
+          parentId="numapplications-donut-chart"
+          ratio=0.6
+          maxHeight=350}}
+        </div>
+      </div>
+    </div>
+
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue.hbs
new file mode 100644
index 0000000..0341108
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue.hbs
@@ -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.
+}}
+
+{{queue-navigator model=model.queues selected=model.selected
+  used="usedResources.memory" max="clusterResources.memory"}}
+
+<div class="row">
+  <div class="col-lg-4 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Information: {{model.selected}}
+      </div>
+        {{yarn-queue.fair-queue-conf-table queue=model.selectedQueue}}
+    </div>
+  </div>
+
+  <div class="col-lg-4 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Capacities: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="capacity-bar-chart">
+        <br/>
+        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+        title=""
+        parentId="capacity-bar-chart"
+        textWidth=150
+        ratio=0.55
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+
+  <div class="col-lg-4 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Running Apps: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="numapplications-donut-chart">
+        {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
+        showLabels=true
+        parentId="numapplications-donut-chart"
+        ratio=0.6
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-conf-table.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-conf-table.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-conf-table.hbs
new file mode 100644
index 0000000..4ced3e7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-conf-table.hbs
@@ -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.
+}}
+
+<table id="queue-configuration-table" class="table table-striped table-bordered" cellspacing="0" width="100%" height="100%">
+  <thead>
+    <tr>
+      <td><b>Configurations</b></td>
+      <td>Value</td>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>Configured Capacity</td>
+      <td>{{queue.capacity}}</td>
+    </tr>
+    <tr>
+      <td>Used Capacity</td>
+      <td>{{queue.usedCapacity}}</td>
+    </tr>
+    <tr>
+      <td>State</td>
+      <td>{{queue.state}}</td>
+    </tr>
+    <tr>
+      <td>Minimum Queue Memory Capacity</td>
+      <td>{{queue.minQueueMemoryCapacity}}</td>
+    </tr>
+    <tr>
+      <td>Maximum Queue Memory Capacity</td>
+      <td>{{queue.maxQueueMemoryCapacity}}</td>
+    </tr>
+    <tr>
+      <td>Number of Nodes</td>
+      <td>{{queue.numNodes}}</td>
+    </tr>
+    <tr>
+      <td>Number of Containers</td>
+      <td>{{queue.numContainers}}</td>
+    </tr>
+  </tbody>
+</table>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-info.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-info.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-info.hbs
new file mode 100644
index 0000000..7f4e8a7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-info.hbs
@@ -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.
+}}
+
+<div class="row">
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Capacities: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="capacity-bar-chart">
+        <br/>
+        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+        title=""
+        parentId="capacity-bar-chart"
+        textWidth=170
+        ratio=0.55
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Information: {{model.selected}}
+      </div>
+        {{yarn-queue.fifo-queue-conf-table queue=model.selectedQueue}}
+    </div>
+  </div>
+
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue.hbs
new file mode 100644
index 0000000..46d79f0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue.hbs
@@ -0,0 +1,48 @@
+{{!
+ * 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.
+}}
+
+{{queue-navigator model=model.queues selected=model.selected
+  used="usedNodeCapacity" max="totalNodeCapacity"}}
+
+<div class="row">
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Information: {{model.selected}}
+      </div>
+        {{yarn-queue.fifo-queue-conf-table queue=model.selectedQueue}}
+    </div>
+  </div>
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Capacities: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="capacity-bar-chart">
+        <br/>
+        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+        title=""
+        parentId="capacity-bar-chart"
+        textWidth=150
+        ratio=0.55
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs
index c112ef9..2f138a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs
@@ -16,69 +16,10 @@
  * limitations under the License.
 }}
 
-<div class="row">
-
-  <div class="col-lg-6 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Queue Capacities: {{model.selected}}
-      </div>
-      <div class="container-fluid" id="capacity-bar-chart">
-        <br/>
-        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
-        title=""
-        parentId="capacity-bar-chart"
-        textWidth=170
-        ratio=0.55
-        maxHeight=350}}
-      </div>
-    </div>
-  </div>
-
-  <div class="col-lg-6 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Queue Information: {{model.selected}}
-      </div>
-      {{queue-configuration-table queue=model.selectedQueue}}
-    </div>
-  </div>
-
-</div>
-
-<div class="row">
-
-  <div class="col-lg-6 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Running Apps: {{model.selected}}
-      </div>
-      <div class="container-fluid" id="numapplications-donut-chart">
-        {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
-        showLabels=true
-        parentId="numapplications-donut-chart"
-        ratio=0.6
-        maxHeight=350}}
-      </div>
-    </div>
-  </div>
-
-  {{#if model.selectedQueue.hasUserUsages}}
-    <div class="col-lg-6 container-fluid">
-      <div class="panel panel-default">
-        <div class="panel-heading">
-          User Usages: {{model.selected}}
-        </div>
-        <div class="container-fluid"  id="userusage-donut-chart">
-          {{donut-chart data=model.selectedQueue.userUsagesDonutChartData
-          showLabels=true
-          parentId="userusage-donut-chart"
-          type="memory"
-          ratio=0.6
-          maxHeight=350}}
-        </div>
-      </div>
-    </div>
-  {{/if}}
-
-</div>
+{{#if (eq model.queues.firstObject.type "capacity")}}
+  {{yarn-queue.capacity-queue-info model=model}}
+{{else if (eq model.queues.firstObject.type "fair")}}
+  {{yarn-queue.fair-queue-info model=model}}
+{{else}}
+  {{yarn-queue.fifo-queue-info model=model}}
+{{/if}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
index 6dfb220..fccdb5b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
@@ -17,54 +17,14 @@
 }}
 
 {{breadcrumb-bar breadcrumbs=breadcrumbs}}
-
 <div class="container-fluid">
-  {{queue-navigator model=model.queues selected=model.selected}}
-
-  <div class="row">
-
-    <div class="col-lg-4 container-fluid">
-      <div class="panel panel-default">
-        <div class="panel-heading">
-          Queue Information: {{model.selected}}
-        </div>
-        {{queue-configuration-table queue=model.selectedQueue}}
-      </div>
-    </div>
-
-    <div class="col-lg-4 container-fluid">
-      <div class="panel panel-default">
-        <div class="panel-heading">
-          Queue Capacities: {{model.selected}}
-        </div>
-        <div class="container-fluid" id="capacity-bar-chart">
-          <br/>
-          {{bar-chart data=model.selectedQueue.capacitiesBarChartData
-          title=""
-          parentId="capacity-bar-chart"
-          textWidth=150
-          ratio=0.55
-          maxHeight=350}}
-        </div>
-      </div>
-    </div>
-
-    <div class="col-lg-4 container-fluid">
-      <div class="panel panel-default">
-        <div class="panel-heading">
-          Running Apps: {{model.selected}}
-        </div>
-        <div class="container-fluid" id="numapplications-donut-chart">
-          {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
-          showLabels=true
-          parentId="numapplications-donut-chart"
-          ratio=0.6
-          maxHeight=350}}
-        </div>
-      </div>
-    </div>
-
-  </div>
+  {{#if (eq model.queues.firstObject.type "capacity")}}
+    {{yarn-queue.capacity-queue model=model}}
+  {{else if (eq model.queues.firstObject.type "fair")}}
+    {{yarn-queue.fair-queue model=model}}
+  {{else}}
+    {{yarn-queue.fifo-queue model=model}}
+  {{/if}}
 </div>
 
 {{outlet}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/color-utils.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/color-utils.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/color-utils.js
index 6c0cfee..af0cdf4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/color-utils.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/color-utils.js
@@ -55,7 +55,6 @@ export default {
       }
     }
 
-    console.log(colors);
     return colors;
   },
 


---------------------------------------------------------------------
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: HADOOP-14627. Support MSI and DeviceCode token provider in ADLS. Contributed by Atul Sikaria.

Posted by ae...@apache.org.
HADOOP-14627. Support MSI and DeviceCode token provider in ADLS. Contributed by Atul Sikaria.


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

Branch: refs/heads/HDFS-7240
Commit: 7769e9614956283a86eda9e4e69aaa592c0ca960
Parents: 8b242f0
Author: John Zhuge <jz...@cloudera.com>
Authored: Thu Aug 10 00:43:40 2017 -0700
Committer: John Zhuge <jz...@apache.org>
Committed: Sun Aug 13 00:22:34 2017 -0700

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         | 37 +++++++-
 hadoop-tools/hadoop-azure-datalake/pom.xml      |  2 +-
 .../org/apache/hadoop/fs/adl/AdlConfKeys.java   |  8 ++
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java | 21 +++++
 .../apache/hadoop/fs/adl/TokenProviderType.java |  2 +
 .../src/site/markdown/index.md                  | 98 ++++++++++++++++++--
 .../hadoop/fs/adl/TestAzureADTokenProvider.java | 40 ++++++++
 7 files changed, 198 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/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 ffcab2c..7c4b0f1 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
@@ -2586,11 +2586,16 @@
     <value>ClientCredential</value>
     <description>
       Defines Azure Active Directory OAuth2 access token provider type.
-      Supported types are ClientCredential, RefreshToken, and Custom.
+      Supported types are ClientCredential, RefreshToken, MSI, DeviceCode,
+      and Custom.
       The ClientCredential type requires property fs.adl.oauth2.client.id,
       fs.adl.oauth2.credential, and fs.adl.oauth2.refresh.url.
       The RefreshToken type requires property fs.adl.oauth2.client.id and
       fs.adl.oauth2.refresh.token.
+      The MSI type requires properties fs.adl.oauth2.msi.port and
+      fs.adl.oauth2.msi.tenantguid.
+      The DeviceCode type requires property
+      fs.adl.oauth2.devicecode.clientapp.id.
       The Custom type requires property fs.adl.oauth2.access.token.provider.
     </description>
   </property>
@@ -2627,6 +2632,36 @@
     </description>
   </property>
 
+  <property>
+    <name>fs.adl.oauth2.msi.port</name>
+    <value></value>
+    <description>
+      The localhost port for the MSI token service. This is the port specified
+      when creating the Azure VM.
+      Used by MSI token provider.
+    </description>
+  </property>
+
+  <property>
+    <name>fs.adl.oauth2.msi.tenantguid</name>
+    <value></value>
+    <description>
+      The tenant guid for the Azure AAD tenant under which the azure data lake
+      store account is created.
+      Used by MSI token provider.
+    </description>
+  </property>
+
+  <property>
+    <name>fs.adl.oauth2.devicecode.clientapp.id</name>
+    <value></value>
+    <description>
+      The app id of the AAD native app in whose context the auth request
+      should be made.
+      Used by DeviceCode token provider.
+    </description>
+  </property>
+
   <!-- Azure Data Lake File System Configurations Ends Here-->
 
   <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/hadoop-tools/hadoop-azure-datalake/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/pom.xml b/hadoop-tools/hadoop-azure-datalake/pom.xml
index 3aed5e1..47f12df 100644
--- a/hadoop-tools/hadoop-azure-datalake/pom.xml
+++ b/hadoop-tools/hadoop-azure-datalake/pom.xml
@@ -110,7 +110,7 @@
     <dependency>
       <groupId>com.microsoft.azure</groupId>
       <artifactId>azure-data-lake-store-sdk</artifactId>
-      <version>2.1.4</version>
+      <version>2.2.1</version>
     </dependency>
     <!--  ENDS HERE-->
     <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
index 31df222..f77d981 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
@@ -54,6 +54,14 @@ public final class AdlConfKeys {
   public static final String TOKEN_PROVIDER_TYPE_CLIENT_CRED =
       "ClientCredential";
 
+  // MSI Auth Configuration
+  public static final String MSI_PORT = "fs.adl.oauth2.msi.port";
+  public static final String MSI_TENANT_GUID = "fs.adl.oauth2.msi.tenantguid";
+
+  // DeviceCode Auth configuration
+  public static final String DEVICE_CODE_CLIENT_APP_ID =
+      "fs.adl.oauth2.devicecode.clientapp.id";
+
   public static final String READ_AHEAD_BUFFER_SIZE_KEY =
       "adl.feature.client.cache.readahead";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
index 76ce43e..a5e31e1 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
@@ -34,6 +34,8 @@ import com.microsoft.azure.datalake.store.LatencyTracker;
 import com.microsoft.azure.datalake.store.UserGroupRepresentation;
 import com.microsoft.azure.datalake.store.oauth2.AccessTokenProvider;
 import com.microsoft.azure.datalake.store.oauth2.ClientCredsTokenProvider;
+import com.microsoft.azure.datalake.store.oauth2.DeviceCodeTokenProvider;
+import com.microsoft.azure.datalake.store.oauth2.MsiTokenProvider;
 import com.microsoft.azure.datalake.store.oauth2.RefreshTokenBasedTokenProvider;
 
 import org.apache.commons.lang.StringUtils;
@@ -254,6 +256,12 @@ public class AdlFileSystem extends FileSystem {
     case ClientCredential:
       tokenProvider = getConfCredentialBasedTokenProvider(conf);
       break;
+    case MSI:
+      tokenProvider = getMsiBasedTokenProvider(conf);
+      break;
+    case DeviceCode:
+      tokenProvider = getDeviceCodeTokenProvider(conf);
+      break;
     case Custom:
     default:
       AzureADTokenProvider azureADTokenProvider = getCustomAccessTokenProvider(
@@ -280,6 +288,19 @@ public class AdlFileSystem extends FileSystem {
     return new RefreshTokenBasedTokenProvider(clientId, refreshToken);
   }
 
+  private AccessTokenProvider getMsiBasedTokenProvider(
+          Configuration conf) throws IOException {
+    int port = Integer.parseInt(getNonEmptyVal(conf, MSI_PORT));
+    String tenantGuid = getPasswordString(conf, MSI_TENANT_GUID);
+    return new MsiTokenProvider(port, tenantGuid);
+  }
+
+  private AccessTokenProvider getDeviceCodeTokenProvider(
+          Configuration conf) throws IOException {
+    String clientAppId = getNonEmptyVal(conf, DEVICE_CODE_CLIENT_APP_ID);
+    return new DeviceCodeTokenProvider(clientAppId);
+  }
+
   @VisibleForTesting
   AccessTokenProvider getTokenProvider() {
     return tokenProvider;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/TokenProviderType.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/TokenProviderType.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/TokenProviderType.java
index 9fd4f4f..1c11d84 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/TokenProviderType.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/TokenProviderType.java
@@ -21,5 +21,7 @@ package org.apache.hadoop.fs.adl;
 enum TokenProviderType {
   RefreshToken,
   ClientCredential,
+  MSI,
+  DeviceCode,
   Custom
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
index d4b7d8e..e34da36 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
+++ b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
@@ -111,20 +111,24 @@ service associated with the client id. See [*Active Directory Library For Java*]
 ##### Generating the Service Principal
 
 1.  Go to [the portal](https://portal.azure.com)
-2.  Under "Browse", look for Active Directory and click on it.
-3.  Create "Web Application". Remember the name you create here - that is what you will add to your ADL account as authorized user.
+2.  Under services in left nav, look for Azure Active Directory and click it.
+3.  Using "App Registrations" in the menu, create "Web Application". Remember
+    the name you create here - that is what you will add to your ADL account
+    as authorized user.
 4.  Go through the wizard
-5.  Once app is created, Go to app configuration, and find the section on "keys"
+5.  Once app is created, go to "keys" under "settings" for the app
 6.  Select a key duration and hit save. Save the generated keys.
-7. Note down the properties you will need to auth:
-    -  The client ID
+7.  Go back to the App Registrations page, and click on the "Endpoints" button
+    at the top
+    a. Note down the  "Token Endpoint" URL
+8. Note down the properties you will need to auth:
+    -  The "Application ID" of the Web App you created above
     -  The key you just generated above
-    -  The token endpoint (select "View endpoints" at the bottom of the page and copy/paste the OAuth2 .0 Token Endpoint value)
-    -  Resource: Always https://management.core.windows.net/ , for all customers
+    -  The token endpoint
 
 ##### Adding the service principal to your ADL Account
 1.  Go to the portal again, and open your ADL account
-2.  Select Users under Settings
+2.  Select `Access control (IAM)`
 3.  Add your user name you created in Step 6 above (note that it does not show up in the list, but will be found if you searched for the name)
 4.  Add "Owner" role
 
@@ -153,6 +157,84 @@ Add the following properties to your `core-site.xml`
 </property>
 ```
 
+#### Using MSI (Managed Service Identity)
+
+Azure VMs can be provisioned with "service identities" that are managed by the
+Identity extension within the VM. The advantage of doing this is that the
+credentials are managed by the extension, and do not have to be put into
+core-site.xml.
+
+To use MSI, the following two steps are needed:
+1. Modify the VM deployment template to specify the port number of the token
+ service exposed to localhost by the identity extension in the VM.
+2. Get your Azure ActiveDirectory Tenant ID:
+   1. Go to [the portal](https://portal.azure.com)
+   2. Under services in left nav, look for Azure Active Directory and click on it.
+   3. Click on Properties
+   4. Note down the GUID shown under "Directory ID" - this is your AAD tenant ID
+
+
+##### Configure core-site.xml
+Add the following properties to your `core-site.xml`
+
+```xml
+<property>
+  <name>fs.adl.oauth2.access.token.provider.type</name>
+  <value>Msi</value>
+</property>
+
+<property>
+  <name>fs.adl.oauth2.msi.port</name>
+  <value>PORT NUMBER FROM STEP 1 ABOVE</value>
+</property>
+
+<property>
+  <name>fs.adl.oauth2.msi.TenantGuid</name>
+  <value>AAD TENANT ID GUID FROM STEP 2 ABOVE</value>
+</property>
+```
+
+### Using Device Code Auth for interactive login
+
+**Note:** This auth method is suitable for running interactive tools, but will
+not work for jobs submitted to a cluster.
+
+To use user-based login, Azure ActiveDirectory provides login flow using
+device code.
+
+To use device code flow, user must first create a **Native** app registration
+in the Azure portal, and provide the client ID for the app as a config. Here
+are the steps:
+
+1.  Go to [the portal](https://portal.azure.com)
+2.  Under services in left nav, look for Azure Active Directory and click on it.
+3.  Using "App Registrations" in the menu, create "Native Application".
+4.  Go through the wizard
+5.  Once app is created, note down the "Appplication ID" of the app
+6. Grant permissions to the app:
+    1. Click on "Permissions" for the app, and then add "Azure Data Lake" and
+       "Windows Azure Service Management API" permissions
+    2. Click on "Grant Permissions" to add the permissions to the app
+
+Add the following properties to your `core-site.xml`
+
+```xml
+<property>
+  <name>fs.adl.oauth2.devicecode.clientappid</name>
+  <value>APP ID FROM STEP 5 ABOVE</value>
+</property>
+```
+
+It is usually not desirable to add DeviceCode as the default token provider
+type. But it can be used when using a local command:
+```
+ hadoop fs -Dfs.adl.oauth2.access.token.provider.type=DeviceCode -ls ...
+```
+Running this will print a URL and device code that can be used to login from
+any browser (even on a different machine, outside of the ssh session). Once
+the login is done, the command continues.
+
+
 #### Protecting the Credentials with Credential Providers
 
 In many Hadoop clusters, the `core-site.xml` file is world-readable. To protect

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java
index 36498c6..929b33a 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java
@@ -23,6 +23,8 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 
+import com.microsoft.azure.datalake.store.oauth2.DeviceCodeTokenProvider;
+import com.microsoft.azure.datalake.store.oauth2.MsiTokenProvider;
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.adl.common.CustomMockTokenProvider;
@@ -40,6 +42,9 @@ import static org.apache.hadoop.fs.adl.AdlConfKeys
     .AZURE_AD_TOKEN_PROVIDER_CLASS_KEY;
 import static org.apache.hadoop.fs.adl.AdlConfKeys
     .AZURE_AD_TOKEN_PROVIDER_TYPE_KEY;
+import static org.apache.hadoop.fs.adl.AdlConfKeys.DEVICE_CODE_CLIENT_APP_ID;
+import static org.apache.hadoop.fs.adl.AdlConfKeys.MSI_PORT;
+import static org.apache.hadoop.fs.adl.AdlConfKeys.MSI_TENANT_GUID;
 import static org.apache.hadoop.fs.adl.TokenProviderType.*;
 import static org.junit.Assert.assertEquals;
 
@@ -98,6 +103,41 @@ public class TestAzureADTokenProvider {
   }
 
   @Test
+  public void testMSITokenProvider()
+          throws IOException, URISyntaxException {
+    Configuration conf = new Configuration();
+    conf.setEnum(AZURE_AD_TOKEN_PROVIDER_TYPE_KEY, MSI);
+    conf.set(MSI_PORT, "54321");
+    conf.set(MSI_TENANT_GUID, "TENANT_GUID");
+
+    URI uri = new URI("adl://localhost:8080");
+    AdlFileSystem fileSystem = new AdlFileSystem();
+    fileSystem.initialize(uri, conf);
+    AccessTokenProvider tokenProvider = fileSystem.getTokenProvider();
+    Assert.assertTrue(tokenProvider instanceof MsiTokenProvider);
+  }
+
+  @Test
+  public void testDeviceCodeTokenProvider()
+          throws IOException, URISyntaxException {
+    boolean runTest = false;
+    if (runTest) {
+      // Device code auth method causes an interactive prompt, so run this only
+      // when running the test interactively at a local terminal. Disabling
+      // test by default, to not break any automation.
+      Configuration conf = new Configuration();
+      conf.setEnum(AZURE_AD_TOKEN_PROVIDER_TYPE_KEY, DeviceCode);
+      conf.set(DEVICE_CODE_CLIENT_APP_ID, "CLIENT_APP_ID_GUID");
+
+      URI uri = new URI("adl://localhost:8080");
+      AdlFileSystem fileSystem = new AdlFileSystem();
+      fileSystem.initialize(uri, conf);
+      AccessTokenProvider tokenProvider = fileSystem.getTokenProvider();
+      Assert.assertTrue(tokenProvider instanceof DeviceCodeTokenProvider);
+    }
+  }
+
+  @Test
   public void testCustomCredTokenProvider()
       throws URISyntaxException, IOException {
     Configuration conf = new Configuration();


---------------------------------------------------------------------
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: YARN-6917. Queue path is recomputed from scratch on every allocation. Contributed by Eric Payne

Posted by ae...@apache.org.
YARN-6917. Queue path is recomputed from scratch on every allocation. Contributed by Eric Payne


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

Branch: refs/heads/HDFS-7240
Commit: 5558792894169425bff054364a1ab4c48b347fb9
Parents: 3325ef6
Author: Jason Lowe <jl...@yahoo-inc.com>
Authored: Mon Aug 14 15:31:34 2017 -0500
Committer: Jason Lowe <jl...@yahoo-inc.com>
Committed: Mon Aug 14 15:31:34 2017 -0500

----------------------------------------------------------------------
 .../resourcemanager/scheduler/capacity/AbstractCSQueue.java  | 8 ++++++++
 .../server/resourcemanager/scheduler/capacity/LeafQueue.java | 5 -----
 .../resourcemanager/scheduler/capacity/ParentQueue.java      | 6 ------
 3 files changed, 8 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/55587928/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 5fbdead..d7c452a 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
@@ -76,6 +76,7 @@ public abstract class AbstractCSQueue implements CSQueue {
   private static final Log LOG = LogFactory.getLog(AbstractCSQueue.class);  
   volatile CSQueue parent;
   final String queueName;
+  private final String queuePath;
   volatile int numContainers;
   
   final Resource minimumAllocation;
@@ -119,6 +120,8 @@ public abstract class AbstractCSQueue implements CSQueue {
     this.labelManager = cs.getRMContext().getNodeLabelManager();
     this.parent = parent;
     this.queueName = queueName;
+    this.queuePath =
+      ((parent == null) ? "" : (parent.getQueuePath() + ".")) + this.queueName;
     this.resourceCalculator = cs.getResourceCalculator();
     this.activitiesManager = cs.getActivitiesManager();
     
@@ -150,6 +153,11 @@ public abstract class AbstractCSQueue implements CSQueue {
         queueCapacities,
         parent == null ? null : parent.getQueueCapacities());
   }
+
+  @Override
+  public String getQueuePath() {
+    return queuePath;
+  }
   
   @Override
   public float getCapacity() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55587928/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 2e502b7..d15431e 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
@@ -299,11 +299,6 @@ public class LeafQueue extends AbstractCSQueue {
     }
   }
 
-  @Override
-  public String getQueuePath() {
-    return getParent().getQueuePath() + "." + getQueueName();
-  }
-
   /**
    * Used only by tests.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55587928/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 e0baa07..2e48000 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
@@ -199,12 +199,6 @@ public class ParentQueue extends AbstractCSQueue {
   }
 
   @Override
-  public String getQueuePath() {
-    String parentPath = ((parent == null) ? "" : (parent.getQueuePath() + "."));
-    return parentPath + getQueueName();
-  }
-
-  @Override
   public QueueInfo getQueueInfo(
       boolean includeChildQueues, boolean recursive) {
     try {


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


[25/50] [abbrv] hadoop git commit: YARN-6965. Duplicate instantiation in FairSchedulerQueueInfo. Contributed by Masahiro Tanaka.

Posted by ae...@apache.org.
YARN-6965. Duplicate instantiation in FairSchedulerQueueInfo. Contributed by Masahiro Tanaka.


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

Branch: refs/heads/HDFS-7240
Commit: 588c190afd49bdbd5708f7805bf6c68f09fee142
Parents: 75dd866
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Aug 16 14:06:22 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Aug 16 14:06:22 2017 +0900

----------------------------------------------------------------------
 .../server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java   | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/588c190a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.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/FairSchedulerQueueInfo.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/FairSchedulerQueueInfo.java
index a4607c2..79339c7 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/FairSchedulerQueueInfo.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/FairSchedulerQueueInfo.java
@@ -99,7 +99,6 @@ public class FairSchedulerQueueInfo {
     steadyFairResources = new ResourceInfo(queue.getSteadyFairShare());
     fairResources = new ResourceInfo(queue.getFairShare());
     minResources = new ResourceInfo(queue.getMinShare());
-    maxResources = new ResourceInfo(queue.getMaxShare());
     maxResources = new ResourceInfo(
         Resources.componentwiseMin(queue.getMaxShare(),
             scheduler.getClusterResource()));


---------------------------------------------------------------------
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: HADOOP-14726. Mark FileStatus::isDir as final

Posted by ae...@apache.org.
HADOOP-14726. Mark FileStatus::isDir as final


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

Branch: refs/heads/HDFS-7240
Commit: 645a8f2a4d09acb5a21820f52ee78784d9e4cc8a
Parents: 4d7be1d
Author: Chris Douglas <cd...@apache.org>
Authored: Mon Aug 14 21:57:20 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Mon Aug 14 21:57:20 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FileStatus.java    | 19 +++++++++----------
 .../hadoop/fs/viewfs/ViewFsFileStatus.java       |  8 +-------
 .../fs/viewfs/ViewFsLocatedFileStatus.java       |  6 ------
 .../hadoop/hdfs/protocolPB/PBHelperClient.java   |  2 +-
 .../apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java   |  6 ++++--
 .../hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java     |  8 ++++----
 .../apache/hadoop/hdfs/server/mover/Mover.java   |  2 +-
 .../hdfs/server/namenode/NamenodeFsck.java       |  4 ++--
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java     |  3 +--
 .../hdfs/server/mover/TestStorageMover.java      |  2 +-
 .../hadoop/hdfs/server/namenode/TestStartup.java |  4 ++--
 .../server/namenode/ha/TestEditLogTailer.java    |  4 ++--
 .../namenode/ha/TestFailureToReadEdits.java      |  6 +++---
 .../namenode/ha/TestInitializeSharedEdits.java   |  2 +-
 .../lib/input/TestCombineFileInputFormat.java    |  2 +-
 .../azure/TestOutOfBandAzureBlobOperations.java  |  8 ++++----
 .../hadoop/fs/swift/snative/SwiftFileStatus.java | 16 ++++------------
 .../snative/SwiftNativeFileSystemStore.java      |  4 ++--
 .../fs/swift/TestSwiftFileSystemDirectories.java |  4 ++--
 .../TestSwiftFileSystemPartitionedUploads.java   |  2 +-
 20 files changed, 46 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
index 2f22ea0..8575439 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
@@ -172,7 +172,7 @@ public class FileStatus implements Writable, Comparable<Object>,
    * @return true if this is a file
    */
   public boolean isFile() {
-    return !isdir && !isSymlink();
+    return !isDirectory() && !isSymlink();
   }
 
   /**
@@ -182,20 +182,20 @@ public class FileStatus implements Writable, Comparable<Object>,
   public boolean isDirectory() {
     return isdir;
   }
-  
+
   /**
-   * Old interface, instead use the explicit {@link FileStatus#isFile()}, 
-   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()} 
+   * Old interface, instead use the explicit {@link FileStatus#isFile()},
+   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()}
    * @return true if this is a directory.
-   * @deprecated Use {@link FileStatus#isFile()},  
-   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()} 
+   * @deprecated Use {@link FileStatus#isFile()},
+   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()}
    * instead.
    */
   @Deprecated
-  public boolean isDir() {
-    return isdir;
+  public final boolean isDir() {
+    return isDirectory();
   }
-  
+
   /**
    * Is this a symbolic link?
    * @return true if this is a symbolic link
@@ -448,7 +448,6 @@ public class FileStatus implements Writable, Comparable<Object>,
     FileStatus other = PBHelper.convert(proto);
     isdir = other.isDirectory();
     length = other.getLen();
-    isdir = other.isDirectory();
     block_replication = other.getReplication();
     blocksize = other.getBlockSize();
     modification_time = other.getModificationTime();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java
index e0f62e4..ce03ced 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java
@@ -61,13 +61,7 @@ class ViewFsFileStatus extends FileStatus {
    public boolean isDirectory() {
      return  myFs.isDirectory();
    }
-   
-   @Override
-   @SuppressWarnings("deprecation")
-   public boolean isDir() {
-     return myFs.isDirectory();
-   }
-   
+
    @Override
    public boolean isSymlink() {
      return myFs.isSymlink();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java
index 4e681a7..c7c8b28 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java
@@ -50,12 +50,6 @@ class ViewFsLocatedFileStatus extends LocatedFileStatus {
   }
 
   @Override
-  @SuppressWarnings("deprecation")
-  public boolean isDir() {
-    return myFs.isDirectory();
-  }
-
-  @Override
   public boolean isSymlink() {
     return myFs.isSymlink();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index a0d2c5a..5b1a687 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -2095,7 +2095,7 @@ public class PBHelperClient {
     if (fs == null)
       return null;
     FileType fType = FileType.IS_FILE;
-    if (fs.isDir()) {
+    if (fs.isDirectory()) {
       fType = FileType.IS_DIR;
     } else if (fs.isSymlink()) {
       fType = FileType.IS_SYMLINK;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
index cc17394..abaa5ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
@@ -65,7 +65,9 @@ public class Nfs3Utils {
      * client takes only the lower 32bit of the fileId and treats it as signed
      * int. When the 32th bit is 1, the client considers it invalid.
      */
-    NfsFileType fileType = fs.isDir() ? NfsFileType.NFSDIR : NfsFileType.NFSREG;
+    NfsFileType fileType = fs.isDirectory()
+        ? NfsFileType.NFSDIR
+        : NfsFileType.NFSREG;
     fileType = fs.isSymlink() ? NfsFileType.NFSLNK : fileType;
     int nlink = (fileType == NfsFileType.NFSDIR) ? fs.getChildrenNum() + 2 : 1;
     long size = (fileType == NfsFileType.NFSDIR) ? getDirSize(fs
@@ -98,7 +100,7 @@ public class Nfs3Utils {
       return null;
     }
 
-    long size = fstat.isDir() ? getDirSize(fstat.getChildrenNum()) : fstat
+    long size = fstat.isDirectory() ? getDirSize(fstat.getChildrenNum()) : fstat
         .getLen();
     return new WccAttr(size, new NfsTime(fstat.getModificationTime()),
         new NfsTime(fstat.getModificationTime()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
index d6bb71d..7a6aa89 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
@@ -1208,7 +1208,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       if (fstat == null) {
         return new REMOVE3Response(Nfs3Status.NFS3ERR_NOENT, errWcc);
       }
-      if (fstat.isDir()) {
+      if (fstat.isDirectory()) {
         return new REMOVE3Response(Nfs3Status.NFS3ERR_ISDIR, errWcc);
       }
 
@@ -1289,7 +1289,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       if (fstat == null) {
         return new RMDIR3Response(Nfs3Status.NFS3ERR_NOENT, errWcc);
       }
-      if (!fstat.isDir()) {
+      if (!fstat.isDirectory()) {
         return new RMDIR3Response(Nfs3Status.NFS3ERR_NOTDIR, errWcc);
       }
 
@@ -1565,7 +1565,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         LOG.info("Can't get path for fileId: " + handle.getFileId());
         return new READDIR3Response(Nfs3Status.NFS3ERR_STALE);
       }
-      if (!dirStatus.isDir()) {
+      if (!dirStatus.isDirectory()) {
         LOG.error("Can't readdir for regular file, fileId: "
             + handle.getFileId());
         return new READDIR3Response(Nfs3Status.NFS3ERR_NOTDIR);
@@ -1732,7 +1732,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         LOG.info("Can't get path for fileId: " + handle.getFileId());
         return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_STALE);
       }
-      if (!dirStatus.isDir()) {
+      if (!dirStatus.isDirectory()) {
         LOG.error("Can't readdirplus for regular file, fileId: "
             + handle.getFileId());
         return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_NOTDIR);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index 1a2c889..b653f4f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -348,7 +348,7 @@ public class Mover {
     private void processRecursively(String parent, HdfsFileStatus status,
         Result result) {
       String fullPath = status.getFullName(parent);
-      if (status.isDir()) {
+      if (status.isDirectory()) {
         if (!fullPath.endsWith(Path.SEPARATOR)) {
           fullPath = fullPath + Path.SEPARATOR;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index eddab3f..5872955 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -471,7 +471,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   void check(String parent, HdfsFileStatus file, Result replRes, Result ecRes)
       throws IOException {
     String path = file.getFullName(parent);
-    if (file.isDir()) {
+    if (file.isDirectory()) {
       checkDir(path, replRes, ecRes);
       return;
     }
@@ -1115,7 +1115,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       if (lfStatus == null) { // not exists
         lfInitedOk = dfs.mkdirs(lfName, null, true);
         lostFound = lfName;
-      } else if (!lfStatus.isDir()) { // exists but not a directory
+      } else if (!lfStatus.isDirectory()) { // exists but not a directory
         LOG.warn("Cannot use /lost+found : a regular file with this name exists.");
         lfInitedOk = false;
       }  else { // exists and is a directory

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
index 5dee6e0..e42e08c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
@@ -51,7 +51,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Logger;
 import org.junit.Test;
 
-import static org.apache.hadoop.hdfs.inotify.Event.CreateEvent;
 import static org.junit.Assert.*;
 
 /**
@@ -572,7 +571,7 @@ public class TestDFSUpgradeFromImage {
       Path path) throws IOException {
     String pathStr = path.toString();
     HdfsFileStatus status = dfs.getFileInfo(pathStr);
-    if (!status.isDir()) {
+    if (!status.isDirectory()) {
       for (int retries = 10; retries > 0; retries--) {
         if (dfs.recoverLease(pathStr)) {
           return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
index 077997f..764a0db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
@@ -283,7 +283,7 @@ public class TestStorageMover {
 
     private void verifyRecursively(final Path parent,
         final HdfsFileStatus status) throws Exception {
-      if (status.isDir()) {
+      if (status.isDirectory()) {
         Path fullPath = parent == null ?
             new Path("/") : status.getFullPath(parent);
         DirectoryListing children = dfs.getClient().listPaths(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
index 29a6064..94172bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
@@ -450,7 +450,7 @@ public class TestStartup {
     namenode.getNamesystem().mkdirs("/test",
         new PermissionStatus("hairong", null, FsPermission.getDefault()), true);
     NamenodeProtocols nnRpc = namenode.getRpcServer();
-    assertTrue(nnRpc.getFileInfo("/test").isDir());
+    assertTrue(nnRpc.getFileInfo("/test").isDirectory());
     nnRpc.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
     nnRpc.saveNamespace(0, 0);
     namenode.stop();
@@ -481,7 +481,7 @@ public class TestStartup {
   private void checkNameSpace(Configuration conf) throws IOException {
     NameNode namenode = new NameNode(conf);
     NamenodeProtocols nnRpc = namenode.getRpcServer();
-    assertTrue(nnRpc.getFileInfo("/test").isDir());
+    assertTrue(nnRpc.getFileInfo("/test").isDirectory());
     nnRpc.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
     nnRpc.saveNamespace(0, 0);
     namenode.stop();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
index 5a2aff9..b3bb3dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
@@ -124,7 +124,7 @@ public class TestEditLogTailer {
       
       for (int i = 0; i < DIRS_TO_MAKE / 2; i++) {
         assertTrue(NameNodeAdapter.getFileInfo(nn2,
-            getDirPath(i), false).isDir());
+            getDirPath(i), false).isDirectory());
       }
       
       for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
@@ -137,7 +137,7 @@ public class TestEditLogTailer {
       
       for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
         assertTrue(NameNodeAdapter.getFileInfo(nn2,
-            getDirPath(i), false).isDir());
+            getDirPath(i), false).isDirectory());
       }
     } finally {
       cluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
index 38c2b2d..93c717c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
@@ -205,7 +205,7 @@ public class TestFailureToReadEdits {
         TEST_DIR1, false));
     // Should have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR2, false).isDir());
+        TEST_DIR2, false).isDirectory());
     // Null because it hasn't been created yet.
     assertNull(NameNodeAdapter.getFileInfo(nn1,
         TEST_DIR3, false));
@@ -219,10 +219,10 @@ public class TestFailureToReadEdits {
         TEST_DIR1, false));
     // Should have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR2, false).isDir());
+        TEST_DIR2, false).isDirectory());
     // Should now have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR3, false).isDir());
+        TEST_DIR3, false).isDirectory());
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
index 856ed8f..8eeb853 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
@@ -128,7 +128,7 @@ public class TestInitializeSharedEdits {
       HATestUtil.waitForStandbyToCatchUp(cluster.getNameNode(0),
           cluster.getNameNode(1));
       assertTrue(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-          newPath.toString(), false).isDir());
+          newPath.toString(), false).isDirectory());
     } finally {
       if (fs != null) {
         fs.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
index 1fca5c9..cc97a14d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
@@ -154,7 +154,7 @@ public class TestCombineFileInputFormat {
     @Override
     public BlockLocation[] getFileBlockLocations(
         FileStatus stat, long start, long len) throws IOException {
-      if (stat.isDir()) {
+      if (stat.isDirectory()) {
         return null;
       }
       System.out.println("File " + stat.getPath());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
index 9d5d6a22c..544d6ab 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
@@ -73,20 +73,20 @@ public class TestOutOfBandAzureBlobOperations {
     FileStatus[] obtained = fs.listStatus(new Path("/root/b"));
     assertNotNull(obtained);
     assertEquals(1, obtained.length);
-    assertFalse(obtained[0].isDir());
+    assertFalse(obtained[0].isDirectory());
     assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
 
     // List the directory
     obtained = fs.listStatus(new Path("/root"));
     assertNotNull(obtained);
     assertEquals(1, obtained.length);
-    assertFalse(obtained[0].isDir());
+    assertFalse(obtained[0].isDirectory());
     assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
 
     // Get the directory's file status
     FileStatus dirStatus = fs.getFileStatus(new Path("/root"));
     assertNotNull(dirStatus);
-    assertTrue(dirStatus.isDir());
+    assertTrue(dirStatus.isDirectory());
     assertEquals("/root", dirStatus.getPath().toUri().getPath());
   }
 
@@ -114,7 +114,7 @@ public class TestOutOfBandAzureBlobOperations {
     FileStatus[] listResult = fs.listStatus(new Path("/root/b"));
     // File should win.
     assertEquals(1, listResult.length);
-    assertFalse(listResult[0].isDir());
+    assertFalse(listResult[0].isDirectory());
     try {
       // Trying to delete root/b/c would cause a dilemma for WASB, so
       // it should throw.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java b/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java
index d010d08..725cae1 100644
--- a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java
+++ b/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java
@@ -71,7 +71,7 @@ public class SwiftFileStatus extends FileStatus {
    * @return true if the status is considered to be a file
    */
   @Override
-  public boolean isDir() {
+  public boolean isDirectory() {
     return super.isDirectory() || getLen() == 0;
   }
 
@@ -79,19 +79,11 @@ public class SwiftFileStatus extends FileStatus {
    * A entry is a file if it is not a directory.
    * By implementing it <i>and not marking as an override</i> this
    * subclass builds and runs in both Hadoop versions.
-   * @return the opposite value to {@link #isDir()}
+   * @return the opposite value to {@link #isDirectory()}
    */
   @Override
   public boolean isFile() {
-    return !isDir();
-  }
-
-  /**
-   * Directory test
-   * @return true if the file is considered to be a directory
-   */
-  public boolean isDirectory() {
-    return isDir();
+    return !this.isDirectory();
   }
 
   @Override
@@ -100,7 +92,7 @@ public class SwiftFileStatus extends FileStatus {
     sb.append(getClass().getSimpleName());
     sb.append("{ ");
     sb.append("path=").append(getPath());
-    sb.append("; isDirectory=").append(isDir());
+    sb.append("; isDirectory=").append(isDirectory());
     sb.append("; length=").append(getLen());
     sb.append("; blocksize=").append(getBlockSize());
     sb.append("; modification_time=").append(getModificationTime());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java b/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
index f2ecb0f..a44051a 100644
--- a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
@@ -578,7 +578,7 @@ public class SwiftNativeFileSystemStore {
 
     //enum the child entries and everything underneath
     List<FileStatus> childStats = listDirectory(srcObject, true, true);
-    boolean srcIsFile = !srcMetadata.isDir();
+    boolean srcIsFile = !srcMetadata.isDirectory();
     if (srcIsFile) {
 
       //source is a simple file OR a partitioned file
@@ -945,7 +945,7 @@ public class SwiftNativeFileSystemStore {
     //>1 entry implies directory with children. Run through them,
     // but first check for the recursive flag and reject it *unless it looks
     // like a partitioned file (len > 0 && has children)
-    if (!fileStatus.isDir()) {
+    if (!fileStatus.isDirectory()) {
       LOG.debug("Multiple child entries but entry has data: assume partitioned");
     } else if (!recursive) {
       //if there are children, unless this is a recursive operation, fail immediately

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java b/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java
index 21fe918..9b4ba5e 100644
--- a/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java
+++ b/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java
@@ -87,7 +87,7 @@ public class TestSwiftFileSystemDirectories extends SwiftFileSystemBaseTest {
     assertEquals("Wrong number of elements in file status " + statusString, 1,
                  statuses.length);
     SwiftFileStatus stat = (SwiftFileStatus) statuses[0];
-    assertTrue("isDir(): Not a directory: " + stat, stat.isDir());
+    assertTrue("isDir(): Not a directory: " + stat, stat.isDirectory());
     extraStatusAssertions(stat);
   }
 
@@ -135,7 +135,7 @@ public class TestSwiftFileSystemDirectories extends SwiftFileSystemBaseTest {
     SwiftTestUtils.writeTextFile(fs, src, "testMultiByteFilesAreFiles", false);
     assertIsFile(src);
     FileStatus status = fs.getFileStatus(src);
-    assertFalse(status.isDir());
+    assertFalse(status.isDirectory());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java b/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java
index b42abcd..419d030 100644
--- a/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java
+++ b/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java
@@ -228,7 +228,7 @@ public class TestSwiftFileSystemPartitionedUploads extends
                  status.getLen());
     String fileInfo = qualifiedPath + "  " + status;
     assertFalse("File claims to be a directory " + fileInfo,
-                status.isDir());
+                status.isDirectory());
 
     FileStatus listedFileStat = resolveChild(parentDirListing, qualifiedPath);
     assertNotNull("Did not find " + path + " in " + parentDirLS,


---------------------------------------------------------------------
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: HADOOP-14673. Remove leftover hadoop_xml_escape from functions. Contributed by Ajay Kumar.

Posted by ae...@apache.org.
HADOOP-14673. Remove leftover hadoop_xml_escape from functions. 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/04465113
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/04465113
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/04465113

Branch: refs/heads/HDFS-7240
Commit: 044651139800b9e2e5b8f224772e6dbd6ded58c6
Parents: 8bef4ec
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Aug 14 16:22:10 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Aug 14 16:22:10 2017 -0700

----------------------------------------------------------------------
 .../src/main/bin/hadoop-functions.sh            | 23 --------------
 .../src/test/scripts/hadoop_escape_chars.bats   | 32 --------------------
 2 files changed, 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/04465113/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
index 3cf21cf..9ea4587 100755
--- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
+++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
@@ -2578,29 +2578,6 @@ function hadoop_parse_args
   hadoop_debug "hadoop_parse: asking caller to skip ${HADOOP_PARSE_COUNTER}"
 }
 
-## @description  XML-escapes the characters (&'"<>) in the given parameter.
-## @audience     private
-## @stability    evolving
-## @replaceable  yes
-## @param        string
-## @return       XML-escaped string
-function hadoop_xml_escape
-{
-  sed -e 's/&/\&amp;/g' -e 's/"/\\\&quot;/g' \
-    -e "s/'/\\\\\&apos;/g" -e 's/</\\\&lt;/g' -e 's/>/\\\&gt;/g' <<< "$1"
-}
-
-## @description  sed-escapes the characters (\/&) in the given parameter.
-## @audience     private
-## @stability    evolving
-## @replaceable  yes
-## @param        string
-## @return       sed-escaped string
-function hadoop_sed_escape
-{
-  sed -e 's/[\/&]/\\&/g' <<< "$1"
-}
-
 ## @description Handle subcommands from main program entries
 ## @audience private
 ## @stability evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/04465113/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_escape_chars.bats
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_escape_chars.bats b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_escape_chars.bats
deleted file mode 100755
index 9b031f2..0000000
--- a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_escape_chars.bats
+++ /dev/null
@@ -1,32 +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.
-
-load hadoop-functions_test_helper
-
-@test "hadoop_escape_sed (positive 1)" {
-  ret="$(hadoop_sed_escape "\pass&&word\0#\$asdf/g  ><'\"~\`!@#$%^&*()_+-=")"
-  expected="\\\\pass\&\&word\\\0#\$asdf\/g  ><'\"~\`!@#$%^\&*()_+-="
-  echo "actual >${ret}<"
-  echo "expected >${expected}<"
-  [ "${ret}" = "${expected}" ]
-}
-
-@test "hadoop_escape_xml (positive 1)" {
-  ret="$(hadoop_xml_escape "\pass&&word\0#\$asdf/g  ><'\"~\`!@#$%^&*()_+-=")"
-  expected="\\pass&amp;&amp;word\0#\$asdf/g  \&gt;\&lt;\&apos;\&quot;~\`!@#\$%^&amp;*()_+-="
-  echo "actual >${ret}<"
-  echo "expected >${expected}<"
-  [ "${ret}" = "${expected}" ]
-}
\ No newline at end of file


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

[34/50] [abbrv] hadoop git commit: HDFS-12250. Reduce usage of FsPermissionExtension in unit tests. Contributed by Chris Douglas.

Posted by ae...@apache.org.
HDFS-12250. Reduce usage of FsPermissionExtension in unit tests. Contributed by Chris Douglas.


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

Branch: refs/heads/HDFS-7240
Commit: dd7916d3cd5d880d0b257d229f43f10feff04c93
Parents: f9a0e23
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Aug 17 09:35:36 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Aug 17 09:35:36 2017 -0700

----------------------------------------------------------------------
 .../hadoop/fs/permission/FsPermission.java      |  2 +-
 .../org/apache/hadoop/fs/shell/AclCommands.java |  6 ++---
 .../hadoop/fs/shell/CommandWithDestination.java |  4 ++--
 .../java/org/apache/hadoop/fs/shell/Ls.java     |  4 ++--
 .../fs/http/client/BaseTestHttpFSWith.java      |  1 +
 .../org/apache/hadoop/hdfs/TestDFSShell.java    | 24 ++++++++++----------
 .../hdfs/server/namenode/FSAclBaseTest.java     |  6 +++++
 .../ClientDistributedCacheManager.java          |  6 ++---
 .../apache/hadoop/fs/adl/TestGetFileStatus.java |  1 +
 .../hadoop/tools/CopyListingFileStatus.java     |  4 ++--
 .../apache/hadoop/tools/util/DistCpUtils.java   |  4 +---
 11 files changed, 33 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
index 23692de..031092b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
@@ -163,7 +163,7 @@ public class FsPermission implements Writable, Serializable,
    */
   public static FsPermission read(DataInput in) throws IOException {
     FsPermission p = new FsPermission();
-    p.readFields(in);
+    p.fromShort(in.readShort());
     return p;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
index a5e386c..701c9de 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
@@ -86,9 +86,9 @@ class AclCommands extends FsCommand {
           (perm.getOtherAction().implies(FsAction.EXECUTE) ? "t" : "T"));
       }
 
-      AclStatus aclStatus = null;
-      List<AclEntry> entries = null;
-      if (perm.getAclBit()) {
+      final AclStatus aclStatus;
+      final List<AclEntry> entries;
+      if (item.stat.hasAcl()) {
         aclStatus = item.fs.getAclStatus(item.path);
         entries = aclStatus.getEntries();
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
index 2a483c0..0bd4882 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
@@ -444,8 +444,8 @@ abstract class CommandWithDestination extends FsCommand {
           src.stat.getPermission());
     }
     if (shouldPreserve(FileAttribute.ACL)) {
-      FsPermission perm = src.stat.getPermission();
-      if (perm.getAclBit()) {
+      if (src.stat.hasAcl()) {
+        FsPermission perm = src.stat.getPermission();
         List<AclEntry> srcEntries =
             src.fs.getAclStatus(src.path).getEntries();
         List<AclEntry> srcFullEntries =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
index 221b3cb..a2d5017 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
@@ -255,7 +255,7 @@ class Ls extends FsCommand {
       ContentSummary contentSummary = item.fs.getContentSummary(item.path);
       String line = String.format(lineFormat,
           (stat.isDirectory() ? "d" : "-"),
-          stat.getPermission() + (stat.getPermission().getAclBit() ? "+" : " "),
+          stat.getPermission() + (stat.hasAcl() ? "+" : " "),
           (stat.isFile() ? stat.getReplication() : "-"),
           stat.getOwner(),
           stat.getGroup(),
@@ -269,7 +269,7 @@ class Ls extends FsCommand {
     } else {
       String line = String.format(lineFormat,
           (stat.isDirectory() ? "d" : "-"),
-          stat.getPermission() + (stat.getPermission().getAclBit() ? "+" : " "),
+          stat.getPermission() + (stat.hasAcl() ? "+" : " "),
           (stat.isFile() ? stat.getReplication() : "-"),
           stat.getOwner(),
           stat.getGroup(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
index 553bbce..2cd8934 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
@@ -859,6 +859,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     FileStatus expectedFileStatus = expected.getFileStatus(path);
     FileStatus actualFileStatus = actual.getFileStatus(path);
     assertEquals(actualFileStatus.hasAcl(), expectedFileStatus.hasAcl());
+    // backwards compat
     assertEquals(actualFileStatus.getPermission().getAclBit(),
         expectedFileStatus.getPermission().getAclBit());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
index 27d41b4..9ae49aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
@@ -2189,7 +2189,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       List<AclEntry> acls = dfs.getAclStatus(target1).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptop
       Path target2 = new Path(hdfsTestDir, "targetfile2");
@@ -2208,7 +2208,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       acls = dfs.getAclStatus(target2).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptopx
       Path target3 = new Path(hdfsTestDir, "targetfile3");
@@ -2229,7 +2229,7 @@ public class TestDFSShell {
       assertArrayEquals(TRUSTED_A1_VALUE, xattrs.get(TRUSTED_A1));
       acls = dfs.getAclStatus(target3).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptopa
       Path target4 = new Path(hdfsTestDir, "targetfile4");
@@ -2248,7 +2248,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       acls = dfs.getAclStatus(target4).getEntries();
       assertFalse(acls.isEmpty());
-      assertTrue(targetPerm.getAclBit());
+      assertTrue(targetStatus.hasAcl());
       assertEquals(dfs.getAclStatus(src), dfs.getAclStatus(target4));
 
       // -ptoa (verify -pa option will preserve permissions also)
@@ -2268,7 +2268,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       acls = dfs.getAclStatus(target5).getEntries();
       assertFalse(acls.isEmpty());
-      assertTrue(targetPerm.getAclBit());
+      assertTrue(targetStatus.hasAcl());
       assertEquals(dfs.getAclStatus(src), dfs.getAclStatus(target5));
     } finally {
       if (null != shell) {
@@ -2480,7 +2480,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       List<AclEntry> acls = dfs.getAclStatus(targetDir1).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptop
       Path targetDir2 = new Path(hdfsTestDir, "targetDir2");
@@ -2499,7 +2499,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       acls = dfs.getAclStatus(targetDir2).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptopx
       Path targetDir3 = new Path(hdfsTestDir, "targetDir3");
@@ -2520,7 +2520,7 @@ public class TestDFSShell {
       assertArrayEquals(TRUSTED_A1_VALUE, xattrs.get(TRUSTED_A1));
       acls = dfs.getAclStatus(targetDir3).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptopa
       Path targetDir4 = new Path(hdfsTestDir, "targetDir4");
@@ -2539,7 +2539,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       acls = dfs.getAclStatus(targetDir4).getEntries();
       assertFalse(acls.isEmpty());
-      assertTrue(targetPerm.getAclBit());
+      assertTrue(targetStatus.hasAcl());
       assertEquals(dfs.getAclStatus(srcDir), dfs.getAclStatus(targetDir4));
 
       // -ptoa (verify -pa option will preserve permissions also)
@@ -2559,7 +2559,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       acls = dfs.getAclStatus(targetDir5).getEntries();
       assertFalse(acls.isEmpty());
-      assertTrue(targetPerm.getAclBit());
+      assertTrue(targetStatus.hasAcl());
       assertEquals(dfs.getAclStatus(srcDir), dfs.getAclStatus(targetDir5));
     } finally {
       if (shell != null) {
@@ -2615,7 +2615,7 @@ public class TestDFSShell {
       assertTrue(perm.equals(targetPerm));
       List<AclEntry> acls = dfs.getAclStatus(target1).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptopa preserves both sticky bit and ACL
       Path target2 = new Path(hdfsTestDir, "targetfile2");
@@ -2632,7 +2632,7 @@ public class TestDFSShell {
       assertTrue(perm.equals(targetPerm));
       acls = dfs.getAclStatus(target2).getEntries();
       assertFalse(acls.isEmpty());
-      assertTrue(targetPerm.getAclBit());
+      assertTrue(targetStatus.hasAcl());
       assertEquals(dfs.getAclStatus(src), dfs.getAclStatus(target2));
     } finally {
       if (null != shell) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
index 93a83fd..ee92217 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
@@ -32,6 +32,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -886,6 +887,11 @@ public abstract class FSAclBaseTest {
     FsPermission perm = inode.getFsPermission();
     assertNotNull(perm);
     assertEquals(0755, perm.toShort());
+    FileStatus stat = fs.getFileStatus(path);
+    assertFalse(stat.hasAcl());
+    assertFalse(stat.isEncrypted());
+    assertFalse(stat.isErasureCoded());
+    // backwards-compat check
     assertEquals(0755, perm.toExtendedShort());
     assertAclFeature(false);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java
index 9f8edb5..ada14db 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.security.Credentials;
@@ -293,7 +292,6 @@ public class ClientDistributedCacheManager {
   private static boolean checkPermissionOfOther(FileSystem fs, Path path,
       FsAction action, Map<URI, FileStatus> statCache) throws IOException {
     FileStatus status = getFileStatus(fs, path.toUri(), statCache);
-    FsPermission perms = status.getPermission();
 
     // Encrypted files are always treated as private. This stance has two
     // important side effects.  The first is that the encrypted files will be
@@ -302,8 +300,8 @@ public class ClientDistributedCacheManager {
     // world readable permissions that is stored in an encryption zone from
     // being localized as a publicly shared file with world readable
     // permissions.
-    if (!perms.getEncryptedBit()) {
-      FsAction otherAction = perms.getOtherAction();
+    if (!status.isEncrypted()) {
+      FsAction otherAction = status.getPermission().getOtherAction();
       if (otherAction.implies(action)) {
         return true;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
index d9e22db..95c2363 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
@@ -98,4 +98,5 @@ public class TestGetFileStatus extends AdlMockWebServer {
     Assert.assertFalse(fileStatus.hasAcl());
     Assert.assertFalse(fileStatus.getPermission().getAclBit());
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
index 29c59ac..138b491 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
@@ -280,7 +280,7 @@ public final class CopyListingFileStatus implements Writable {
     out.writeLong(getBlockSize());
     out.writeLong(getModificationTime());
     out.writeLong(getAccessTime());
-    getPermission().write(out);
+    out.writeShort(getPermission().toShort());
     Text.writeString(out, getOwner(), Text.DEFAULT_MAX_LEN);
     Text.writeString(out, getGroup(), Text.DEFAULT_MAX_LEN);
     if (aclEntries != null) {
@@ -330,7 +330,7 @@ public final class CopyListingFileStatus implements Writable {
     blocksize = in.readLong();
     modificationTime = in.readLong();
     accessTime = in.readLong();
-    permission.readFields(in);
+    permission.fromShort(in.readShort());
     owner = Text.readString(in, Text.DEFAULT_MAX_LEN);
     group = Text.readString(in, Text.DEFAULT_MAX_LEN);
     byte aclEntriesSize = in.readByte();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
index dbe750a..2b3b529 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclUtil;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
@@ -403,8 +402,7 @@ public class DistCpUtils {
     CopyListingFileStatus copyListingFileStatus =
         new CopyListingFileStatus(fileStatus, chunkOffset, chunkLength);
     if (preserveAcls) {
-      FsPermission perm = fileStatus.getPermission();
-      if (perm.getAclBit()) {
+      if (fileStatus.hasAcl()) {
         List<AclEntry> aclEntries = fileSystem.getAclStatus(
           fileStatus.getPath()).getEntries();
         copyListingFileStatus.setAclEntries(aclEntries);


---------------------------------------------------------------------
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: HADOOP-14732. ProtobufRpcEngine should use Time.monotonicNow to measure durations. Contributed by Hanisha Koneru.

Posted by ae...@apache.org.
HADOOP-14732. ProtobufRpcEngine should use Time.monotonicNow to measure durations. Contributed by Hanisha Koneru.


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

Branch: refs/heads/HDFS-7240
Commit: 8bef4eca28a3466707cc4ea0de0330449319a5eb
Parents: 5558792
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Aug 14 15:53:35 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Aug 14 15:53:35 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/ipc/ProtobufRpcEngine.java | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8bef4eca/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
index 639bbad..2c0cfe5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -190,7 +190,7 @@ public class ProtobufRpcEngine implements RpcEngine {
         throws ServiceException {
       long startTime = 0;
       if (LOG.isDebugEnabled()) {
-        startTime = Time.now();
+        startTime = Time.monotonicNow();
       }
       
       if (args.length != 2) { // RpcController + Message
@@ -245,7 +245,7 @@ public class ProtobufRpcEngine implements RpcEngine {
       }
 
       if (LOG.isDebugEnabled()) {
-        long callTime = Time.now() - startTime;
+        long callTime = Time.monotonicNow() - startTime;
         LOG.debug("Call: " + method.getName() + " took " + callTime + "ms");
       }
       
@@ -373,19 +373,19 @@ public class ProtobufRpcEngine implements RpcEngine {
         this.server = currentCallInfo.get().server;
         this.call = Server.getCurCall().get();
         this.methodName = currentCallInfo.get().methodName;
-        this.setupTime = Time.now();
+        this.setupTime = Time.monotonicNow();
       }
 
       @Override
       public void setResponse(Message message) {
-        long processingTime = Time.now() - setupTime;
+        long processingTime = Time.monotonicNow() - setupTime;
         call.setDeferredResponse(RpcWritable.wrap(message));
         server.updateDeferredMetrics(methodName, processingTime);
       }
 
       @Override
       public void error(Throwable t) {
-        long processingTime = Time.now() - setupTime;
+        long processingTime = Time.monotonicNow() - setupTime;
         String detailedMetricsName = t.getClass().getSimpleName();
         server.updateDeferredMetrics(detailedMetricsName, processingTime);
         call.setDeferredError(t);
@@ -513,7 +513,7 @@ public class ProtobufRpcEngine implements RpcEngine {
         Message param = request.getValue(prototype);
 
         Message result;
-        long startTime = Time.now();
+        long startTime = Time.monotonicNow();
         int qTime = (int) (startTime - receiveTime);
         Exception exception = null;
         boolean isDeferred = false;
@@ -537,7 +537,7 @@ public class ProtobufRpcEngine implements RpcEngine {
           throw e;
         } finally {
           currentCallInfo.set(null);
-          int processingTime = (int) (Time.now() - startTime);
+          int processingTime = (int) (Time.monotonicNow() - startTime);
           if (LOG.isDebugEnabled()) {
             String msg =
                 "Served: " + methodName + (isDeferred ? ", deferred" : "") +


---------------------------------------------------------------------
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: HDFS-12072. Provide fairness between EC and non-EC recovery tasks. Contributed by Eddy Xu.

Posted by ae...@apache.org.
HDFS-12072. Provide fairness between EC and non-EC recovery tasks. Contributed by Eddy Xu.


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

Branch: refs/heads/HDFS-7240
Commit: b29894889742dda654cd88a7ce72a4e51fccb328
Parents: ab1a8ae
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Aug 17 15:26:11 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Aug 17 15:26:11 2017 -0700

----------------------------------------------------------------------
 .../blockmanagement/DatanodeDescriptor.java     |  6 +-
 .../server/blockmanagement/DatanodeManager.java | 45 ++++++---
 .../blockmanagement/TestDatanodeManager.java    | 96 +++++++++++++++-----
 3 files changed, 108 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2989488/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 2bd4a20..d35894c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -661,7 +661,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return erasurecodeBlocks.size();
   }
 
-  public List<BlockTargetPair> getReplicationCommand(int maxTransfers) {
+  int getNumberOfReplicateBlocks() {
+    return replicateBlocks.size();
+  }
+
+  List<BlockTargetPair> getReplicationCommand(int maxTransfers) {
     return replicateBlocks.poll(maxTransfers);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2989488/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 78783ca..c75bcea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -1663,21 +1663,38 @@ public class DatanodeManager {
     }
 
     final List<DatanodeCommand> cmds = new ArrayList<>();
-    // check pending replication
-    List<BlockTargetPair> pendingList = nodeinfo.getReplicationCommand(
-        maxTransfers);
-    if (pendingList != null) {
-      cmds.add(new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blockPoolId,
-          pendingList));
-      maxTransfers -= pendingList.size();
-    }
-    // check pending erasure coding tasks
-    List<BlockECReconstructionInfo> pendingECList = nodeinfo
-        .getErasureCodeCommand(maxTransfers);
-    if (pendingECList != null) {
-      cmds.add(new BlockECReconstructionCommand(
-          DNA_ERASURE_CODING_RECONSTRUCTION, pendingECList));
+    // Allocate _approximately_ maxTransfers pending tasks to DataNode.
+    // NN chooses pending tasks based on the ratio between the lengths of
+    // replication and erasure-coded block queues.
+    int totalReplicateBlocks = nodeinfo.getNumberOfReplicateBlocks();
+    int totalECBlocks = nodeinfo.getNumberOfBlocksToBeErasureCoded();
+    int totalBlocks = totalReplicateBlocks + totalECBlocks;
+    if (totalBlocks > 0) {
+      int numReplicationTasks = (int) Math.ceil(
+          (double) (totalReplicateBlocks * maxTransfers) / totalBlocks);
+      int numECTasks = (int) Math.ceil(
+          (double) (totalECBlocks * maxTransfers) / totalBlocks);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Pending replication tasks: " + numReplicationTasks
+            + " erasure-coded tasks: " + numECTasks);
+      }
+      // check pending replication tasks
+      List<BlockTargetPair> pendingList = nodeinfo.getReplicationCommand(
+          numReplicationTasks);
+      if (pendingList != null && !pendingList.isEmpty()) {
+        cmds.add(new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blockPoolId,
+            pendingList));
+      }
+      // check pending erasure coding tasks
+      List<BlockECReconstructionInfo> pendingECList = nodeinfo
+          .getErasureCodeCommand(numECTasks);
+      if (pendingECList != null && !pendingECList.isEmpty()) {
+        cmds.add(new BlockECReconstructionCommand(
+            DNA_ERASURE_CODING_RECONSTRUCTION, pendingECList));
+      }
     }
+
     // check block invalidation
     Block[] blks = nodeinfo.getInvalidateBlocks(blockInvalidateLimit);
     if (blks != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2989488/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
index de002f4..286f4a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
@@ -25,6 +25,7 @@ import java.net.URL;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -500,46 +501,93 @@ public class TestDatanodeManager {
         "127.0.0.1:23456", bothAgain.get(1).getInfoAddr());
   }
 
-  @Test
-  public void testPendingRecoveryTasks() throws IOException {
+  /**
+   * Verify the correctness of pending recovery process.
+   *
+   * @param numReplicationBlocks the number of replication blocks in the queue.
+   * @param numECBlocks number of EC blocks in the queue.
+   * @param maxTransfers the maxTransfer value.
+   * @param numReplicationTasks the number of replication tasks polled from
+   *                            the queue.
+   * @param numECTasks the number of EC tasks polled from the queue.
+   *
+   * @throws IOException
+   */
+  private void verifyPendingRecoveryTasks(
+      int numReplicationBlocks, int numECBlocks,
+      int maxTransfers, int numReplicationTasks, int numECTasks)
+      throws IOException {
     FSNamesystem fsn = Mockito.mock(FSNamesystem.class);
     Mockito.when(fsn.hasWriteLock()).thenReturn(true);
     Configuration conf = new Configuration();
     DatanodeManager dm = Mockito.spy(mockDatanodeManager(fsn, conf));
 
-    int maxTransfers = 20;
-    int numPendingTasks = 7;
-    int numECTasks = maxTransfers - numPendingTasks;
-
     DatanodeDescriptor nodeInfo = Mockito.mock(DatanodeDescriptor.class);
     Mockito.when(nodeInfo.isRegistered()).thenReturn(true);
     Mockito.when(nodeInfo.getStorageInfos())
         .thenReturn(new DatanodeStorageInfo[0]);
 
-    List<BlockTargetPair> pendingList =
-        Collections.nCopies(numPendingTasks, new BlockTargetPair(null, null));
-    Mockito.when(nodeInfo.getReplicationCommand(maxTransfers))
-        .thenReturn(pendingList);
-    List<BlockECReconstructionInfo> ecPendingList =
-        Collections.nCopies(numECTasks, null);
+    if (numReplicationBlocks > 0) {
+      Mockito.when(nodeInfo.getNumberOfReplicateBlocks())
+          .thenReturn(numReplicationBlocks);
+
+      List<BlockTargetPair> tasks =
+          Collections.nCopies(
+              Math.min(numReplicationTasks, numReplicationBlocks),
+              new BlockTargetPair(null, null));
+      Mockito.when(nodeInfo.getReplicationCommand(numReplicationTasks))
+          .thenReturn(tasks);
+    }
+
+    if (numECBlocks > 0) {
+      Mockito.when(nodeInfo.getNumberOfBlocksToBeErasureCoded())
+          .thenReturn(numECBlocks);
+
+      List<BlockECReconstructionInfo> tasks =
+          Collections.nCopies(numECTasks, null);
+      Mockito.when(nodeInfo.getErasureCodeCommand(numECTasks))
+          .thenReturn(tasks);
+    }
 
-    Mockito.when(nodeInfo.getErasureCodeCommand(numECTasks))
-        .thenReturn(ecPendingList);
     DatanodeRegistration dnReg = Mockito.mock(DatanodeRegistration.class);
     Mockito.when(dm.getDatanode(dnReg)).thenReturn(nodeInfo);
-
     DatanodeCommand[] cmds = dm.handleHeartbeat(
         dnReg, new StorageReport[1], "bp-123", 0, 0, 10, maxTransfers, 0, null,
         SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT);
 
-    assertEquals(2, cmds.length);
-    assertTrue(cmds[0] instanceof BlockCommand);
-    BlockCommand replicaCmd = (BlockCommand) cmds[0];
-    assertEquals(numPendingTasks, replicaCmd.getBlocks().length);
-    assertEquals(numPendingTasks, replicaCmd.getTargets().length);
-    assertTrue(cmds[1] instanceof BlockECReconstructionCommand);
-    BlockECReconstructionCommand ecRecoveryCmd =
-        (BlockECReconstructionCommand) cmds[1];
-    assertEquals(numECTasks, ecRecoveryCmd.getECTasks().size());
+    long expectedNumCmds = Arrays.stream(
+        new int[]{numReplicationTasks, numECTasks})
+        .filter(x -> x > 0)
+        .count();
+    assertEquals(expectedNumCmds, cmds.length);
+
+    int idx = 0;
+    if (numReplicationTasks > 0) {
+      assertTrue(cmds[idx] instanceof BlockCommand);
+      BlockCommand cmd = (BlockCommand) cmds[0];
+      assertEquals(numReplicationTasks, cmd.getBlocks().length);
+      assertEquals(numReplicationTasks, cmd.getTargets().length);
+      idx++;
+    }
+
+    if (numECTasks > 0) {
+      assertTrue(cmds[idx] instanceof BlockECReconstructionCommand);
+      BlockECReconstructionCommand cmd =
+          (BlockECReconstructionCommand) cmds[idx];
+      assertEquals(numECTasks, cmd.getECTasks().size());
+    }
+
+    Mockito.verify(nodeInfo).getReplicationCommand(numReplicationTasks);
+    Mockito.verify(nodeInfo).getErasureCodeCommand(numECTasks);
+  }
+
+  @Test
+  public void testPendingRecoveryTasks() throws IOException {
+    // Tasks are slitted according to the ratio between queue lengths.
+    verifyPendingRecoveryTasks(20, 20, 20, 10, 10);
+    verifyPendingRecoveryTasks(40, 10, 20, 16, 4);
+
+    // Approximately load tasks if the ratio between queue length is large.
+    verifyPendingRecoveryTasks(400, 1, 20, 20, 1);
   }
 }


---------------------------------------------------------------------
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: MAPREDUCE-6940. Copy-paste error in the TaskAttemptUnsuccessfulCompletionEvent constructor. Contributed by Oleg Danilov

Posted by ae...@apache.org.
MAPREDUCE-6940. Copy-paste error in the TaskAttemptUnsuccessfulCompletionEvent constructor. Contributed by Oleg Danilov


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

Branch: refs/heads/HDFS-7240
Commit: 0acc5e00362602f027524637a86ca1bf80982986
Parents: de462da
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Aug 16 16:34:06 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Aug 16 16:34:06 2017 -0500

----------------------------------------------------------------------
 .../TaskAttemptUnsuccessfulCompletionEvent.java | 28 ++++++++++----------
 1 file changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0acc5e00/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
index 1732d91..1752967 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
@@ -60,7 +60,7 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
   int[] physMemKbytes;
   private static final Counters EMPTY_COUNTERS = new Counters();
 
-  /** 
+  /**
    * Create an event to record the unsuccessful completion of attempts
    * @param id Attempt ID
    * @param taskType Type of the task
@@ -74,7 +74,7 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
    * @param allSplits the "splits", or a pixelated graph of various
    *        measurable worker node state variables against progress.
    *        Currently there are four; wallclock time, CPU time,
-   *        virtual memory and physical memory.  
+   *        virtual memory and physical memory.
    */
   public TaskAttemptUnsuccessfulCompletionEvent
        (TaskAttemptID id, TaskType taskType,
@@ -101,7 +101,7 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
         ProgressSplitsBlock.arrayGetPhysMemKbytes(allSplits);
   }
 
-  /** 
+  /**
    * @deprecated please use the constructor with an additional
    *              argument, an array of splits arrays instead.  See
    *              {@link org.apache.hadoop.mapred.ProgressSplitsBlock}
@@ -117,19 +117,19 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
    */
   public TaskAttemptUnsuccessfulCompletionEvent
        (TaskAttemptID id, TaskType taskType,
-        String status, long finishTime, 
+        String status, long finishTime,
         String hostname, String error) {
     this(id, taskType, status, finishTime, hostname, -1, "",
         error, EMPTY_COUNTERS, null);
   }
-  
+
   public TaskAttemptUnsuccessfulCompletionEvent
       (TaskAttemptID id, TaskType taskType,
        String status, long finishTime,
        String hostname, int port, String rackName,
        String error, int[][] allSplits) {
     this(id, taskType, status, finishTime, hostname, port,
-        rackName, error, EMPTY_COUNTERS, null);
+        rackName, error, EMPTY_COUNTERS, allSplits);
   }
 
   TaskAttemptUnsuccessfulCompletionEvent() {}
@@ -162,9 +162,9 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
     }
     return datum;
   }
-  
-  
-  
+
+
+
   public void setDatum(Object odatum) {
     this.datum =
         (TaskAttemptUnsuccessfulCompletion)odatum;
@@ -208,12 +208,12 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
   public String getHostname() { return hostname; }
   /** Get the rpc port for the host where the attempt executed */
   public int getPort() { return port; }
-  
+
   /** Get the rack name of the node where the attempt ran */
   public String getRackName() {
     return rackName == null ? null : rackName.toString();
   }
-  
+
   /** Get the error string */
   public String getError() { return error.toString(); }
   /** Get the task status */
@@ -224,12 +224,12 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
   Counters getCounters() { return counters; }
   /** Get the event type */
   public EventType getEventType() {
-    // Note that the task type can be setup/map/reduce/cleanup but the 
+    // Note that the task type can be setup/map/reduce/cleanup but the
     // attempt-type can only be map/reduce.
     // find out if the task failed or got killed
     boolean failed = TaskStatus.State.FAILED.toString().equals(getTaskStatus());
-    return getTaskId().getTaskType() == TaskType.MAP 
-           ? (failed 
+    return getTaskId().getTaskType() == TaskType.MAP
+           ? (failed
               ? EventType.MAP_ATTEMPT_FAILED
               : EventType.MAP_ATTEMPT_KILLED)
            : (failed


---------------------------------------------------------------------
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: MAPREDUCE-6936. Remove unnecessary dependency of hadoop-yarn-server-common from hadoop-mapreduce-client-common (haibochen via rkanter)

Posted by ae...@apache.org.
MAPREDUCE-6936. Remove unnecessary dependency of hadoop-yarn-server-common from hadoop-mapreduce-client-common (haibochen via rkanter)


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

Branch: refs/heads/HDFS-7240
Commit: ab051bd42ee1d7c4d3b7cc71e6b2734a0955e767
Parents: 0acc5e0
Author: Robert Kanter <rk...@apache.org>
Authored: Wed Aug 16 16:14:04 2017 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Wed Aug 16 16:14:04 2017 -0700

----------------------------------------------------------------------
 .../hadoop-mapreduce-client-common/pom.xml                       | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab051bd4/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml
index db8ae49..b88b012 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml
@@ -46,10 +46,6 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-mapreduce-client-core</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-server-common</artifactId>
-    </dependency>
   </dependencies>
 
   <build>


---------------------------------------------------------------------
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: HDFS-12269. Better to return a Map rather than HashMap in getErasureCodingCodecs. Contributed by Huafeng Wang.

Posted by ae...@apache.org.
HDFS-12269. Better to return a Map rather than HashMap in getErasureCodingCodecs. Contributed by Huafeng Wang.


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

Branch: refs/heads/HDFS-7240
Commit: 08aaa4b36fab44c3f47878b3c487db3b373ffccf
Parents: ab051bd
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Aug 17 13:20:27 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Aug 17 13:20:27 2017 +0900

----------------------------------------------------------------------
 .../java/org/apache/hadoop/io/erasurecode/CodecRegistry.java    | 2 +-
 .../src/main/java/org/apache/hadoop/hdfs/DFSClient.java         | 2 +-
 .../main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java | 3 +--
 .../java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java    | 4 ++--
 .../hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java     | 5 +++--
 .../ClientNamenodeProtocolServerSideTranslatorPB.java           | 3 +--
 .../hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java       | 4 ++--
 .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java    | 4 ++--
 .../apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java   | 4 ++--
 .../src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java     | 3 +--
 .../java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java  | 4 ++--
 11 files changed, 18 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java
index fcf1349..daf91e2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java
@@ -176,7 +176,7 @@ public final class CodecRegistry {
    * @return a map of all codec names, and their corresponding code list
    * separated by ','.
    */
-  public HashMap<String, String> getCodec2CoderCompactMap() {
+  public Map<String, String> getCodec2CoderCompactMap() {
     return coderNameCompactMap;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 88b273a..969522d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -2764,7 +2764,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  public HashMap<String, String> getErasureCodingCodecs() throws IOException {
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
     checkOpen();
     try (TraceScope ignored = tracer.newScope("getErasureCodingCodecs")) {
       return namenode.getErasureCodingCodecs();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index cd368d4..8f82d03 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -26,7 +26,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -2585,7 +2584,7 @@ public class DistributedFileSystem extends FileSystem {
    * @return all erasure coding codecs and coders supported by this file system.
    * @throws IOException
    */
-  public HashMap<String, String> getAllErasureCodingCodecs()
+  public Map<String, String> getAllErasureCodingCodecs()
       throws IOException {
     return dfs.getErasureCodingCodecs();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 45c6b32..eb9380d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -19,8 +19,8 @@ package org.apache.hadoop.hdfs.protocol;
 
 import java.io.IOException;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -1601,7 +1601,7 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @Idempotent
-  HashMap<String, String> getErasureCodingCodecs() throws IOException;
+  Map<String, String> getErasureCodingCodecs() throws IOException;
 
   /**
    * Get the information about the EC policy for the path.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index aed4117..ac06c1a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -26,6 +26,7 @@ import java.util.List;
 
 import com.google.common.collect.Lists;
 
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
@@ -1760,11 +1761,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public HashMap<String, String> getErasureCodingCodecs() throws IOException {
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
     try {
       GetErasureCodingCodecsResponseProto response = rpcProxy
           .getErasureCodingCodecs(null, VOID_GET_EC_CODEC_REQUEST);
-      HashMap<String, String> ecCodecs = new HashMap<String, String>();
+      Map<String, String> ecCodecs = new HashMap<>();
       for (CodecProto codec : response.getCodecList()) {
         ecCodecs.put(codec.getCodec(), codec.getCoders());
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 38b81c6..a446276 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
@@ -1664,7 +1663,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, GetErasureCodingCodecsRequestProto request)
       throws ServiceException {
     try {
-      HashMap<String, String> codecs = server.getErasureCodingCodecs();
+      Map<String, String> codecs = server.getErasureCodingCodecs();
       GetErasureCodingCodecsResponseProto.Builder resBuilder =
           GetErasureCodingCodecsResponseProto.newBuilder();
       for (Map.Entry<String, String> codec : codecs.entrySet()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/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 486503c..7895433 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
@@ -25,8 +25,8 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.stream.Collectors;
 
 import com.google.common.base.Preconditions;
@@ -344,7 +344,7 @@ final class FSDirErasureCodingOp {
    * @param fsn namespace
    * @return {@link java.util.HashMap} array
    */
-  static HashMap<String, String> getErasureCodingCodecs(final FSNamesystem fsn)
+  static Map<String, String> getErasureCodingCodecs(final FSNamesystem fsn)
       throws IOException {
     assert fsn.hasReadLock();
     return CodecRegistry.getInstance().getCodec2CoderCompactMap();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/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 1cfaa54..2313335 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
@@ -7255,14 +7255,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Get available erasure coding codecs and corresponding coders.
    */
-  HashMap<String, String> getErasureCodingCodecs() throws IOException {
+  Map<String, String> getErasureCodingCodecs() throws IOException {
     final String operationName = "getErasureCodingCodecs";
     boolean success = false;
     checkOperation(OperationCategory.READ);
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      final HashMap<String, String> ret =
+      final Map<String, String> ret =
           FSDirErasureCodingOp.getErasureCodingCodecs(this);
       success = true;
       return ret;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/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 d304d3d..7871202 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
@@ -37,9 +37,9 @@ import java.net.InetSocketAddress;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
 
@@ -2278,7 +2278,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public HashMap<String, String> getErasureCodingCodecs() throws IOException {
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
     checkNNStartup();
     return namesystem.getErasureCodingCodecs();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
index 46600a0..17a84f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.util.ToolRunner;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -441,7 +440,7 @@ public class ECAdmin extends Configured implements Tool {
 
       final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       try {
-        HashMap<String, String> codecs =
+        Map<String, String> codecs =
             dfs.getAllErasureCodingCodecs();
         if (codecs.isEmpty()) {
           System.out.println("No erasure coding codecs are supported on the " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index 06edb1a..22e118f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -50,8 +50,8 @@ import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.*;
@@ -647,7 +647,7 @@ public class TestErasureCodingPolicies {
 
   @Test
   public void testGetAllErasureCodingCodecs() throws Exception {
-    HashMap<String, String> allECCodecs = fs
+    Map<String, String> allECCodecs = fs
         .getAllErasureCodingCodecs();
     assertTrue("At least 3 system codecs should be enabled",
         allECCodecs.size() >= 3);


---------------------------------------------------------------------
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-12221. Replace xcerces in XmlEditsVisitor. (Ajay Kumar via lei)

Posted by ae...@apache.org.
HDFS-12221. Replace xcerces in XmlEditsVisitor. (Ajay Kumar via lei)


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

Branch: refs/heads/HDFS-7240
Commit: ce797a170669524224cfeaaf70647047e7626816
Parents: d8f74c3
Author: Lei Xu <le...@apache.org>
Authored: Mon Aug 14 10:27:47 2017 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Mon Aug 14 10:27:47 2017 -0700

----------------------------------------------------------------------
 .../hadoop-client-minicluster/pom.xml           |   6 --
 .../hadoop-client-runtime/pom.xml               |   7 ---
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   5 --
 .../offlineEditsViewer/XmlEditsVisitor.java     |  41 ++++++++----
 .../hadoop-hdfs/src/test/resources/editsStored  | Bin 5850 -> 5850 bytes
 .../src/test/resources/editsStored.xml          |  62 +++++++++----------
 .../hadoop-mapreduce-client/pom.xml             |  10 +--
 hadoop-project-dist/pom.xml                     |  10 +--
 hadoop-project/pom.xml                          |   8 ---
 hadoop-yarn-project/hadoop-yarn/pom.xml         |  10 +--
 10 files changed, 62 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-client-modules/hadoop-client-minicluster/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-minicluster/pom.xml b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
index 5255640..5cf1fad 100644
--- a/hadoop-client-modules/hadoop-client-minicluster/pom.xml
+++ b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
@@ -629,12 +629,6 @@
                       </excludes>
                     </filter>
                     <filter>
-                      <artifact>xerces:xercesImpl</artifact>
-                      <excludes>
-                        <exclude>**/*</exclude>
-                      </excludes>
-                    </filter>
-                    <filter>
                       <artifact>org.apache.hadoop:hadoop-mapreduce-client-jobclient:*</artifact>
                       <excludes>
                         <exclude>testjar/*</exclude>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-client-modules/hadoop-client-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-runtime/pom.xml b/hadoop-client-modules/hadoop-client-runtime/pom.xml
index 2f64152..24c6b7a 100644
--- a/hadoop-client-modules/hadoop-client-runtime/pom.xml
+++ b/hadoop-client-modules/hadoop-client-runtime/pom.xml
@@ -174,13 +174,6 @@
                         <exclude>org/apache/jasper/compiler/Localizer.class</exclude>
                       </excludes>
                     </filter>
-                    <!-- We only have xerces as a dependency for XML output for the fsimage edits, we don't need anything specific to it for javax xml support -->
-                    <filter>
-                      <artifact>xerces:xercesImpl</artifact>
-                      <excludes>
-                        <exclude>META-INF/services/*</exclude>
-                      </excludes>
-                    </filter>
                     <!-- We rely on jersey for our web interfaces. We want to use its java services stuff only internal to jersey -->
                     <filter>
                       <artifact>com.sun.jersey:*</artifact>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index 1c50d31..fa1044d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -174,11 +174,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>xerces</groupId>
-      <artifactId>xercesImpl</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
       <groupId>org.apache.htrace</groupId>
       <artifactId>htrace-core4</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java
index 7a39ba6..ddf7933 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java
@@ -20,17 +20,21 @@ package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 import java.io.IOException;
 import java.io.OutputStream;
 
+import javax.xml.transform.OutputKeys;
+import javax.xml.transform.TransformerConfigurationException;
+import javax.xml.transform.sax.SAXTransformerFactory;
+import javax.xml.transform.sax.TransformerHandler;
+import javax.xml.transform.stream.StreamResult;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.AttributesImpl;
+
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.xml.sax.ContentHandler;
-import org.xml.sax.SAXException;
-import org.xml.sax.helpers.AttributesImpl;
 
-import org.apache.xml.serialize.OutputFormat;
-import org.apache.xml.serialize.XMLSerializer;
 
 /**
  * An XmlEditsVisitor walks over an EditLog structure and writes out
@@ -41,26 +45,37 @@ import org.apache.xml.serialize.XMLSerializer;
 public class XmlEditsVisitor implements OfflineEditsVisitor {
   private final OutputStream out;
   private ContentHandler contentHandler;
+  private final SAXTransformerFactory factory;
+  private final static String XML_INDENTATION_PROP ="{http://xml.apache.org/" +
+          "xslt}indent-amount";
+  private final static String XML_INDENTATION_NUM ="2";
 
   /**
    * Create a processor that writes to the file named and may or may not
    * also output to the screen, as specified.
    *
-   * @param filename Name of file to write output to
-   * @param printToScreen Mirror output to screen?
+   * @param out output stream to write
+   * @throws IOException on any error
    */
   public XmlEditsVisitor(OutputStream out)
       throws IOException {
     this.out = out;
-    OutputFormat outFormat = new OutputFormat("XML", "UTF-8", true);
-    outFormat.setIndenting(true);
-    outFormat.setIndent(2);
-    outFormat.setDoctype(null, null);
-    XMLSerializer serializer = new XMLSerializer(out, outFormat);
-    contentHandler = serializer.asContentHandler();
+    factory =(SAXTransformerFactory)SAXTransformerFactory.newInstance();
     try {
+      TransformerHandler handler = factory.newTransformerHandler();
+      handler.getTransformer().setOutputProperty(OutputKeys.METHOD, "xml");
+      handler.getTransformer().setOutputProperty(OutputKeys.ENCODING, "UTF-8");
+      handler.getTransformer().setOutputProperty(OutputKeys.INDENT, "yes");
+      handler.getTransformer().setOutputProperty(XML_INDENTATION_PROP,
+              XML_INDENTATION_NUM);
+      handler.getTransformer().setOutputProperty(OutputKeys.STANDALONE, "yes");
+      handler.setResult(new StreamResult(out));
+      contentHandler = handler;
+      
       contentHandler.startDocument();
       contentHandler.startElement("", "", "EDITS", new AttributesImpl());
+    } catch (TransformerConfigurationException e) {
+      throw new IOException("SAXTransformer error: " + e.getMessage());
     } catch (SAXException e) {
       throw new IOException("SAX error: " + e.getMessage());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
index a32cd98..e271cb5 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
index 201db9e..f901145 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
@@ -1,6 +1,6 @@
-<?xml version="1.0" encoding="UTF-8"?>
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
 <EDITS>
-  <EDITS_VERSION>-63</EDITS_VERSION>
+  <EDITS_VERSION>-64</EDITS_VERSION>
   <RECORD>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <DATA>
@@ -63,8 +63,8 @@
       <MTIME>1422406380369</MTIME>
       <ATIME>1422406380345</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>xyao</USERNAME>
@@ -96,8 +96,8 @@
       <MTIME>1422406380376</MTIME>
       <ATIME>1422406380345</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>xyao</USERNAME>
@@ -238,8 +238,8 @@
       <MTIME>1422406380425</MTIME>
       <ATIME>1422406380423</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>xyao</USERNAME>
@@ -360,7 +360,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1001</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -393,7 +393,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1002</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -426,7 +426,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1003</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -441,8 +441,8 @@
       <MTIME>1422406380534</MTIME>
       <ATIME>1422406380446</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741825</BLOCK_ID>
@@ -513,7 +513,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1004</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -546,7 +546,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1005</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -579,7 +579,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1006</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -594,8 +594,8 @@
       <MTIME>1422406380558</MTIME>
       <ATIME>1422406380537</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741828</BLOCK_ID>
@@ -666,7 +666,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1007</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -699,7 +699,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1008</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -732,7 +732,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1009</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -747,8 +747,8 @@
       <MTIME>1422406380579</MTIME>
       <ATIME>1422406380560</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741831</BLOCK_ID>
@@ -834,7 +834,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1010</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -867,7 +867,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1011</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -882,8 +882,8 @@
       <MTIME>1422406380599</MTIME>
       <ATIME>1422406380586</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741834</BLOCK_ID>
@@ -979,7 +979,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1012</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -993,7 +993,7 @@
         <NUM_BYTES>11</NUM_BYTES>
         <GENSTAMP>1012</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -1024,8 +1024,8 @@
       <MTIME>1422406383261</MTIME>
       <ATIME>1422406380608</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741836</BLOCK_ID>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
index 4e7a0ae..212078b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
@@ -238,14 +238,6 @@
                       <outputDirectory>${project.build.directory}</outputDirectory>
                       <destFileName>hadoop-annotations.jar</destFileName>
                     </artifactItem>
-                    <artifactItem>
-                      <groupId>xerces</groupId>
-                      <artifactId>xercesImpl</artifactId>
-                      <version>${xerces.version.jdiff}</version>
-                      <overWrite>false</overWrite>
-                      <outputDirectory>${project.build.directory}</outputDirectory>
-                      <destFileName>xerces.jar</destFileName>
-                    </artifactItem>
                   </artifactItems>
                 </configuration>
               </execution>
@@ -283,7 +275,7 @@
                            sourceFiles="${dev-support.relative.dir}/jdiff/Null.java"
                            maxmemory="${jdiff.javadoc.maxmemory}">
                     <doclet name="org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet"
-                            path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar:${project.build.directory}/xerces.jar">
+                            path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar">
                       <param name="-oldapi" value="${project.name} ${jdiff.stable.api}"/>
                       <param name="-newapi" value="${project.name} ${project.version}"/>
                       <param name="-oldapidir" value="${basedir}/${dev-support.relative.dir}/jdiff"/>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-project-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml
index 9da5e53..cf43218 100644
--- a/hadoop-project-dist/pom.xml
+++ b/hadoop-project-dist/pom.xml
@@ -194,14 +194,6 @@
                       <outputDirectory>${project.build.directory}</outputDirectory>
                       <destFileName>hadoop-annotations.jar</destFileName>
                     </artifactItem>
-                    <artifactItem>
-                      <groupId>xerces</groupId>
-                      <artifactId>xercesImpl</artifactId>
-                      <version>${xerces.jdiff.version}</version>
-                      <overWrite>false</overWrite>
-                      <outputDirectory>${project.build.directory}</outputDirectory>
-                      <destFileName>xerces.jar</destFileName>
-                    </artifactItem>
                   </artifactItems>
                 </configuration>
               </execution>
@@ -267,7 +259,7 @@
                              sourceFiles="${basedir}/dev-support/jdiff/Null.java"
                              maxmemory="${jdiff.javadoc.maxmemory}">
                       <doclet name="org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet"
-                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar:${project.build.directory}/xerces.jar">
+                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar">
                         <param name="-oldapi" value="${project.name} ${jdiff.stable.api}"/>
                         <param name="-newapi" value="${project.name} ${project.version}"/>
                         <param name="-oldapidir" value="${basedir}/dev-support/jdiff"/>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 8151016..6311cd9 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -45,8 +45,6 @@
     <!-- These 2 versions are defined here because they are used -->
     <!-- JDIFF generation from embedded ant in the antrun plugin -->
     <jdiff.version>1.0.9</jdiff.version>
-    <!-- Version number for xerces used by JDiff -->
-    <xerces.jdiff.version>2.11.0</xerces.jdiff.version>
 
     <kafka.version>0.8.2.1</kafka.version>
     <hbase.version>1.2.6</hbase.version>
@@ -1156,12 +1154,6 @@
      </dependency>
 
      <dependency>
-       <groupId>xerces</groupId>
-       <artifactId>xercesImpl</artifactId>
-       <version>2.9.1</version>
-     </dependency>
-
-     <dependency>
        <groupId>org.apache.curator</groupId>
        <artifactId>curator-recipes</artifactId>
        <version>${curator.version}</version>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-yarn-project/hadoop-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/pom.xml b/hadoop-yarn-project/hadoop-yarn/pom.xml
index 2e8ec69..12e4604 100644
--- a/hadoop-yarn-project/hadoop-yarn/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/pom.xml
@@ -155,14 +155,6 @@
                       <outputDirectory>${project.build.directory}</outputDirectory>
                       <destFileName>hadoop-annotations.jar</destFileName>
                     </artifactItem>
-                    <artifactItem>
-                      <groupId>xerces</groupId>
-                      <artifactId>xercesImpl</artifactId>
-                      <version>${xerces.version.jdiff}</version>
-                      <overWrite>false</overWrite>
-                      <outputDirectory>${project.build.directory}</outputDirectory>
-                      <destFileName>xerces.jar</destFileName>
-                    </artifactItem>
                   </artifactItems>
                 </configuration>
               </execution>
@@ -201,7 +193,7 @@
                       sourceFiles="${dev-support.relative.dir}/jdiff/Null.java"
                              maxmemory="${jdiff.javadoc.maxmemory}">
                       <doclet name="org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet"
-                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar:${project.build.directory}/xerces.jar">
+                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar">
                         <param name="-oldapi" value="${project.name} ${jdiff.stable.api}"/>
                         <param name="-newapi" value="${project.name} ${project.version}"/>
                         <param name="-oldapidir" value="${basedir}/${dev-support.relative.dir}/jdiff"/>


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


[08/50] [abbrv] hadoop git commit: HDFS-12162. Update listStatus document to describe the behavior when the argument is a file. Contributed by Ajay Kumar.

Posted by ae...@apache.org.
HDFS-12162. Update listStatus document to describe the behavior when the argument is a file. 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/d72124a4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d72124a4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d72124a4

Branch: refs/heads/HDFS-7240
Commit: d72124a44268e21ada036242bfbccafc23c52ed0
Parents: 18f3603
Author: Anu Engineer <ae...@apache.org>
Authored: Mon Aug 14 11:32:49 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Mon Aug 14 11:32:49 2017 -0700

----------------------------------------------------------------------
 .../hadoop/fs/http/server/FSOperations.java     |  2 +-
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    | 39 ++++++++++++++++++++
 2 files changed, 40 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d72124a4/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
index c008802..4b5918a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
@@ -669,7 +669,7 @@ public class FSOperations {
     /**
      * Creates a list-status executor.
      *
-     * @param path the directory to retrieve the status of its contents.
+     * @param path the directory/file to retrieve the status of its contents.
      * @param filter glob filter to use.
      *
      * @throws IOException thrown if the filter expression is incorrect.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d72124a4/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
index 7544c80..03834eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
@@ -495,6 +495,45 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getFileSt
 
 See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).listStatus
 
+### List a File
+
+* Submit a HTTP GET request.
+
+        curl -i  "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=LISTSTATUS"
+
+    The client receives a response with a [`FileStatuses` JSON object](#FileStatuses_JSON_Schema):
+
+        HTTP/1.1 200 OK
+        Content-Type: application/json
+        Content-Length: 427
+
+        {
+          "FileStatuses":
+          {
+            "FileStatus":
+            [
+              {
+                "accessTime"      : 1320171722771,
+                "blockSize"       : 33554432,
+                "childrenNum"     : 0,
+                "fileId"          : 16390,
+                "group"           : "supergroup",
+                "length"          : 1366,
+                "modificationTime": 1501770633062,
+                "owner"           : "webuser",
+                "pathSuffix"      : "",
+                "permission"      : "644",
+                "replication"     : 1,
+                "storagePolicy"   : 0,
+                "type"            : "FILE"
+              }
+            ]
+          }
+        }
+
+See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).listStatus
+
+
 ### Iteratively List a Directory
 
 * Submit a HTTP GET request.


---------------------------------------------------------------------
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: HADOOP-14194. Aliyun OSS should not use empty endpoint as default. Contributed by Genmao Yu

Posted by ae...@apache.org.
HADOOP-14194. Aliyun OSS should not use empty endpoint as default. Contributed by Genmao Yu


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

Branch: refs/heads/HDFS-7240
Commit: 267e19a09f366a965b30c8d4dc75e377b0d92fff
Parents: 7a82d7b
Author: Kai Zheng <ka...@intel.com>
Authored: Mon Aug 21 13:36:28 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Mon Aug 21 13:36:28 2017 +0800

----------------------------------------------------------------------
 .../apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java    | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/267e19a0/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
index a944fc1..a85a739 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
@@ -129,6 +129,10 @@ public class AliyunOSSFileSystemStore {
     }
 
     String endPoint = conf.getTrimmed(ENDPOINT_KEY, "");
+    if (StringUtils.isEmpty(endPoint)) {
+      throw new IllegalArgumentException("Aliyun OSS endpoint should not be " +
+        "null or empty. Please set proper endpoint with 'fs.oss.endpoint'.");
+    }
     CredentialsProvider provider =
         AliyunOSSUtils.getCredentialsProvider(conf);
     ossClient = new OSSClient(endPoint, provider, clientConf);


---------------------------------------------------------------------
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: HDFS-11303. Hedged read might hang infinitely if read data from all DN failed . Contributed by Chen Zhang, Wei-chiu Chuang, and John Zhuge.

Posted by ae...@apache.org.
HDFS-11303. Hedged read might hang infinitely if read data from all DN failed . Contributed by Chen Zhang, Wei-chiu Chuang, and John Zhuge.


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

Branch: refs/heads/HDFS-7240
Commit: 8b242f09a61a7536d2422546bfa6c2aaf1d57ed6
Parents: 28d97b7
Author: John Zhuge <jz...@cloudera.com>
Authored: Thu Aug 10 14:04:36 2017 -0700
Committer: John Zhuge <jz...@apache.org>
Committed: Fri Aug 11 19:42:07 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 11 ++--
 .../java/org/apache/hadoop/hdfs/TestPread.java  | 63 ++++++++++++++++++++
 2 files changed, 70 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b242f09/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index dcc997c..6bff172 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -1131,8 +1131,9 @@ public class DFSInputStream extends FSInputStream
         Future<ByteBuffer> firstRequest = hedgedService
             .submit(getFromDataNodeCallable);
         futures.add(firstRequest);
+        Future<ByteBuffer> future = null;
         try {
-          Future<ByteBuffer> future = hedgedService.poll(
+          future = hedgedService.poll(
               conf.getHedgedReadThresholdMillis(), TimeUnit.MILLISECONDS);
           if (future != null) {
             ByteBuffer result = future.get();
@@ -1142,16 +1143,18 @@ public class DFSInputStream extends FSInputStream
           }
           DFSClient.LOG.debug("Waited {}ms to read from {}; spawning hedged "
               + "read", conf.getHedgedReadThresholdMillis(), chosenNode.info);
-          // Ignore this node on next go around.
-          ignored.add(chosenNode.info);
           dfsClient.getHedgedReadMetrics().incHedgedReadOps();
           // continue; no need to refresh block locations
         } catch (ExecutionException e) {
-          // Ignore
+          futures.remove(future);
         } catch (InterruptedException e) {
           throw new InterruptedIOException(
               "Interrupted while waiting for reading task");
         }
+        // Ignore this node on next go around.
+        // If poll timeout and the request still ongoing, don't consider it
+        // again. If read data failed, don't consider it either.
+        ignored.add(chosenNode.info);
       } else {
         // We are starting up a 'hedged' read. We have a read already
         // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b242f09/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
index 85fc97b..bcb02b3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
@@ -59,6 +59,8 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 import com.google.common.base.Supplier;
+import org.slf4j.LoggerFactory;
+import org.slf4j.Logger;
 
 /**
  * This class tests the DFS positional read functionality in a single node
@@ -72,6 +74,9 @@ public class TestPread {
   boolean simulatedStorage;
   boolean isHedgedRead;
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestPread.class.getName());
+
   @Before
   public void setup() {
     simulatedStorage = false;
@@ -551,6 +556,64 @@ public class TestPread {
     }
   }
 
+  @Test(timeout=30000)
+  public void testHedgedReadFromAllDNFailed() throws IOException {
+    Configuration conf = new Configuration();
+    int numHedgedReadPoolThreads = 5;
+    final int hedgedReadTimeoutMillis = 50;
+
+    conf.setInt(HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY,
+        numHedgedReadPoolThreads);
+    conf.setLong(HdfsClientConfigKeys.HedgedRead.THRESHOLD_MILLIS_KEY,
+        hedgedReadTimeoutMillis);
+    conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 0);
+    // Set up the InjectionHandler
+    DFSClientFaultInjector.set(Mockito.mock(DFSClientFaultInjector.class));
+    DFSClientFaultInjector injector = DFSClientFaultInjector.get();
+    Mockito.doAnswer(new Answer<Void>() {
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        if (true) {
+          LOG.info("-------------- throw Checksum Exception");
+          throw new ChecksumException("ChecksumException test", 100);
+        }
+        return null;
+      }
+    }).when(injector).fetchFromDatanodeException();
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
+        .format(true).build();
+    DistributedFileSystem fileSys = cluster.getFileSystem();
+    DFSClient dfsClient = fileSys.getClient();
+    FSDataOutputStream output = null;
+    DFSInputStream input = null;
+    String filename = "/hedgedReadMaxOut.dat";
+    DFSHedgedReadMetrics metrics = dfsClient.getHedgedReadMetrics();
+    // Metrics instance is static, so we need to reset counts from prior tests.
+    metrics.hedgedReadOps.set(0);
+    try {
+      Path file = new Path(filename);
+      output = fileSys.create(file, (short) 2);
+      byte[] data = new byte[64 * 1024];
+      output.write(data);
+      output.flush();
+      output.close();
+      byte[] buffer = new byte[64 * 1024];
+      input = dfsClient.open(filename);
+      input.read(0, buffer, 0, 1024);
+      Assert.fail("Reading the block should have thrown BlockMissingException");
+    } catch (BlockMissingException e) {
+      assertEquals(3, input.getHedgedReadOpsLoopNumForTesting());
+      assertTrue(metrics.getHedgedReadOps() == 0);
+    } finally {
+      Mockito.reset(injector);
+      IOUtils.cleanupWithLogger(LOG, input);
+      IOUtils.cleanupWithLogger(LOG, output);
+      fileSys.close();
+      cluster.shutdown();
+    }
+  }
+
   /**
    * Scenario: 1. Write a file with RF=2, DN1 and DN2<br>
    * 2. Open the stream, Consider Locations are [DN1, DN2] in LocatedBlock.<br>


---------------------------------------------------------------------
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: YARN-5978. ContainerScheduler and ContainerManager changes to support ExecType update. (Kartheek Muthyala via asuresh)

Posted by ae...@apache.org.
YARN-5978. ContainerScheduler and ContainerManager changes to support ExecType update. (Kartheek Muthyala via asuresh)


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

Branch: refs/heads/HDFS-7240
Commit: 4d7be1d8575e9254c59d41460960708e3718503a
Parents: 0446511
Author: Arun Suresh <as...@apache.org>
Authored: Mon Aug 14 19:46:17 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Mon Aug 14 19:46:17 2017 -0700

----------------------------------------------------------------------
 .../yarn/client/api/impl/TestAMRMClient.java    | 395 +++++++++++++++++--
 .../yarn/client/api/impl/TestNMClient.java      |   7 +-
 .../containermanager/ContainerManagerImpl.java  | 132 ++++---
 .../containermanager/container/Container.java   |   4 +-
 .../container/ContainerImpl.java                |  37 +-
 .../monitor/ContainersMonitorImpl.java          |  15 -
 .../scheduler/ContainerScheduler.java           |  73 ++++
 .../scheduler/ContainerSchedulerEventType.java  |   1 +
 .../UpdateContainerSchedulerEvent.java          |  85 ++++
 .../nodemanager/TestNodeManagerResync.java      |  11 +-
 .../BaseContainerManagerTest.java               |  33 +-
 .../containermanager/TestContainerManager.java  | 267 ++++++++-----
 .../TestContainerManagerRecovery.java           |   2 +-
 .../TestContainerSchedulerQueuing.java          |  96 +++++
 .../nodemanager/webapp/MockContainer.java       |   2 +-
 .../scheduler/SchedulerApplicationAttempt.java  |   2 +-
 .../security/RMContainerTokenSecretManager.java |  30 +-
 17 files changed, 964 insertions(+), 228 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
index 1b2bca3..09b12f2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.client.api.impl;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
@@ -36,6 +37,7 @@ 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.TreeSet;
 
@@ -142,6 +144,10 @@ public class TestAMRMClient {
     // set the minimum allocation so that resource decrease can go under 1024
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512);
     conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1);
+    conf.setBoolean(
+        YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
+    conf.setInt(
+        YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, 10);
     yarnCluster = new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1);
     yarnCluster.init(conf);
     yarnCluster.start();
@@ -924,8 +930,8 @@ public class TestAMRMClient {
     // add exp=x to ANY
     client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
         1), null, null, Priority.UNDEFINED, true, "x"));
-    Assert.assertEquals(1, client.ask.size());
-    Assert.assertEquals("x", client.ask.iterator().next()
+    assertEquals(1, client.ask.size());
+    assertEquals("x", client.ask.iterator().next()
         .getNodeLabelExpression());
 
     // add exp=x then add exp=a to ANY in same priority, only exp=a should kept
@@ -933,8 +939,8 @@ public class TestAMRMClient {
         1), null, null, Priority.UNDEFINED, true, "x"));
     client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
         1), null, null, Priority.UNDEFINED, true, "a"));
-    Assert.assertEquals(1, client.ask.size());
-    Assert.assertEquals("a", client.ask.iterator().next()
+    assertEquals(1, client.ask.size());
+    assertEquals("a", client.ask.iterator().next()
         .getNodeLabelExpression());
     
     // add exp=x to ANY, rack and node, only resource request has ANY resource
@@ -943,10 +949,10 @@ public class TestAMRMClient {
     client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
         1), null, null, Priority.UNDEFINED, true,
         "y"));
-    Assert.assertEquals(1, client.ask.size());
+    assertEquals(1, client.ask.size());
     for (ResourceRequest req : client.ask) {
       if (ResourceRequest.ANY.equals(req.getResourceName())) {
-        Assert.assertEquals("y", req.getNodeLabelExpression());
+        assertEquals("y", req.getNodeLabelExpression());
       } else {
         Assert.assertNull(req.getNodeLabelExpression());
       }
@@ -957,7 +963,7 @@ public class TestAMRMClient {
         new String[] { "node1", "node2" }, Priority.UNDEFINED, true, "y"));
     for (ResourceRequest req : client.ask) {
       if (ResourceRequest.ANY.equals(req.getResourceName())) {
-        Assert.assertEquals("y", req.getNodeLabelExpression());
+        assertEquals("y", req.getNodeLabelExpression());
       } else {
         Assert.assertNull(req.getNodeLabelExpression());
       }
@@ -971,7 +977,7 @@ public class TestAMRMClient {
     } catch (InvalidContainerRequestException e) {
       return;
     }
-    Assert.fail();
+    fail();
   }
   
   @Test(timeout=30000)
@@ -1042,7 +1048,8 @@ public class TestAMRMClient {
     // get allocations
     AllocateResponse allocResponse = amClient.allocate(0.1f);
     List<Container> containers = allocResponse.getAllocatedContainers();
-    Assert.assertEquals(num, containers.size());
+    assertEquals(num, containers.size());
+
     // build container launch context
     Credentials ts = new Credentials();
     DataOutputBuffer dob = new DataOutputBuffer();
@@ -1083,14 +1090,14 @@ public class TestAMRMClient {
   private void doContainerResourceChange(
       final AMRMClient<ContainerRequest> amClient, List<Container> containers)
       throws YarnException, IOException {
-    Assert.assertEquals(3, containers.size());
+    assertEquals(3, containers.size());
     // remember the container IDs
     Container container1 = containers.get(0);
     Container container2 = containers.get(1);
     Container container3 = containers.get(2);
     AMRMClientImpl<ContainerRequest> amClientImpl =
         (AMRMClientImpl<ContainerRequest>) amClient;
-    Assert.assertEquals(0, amClientImpl.change.size());
+    assertEquals(0, amClientImpl.change.size());
     // verify newer request overwrites older request for the container1
     amClientImpl.requestContainerUpdate(container1,
         UpdateContainerRequest.newInstance(container1.getVersion(),
@@ -1100,21 +1107,21 @@ public class TestAMRMClient {
         UpdateContainerRequest.newInstance(container1.getVersion(),
             container1.getId(), ContainerUpdateType.INCREASE_RESOURCE,
             Resource.newInstance(4096, 1), null));
-    Assert.assertEquals(Resource.newInstance(4096, 1),
+    assertEquals(Resource.newInstance(4096, 1),
         amClientImpl.change.get(container1.getId()).getValue().getCapability());
     // verify new decrease request cancels old increase request for container1
     amClientImpl.requestContainerUpdate(container1,
         UpdateContainerRequest.newInstance(container1.getVersion(),
             container1.getId(), ContainerUpdateType.DECREASE_RESOURCE,
             Resource.newInstance(512, 1), null));
-    Assert.assertEquals(Resource.newInstance(512, 1),
+    assertEquals(Resource.newInstance(512, 1),
         amClientImpl.change.get(container1.getId()).getValue().getCapability());
     // request resource increase for container2
     amClientImpl.requestContainerUpdate(container2,
         UpdateContainerRequest.newInstance(container2.getVersion(),
             container2.getId(), ContainerUpdateType.INCREASE_RESOURCE,
             Resource.newInstance(2048, 1), null));
-    Assert.assertEquals(Resource.newInstance(2048, 1),
+    assertEquals(Resource.newInstance(2048, 1),
         amClientImpl.change.get(container2.getId()).getValue().getCapability());
     // verify release request will cancel pending change requests for the same
     // container
@@ -1122,27 +1129,357 @@ public class TestAMRMClient {
         UpdateContainerRequest.newInstance(container3.getVersion(),
             container3.getId(), ContainerUpdateType.INCREASE_RESOURCE,
             Resource.newInstance(2048, 1), null));
-    Assert.assertEquals(3, amClientImpl.pendingChange.size());
+    assertEquals(3, amClientImpl.pendingChange.size());
     amClientImpl.releaseAssignedContainer(container3.getId());
-    Assert.assertEquals(2, amClientImpl.pendingChange.size());
+    assertEquals(2, amClientImpl.pendingChange.size());
     // as of now: container1 asks to decrease to (512, 1)
     //            container2 asks to increase to (2048, 1)
     // send allocation requests
     AllocateResponse allocResponse = amClient.allocate(0.1f);
-    Assert.assertEquals(0, amClientImpl.change.size());
+    assertEquals(0, amClientImpl.change.size());
     // we should get decrease confirmation right away
     List<UpdatedContainer> updatedContainers =
         allocResponse.getUpdatedContainers();
-    Assert.assertEquals(1, updatedContainers.size());
+    assertEquals(1, updatedContainers.size());
     // we should get increase allocation after the next NM's heartbeat to RM
     triggerSchedulingWithNMHeartBeat();
     // get allocations
     allocResponse = amClient.allocate(0.1f);
     updatedContainers =
         allocResponse.getUpdatedContainers();
-    Assert.assertEquals(1, updatedContainers.size());
+    assertEquals(1, updatedContainers.size());
+  }
+
+  @Test(timeout=60000)
+  public void testAMRMClientWithContainerPromotion()
+      throws YarnException, IOException {
+    AMRMClientImpl<AMRMClient.ContainerRequest> amClient =
+        (AMRMClientImpl<AMRMClient.ContainerRequest>) AMRMClient
+            .createAMRMClient();
+    //asserting we are not using the singleton instance cache
+    Assert.assertSame(NMTokenCache.getSingleton(),
+        amClient.getNMTokenCache());
+    amClient.init(conf);
+    amClient.start();
+
+    // start am nm client
+    NMClientImpl nmClient = (NMClientImpl) NMClient.createNMClient();
+    Assert.assertNotNull(nmClient);
+    // asserting we are using the singleton instance cache
+    Assert.assertSame(
+        NMTokenCache.getSingleton(), nmClient.getNMTokenCache());
+    nmClient.init(conf);
+    nmClient.start();
+    assertEquals(STATE.STARTED, nmClient.getServiceState());
+
+    amClient.registerApplicationMaster("Host", 10000, "");
+    // setup container request
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // START OPPORTUNISTIC Container, Send allocation request to RM
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, null, null, priority2, 0,
+            true, null, ExecutionTypeRequest
+            .newInstance(ExecutionType.OPPORTUNISTIC, true)));
+
+    int oppContainersRequestedAny =
+        amClient.getTable(0).get(priority2, ResourceRequest.ANY,
+            ExecutionType.OPPORTUNISTIC, capability).remoteRequest
+            .getNumContainers();
+
+    assertEquals(1, oppContainersRequestedAny);
+    assertEquals(1, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // RM should allocate container within 2 calls to allocate()
+    int allocatedContainerCount = 0;
+    Map<ContainerId, Container> allocatedOpportContainers = new HashMap<>();
+    int iterationsLeft = 50;
+
+    amClient.getNMTokenCache().clearCache();
+    assertEquals(0,
+        amClient.getNMTokenCache().numberOfTokensInCache());
+
+    AllocateResponse allocResponse = null;
+    while (allocatedContainerCount < oppContainersRequestedAny
+        && iterationsLeft-- > 0) {
+      allocResponse = amClient.allocate(0.1f);
+      // let NM heartbeat to RM and trigger allocations
+      //triggerSchedulingWithNMHeartBeat();
+      assertEquals(0, amClient.ask.size());
+      assertEquals(0, amClient.release.size());
+
+      allocatedContainerCount +=
+          allocResponse.getAllocatedContainers().size();
+      for (Container container : allocResponse.getAllocatedContainers()) {
+        if (container.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
+          allocatedOpportContainers.put(container.getId(), container);
+        }
+      }
+      if (allocatedContainerCount < oppContainersRequestedAny) {
+        // sleep to let NM's heartbeat to RM and trigger allocations
+        sleep(100);
+      }
+    }
+
+    assertEquals(oppContainersRequestedAny, allocatedContainerCount);
+    assertEquals(oppContainersRequestedAny, allocatedOpportContainers.size());
+
+    startContainer(allocResponse, nmClient);
+
+    // SEND PROMOTION REQUEST TO RM
+    try {
+      Container c = allocatedOpportContainers.values().iterator().next();
+      amClient.requestContainerUpdate(
+          c, UpdateContainerRequest.newInstance(c.getVersion(),
+              c.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+              null, ExecutionType.OPPORTUNISTIC));
+      fail("Should throw Exception..");
+    } catch (IllegalArgumentException e) {
+      System.out.println("## " + e.getMessage());
+      assertTrue(e.getMessage().contains(
+          "target should be GUARANTEED and original should be OPPORTUNISTIC"));
+    }
+
+    Container c = allocatedOpportContainers.values().iterator().next();
+    amClient.requestContainerUpdate(
+        c, UpdateContainerRequest.newInstance(c.getVersion(),
+            c.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+            null, ExecutionType.GUARANTEED));
+    iterationsLeft = 120;
+    Map<ContainerId, UpdatedContainer> updatedContainers = new HashMap<>();
+    // do a few iterations to ensure RM is not going to send new containers
+    while (iterationsLeft-- > 0 && updatedContainers.isEmpty()) {
+      // inform RM of rejection
+      allocResponse = amClient.allocate(0.1f);
+      // RM did not send new containers because AM does not need any
+      if (allocResponse.getUpdatedContainers() != null) {
+        for (UpdatedContainer updatedContainer : allocResponse
+            .getUpdatedContainers()) {
+          System.out.println("Got update..");
+          updatedContainers.put(updatedContainer.getContainer().getId(),
+              updatedContainer);
+        }
+      }
+      if (iterationsLeft > 0) {
+        // sleep to make sure NM's heartbeat
+        sleep(100);
+      }
+    }
+    assertEquals(1, updatedContainers.size());
+
+    for (ContainerId cId : allocatedOpportContainers.keySet()) {
+      Container orig = allocatedOpportContainers.get(cId);
+      UpdatedContainer updatedContainer = updatedContainers.get(cId);
+      assertNotNull(updatedContainer);
+      assertEquals(ExecutionType.GUARANTEED,
+          updatedContainer.getContainer().getExecutionType());
+      assertEquals(orig.getResource(),
+          updatedContainer.getContainer().getResource());
+      assertEquals(orig.getNodeId(),
+          updatedContainer.getContainer().getNodeId());
+      assertEquals(orig.getVersion() + 1,
+          updatedContainer.getContainer().getVersion());
+    }
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // SEND UPDATE EXECTYPE UPDATE TO NM
+    updateContainerExecType(allocResponse, ExecutionType.GUARANTEED, nmClient);
+
+    amClient.ask.clear();
+  }
+
+  @Test(timeout=60000)
+  public void testAMRMClientWithContainerDemotion()
+      throws YarnException, IOException {
+    AMRMClientImpl<AMRMClient.ContainerRequest> amClient =
+        (AMRMClientImpl<AMRMClient.ContainerRequest>) AMRMClient
+            .createAMRMClient();
+    //asserting we are not using the singleton instance cache
+    Assert.assertSame(NMTokenCache.getSingleton(),
+        amClient.getNMTokenCache());
+    amClient.init(conf);
+    amClient.start();
+
+    NMClientImpl nmClient = (NMClientImpl) NMClient.createNMClient();
+    Assert.assertNotNull(nmClient);
+    // asserting we are using the singleton instance cache
+    Assert.assertSame(
+        NMTokenCache.getSingleton(), nmClient.getNMTokenCache());
+    nmClient.init(conf);
+    nmClient.start();
+    assertEquals(STATE.STARTED, nmClient.getServiceState());
+
+    amClient.registerApplicationMaster("Host", 10000, "");
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // START OPPORTUNISTIC Container, Send allocation request to RM
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, null, null, priority2, 0,
+            true, null, ExecutionTypeRequest
+            .newInstance(ExecutionType.GUARANTEED, true)));
+
+    int oppContainersRequestedAny =
+        amClient.getTable(0).get(priority2, ResourceRequest.ANY,
+            ExecutionType.GUARANTEED, capability).remoteRequest
+            .getNumContainers();
+
+    assertEquals(1, oppContainersRequestedAny);
+    assertEquals(1, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // RM should allocate container within 2 calls to allocate()
+    int allocatedContainerCount = 0;
+    Map<ContainerId, Container> allocatedGuaranteedContainers = new HashMap<>();
+    int iterationsLeft = 50;
+
+    amClient.getNMTokenCache().clearCache();
+    assertEquals(0,
+        amClient.getNMTokenCache().numberOfTokensInCache());
+
+    AllocateResponse allocResponse = null;
+    while (allocatedContainerCount < oppContainersRequestedAny
+        && iterationsLeft-- > 0) {
+      allocResponse = amClient.allocate(0.1f);
+      // let NM heartbeat to RM and trigger allocations
+      //triggerSchedulingWithNMHeartBeat();
+      assertEquals(0, amClient.ask.size());
+      assertEquals(0, amClient.release.size());
+
+      allocatedContainerCount +=
+          allocResponse.getAllocatedContainers().size();
+      for (Container container : allocResponse.getAllocatedContainers()) {
+        if (container.getExecutionType() == ExecutionType.GUARANTEED) {
+          allocatedGuaranteedContainers.put(container.getId(), container);
+        }
+      }
+      if (allocatedContainerCount < oppContainersRequestedAny) {
+        // sleep to let NM's heartbeat to RM and trigger allocations
+        sleep(100);
+      }
+    }
+    assertEquals(oppContainersRequestedAny, allocatedContainerCount);
+    assertEquals(oppContainersRequestedAny,
+        allocatedGuaranteedContainers.size());
+    startContainer(allocResponse, nmClient);
+
+    // SEND DEMOTION REQUEST TO RM
+    try {
+      Container c = allocatedGuaranteedContainers.values().iterator().next();
+      amClient.requestContainerUpdate(
+          c, UpdateContainerRequest.newInstance(c.getVersion(),
+              c.getId(), ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
+              null, ExecutionType.GUARANTEED));
+      fail("Should throw Exception..");
+    } catch (IllegalArgumentException e) {
+      System.out.println("## " + e.getMessage());
+      assertTrue(e.getMessage().contains(
+          "target should be OPPORTUNISTIC and original should be GUARANTEED"));
+    }
+
+    Container c = allocatedGuaranteedContainers.values().iterator().next();
+    amClient.requestContainerUpdate(
+        c, UpdateContainerRequest.newInstance(c.getVersion(),
+            c.getId(), ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
+            null, ExecutionType.OPPORTUNISTIC));
+    iterationsLeft = 120;
+    Map<ContainerId, UpdatedContainer> updatedContainers = new HashMap<>();
+    // do a few iterations to ensure RM is not going to send new containers
+    while (iterationsLeft-- > 0 && updatedContainers.isEmpty()) {
+      // inform RM of rejection
+      allocResponse = amClient.allocate(0.1f);
+      // RM did not send new containers because AM does not need any
+      if (allocResponse.getUpdatedContainers() != null) {
+        for (UpdatedContainer updatedContainer : allocResponse
+            .getUpdatedContainers()) {
+          System.out.println("Got update..");
+          updatedContainers.put(updatedContainer.getContainer().getId(),
+              updatedContainer);
+        }
+      }
+      if (iterationsLeft > 0) {
+        // sleep to make sure NM's heartbeat
+        sleep(100);
+      }
+    }
+    assertEquals(1, updatedContainers.size());
+
+    for (ContainerId cId : allocatedGuaranteedContainers.keySet()) {
+      Container orig = allocatedGuaranteedContainers.get(cId);
+      UpdatedContainer updatedContainer = updatedContainers.get(cId);
+      assertNotNull(updatedContainer);
+      assertEquals(ExecutionType.OPPORTUNISTIC,
+          updatedContainer.getContainer().getExecutionType());
+      assertEquals(orig.getResource(),
+          updatedContainer.getContainer().getResource());
+      assertEquals(orig.getNodeId(),
+          updatedContainer.getContainer().getNodeId());
+      assertEquals(orig.getVersion() + 1,
+          updatedContainer.getContainer().getVersion());
+    }
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    updateContainerExecType(allocResponse, ExecutionType.OPPORTUNISTIC,
+        nmClient);
+    amClient.ask.clear();
+  }
+
+  private void updateContainerExecType(AllocateResponse allocResponse,
+      ExecutionType expectedExecType, NMClientImpl nmClient)
+      throws IOException, YarnException {
+    for (UpdatedContainer updatedContainer : allocResponse
+        .getUpdatedContainers()) {
+      Container container = updatedContainer.getContainer();
+      nmClient.increaseContainerResource(container);
+      // NodeManager may still need some time to get the stable
+      // container status
+      while (true) {
+        ContainerStatus status = nmClient
+            .getContainerStatus(container.getId(), container.getNodeId());
+        if (status.getExecutionType() == expectedExecType) {
+          break;
+        }
+        sleep(10);
+      }
+    }
+  }
+
+  private void startContainer(AllocateResponse allocResponse,
+      NMClientImpl nmClient) throws IOException, YarnException {
+    // START THE CONTAINER IN NM
+    // build container launch context
+    Credentials ts = new Credentials();
+    DataOutputBuffer dob = new DataOutputBuffer();
+    ts.writeTokenStorageToStream(dob);
+    ByteBuffer securityTokens =
+        ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+    // start a process long enough for increase/decrease action to take effect
+    ContainerLaunchContext clc = BuilderUtils.newContainerLaunchContext(
+        Collections.<String, LocalResource>emptyMap(),
+        new HashMap<String, String>(), Arrays.asList("sleep", "100"),
+        new HashMap<String, ByteBuffer>(), securityTokens,
+        new HashMap<ApplicationAccessType, String>());
+    // start the containers and make sure they are in RUNNING state
+    for (Container container : allocResponse.getAllocatedContainers()) {
+      nmClient.startContainer(container, clc);
+      // NodeManager may still need some time to get the stable
+      // container status
+      while (true) {
+        ContainerStatus status = nmClient
+            .getContainerStatus(container.getId(), container.getNodeId());
+        if (status.getState() == ContainerState.RUNNING) {
+          break;
+        }
+        sleep(10);
+      }
+    }
   }
 
+
   private void testAllocation(final AMRMClientImpl<ContainerRequest> amClient)
       throws YarnException, IOException {
     // setup container request
@@ -1172,7 +1509,7 @@ public class TestAMRMClient {
     Set<ContainerId> releases = new TreeSet<ContainerId>();
     
     amClient.getNMTokenCache().clearCache();
-    Assert.assertEquals(0, amClient.getNMTokenCache().numberOfTokensInCache());
+    assertEquals(0, amClient.getNMTokenCache().numberOfTokensInCache());
     HashMap<String, Token> receivedNMTokens = new HashMap<String, Token>();
     
     while (allocatedContainerCount < containersRequestedAny
@@ -1192,7 +1529,7 @@ public class TestAMRMClient {
       for (NMToken token : allocResponse.getNMTokens()) {
         String nodeID = token.getNodeId().toString();
         if (receivedNMTokens.containsKey(nodeID)) {
-          Assert.fail("Received token again for : " + nodeID);          
+          fail("Received token again for : " + nodeID);
         }
         receivedNMTokens.put(nodeID, token.getToken());
       }
@@ -1204,7 +1541,7 @@ public class TestAMRMClient {
     }
     
     // Should receive atleast 1 token
-    Assert.assertTrue(receivedNMTokens.size() > 0
+    assertTrue(receivedNMTokens.size() > 0
         && receivedNMTokens.size() <= nodeCount);
     
     assertEquals(allocatedContainerCount, containersRequestedAny);
@@ -1444,7 +1781,7 @@ public class TestAMRMClient {
       org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> amrmToken_1 =
           getAMRMToken();
       Assert.assertNotNull(amrmToken_1);
-      Assert.assertEquals(amrmToken_1.decodeIdentifier().getKeyId(),
+      assertEquals(amrmToken_1.decodeIdentifier().getKeyId(),
         amrmTokenSecretManager.getMasterKey().getMasterKey().getKeyId());
 
       // Wait for enough time and make sure the roll_over happens
@@ -1459,7 +1796,7 @@ public class TestAMRMClient {
       org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> amrmToken_2 =
           getAMRMToken();
       Assert.assertNotNull(amrmToken_2);
-      Assert.assertEquals(amrmToken_2.decodeIdentifier().getKeyId(),
+      assertEquals(amrmToken_2.decodeIdentifier().getKeyId(),
         amrmTokenSecretManager.getMasterKey().getMasterKey().getKeyId());
 
       Assert.assertNotEquals(amrmToken_1, amrmToken_2);
@@ -1474,7 +1811,7 @@ public class TestAMRMClient {
       AMRMTokenIdentifierForTest newVersionTokenIdentifier = 
           new AMRMTokenIdentifierForTest(amrmToken_2.decodeIdentifier(), "message");
       
-      Assert.assertEquals("Message is changed after set to newVersionTokenIdentifier",
+      assertEquals("Message is changed after set to newVersionTokenIdentifier",
           "message", newVersionTokenIdentifier.getMessage());
       org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> newVersionToken = 
           new org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> (
@@ -1530,10 +1867,10 @@ public class TestAMRMClient {
                 .getBindAddress(), conf);
           }
         }).allocate(Records.newRecord(AllocateRequest.class));
-        Assert.fail("The old Token should not work");
+        fail("The old Token should not work");
       } catch (Exception ex) {
-        Assert.assertTrue(ex instanceof InvalidToken);
-        Assert.assertTrue(ex.getMessage().contains(
+        assertTrue(ex instanceof InvalidToken);
+        assertTrue(ex.getMessage().contains(
           "Invalid AMRMToken from "
               + amrmToken_2.decodeIdentifier().getApplicationAttemptId()));
       }
@@ -1560,7 +1897,7 @@ public class TestAMRMClient {
       org.apache.hadoop.security.token.Token<?> token = iter.next();
       if (token.getKind().equals(AMRMTokenIdentifier.KIND_NAME)) {
         if (result != null) {
-          Assert.fail("credentials has more than one AMRM token."
+          fail("credentials has more than one AMRM token."
               + " token1: " + result + " token2: " + token);
         }
         result = (org.apache.hadoop.security.token.Token<AMRMTokenIdentifier>)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
index 6bd0816..9b79e2d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
@@ -301,7 +301,6 @@ public class TestNMClient {
         assertTrue("The thrown exception is not expected",
             e.getMessage().contains("is not handled by this NodeManager"));
       }
-
       // increaseContainerResource shouldn't be called before startContainer,
       // otherwise, NodeManager cannot find the container
       try {
@@ -475,10 +474,10 @@ public class TestNMClient {
     try {
       nmClient.increaseContainerResource(container);
     } catch (YarnException e) {
-      // NM container will only be in SCHEDULED state, so expect the increase
-      // action to fail.
+      // NM container increase container resource should fail without a version
+      // increase action to fail.
       if (!e.getMessage().contains(
-          "can only be changed when a container is in RUNNING state")) {
+          container.getId() + " has update version ")) {
         throw (AssertionError)
             (new AssertionError("Exception is not expected: " + e)
                 .initCause(e));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/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 84ed3c1..a1e8ca0 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
@@ -66,6 +66,7 @@ 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.ExecutionType;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
@@ -136,13 +137,14 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.LogHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.NonAggregatingLogHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEventType;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ChangeMonitoringContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
 
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerScheduler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerSchedulerEventType;
+
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.UpdateContainerSchedulerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredApplicationsState;
@@ -410,8 +412,24 @@ public class ContainerManagerImpl extends CompositeService implements
       throws IOException {
     StartContainerRequest req = rcs.getStartRequest();
     ContainerLaunchContext launchContext = req.getContainerLaunchContext();
-    ContainerTokenIdentifier token =
-        BuilderUtils.newContainerTokenIdentifier(req.getContainerToken());
+    ContainerTokenIdentifier token = null;
+    if(rcs.getCapability() != null) {
+      ContainerTokenIdentifier originalToken =
+          BuilderUtils.newContainerTokenIdentifier(req.getContainerToken());
+      token = new ContainerTokenIdentifier(originalToken.getContainerID(),
+          originalToken.getVersion(), originalToken.getNmHostAddress(),
+          originalToken.getApplicationSubmitter(), rcs.getCapability(),
+          originalToken.getExpiryTimeStamp(), originalToken.getMasterKeyId(),
+          originalToken.getRMIdentifier(), originalToken.getPriority(),
+          originalToken.getCreationTime(),
+          originalToken.getLogAggregationContext(),
+          originalToken.getNodeLabelExpression(),
+          originalToken.getContainerType(), originalToken.getExecutionType());
+
+    } else {
+      token = BuilderUtils.newContainerTokenIdentifier(req.getContainerToken());
+    }
+
     ContainerId containerId = token.getContainerID();
     ApplicationId appId =
         containerId.getApplicationAttemptId().getApplicationId();
@@ -1183,9 +1201,7 @@ public class ContainerManagerImpl extends CompositeService implements
           // as container resource increase request will have come with
           // an updated NMToken.
           updateNMTokenIdentifier(nmTokenIdentifier);
-          Resource resource = containerTokenIdentifier.getResource();
-          changeContainerResourceInternal(containerId,
-              containerTokenIdentifier.getVersion(), resource, true);
+          updateContainerInternal(containerId, containerTokenIdentifier);
           successfullyUpdatedContainers.add(containerId);
         } catch (YarnException | InvalidToken e) {
           failedContainers.put(containerId, SerializedException.newInstance(e));
@@ -1199,9 +1215,9 @@ public class ContainerManagerImpl extends CompositeService implements
   }
 
   @SuppressWarnings("unchecked")
-  private void changeContainerResourceInternal(ContainerId containerId,
-      int containerVersion, Resource targetResource, boolean increase)
-          throws YarnException, IOException {
+  private void updateContainerInternal(ContainerId containerId,
+      ContainerTokenIdentifier containerTokenIdentifier)
+      throws YarnException, IOException {
     Container container = context.getContainers().get(containerId);
     // Check container existence
     if (container == null) {
@@ -1213,64 +1229,77 @@ public class ContainerManagerImpl extends CompositeService implements
             + " is not handled by this NodeManager");
       }
     }
+    // Check container version.
+    int currentVersion = container.getContainerTokenIdentifier().getVersion();
+    if (containerTokenIdentifier.getVersion() <= currentVersion) {
+      throw RPCUtil.getRemoteException("Container " + containerId.toString()
+          + " has update version [" + currentVersion + "] >= requested version"
+          + " [" + containerTokenIdentifier.getVersion() + "]");
+    }
+
     // Check container state
     org.apache.hadoop.yarn.server.nodemanager.
         containermanager.container.ContainerState currentState =
         container.getContainerState();
     if (currentState != org.apache.hadoop.yarn.server.
-        nodemanager.containermanager.container.ContainerState.RUNNING) {
+            nodemanager.containermanager.container.ContainerState.RUNNING &&
+        currentState != org.apache.hadoop.yarn.server.
+            nodemanager.containermanager.container.ContainerState.SCHEDULED) {
       throw RPCUtil.getRemoteException("Container " + containerId.toString()
           + " is in " + currentState.name() + " state."
           + " Resource can only be changed when a container is in"
-          + " RUNNING state");
+          + " RUNNING or SCHEDULED state");
     }
+
     // Check validity of the target resource.
     Resource currentResource = container.getResource();
-    if (currentResource.equals(targetResource)) {
-      LOG.warn("Unable to change resource for container "
-          + containerId.toString()
-          + ". The target resource "
-          + targetResource.toString()
-          + " is the same as the current resource");
-      return;
-    }
-    if (increase && !Resources.fitsIn(currentResource, targetResource)) {
-      throw RPCUtil.getRemoteException("Unable to increase resource for "
-          + "container " + containerId.toString()
-          + ". The target resource "
-          + targetResource.toString()
-          + " is smaller than the current resource "
-          + currentResource.toString());
-    }
-    if (!increase &&
-        (!Resources.fitsIn(Resources.none(), targetResource)
-            || !Resources.fitsIn(targetResource, currentResource))) {
-      throw RPCUtil.getRemoteException("Unable to decrease resource for "
-          + "container " + containerId.toString()
-          + ". The target resource "
-          + targetResource.toString()
-          + " is not smaller than the current resource "
-          + currentResource.toString());
-    }
-    if (increase) {
-      org.apache.hadoop.yarn.api.records.Container increasedContainer =
-          org.apache.hadoop.yarn.api.records.Container.newInstance(
-              containerId, null, null, targetResource, null, null);
+    ExecutionType currentExecType =
+        container.getContainerTokenIdentifier().getExecutionType();
+    boolean isResourceChange = false;
+    boolean isExecTypeUpdate = false;
+    Resource targetResource = containerTokenIdentifier.getResource();
+    ExecutionType targetExecType = containerTokenIdentifier.getExecutionType();
+
+    // Is true if either the resources has increased or execution type
+    // updated from opportunistic to guaranteed
+    boolean isIncrease = false;
+    if (!currentResource.equals(targetResource)) {
+      isResourceChange = true;
+      isIncrease = Resources.fitsIn(currentResource, targetResource)
+          && !Resources.fitsIn(targetResource, currentResource);
+    } else if (!currentExecType.equals(targetExecType)) {
+      isExecTypeUpdate = true;
+      isIncrease = currentExecType == ExecutionType.OPPORTUNISTIC &&
+          targetExecType == ExecutionType.GUARANTEED;
+    }
+    if (isIncrease) {
+      org.apache.hadoop.yarn.api.records.Container increasedContainer = null;
+      if (isResourceChange) {
+        increasedContainer =
+            org.apache.hadoop.yarn.api.records.Container.newInstance(
+                containerId, null, null, targetResource, null, null,
+                currentExecType);
+      } else {
+        increasedContainer =
+            org.apache.hadoop.yarn.api.records.Container.newInstance(
+                containerId, null, null, currentResource, null, null,
+                targetExecType);
+      }
       if (context.getIncreasedContainers().putIfAbsent(containerId,
           increasedContainer) != null){
         throw RPCUtil.getRemoteException("Container " + containerId.toString()
-            + " resource is being increased.");
+            + " resource is being increased -or- " +
+            "is undergoing ExecutionType promoted.");
       }
     }
     this.readLock.lock();
     try {
       if (!serviceStopped) {
-        // Persist container resource change for recovery
-        this.context.getNMStateStore().storeContainerResourceChanged(
-            containerId, containerVersion, targetResource);
-        getContainersMonitor().handle(
-            new ChangeMonitoringContainerResourceEvent(
-                containerId, targetResource));
+        // Dispatch message to ContainerScheduler to actually
+        // make the change.
+        dispatcher.getEventHandler().handle(new UpdateContainerSchedulerEvent(
+            container, containerTokenIdentifier, isResourceChange,
+            isExecTypeUpdate, isIncrease));
       } else {
         throw new YarnException(
             "Unable to change container resource as the NodeManager is "
@@ -1571,8 +1600,11 @@ public class ContainerManagerImpl extends CompositeService implements
       for (org.apache.hadoop.yarn.api.records.Container container
           : containersDecreasedEvent.getContainersToDecrease()) {
         try {
-          changeContainerResourceInternal(container.getId(),
-              container.getVersion(), container.getResource(), false);
+          ContainerTokenIdentifier containerTokenIdentifier =
+              BuilderUtils.newContainerTokenIdentifier(
+                  container.getContainerToken());
+          updateContainerInternal(container.getId(),
+              containerTokenIdentifier);
         } catch (YarnException e) {
           LOG.error("Unable to decrease container resource", e);
         } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.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/container/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
index bd3f06d..f6e567c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
@@ -39,10 +39,10 @@ public interface Container extends EventHandler<ContainerEvent> {
 
   Resource getResource();
 
-  void setResource(Resource targetResource);
-
   ContainerTokenIdentifier getContainerTokenIdentifier();
 
+  void setContainerTokenIdentifier(ContainerTokenIdentifier token);
+
   String getUser();
   
   ContainerState getContainerState();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.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/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index c0aa6b0..734a27b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -148,9 +148,8 @@ public class ContainerImpl implements Container {
   private final Credentials credentials;
   private final NodeManagerMetrics metrics;
   private volatile ContainerLaunchContext launchContext;
-  private final ContainerTokenIdentifier containerTokenIdentifier;
+  private volatile ContainerTokenIdentifier containerTokenIdentifier;
   private final ContainerId containerId;
-  private volatile Resource resource;
   private final String user;
   private int version;
   private int exitCode = ContainerExitStatus.INVALID;
@@ -201,7 +200,6 @@ public class ContainerImpl implements Container {
         YarnConfiguration.DEFAULT_NM_CONTAINER_DIAGNOSTICS_MAXIMUM_SIZE);
     this.containerTokenIdentifier = containerTokenIdentifier;
     this.containerId = containerTokenIdentifier.getContainerID();
-    this.resource = containerTokenIdentifier.getResource();
     this.diagnostics = new StringBuilder();
     this.credentials = creds;
     this.metrics = metrics;
@@ -269,13 +267,6 @@ public class ContainerImpl implements Container {
     this.exitCode = rcs.getExitCode();
     this.recoveredAsKilled = rcs.getKilled();
     this.diagnostics.append(rcs.getDiagnostics());
-    Resource recoveredCapability = rcs.getCapability();
-    if (recoveredCapability != null
-        && !this.resource.equals(recoveredCapability)) {
-      // resource capability had been updated before NM was down
-      this.resource = Resource.newInstance(recoveredCapability.getMemorySize(),
-          recoveredCapability.getVirtualCores());
-    }
     this.version = rcs.getVersion();
     this.remainingRetryAttempts = rcs.getRemainingRetryAttempts();
     this.workDir = rcs.getWorkDir();
@@ -640,14 +631,8 @@ public class ContainerImpl implements Container {
 
   @Override
   public Resource getResource() {
-    return Resources.clone(this.resource);
-  }
-
-  @Override
-  public void setResource(Resource targetResource) {
-    Resource currentResource = getResource();
-    this.resource = Resources.clone(targetResource);
-    this.metrics.changeContainer(currentResource, targetResource);
+    return Resources.clone(
+        this.containerTokenIdentifier.getResource());
   }
 
   @Override
@@ -661,6 +646,16 @@ public class ContainerImpl implements Container {
   }
 
   @Override
+  public void setContainerTokenIdentifier(ContainerTokenIdentifier token) {
+    this.writeLock.lock();
+    try {
+      this.containerTokenIdentifier = token;
+    } finally {
+      this.writeLock.unlock();
+    }
+  }
+
+  @Override
   public String getWorkDir() {
     return workDir;
   }
@@ -833,7 +828,8 @@ public class ContainerImpl implements Container {
             AuditConstants.FINISH_KILLED_CONTAINER, "ContainerImpl",
             container.containerId.getApplicationAttemptId().getApplicationId(),
             container.containerId);
-        container.metrics.releaseContainer(container.resource);
+        container.metrics.releaseContainer(
+            container.containerTokenIdentifier.getResource());
         container.sendFinishedEvents();
         return ContainerState.DONE;
       }
@@ -1517,7 +1513,8 @@ public class ContainerImpl implements Container {
     @Override
     @SuppressWarnings("unchecked")
     public void transition(ContainerImpl container, ContainerEvent event) {
-      container.metrics.releaseContainer(container.resource);
+      container.metrics.releaseContainer(
+          container.containerTokenIdentifier.getResource());
       if (container.containerMetrics != null) {
         container.containerMetrics
             .recordFinishTimeAndExitCode(clock.getTime(), container.exitCode);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.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/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
index 6ee60bd..13e7491 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
@@ -741,19 +741,6 @@ public class ContainersMonitorImpl extends AbstractService implements
     }
   }
 
-  private void changeContainerResource(
-      ContainerId containerId, Resource resource) {
-    Container container = context.getContainers().get(containerId);
-    // Check container existence
-    if (container == null) {
-      LOG.warn("Container " + containerId.toString() + "does not exist");
-      return;
-    }
-    // YARN-5860: Route this through the ContainerScheduler to
-    //       fix containerAllocation
-    container.setResource(resource);
-  }
-
   private void updateContainerMetrics(ContainersMonitorEvent monitoringEvent) {
     if (!containerMetricsEnabled || monitoringEvent == null) {
       return;
@@ -902,8 +889,6 @@ public class ContainersMonitorImpl extends AbstractService implements
       int cpuVcores = changeEvent.getResource().getVirtualCores();
       processTreeInfo.setResourceLimit(pmemLimit, vmemLimit, cpuVcores);
     }
-
-    changeContainerResource(containerId, changeEvent.getResource());
   }
 
   private void onStopMonitoringContainer(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.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/scheduler/ContainerScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
index 60d6213..19b4505 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
@@ -31,6 +31,9 @@ import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
 import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor
+    .ChangeMonitoringContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 
 
@@ -136,6 +139,13 @@ public class ContainerScheduler extends AbstractService implements
     case CONTAINER_COMPLETED:
       onContainerCompleted(event.getContainer());
       break;
+    case UPDATE_CONTAINER:
+      if (event instanceof UpdateContainerSchedulerEvent) {
+        onUpdateContainer((UpdateContainerSchedulerEvent) event);
+      } else {
+        LOG.error("Unknown event type on UpdateCOntainer: " + event.getType());
+      }
+      break;
     case SHED_QUEUED_CONTAINERS:
       shedQueuedOpportunisticContainers();
       break;
@@ -146,6 +156,69 @@ public class ContainerScheduler extends AbstractService implements
   }
 
   /**
+   * We assume that the ContainerManager has already figured out what kind
+   * of update this is.
+   */
+  private void onUpdateContainer(UpdateContainerSchedulerEvent updateEvent) {
+    ContainerId containerId = updateEvent.getContainer().getContainerId();
+    if (updateEvent.isResourceChange()) {
+      if (runningContainers.containsKey(containerId)) {
+        this.utilizationTracker.subtractContainerResource(
+            updateEvent.getContainer());
+        updateEvent.getContainer().setContainerTokenIdentifier(
+            updateEvent.getUpdatedToken());
+        this.utilizationTracker.addContainerResources(
+            updateEvent.getContainer());
+        getContainersMonitor().handle(
+            new ChangeMonitoringContainerResourceEvent(containerId,
+                updateEvent.getUpdatedToken().getResource()));
+      } else {
+        updateEvent.getContainer().setContainerTokenIdentifier(
+            updateEvent.getUpdatedToken());
+      }
+      try {
+        // Persist change in the state store.
+        this.context.getNMStateStore().storeContainerResourceChanged(
+            containerId,
+            updateEvent.getUpdatedToken().getVersion(),
+            updateEvent.getUpdatedToken().getResource());
+      } catch (IOException e) {
+        LOG.warn("Could not store container [" + containerId + "] resource " +
+            "change..", e);
+      }
+    }
+
+    if (updateEvent.isExecTypeUpdate()) {
+      updateEvent.getContainer().setContainerTokenIdentifier(
+          updateEvent.getUpdatedToken());
+      // If this is a running container.. just change the execution type
+      // and be done with it.
+      if (!runningContainers.containsKey(containerId)) {
+        // Promotion or not (Increase signifies either a promotion
+        // or container size increase)
+        if (updateEvent.isIncrease()) {
+          // Promotion of queued container..
+          if (queuedOpportunisticContainers.remove(containerId) != null) {
+            queuedGuaranteedContainers.put(containerId,
+                updateEvent.getContainer());
+          }
+          //Kill opportunistic containers if any to make room for
+          // promotion request
+          killOpportunisticContainers(updateEvent.getContainer());
+        } else {
+          // Demotion of queued container.. Should not happen too often
+          // since you should not find too many queued guaranteed
+          // containers
+          if (queuedGuaranteedContainers.remove(containerId) != null) {
+            queuedOpportunisticContainers.put(containerId,
+                updateEvent.getContainer());
+          }
+        }
+      }
+    }
+  }
+
+  /**
    * Return number of queued containers.
    * @return Number of queued containers.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.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/scheduler/ContainerSchedulerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.java
index 086cb9b..917eda0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.java
@@ -24,6 +24,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
 public enum ContainerSchedulerEventType {
   SCHEDULE_CONTAINER,
   CONTAINER_COMPLETED,
+  UPDATE_CONTAINER,
   // Producer: Node HB response - RM has asked to shed the queue
   SHED_QUEUED_CONTAINERS,
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/UpdateContainerSchedulerEvent.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/scheduler/UpdateContainerSchedulerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/UpdateContainerSchedulerEvent.java
new file mode 100644
index 0000000..5384b7e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/UpdateContainerSchedulerEvent.java
@@ -0,0 +1,85 @@
+/**
+ * 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.containermanager.scheduler;
+
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container
+    .Container;
+/**
+ * Update Event consumed by the {@link ContainerScheduler}.
+ */
+public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
+
+  private ContainerTokenIdentifier updatedToken;
+  private boolean isResourceChange;
+  private boolean isExecTypeUpdate;
+  private boolean isIncrease;
+
+  /**
+   * Create instance of Event.
+   *
+   * @param originalContainer Original Container.
+   * @param updatedToken Updated Container Token.
+   * @param isResourceChange is this a Resource Change.
+   * @param isExecTypeUpdate is this an ExecTypeUpdate.
+   * @param isIncrease is this a Container Increase.
+   */
+  public UpdateContainerSchedulerEvent(Container originalContainer,
+      ContainerTokenIdentifier updatedToken, boolean isResourceChange,
+      boolean isExecTypeUpdate, boolean isIncrease) {
+    super(originalContainer, ContainerSchedulerEventType.UPDATE_CONTAINER);
+    this.updatedToken = updatedToken;
+    this.isResourceChange = isResourceChange;
+    this.isExecTypeUpdate = isExecTypeUpdate;
+    this.isIncrease = isIncrease;
+  }
+
+  /**
+   * Update Container Token.
+   *
+   * @return Container Token.
+   */
+  public ContainerTokenIdentifier getUpdatedToken() {
+    return updatedToken;
+  }
+
+  /**
+   * isResourceChange.
+   * @return isResourceChange.
+   */
+  public boolean isResourceChange() {
+    return isResourceChange;
+  }
+
+  /**
+   * isExecTypeUpdate.
+   * @return isExecTypeUpdate.
+   */
+  public boolean isExecTypeUpdate() {
+    return isExecTypeUpdate;
+  }
+
+  /**
+   * isIncrease.
+   * @return isIncrease.
+   */
+  public boolean isIncrease() {
+    return isIncrease;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.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/TestNodeManagerResync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
index 0c025ac..b8cd7dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
@@ -682,7 +682,7 @@ public class TestNodeManagerResync {
         try{
           try {
             updateBarrier.await();
-            increaseTokens.add(getContainerToken(targetResource));
+            increaseTokens.add(getContainerToken(targetResource, 1));
             ContainerUpdateRequest updateRequest =
                 ContainerUpdateRequest.newInstance(increaseTokens);
             ContainerUpdateResponse updateResponse =
@@ -710,6 +710,15 @@ public class TestNodeManagerResync {
           getNMContext().getNodeId(), user, resource,
           getNMContext().getContainerTokenSecretManager(), null);
     }
+
+    private Token getContainerToken(Resource resource, int version)
+        throws IOException {
+      ContainerId cId = TestContainerManager.createContainerId(0);
+      return TestContainerManager.createContainerToken(
+          cId, version, DUMMY_RM_IDENTIFIER,
+          getNMContext().getNodeId(), user, resource,
+          getNMContext().getContainerTokenSecretManager(), null);
+    }
   }
 
   public static NMContainerStatus createNMContainerStatus(int id,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.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/BaseContainerManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
index d266ac1..6c96a47 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
@@ -421,6 +421,20 @@ public abstract class BaseContainerManagerTest {
             containerTokenIdentifier);
   }
 
+  public static Token createContainerToken(ContainerId cId, int version,
+      long rmIdentifier, NodeId nodeId, String user, Resource resource,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext) throws IOException {
+    ContainerTokenIdentifier containerTokenIdentifier =
+        new ContainerTokenIdentifier(cId, version, nodeId.toString(), user,
+            resource, System.currentTimeMillis() + 100000L, 123, rmIdentifier,
+            Priority.newInstance(0), 0, logAggregationContext, null,
+            ContainerType.TASK, ExecutionType.GUARANTEED);
+    return BuilderUtils.newContainerToken(nodeId,
+        containerTokenSecretManager.retrievePassword(containerTokenIdentifier),
+        containerTokenIdentifier);
+  }
+
   public static Token createContainerToken(ContainerId cId, long rmIdentifier,
       NodeId nodeId, String user, Resource resource,
       NMContainerTokenSecretManager containerTokenSecretManager,
@@ -431,8 +445,23 @@ public abstract class BaseContainerManagerTest {
             System.currentTimeMillis() + 100000L, 123, rmIdentifier,
             Priority.newInstance(0), 0, logAggregationContext, null,
             ContainerType.TASK, executionType);
-    return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager
-            .retrievePassword(containerTokenIdentifier),
+    return BuilderUtils.newContainerToken(nodeId,
+        containerTokenSecretManager.retrievePassword(containerTokenIdentifier),
+        containerTokenIdentifier);
+  }
+
+  public static Token createContainerToken(ContainerId cId, int version,
+      long rmIdentifier, NodeId nodeId, String user, Resource resource,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext, ExecutionType executionType)
+      throws IOException {
+    ContainerTokenIdentifier containerTokenIdentifier =
+        new ContainerTokenIdentifier(cId, version, nodeId.toString(), user,
+            resource, System.currentTimeMillis() + 100000L, 123, rmIdentifier,
+            Priority.newInstance(0), 0, logAggregationContext, null,
+            ContainerType.TASK, executionType);
+    return BuilderUtils.newContainerToken(nodeId,
+        containerTokenSecretManager.retrievePassword(containerTokenIdentifier),
         containerTokenIdentifier);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.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/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index 24d46b6..9844225 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager;
 
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.timeout;
 import static org.mockito.Mockito.verify;
 
@@ -70,6 +71,7 @@ import org.apache.hadoop.yarn.api.records.ContainerRetryContext;
 import org.apache.hadoop.yarn.api.records.ContainerRetryPolicy;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@@ -80,14 +82,15 @@ import org.apache.hadoop.yarn.api.records.SignalContainerCommand;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.ConfigurationException;
 import org.apache.hadoop.yarn.exceptions.InvalidContainerException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
-import org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrSignalContainersEvent;
+import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
@@ -100,6 +103,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.Conta
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext;
+import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
 import org.junit.Before;
@@ -116,10 +120,34 @@ public class TestContainerManager extends BaseContainerManagerTest {
   static {
     LOG = LogFactory.getLog(TestContainerManager.class);
   }
-  
+
+  private boolean delayContainers = false;
+
+  @Override
+  protected ContainerExecutor createContainerExecutor() {
+    DefaultContainerExecutor exec = new DefaultContainerExecutor() {
+      @Override
+      public int launchContainer(ContainerStartContext ctx)
+          throws IOException, ConfigurationException {
+        if (delayContainers) {
+          try {
+            Thread.sleep(10000);
+          } catch (InterruptedException e) {
+            // Nothing..
+          }
+        }
+        return super.launchContainer(ctx);
+      }
+    };
+    exec.setConf(conf);
+    return spy(exec);
+  }
+
   @Override
   @Before
   public void setup() throws IOException {
+    conf.setInt(
+        YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, 10);
     super.setup();
   }
   
@@ -1468,7 +1496,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     Assert.assertEquals(strExceptionMsg,
         ContainerManagerImpl.INVALID_NMTOKEN_MSG);
 
-    ContainerManagerImpl spyContainerMgr = Mockito.spy(cMgrImpl);
+    ContainerManagerImpl spyContainerMgr = spy(cMgrImpl);
     UserGroupInformation ugInfo = UserGroupInformation.createRemoteUser("a");
     Mockito.when(spyContainerMgr.getRemoteUgi()).thenReturn(ugInfo);
     Mockito.when(spyContainerMgr.
@@ -1543,7 +1571,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     // container will have exited, and won't be in RUNNING state
     ContainerId cId0 = createContainerId(0);
     Token containerToken =
-        createContainerToken(cId0, DUMMY_RM_IDENTIFIER,
+        createContainerToken(cId0, 1, DUMMY_RM_IDENTIFIER,
             context.getNodeId(), user,
                 Resource.newInstance(1234, 3),
                     context.getContainerTokenSecretManager(), null);
@@ -1572,7 +1600,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
       if (cId0.equals(entry.getKey())) {
         Assert.assertTrue(entry.getValue().getMessage()
           .contains("Resource can only be changed when a "
-              + "container is in RUNNING state"));
+              + "container is in RUNNING or SCHEDULED state"));
       } else if (cId7.equals(entry.getKey())) {
         Assert.assertTrue(entry.getValue().getMessage()
             .contains("Container " + cId7.toString()
@@ -1585,89 +1613,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
   }
 
   @Test
-  public void testIncreaseContainerResourceWithInvalidResource() throws Exception {
-    containerManager.start();
-    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
-    PrintWriter fileWriter = new PrintWriter(scriptFile);
-    // Construct the Container-id
-    ContainerId cId = createContainerId(0);
-    if (Shell.WINDOWS) {
-      fileWriter.println("@ping -n 100 127.0.0.1 >nul");
-    } else {
-      fileWriter.write("\numask 0");
-      fileWriter.write("\nexec sleep 100");
-    }
-    fileWriter.close();
-    ContainerLaunchContext containerLaunchContext =
-        recordFactory.newRecordInstance(ContainerLaunchContext.class);
-    URL resource_alpha =
-        URL.fromPath(localFS
-            .makeQualified(new Path(scriptFile.getAbsolutePath())));
-    LocalResource rsrc_alpha =
-        recordFactory.newRecordInstance(LocalResource.class);
-    rsrc_alpha.setResource(resource_alpha);
-    rsrc_alpha.setSize(-1);
-    rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
-    rsrc_alpha.setType(LocalResourceType.FILE);
-    rsrc_alpha.setTimestamp(scriptFile.lastModified());
-    String destinationFile = "dest_file";
-    Map<String, LocalResource> localResources =
-        new HashMap<String, LocalResource>();
-    localResources.put(destinationFile, rsrc_alpha);
-    containerLaunchContext.setLocalResources(localResources);
-    List<String> commands =
-        Arrays.asList(Shell.getRunScriptCommand(scriptFile));
-    containerLaunchContext.setCommands(commands);
-
-    StartContainerRequest scRequest =
-        StartContainerRequest.newInstance(
-            containerLaunchContext,
-            createContainerToken(cId, DUMMY_RM_IDENTIFIER, context.getNodeId(),
-            user, context.getContainerTokenSecretManager()));
-    List<StartContainerRequest> list = new ArrayList<>();
-    list.add(scRequest);
-    StartContainersRequest allRequests =
-        StartContainersRequest.newInstance(list);
-    containerManager.startContainers(allRequests);
-    // Make sure the container reaches RUNNING state
-    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
-        org.apache.hadoop.yarn.server.nodemanager.
-            containermanager.container.ContainerState.RUNNING);
-    // Construct container resource increase request,
-    List<Token> increaseTokens = new ArrayList<>();
-    // Add increase request. The increase request should fail
-    // as the current resource does not fit in the target resource
-    Token containerToken =
-        createContainerToken(cId, DUMMY_RM_IDENTIFIER,
-            context.getNodeId(), user,
-            Resource.newInstance(512, 1),
-            context.getContainerTokenSecretManager(), null);
-    increaseTokens.add(containerToken);
-    ContainerUpdateRequest updateRequest =
-        ContainerUpdateRequest.newInstance(increaseTokens);
-    ContainerUpdateResponse updateResponse =
-        containerManager.updateContainer(updateRequest);
-    // Check response
-    Assert.assertEquals(
-        0, updateResponse.getSuccessfullyUpdatedContainers().size());
-    Assert.assertEquals(1, updateResponse.getFailedRequests().size());
-    for (Map.Entry<ContainerId, SerializedException> entry : updateResponse
-        .getFailedRequests().entrySet()) {
-      if (cId.equals(entry.getKey())) {
-        Assert.assertNotNull("Failed message", entry.getValue().getMessage());
-        Assert.assertTrue(entry.getValue().getMessage()
-            .contains("The target resource "
-                + Resource.newInstance(512, 1).toString()
-                + " is smaller than the current resource "
-                + Resource.newInstance(1024, 1)));
-      } else {
-        throw new YarnException("Received failed request from wrong"
-            + " container: " + entry.getKey().toString());
-      }
-    }
-  }
-
-  @Test
   public void testChangeContainerResource() throws Exception {
     containerManager.start();
     File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
@@ -1720,7 +1665,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     List<Token> increaseTokens = new ArrayList<>();
     // Add increase request.
     Resource targetResource = Resource.newInstance(4096, 2);
-    Token containerToken = createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+    Token containerToken = createContainerToken(cId, 1, DUMMY_RM_IDENTIFIER,
         context.getNodeId(), user, targetResource,
             context.getContainerTokenSecretManager(), null);
     increaseTokens.add(containerToken);
@@ -1741,15 +1686,19 @@ public class TestContainerManager extends BaseContainerManagerTest {
     // Check status immediately as resource increase is blocking
     assertEquals(targetResource, containerStatus.getCapability());
     // Simulate a decrease request
-    List<org.apache.hadoop.yarn.api.records.Container> containersToDecrease
-        = new ArrayList<>();
+    List<Token> decreaseTokens = new ArrayList<>();
     targetResource = Resource.newInstance(2048, 2);
-    org.apache.hadoop.yarn.api.records.Container decreasedContainer =
-        org.apache.hadoop.yarn.api.records.Container
-            .newInstance(cId, null, null, targetResource, null, null);
-    containersToDecrease.add(decreasedContainer);
-    containerManager.handle(
-        new CMgrDecreaseContainersResourceEvent(containersToDecrease));
+    Token token = createContainerToken(cId, 2, DUMMY_RM_IDENTIFIER,
+        context.getNodeId(), user, targetResource,
+        context.getContainerTokenSecretManager(), null);
+    decreaseTokens.add(token);
+    updateRequest = ContainerUpdateRequest.newInstance(decreaseTokens);
+    updateResponse = containerManager.updateContainer(updateRequest);
+
+    Assert.assertEquals(
+        1, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
+
     // Check status with retry
     containerStatus = containerManager
         .getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
@@ -1879,7 +1828,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     ContainerLaunchContext spyContainerLaunchContext =
-        Mockito.spy(containerLaunchContext);
+        spy(containerLaunchContext);
     Mockito.when(spyContainerLaunchContext.getLocalResources())
         .thenReturn(localResources);
 
@@ -1924,7 +1873,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     ContainerLaunchContext spyContainerLaunchContext =
-        Mockito.spy(containerLaunchContext);
+        spy(containerLaunchContext);
     Mockito.when(spyContainerLaunchContext.getLocalResources())
         .thenReturn(localResources);
 
@@ -1969,7 +1918,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     ContainerLaunchContext spyContainerLaunchContext =
-        Mockito.spy(containerLaunchContext);
+        spy(containerLaunchContext);
     Mockito.when(spyContainerLaunchContext.getLocalResources())
         .thenReturn(localResources);
 
@@ -1996,4 +1945,122 @@ public class TestContainerManager extends BaseContainerManagerTest {
     Assert.assertTrue(response.getFailedRequests().get(cId).getMessage()
         .contains("Null resource visibility for local resource"));
   }
+
+  @Test
+  public void testContainerUpdateExecTypeOpportunisticToGuaranteed()
+      throws IOException, YarnException, InterruptedException {
+    delayContainers = true;
+    containerManager.start();
+    // Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+            containerLaunchContext,
+            createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+                context.getNodeId(), user, BuilderUtils.newResource(512, 1),
+                context.getContainerTokenSecretManager(), null,
+                ExecutionType.OPPORTUNISTIC));
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+    // Make sure the container reaches RUNNING state
+    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
+        org.apache.hadoop.yarn.server.nodemanager.
+            containermanager.container.ContainerState.RUNNING);
+    // Construct container resource increase request,
+    List<Token> updateTokens = new ArrayList<>();
+    Token containerToken =
+        createContainerToken(cId, 1, DUMMY_RM_IDENTIFIER, context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED);
+    updateTokens.add(containerToken);
+    ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(updateTokens);
+    ContainerUpdateResponse updateResponse =
+        containerManager.updateContainer(updateRequest);
+
+    Assert.assertEquals(
+        1, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
+
+    //Make sure the container is running
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    statList.add(cId);
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    Assert.assertEquals(1, containerStatuses.size());
+    for (ContainerStatus status : containerStatuses) {
+      Assert.assertEquals(
+          org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+          status.getState());
+      Assert.assertEquals(ExecutionType.GUARANTEED, status.getExecutionType());
+    }
+  }
+
+  @Test
+  public void testContainerUpdateExecTypeGuaranteedToOpportunistic()
+      throws IOException, YarnException, InterruptedException {
+    delayContainers = true;
+    containerManager.start();
+    // Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+            containerLaunchContext,
+            createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+                context.getNodeId(), user, BuilderUtils.newResource(512, 1),
+                context.getContainerTokenSecretManager(), null));
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+    // Make sure the container reaches RUNNING state
+    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
+        org.apache.hadoop.yarn.server.nodemanager.
+            containermanager.container.ContainerState.RUNNING);
+    // Construct container resource increase request,
+    List<Token> updateTokens = new ArrayList<>();
+    Token containerToken =
+        createContainerToken(cId, 1, DUMMY_RM_IDENTIFIER, context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC);
+    updateTokens.add(containerToken);
+    ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(updateTokens);
+    ContainerUpdateResponse updateResponse =
+        containerManager.updateContainer(updateRequest);
+
+    Assert.assertEquals(
+        1, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
+
+    //Make sure the container is running
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    statList.add(cId);
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    Assert.assertEquals(1, containerStatuses.size());
+    for (ContainerStatus status : containerStatuses) {
+      Assert.assertEquals(
+          org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+          status.getState());
+      Assert
+          .assertEquals(ExecutionType.OPPORTUNISTIC, status.getExecutionType());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.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/TestContainerManagerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
index d2bd79c..224e99c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
@@ -652,7 +652,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
     final List<Token> increaseTokens = new ArrayList<Token>();
     // add increase request
     Token containerToken = TestContainerManager.createContainerToken(
-        cid, 0, context.getNodeId(), user.getShortUserName(),
+        cid, 1, 0, context.getNodeId(), user.getShortUserName(),
         capability, context.getContainerTokenSecretManager(), null);
     increaseTokens.add(containerToken);
     final ContainerUpdateRequest updateRequest =


---------------------------------------------------------------------
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: HADOOP-14773. Extend ZKCuratorManager API for more reusability. (Íñigo Goiri via Subru).

Posted by ae...@apache.org.
HADOOP-14773. Extend ZKCuratorManager API for more reusability. (Íñigo Goiri via Subru).


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

Branch: refs/heads/HDFS-7240
Commit: 75dd866bfb8b63cb9f13179d4365b05c48e0907d
Parents: f34646d
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Aug 15 16:53:59 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Aug 15 16:53:59 2017 -0700

----------------------------------------------------------------------
 .../hadoop/util/curator/ZKCuratorManager.java   | 54 ++++++++++++++++++--
 .../util/curator/TestZKCuratorManager.java      |  2 +-
 .../recovery/ZKRMStateStore.java                | 19 +------
 3 files changed, 52 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75dd866b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
index 3adf028..9a031af 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
@@ -33,9 +33,12 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.util.ZKUtil;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Helper class that provides utility methods specific to ZK operations.
  */
@@ -179,7 +182,6 @@ public final class ZKCuratorManager {
   /**
    * Get the data in a ZNode.
    * @param path Path of the ZNode.
-   * @param stat Output statistics of the ZNode.
    * @return The data in the ZNode.
    * @throws Exception If it cannot contact Zookeeper.
    */
@@ -190,16 +192,38 @@ public final class ZKCuratorManager {
   /**
    * Get the data in a ZNode.
    * @param path Path of the ZNode.
-   * @param stat Output statistics of the ZNode.
+   * @param stat
+   * @return The data in the ZNode.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public byte[] getData(final String path, Stat stat) throws Exception {
+    return curator.getData().storingStatIn(stat).forPath(path);
+  }
+
+  /**
+   * Get the data in a ZNode.
+   * @param path Path of the ZNode.
    * @return The data in the ZNode.
    * @throws Exception If it cannot contact Zookeeper.
    */
-  public String getSringData(final String path) throws Exception {
+  public String getStringData(final String path) throws Exception {
     byte[] bytes = getData(path);
     return new String(bytes, Charset.forName("UTF-8"));
   }
 
   /**
+   * Get the data in a ZNode.
+   * @param path Path of the ZNode.
+   * @param stat Output statistics of the ZNode.
+   * @return The data in the ZNode.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public String getStringData(final String path, Stat stat) throws Exception {
+    byte[] bytes = getData(path, stat);
+    return new String(bytes, Charset.forName("UTF-8"));
+  }
+
+  /**
    * Set data into a ZNode.
    * @param path Path of the ZNode.
    * @param data Data to set.
@@ -272,14 +296,36 @@ public final class ZKCuratorManager {
   }
 
   /**
+   * Utility function to ensure that the configured base znode exists.
+   * This recursively creates the znode as well as all of its parents.
+   * @param path Path of the znode to create.
+   * @throws Exception If it cannot create the file.
+   */
+  public void createRootDirRecursively(String path) throws Exception {
+    String[] pathParts = path.split("/");
+    Preconditions.checkArgument(
+        pathParts.length >= 1 && pathParts[0].isEmpty(),
+        "Invalid path: %s", path);
+    StringBuilder sb = new StringBuilder();
+
+    for (int i = 1; i < pathParts.length; i++) {
+      sb.append("/").append(pathParts[i]);
+      create(sb.toString());
+    }
+  }
+
+  /**
    * Delete a ZNode.
    * @param path Path of the ZNode.
+   * @return If the znode was deleted.
    * @throws Exception If it cannot contact ZooKeeper.
    */
-  public void delete(final String path) throws Exception {
+  public boolean delete(final String path) throws Exception {
     if (exists(path)) {
       curator.delete().deletingChildrenIfNeeded().forPath(path);
+      return true;
     }
+    return false;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75dd866b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java
index 2bcf508..3e78a44 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java
@@ -67,7 +67,7 @@ public class TestZKCuratorManager {
     curator.create(testZNode);
     assertTrue(curator.exists(testZNode));
     curator.setData(testZNode, expectedString, -1);
-    String testString = curator.getSringData("/test");
+    String testString = curator.getStringData("/test");
     assertEquals(expectedString, testString);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75dd866b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.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/recovery/ZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
index 4b6e82c..a445e75 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.server.resourcemanager.recovery;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.curator.framework.CuratorFramework;
@@ -338,7 +337,7 @@ public class ZKRMStateStore extends RMStateStore {
   @Override
   public synchronized void startInternal() throws Exception {
     // ensure root dirs exist
-    createRootDirRecursively(znodeWorkingPath);
+    zkManager.createRootDirRecursively(znodeWorkingPath);
     create(zkRootNodePath);
     setRootNodeAcls();
     delete(fencingNodePath);
@@ -1147,22 +1146,6 @@ public class ZKRMStateStore extends RMStateStore {
   }
 
   /**
-   * Utility function to ensure that the configured base znode exists.
-   * This recursively creates the znode as well as all of its parents.
-   */
-  private void createRootDirRecursively(String path) throws Exception {
-    String pathParts[] = path.split("/");
-    Preconditions.checkArgument(pathParts.length >= 1 && pathParts[0].isEmpty(),
-        "Invalid path: %s", path);
-    StringBuilder sb = new StringBuilder();
-
-    for (int i = 1; i < pathParts.length; i++) {
-      sb.append("/").append(pathParts[i]);
-      create(sb.toString());
-    }
-  }
-
-  /**
    * Get alternate path for app id if path according to configured split index
    * does not exist. We look for path based on all possible split indices.
    * @param appId


---------------------------------------------------------------------
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-6905 Multiple HBaseTimelineStorage test failures due to missing FastNumberFormat (Contributed by Haibo Chen)

Posted by ae...@apache.org.
YARN-6905 Multiple HBaseTimelineStorage test failures due to missing FastNumberFormat (Contributed by Haibo Chen)


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

Branch: refs/heads/HDFS-7240
Commit: 608a06cca5d68b3155bd70a94bf29ae0942b9ca0
Parents: d72124a
Author: Vrushali C <vr...@apache.org>
Authored: Mon Aug 14 11:40:27 2017 -0700
Committer: Vrushali C <vr...@apache.org>
Committed: Mon Aug 14 11:41:11 2017 -0700

----------------------------------------------------------------------
 .../storage/TestHBaseTimelineStorageApps.java   |  4 +-
 .../TestHBaseTimelineStorageEntities.java       | 14 ++++---
 .../storage/common/AppIdKeyConverter.java       |  3 +-
 .../common/HBaseTimelineStorageUtils.java       | 33 +++++++++++++++++
 .../TestCustomApplicationIdConversion.java      | 39 ++++++++++++++++++++
 5 files changed, 86 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/608a06cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
index b3e5197..3948d23 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
@@ -69,6 +69,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.application.Applica
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
@@ -493,7 +494,8 @@ public class TestHBaseTimelineStorageApps {
     event.addInfo(expKey, expVal);
 
     final TimelineEntity entity = new ApplicationEntity();
-    entity.setId(ApplicationId.newInstance(0, 1).toString());
+    entity.setId(HBaseTimelineStorageUtils.convertApplicationIdToString(
+        ApplicationId.newInstance(0, 1)));
     entity.addEvent(event);
 
     TimelineEntities entities = new TimelineEntities();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/608a06cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
index 4b4c3e1..e18d0d0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefi
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
@@ -202,8 +203,9 @@ public class TestHBaseTimelineStorageEntities {
       String flow = "some_flow_name";
       String flowVersion = "AB7822C10F1111";
       long runid = 1002345678919L;
-      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
-          9000000L, 1).toString();
+      String appName = HBaseTimelineStorageUtils.convertApplicationIdToString(
+          ApplicationId.newInstance(System.currentTimeMillis() + 9000000L, 1)
+      );
       hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
       hbi.stop();
 
@@ -399,8 +401,8 @@ public class TestHBaseTimelineStorageEntities {
       String flow = "other_flow_name";
       String flowVersion = "1111F01C2287BA";
       long runid = 1009876543218L;
-      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
-          9000000L, 1).toString();
+      String appName = HBaseTimelineStorageUtils.convertApplicationIdToString(
+          ApplicationId.newInstance(System.currentTimeMillis() + 9000000L, 1));
       byte[] startRow =
           new EntityRowKeyPrefix(cluster, user, flow, runid, appName)
               .getRowKeyPrefix();
@@ -487,7 +489,9 @@ public class TestHBaseTimelineStorageEntities {
     event.addInfo(expKey, expVal);
 
     final TimelineEntity entity = new ApplicationEntity();
-    entity.setId(ApplicationId.newInstance(0, 1).toString());
+    entity.setId(
+        HBaseTimelineStorageUtils.convertApplicationIdToString(
+            ApplicationId.newInstance(0, 1)));
     entity.addEvent(event);
 
     TimelineEntities entities = new TimelineEntities();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/608a06cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
index c165801..51604f0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
@@ -82,7 +82,8 @@ public final class AppIdKeyConverter implements KeyConverter<String> {
         Bytes.toLong(appIdBytes, 0, Bytes.SIZEOF_LONG));
     int seqId = HBaseTimelineStorageUtils.invertInt(
         Bytes.toInt(appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT));
-    return ApplicationId.newInstance(clusterTs, seqId).toString();
+    return HBaseTimelineStorageUtils.convertApplicationIdToString(
+        ApplicationId.newInstance(clusterTs, seqId));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/608a06cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
index b6f1157..b8c7029 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
@@ -32,6 +33,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.text.NumberFormat;
 import java.util.List;
 import java.util.Map;
 
@@ -240,4 +242,35 @@ public final class HBaseTimelineStorageUtils {
     long dayTimestamp = ts - (ts % MILLIS_ONE_DAY);
     return dayTimestamp;
   }
+
+  private static final ThreadLocal<NumberFormat> APP_ID_FORMAT =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(4);
+          return fmt;
+        }
+      };
+
+  /**
+   * A utility method that converts ApplicationId to string without using
+   * FastNumberFormat in order to avoid the incompatibility issue caused
+   * by mixing hadoop-common 2.5.1 and hadoop-yarn-api 3.0 in this module.
+   * This is a work-around implementation as discussed in YARN-6905.
+   *
+   * @param appId application id
+   * @return the string representation of the given application id
+   *
+   */
+  public static String convertApplicationIdToString(ApplicationId appId) {
+    StringBuilder sb = new StringBuilder(64);
+    sb.append(ApplicationId.appIdStrPrefix);
+    sb.append("_");
+    sb.append(appId.getClusterTimestamp());
+    sb.append('_');
+    sb.append(APP_ID_FORMAT.get().format(appId.getId()));
+    return sb.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/608a06cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java
new file mode 100644
index 0000000..73bc29e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.timelineservice.storage.common;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for HBaseTimelineStorageUtils.convertApplicationIdToString(),
+ * a custom conversion from ApplicationId to String that avoids the
+ * incompatibility issue caused by mixing hadoop-common 2.5.1 and
+ * hadoop-yarn-api 3.0. See YARN-6905.
+ */
+public class TestCustomApplicationIdConversion {
+  @Test
+  public void testConvertAplicationIdToString() {
+    ApplicationId applicationId = ApplicationId.newInstance(0, 1);
+    String applicationIdStr =
+        HBaseTimelineStorageUtils.convertApplicationIdToString(applicationId);
+    Assert.assertEquals(applicationId,
+        ApplicationId.fromString(applicationIdStr));
+  }
+}


---------------------------------------------------------------------
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: YARN-5978. ContainerScheduler and ContainerManager changes to support ExecType update. (Kartheek Muthyala via asuresh)

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.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/scheduler/TestContainerSchedulerQueuing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java
index aeba399..a1c247b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java
@@ -27,6 +27,8 @@ import java.util.List;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
@@ -37,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ConfigurationException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -951,4 +954,97 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
         map.get(org.apache.hadoop.yarn.api.records.ContainerState.SCHEDULED)
             .getContainerId());
   }
+
+  /**
+   * Starts one OPPORTUNISTIC container that takes up the whole node's
+   * resources, and submit one more that will be queued. Now promote the
+   * queued OPPORTUNISTIC container, which should kill the current running
+   * OPPORTUNISTIC container to make room for the promoted request.
+   * @throws Exception
+   */
+  @Test
+  public void testPromotionOfOpportunisticContainers() throws Exception {
+    containerManager.start();
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(2048, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(1024, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    Thread.sleep(5000);
+
+    // Ensure first container is running and others are queued.
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 3; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest = GetContainerStatusesRequest
+        .newInstance(Arrays.asList(createContainerId(0)));
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      if (status.getContainerId().equals(createContainerId(0))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+            status.getState());
+      } else {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.SCHEDULED,
+            status.getState());
+      }
+    }
+
+    ContainerScheduler containerScheduler =
+        containerManager.getContainerScheduler();
+    // Ensure two containers are properly queued.
+    Assert.assertEquals(1, containerScheduler.getNumQueuedContainers());
+    Assert.assertEquals(0,
+        containerScheduler.getNumQueuedGuaranteedContainers());
+    Assert.assertEquals(1,
+        containerScheduler.getNumQueuedOpportunisticContainers());
+
+    // Promote Queued Opportunistic Container
+    Token updateToken =
+        createContainerToken(createContainerId(1), 1, DUMMY_RM_IDENTIFIER,
+            context.getNodeId(), user, BuilderUtils.newResource(1024, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED);
+    List<Token> updateTokens = new ArrayList<Token>();
+    updateTokens.add(updateToken);
+    ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(updateTokens);
+    ContainerUpdateResponse updateResponse =
+        containerManager.updateContainer(updateRequest);
+
+    Assert.assertEquals(1,
+        updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertEquals(0, updateResponse.getFailedRequests().size());
+
+    waitForContainerState(containerManager, createContainerId(0),
+        org.apache.hadoop.yarn.api.records.ContainerState.COMPLETE);
+
+    waitForContainerState(containerManager, createContainerId(1),
+        org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
+
+    // Ensure no containers are queued.
+    Assert.assertEquals(0, containerScheduler.getNumQueuedContainers());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.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/webapp/MockContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
index 022baea..4561e85c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
@@ -140,7 +140,7 @@ public class MockContainer implements Container {
   }
 
   @Override
-  public void setResource(Resource targetResource) {
+  public void setContainerTokenIdentifier(ContainerTokenIdentifier token) {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.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/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index 8b2f9db..397d507 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -655,7 +655,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
               container.getNodeId(), getUser(), container.getResource(),
               container.getPriority(), rmContainer.getCreationTime(),
               this.logAggregationContext, rmContainer.getNodeLabelExpression(),
-              containerType));
+              containerType, container.getExecutionType()));
       updateNMToken(container);
     } catch (IllegalArgumentException e) {
       // DNS might be down, skip returning this container.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.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/security/RMContainerTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
index 8c42255..677aa14 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
@@ -187,6 +187,31 @@ public class RMContainerTokenSecretManager extends
   }
 
   /**
+   * Helper function for creating ContainerTokens.
+   *
+   * @param containerId containerId.
+   * @param containerVersion containerVersion.
+   * @param nodeId nodeId.
+   * @param appSubmitter appSubmitter.
+   * @param capability capability.
+   * @param priority priority.
+   * @param createTime createTime.
+   * @param logAggregationContext logAggregationContext.
+   * @param nodeLabelExpression nodeLabelExpression.
+   * @param containerType containerType.
+   * @return the container-token.
+   */
+  public Token createContainerToken(ContainerId containerId,
+      int containerVersion, NodeId nodeId, String appSubmitter,
+      Resource capability, Priority priority, long createTime,
+      LogAggregationContext logAggregationContext, String nodeLabelExpression,
+      ContainerType containerType) {
+    return createContainerToken(containerId, containerVersion, nodeId,
+        appSubmitter, capability, priority, createTime, null, null,
+        ContainerType.TASK, ExecutionType.GUARANTEED);
+  }
+
+  /**
    * Helper function for creating ContainerTokens
    *
    * @param containerId Container Id
@@ -199,13 +224,14 @@ public class RMContainerTokenSecretManager extends
    * @param logAggregationContext Log Aggregation Context
    * @param nodeLabelExpression Node Label Expression
    * @param containerType Container Type
+   * @param execType Execution Type
    * @return the container-token
    */
   public Token createContainerToken(ContainerId containerId,
       int containerVersion, NodeId nodeId, String appSubmitter,
       Resource capability, Priority priority, long createTime,
       LogAggregationContext logAggregationContext, String nodeLabelExpression,
-      ContainerType containerType) {
+      ContainerType containerType, ExecutionType execType) {
     byte[] password;
     ContainerTokenIdentifier tokenIdentifier;
     long expiryTimeStamp =
@@ -220,7 +246,7 @@ public class RMContainerTokenSecretManager extends
               this.currentMasterKey.getMasterKey().getKeyId(),
               ResourceManager.getClusterTimeStamp(), priority, createTime,
               logAggregationContext, nodeLabelExpression, containerType,
-              ExecutionType.GUARANTEED);
+              execType);
       password = this.createPassword(tokenIdentifier);
 
     } finally {


---------------------------------------------------------------------
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: HDFS-12054. FSNamesystem#addErasureCodingPolicies should call checkNameNodeSafeMode() to ensure Namenode is not in safemode. Contributed by lufei.

Posted by ae...@apache.org.
HDFS-12054. FSNamesystem#addErasureCodingPolicies should call checkNameNodeSafeMode() to ensure Namenode is not in safemode. Contributed by lufei.


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

Branch: refs/heads/HDFS-7240
Commit: 1040bae6fcbae7079d8126368cdeac60831a4d0c
Parents: 2e43c28
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Tue Aug 15 07:38:43 2017 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Tue Aug 15 07:38:43 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/namenode/FSNamesystem.java   |  2 ++
 .../java/org/apache/hadoop/hdfs/TestSafeMode.java   | 16 ++++++++++++++++
 2 files changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1040bae6/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 b1639b2..caf73f7 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
@@ -7081,6 +7081,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       for (ErasureCodingPolicy policy : policies) {
         try {
+          checkOperation(OperationCategory.WRITE);
+          checkNameNodeSafeMode("Cannot add erasure coding policy");
           ErasureCodingPolicy newPolicy =
               FSDirErasureCodingOp.addErasureCodePolicy(this, policy);
           addECPolicyName = newPolicy.getName();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1040bae6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
index f03b440..bc95ec7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -48,6 +49,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -466,6 +468,20 @@ public class TestSafeMode {
       // expected
     }
 
+    ECSchema toAddSchema = new ECSchema("testcodec", 3, 2);
+    ErasureCodingPolicy newPolicy =
+        new ErasureCodingPolicy(toAddSchema, 128 * 1024);
+    ErasureCodingPolicy[] policyArray =
+        new ErasureCodingPolicy[]{newPolicy};
+    try {
+      dfs.addErasureCodingPolicies(policyArray);
+      fail("AddErasureCodingPolicies should have failed.");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains(
+          "Cannot add erasure coding policy", ioe);
+      // expected
+    }
+
     assertFalse("Could not leave SM",
         dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE));
   }


---------------------------------------------------------------------
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: HDFS-12325. SFTPFileSystem operations should restore cwd. Contributed by Chen Liang.

Posted by ae...@apache.org.
HDFS-12325. SFTPFileSystem operations should restore cwd. Contributed by Chen Liang.


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

Branch: refs/heads/HDFS-7240
Commit: 736ceab2f58fb9ab5907c5b5110bd44384038e6b
Parents: 913760c
Author: Arpit Agarwal <ar...@apache.org>
Authored: Sun Aug 20 23:41:06 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Aug 21 11:48:51 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java    | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/736ceab2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java
index 421769d..43eb783 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java
@@ -326,8 +326,10 @@ public class SFTPFileSystem extends FileSystem {
         String parentDir = parent.toUri().getPath();
         boolean succeeded = true;
         try {
+          final String previousCwd = client.pwd();
           client.cd(parentDir);
           client.mkdir(pathName);
+          client.cd(previousCwd);
         } catch (SftpException e) {
           throw new IOException(String.format(E_MAKE_DIR_FORPATH, pathName,
               parentDir));
@@ -474,8 +476,10 @@ public class SFTPFileSystem extends FileSystem {
     }
     boolean renamed = true;
     try {
+      final String previousCwd = channel.pwd();
       channel.cd("/");
       channel.rename(src.toUri().getPath(), dst.toUri().getPath());
+      channel.cd(previousCwd);
     } catch (SftpException e) {
       renamed = false;
     }
@@ -558,8 +562,10 @@ public class SFTPFileSystem extends FileSystem {
     }
     OutputStream os;
     try {
+      final String previousCwd = client.pwd();
       client.cd(parent.toUri().getPath());
       os = client.put(f.getName());
+      client.cd(previousCwd);
     } catch (SftpException e) {
       throw new IOException(e);
     }


---------------------------------------------------------------------
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: HDFS-12316. Verify HDFS snapshot deletion doesn't crash the ongoing file writes.

Posted by ae...@apache.org.
HDFS-12316. Verify HDFS snapshot deletion doesn't crash the ongoing file writes.


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

Branch: refs/heads/HDFS-7240
Commit: 4230872dd66d748172903b1522885b03f34bbf9b
Parents: b298948
Author: Manoj Govindassamy <ma...@apache.org>
Authored: Thu Aug 17 16:23:48 2017 -0700
Committer: Manoj Govindassamy <ma...@apache.org>
Committed: Thu Aug 17 16:23:48 2017 -0700

----------------------------------------------------------------------
 .../snapshot/TestOpenFilesWithSnapshot.java     | 109 +++++++++++++++++++
 1 file changed, 109 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4230872d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
index fb83a3e..bf27f2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
@@ -23,7 +23,11 @@ import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.Random;
 import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
@@ -38,12 +42,15 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.util.Time;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 public class TestOpenFilesWithSnapshot {
+  private static final Log LOG =
+      LogFactory.getLog(TestOpenFilesWithSnapshot.class.getName());
   private final Configuration conf = new Configuration();
   MiniDFSCluster cluster = null;
   DistributedFileSystem fs = null;
@@ -622,6 +629,108 @@ public class TestOpenFilesWithSnapshot {
     hbaseOutputStream.close();
   }
 
+  /**
+   * Test client writing to open files are not interrupted when snapshots
+   * that captured open files get deleted.
+   */
+  @Test (timeout = 240000)
+  public void testOpenFileWritingAcrossSnapDeletion() throws Exception {
+    final Path snapRootDir = new Path("/level_0_A");
+    final String flumeFileName = "flume.log";
+    final String hbaseFileName = "hbase.log";
+    final String snap1Name = "snap_1";
+    final String snap2Name = "snap_2";
+    final String snap3Name = "snap_3";
+
+    // Create files and open streams
+    final Path flumeFile = new Path(snapRootDir, flumeFileName);
+    FSDataOutputStream flumeOut = fs.create(flumeFile, false,
+        8000, (short)3, 1048576);
+    flumeOut.close();
+    final Path hbaseFile = new Path(snapRootDir, hbaseFileName);
+    FSDataOutputStream hbaseOut = fs.create(hbaseFile, false,
+        8000, (short)3, 1048576);
+    hbaseOut.close();
+
+    final AtomicBoolean writerError = new AtomicBoolean(false);
+    final CountDownLatch startLatch = new CountDownLatch(1);
+    final CountDownLatch deleteLatch = new CountDownLatch(1);
+    Thread t = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          FSDataOutputStream flumeOutputStream = fs.append(flumeFile, 8000);
+          FSDataOutputStream hbaseOutputStream = fs.append(hbaseFile, 8000);
+          byte[] bytes = new byte[(int) (1024 * 0.2)];
+          Random r = new Random(Time.now());
+
+          for (int i = 0; i < 200000; i++) {
+            r.nextBytes(bytes);
+            flumeOutputStream.write(bytes);
+            if (hbaseOutputStream != null) {
+              hbaseOutputStream.write(bytes);
+            }
+            if (i == 50000) {
+              startLatch.countDown();
+            } else if (i == 100000) {
+              deleteLatch.countDown();
+            } else if (i == 150000) {
+              hbaseOutputStream.hsync();
+              fs.delete(hbaseFile, true);
+              try {
+                hbaseOutputStream.close();
+              } catch (Exception e) {
+                // since the file is deleted before the open stream close,
+                // it might throw FileNotFoundException. Ignore the
+                // expected exception.
+              }
+              hbaseOutputStream = null;
+            } else if (i % 5000 == 0) {
+              LOG.info("Write pos: " + flumeOutputStream.getPos()
+                  + ", size: " + fs.getFileStatus(flumeFile).getLen()
+                  + ", loop: " + (i + 1));
+            }
+          }
+        } catch (Exception e) {
+          LOG.warn("Writer error: " + e);
+          writerError.set(true);
+        }
+      }
+    });
+    t.start();
+
+    startLatch.await();
+    final Path snap1Dir = SnapshotTestHelper.createSnapshot(
+        fs, snapRootDir, snap1Name);
+    final Path flumeS1Path = new Path(snap1Dir, flumeFileName);
+    LOG.info("Snap1 file status: " + fs.getFileStatus(flumeS1Path));
+    LOG.info("Current file status: " + fs.getFileStatus(flumeFile));
+
+    deleteLatch.await();
+    LOG.info("Snap1 file status: " + fs.getFileStatus(flumeS1Path));
+    LOG.info("Current file status: " + fs.getFileStatus(flumeFile));
+
+    // Verify deletion of snapshot which had the under construction file
+    // captured is not truncating the under construction file and the thread
+    // writing to the same file not crashing on newer block allocations.
+    LOG.info("Deleting " + snap1Name);
+    fs.deleteSnapshot(snapRootDir, snap1Name);
+
+    // Verify creation and deletion of snapshot newer than the oldest
+    // snapshot is not crashing the thread writing to under construction file.
+    SnapshotTestHelper.createSnapshot(fs, snapRootDir, snap2Name);
+    SnapshotTestHelper.createSnapshot(fs, snapRootDir, snap3Name);
+    fs.deleteSnapshot(snapRootDir, snap3Name);
+    fs.deleteSnapshot(snapRootDir, snap2Name);
+    SnapshotTestHelper.createSnapshot(fs, snapRootDir, "test");
+
+    t.join();
+    Assert.assertFalse("Client encountered writing error!", writerError.get());
+
+    restartNameNode();
+    cluster.waitActive();
+  }
+
   private void restartNameNode() throws Exception {
     cluster.triggerBlockReports();
     NameNode nameNode = cluster.getNameNode();


---------------------------------------------------------------------
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: Revert "HADOOP-14732. ProtobufRpcEngine should use Time.monotonicNow to measure durations. Contributed by Hanisha Koneru."

Posted by ae...@apache.org.
Revert "HADOOP-14732. ProtobufRpcEngine should use Time.monotonicNow to measure durations. Contributed by Hanisha Koneru."

This reverts commit 8bef4eca28a3466707cc4ea0de0330449319a5eb.


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

Branch: refs/heads/HDFS-7240
Commit: 2d105a206884b62ccdba61f2de3e2fe65fc43074
Parents: e05fa34
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Aug 18 10:15:52 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Aug 18 10:15:52 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/ipc/ProtobufRpcEngine.java | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d105a20/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
index 2c0cfe5..639bbad 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -190,7 +190,7 @@ public class ProtobufRpcEngine implements RpcEngine {
         throws ServiceException {
       long startTime = 0;
       if (LOG.isDebugEnabled()) {
-        startTime = Time.monotonicNow();
+        startTime = Time.now();
       }
       
       if (args.length != 2) { // RpcController + Message
@@ -245,7 +245,7 @@ public class ProtobufRpcEngine implements RpcEngine {
       }
 
       if (LOG.isDebugEnabled()) {
-        long callTime = Time.monotonicNow() - startTime;
+        long callTime = Time.now() - startTime;
         LOG.debug("Call: " + method.getName() + " took " + callTime + "ms");
       }
       
@@ -373,19 +373,19 @@ public class ProtobufRpcEngine implements RpcEngine {
         this.server = currentCallInfo.get().server;
         this.call = Server.getCurCall().get();
         this.methodName = currentCallInfo.get().methodName;
-        this.setupTime = Time.monotonicNow();
+        this.setupTime = Time.now();
       }
 
       @Override
       public void setResponse(Message message) {
-        long processingTime = Time.monotonicNow() - setupTime;
+        long processingTime = Time.now() - setupTime;
         call.setDeferredResponse(RpcWritable.wrap(message));
         server.updateDeferredMetrics(methodName, processingTime);
       }
 
       @Override
       public void error(Throwable t) {
-        long processingTime = Time.monotonicNow() - setupTime;
+        long processingTime = Time.now() - setupTime;
         String detailedMetricsName = t.getClass().getSimpleName();
         server.updateDeferredMetrics(detailedMetricsName, processingTime);
         call.setDeferredError(t);
@@ -513,7 +513,7 @@ public class ProtobufRpcEngine implements RpcEngine {
         Message param = request.getValue(prototype);
 
         Message result;
-        long startTime = Time.monotonicNow();
+        long startTime = Time.now();
         int qTime = (int) (startTime - receiveTime);
         Exception exception = null;
         boolean isDeferred = false;
@@ -537,7 +537,7 @@ public class ProtobufRpcEngine implements RpcEngine {
           throw e;
         } finally {
           currentCallInfo.set(null);
-          int processingTime = (int) (Time.monotonicNow() - startTime);
+          int processingTime = (int) (Time.now() - startTime);
           if (LOG.isDebugEnabled()) {
             String msg =
                 "Served: " + methodName + (isDeferred ? ", deferred" : "") +


---------------------------------------------------------------------
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-6988. container-executor fails for docker when command length > 4096 B. Contributed by Eric Badger

Posted by ae...@apache.org.
YARN-6988. container-executor fails for docker when command length > 4096 B. Contributed by Eric Badger


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

Branch: refs/heads/HDFS-7240
Commit: ab1a8ae85f8c61304a0f437cdc61cc5aeda36a4b
Parents: dd7916d
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Aug 17 15:50:14 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Aug 17 15:50:14 2017 -0500

----------------------------------------------------------------------
 .../impl/container-executor.c                   | 38 +++++++++++++-------
 .../main/native/container-executor/impl/util.h  |  7 ++++
 2 files changed, 33 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab1a8ae8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
index 9f754c4..7361808 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
@@ -1417,9 +1417,10 @@ int run_docker(const char *command_file) {
   char* docker_command = parse_docker_command_file(command_file);
   char* docker_binary = get_section_value(DOCKER_BINARY_KEY, &executor_cfg);
   docker_binary = check_docker_binary(docker_binary);
+  size_t command_size = MIN(sysconf(_SC_ARG_MAX), 128*1024);
 
-  char* docker_command_with_binary = calloc(sizeof(char), EXECUTOR_PATH_MAX);
-  snprintf(docker_command_with_binary, EXECUTOR_PATH_MAX, "%s %s", docker_binary, docker_command);
+  char* docker_command_with_binary = calloc(sizeof(char), command_size);
+  snprintf(docker_command_with_binary, command_size, "%s %s", docker_binary, docker_command);
   char **args = split_delimiter(docker_command_with_binary, " ");
 
   int exit_code = -1;
@@ -1567,16 +1568,24 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
   char *script_file_dest = NULL;
   char *cred_file_dest = NULL;
   char *exit_code_file = NULL;
-  char docker_command_with_binary[EXECUTOR_PATH_MAX];
-  char docker_wait_command[EXECUTOR_PATH_MAX];
-  char docker_logs_command[EXECUTOR_PATH_MAX];
-  char docker_inspect_command[EXECUTOR_PATH_MAX];
-  char docker_rm_command[EXECUTOR_PATH_MAX];
+  char *docker_command_with_binary = NULL;
+  char *docker_wait_command = NULL;
+  char *docker_logs_command = NULL;
+  char *docker_inspect_command = NULL;
+  char *docker_rm_command = NULL;
   int container_file_source =-1;
   int cred_file_source = -1;
   int BUFFER_SIZE = 4096;
   char buffer[BUFFER_SIZE];
 
+  size_t command_size = MIN(sysconf(_SC_ARG_MAX), 128*1024);
+
+  docker_command_with_binary = calloc(sizeof(char), command_size);
+  docker_wait_command = calloc(sizeof(char), command_size);
+  docker_logs_command = calloc(sizeof(char), command_size);
+  docker_inspect_command = calloc(sizeof(char), command_size);
+  docker_rm_command = calloc(sizeof(char), command_size);
+
   gid_t user_gid = getegid();
   uid_t prev_uid = geteuid();
 
@@ -1621,7 +1630,7 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
     goto cleanup;
   }
 
-  snprintf(docker_command_with_binary, EXECUTOR_PATH_MAX, "%s %s", docker_binary, docker_command);
+  snprintf(docker_command_with_binary, command_size, "%s %s", docker_binary, docker_command);
 
   fprintf(LOGFILE, "Launching docker container...\n");
   FILE* start_docker = popen(docker_command_with_binary, "r");
@@ -1634,7 +1643,7 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
     goto cleanup;
   }
 
-  snprintf(docker_inspect_command, EXECUTOR_PATH_MAX,
+  snprintf(docker_inspect_command, command_size,
     "%s inspect --format {{.State.Pid}} %s",
     docker_binary, container_id);
 
@@ -1679,7 +1688,7 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
       goto cleanup;
     }
 
-    snprintf(docker_wait_command, EXECUTOR_PATH_MAX,
+    snprintf(docker_wait_command, command_size,
       "%s wait %s", docker_binary, container_id);
 
     fprintf(LOGFILE, "Waiting for docker container to finish...\n");
@@ -1693,7 +1702,7 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
     if(exit_code != 0) {
       fprintf(ERRORFILE, "Docker container exit code was not zero: %d\n",
       exit_code);
-      snprintf(docker_logs_command, EXECUTOR_PATH_MAX, "%s logs --tail=250 %s",
+      snprintf(docker_logs_command, command_size, "%s logs --tail=250 %s",
         docker_binary, container_id);
       FILE* logs = popen(docker_logs_command, "r");
       if(logs != NULL) {
@@ -1723,7 +1732,7 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
   }
 
   fprintf(LOGFILE, "Removing docker container post-exit...\n");
-  snprintf(docker_rm_command, EXECUTOR_PATH_MAX,
+  snprintf(docker_rm_command, command_size,
     "%s rm %s", docker_binary, container_id);
   FILE* rm_docker = popen(docker_rm_command, "w");
   if (pclose (rm_docker) != 0)
@@ -1763,6 +1772,11 @@ cleanup:
   free(exit_code_file);
   free(script_file_dest);
   free(cred_file_dest);
+  free(docker_command_with_binary);
+  free(docker_wait_command);
+  free(docker_logs_command);
+  free(docker_inspect_command);
+  free(docker_rm_command);
   return exit_code;
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab1a8ae8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h
index a8a12a9..fa21def 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h
@@ -66,6 +66,13 @@ enum errorcodes {
   ERROR_COMPILING_REGEX = 42
 };
 
+/* Macros for min/max. */
+#ifndef MIN
+#define MIN(a,b) (((a)<(b))?(a):(b))
+#endif /* MIN */
+#ifndef MAX
+#define MAX(a,b) (((a)>(b))?(a):(b))
+#endif  /* MAX */
 
 // the log file for messages
 extern FILE *LOGFILE;


---------------------------------------------------------------------
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: HDFS-12301. NN File Browser UI: Navigate to a path when enter is pressed

Posted by ae...@apache.org.
HDFS-12301. NN File Browser UI: Navigate to a path when enter is pressed


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

Branch: refs/heads/HDFS-7240
Commit: f34646d652310442cb5339aabbbb269f10dfa838
Parents: d265459
Author: Ravi Prakash <ra...@altiscale.com>
Authored: Tue Aug 15 15:44:59 2017 -0700
Committer: Ravi Prakash <ra...@altiscale.com>
Committed: Tue Aug 15 15:44:59 2017 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/webapps/hdfs/explorer.js              | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f34646d6/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
index 3e276a9..dae3519 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
@@ -370,6 +370,12 @@
 
     var b = function() { browse_directory($('#directory').val()); };
     $('#btn-nav-directory').click(b);
+    //Also navigate to the directory when a user presses enter.
+    $('#directory').on('keyup', function (e) {
+      if (e.which == 13) {
+        browse_directory($('#directory').val());
+      }
+    });
     var dir = window.location.hash.slice(1);
     if(dir == "") {
       window.location.hash = "/";


---------------------------------------------------------------------
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-6996. Change javax.cache library implementation from JSR107 to Apache Geronimo. (Ray Chiang via Subru).

Posted by ae...@apache.org.
YARN-6996. Change javax.cache library implementation from JSR107 to Apache Geronimo. (Ray Chiang via Subru).


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

Branch: refs/heads/HDFS-7240
Commit: 18f3603bce37e0e07c9075811b1179afc2c227eb
Parents: e2f6299
Author: Subru Krishnan <su...@apache.org>
Authored: Mon Aug 14 11:10:00 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Aug 14 11:10:00 2017 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                                         | 6 +++---
 .../hadoop-yarn-server/hadoop-yarn-server-common/pom.xml       | 4 ++--
 2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/18f3603b/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 6311cd9..8c1d374 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -96,7 +96,7 @@
     <apacheds.version>2.0.0-M21</apacheds.version>
     <ldap-api.version>1.0.0-M33</ldap-api.version>
 
-    <jcache.version>1.0.0</jcache.version>
+    <jcache.version>1.0-alpha-1</jcache.version>
     <ehcache.version>3.3.1</ehcache.version>
     <hikari.version>2.4.12</hikari.version>
     <mssql.version>6.2.1.jre7</mssql.version>
@@ -1276,8 +1276,8 @@
           <version>1.0.0</version>
         </dependency>
         <dependency>
-          <groupId>javax.cache</groupId>
-          <artifactId>cache-api</artifactId>
+          <groupId>org.apache.geronimo.specs</groupId>
+          <artifactId>geronimo-jcache_1.0_spec</artifactId>
           <version>${jcache.version}</version>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/18f3603b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
index 5f85097..441a574 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
@@ -103,8 +103,8 @@
       <artifactId>leveldbjni-all</artifactId>
     </dependency>
     <dependency>
-      <groupId>javax.cache</groupId>
-      <artifactId>cache-api</artifactId>
+      <groupId>org.apache.geronimo.specs</groupId>
+      <artifactId>geronimo-jcache_1.0_spec</artifactId>
     </dependency>
     <dependency>
       <groupId>org.ehcache</groupId>


---------------------------------------------------------------------
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: YARN-7014. Fix off-by-one error causing heap corruption (Jason Lowe via nroberts)

Posted by ae...@apache.org.
YARN-7014. Fix off-by-one error causing heap corruption (Jason Lowe via nroberts)


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

Branch: refs/heads/HDFS-7240
Commit: d265459024b8e5f5eccf421627f684ca8f162112
Parents: dadb0c2
Author: Nathan Roberts <nr...@apache.org>
Authored: Tue Aug 15 15:52:48 2017 -0500
Committer: Nathan Roberts <nr...@apache.org>
Committed: Tue Aug 15 15:52:48 2017 -0500

----------------------------------------------------------------------
 .../src/main/native/container-executor/impl/utils/string-utils.c  | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2654590/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
index 703d484..063df7e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
@@ -44,8 +44,7 @@ int validate_container_id(const char* input) {
    * container_e17_1410901177871_0001_01_000005
    * container_1410901177871_0001_01_000005
    */
-  char* input_cpy = malloc(strlen(input));
-  strcpy(input_cpy, input);
+  char* input_cpy = strdup(input);
   char* p = strtok(input_cpy, "_");
   int idx = 0;
   while (p != NULL) {


---------------------------------------------------------------------
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: HDFS-12066. When Namenode is in safemode, may not allowed to remove an user's erasure coding policy. Contributed by lufei.

Posted by ae...@apache.org.
HDFS-12066. When Namenode is in safemode,may not allowed to remove an user's erasure coding policy. Contributed by lufei.


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

Branch: refs/heads/HDFS-7240
Commit: e3ae3e26446c2e98b7aebc4ea66256cfdb4a397f
Parents: 1040bae
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Tue Aug 15 07:41:10 2017 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Tue Aug 15 07:41:43 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java    | 3 +++
 .../src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java  | 9 +++++++++
 2 files changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3ae3e26/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 caf73f7..1cfaa54 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
@@ -7113,6 +7113,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean success = false;
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot remove erasure coding policy "
+          + ecPolicyName);
       FSDirErasureCodingOp.removeErasureCodePolicy(this, ecPolicyName);
       success = true;
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3ae3e26/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
index bc95ec7..f25d28f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
@@ -482,6 +482,15 @@ public class TestSafeMode {
       // expected
     }
 
+    try {
+      dfs.removeErasureCodingPolicy("testECName");
+      fail("RemoveErasureCodingPolicy should have failed.");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains(
+          "Cannot remove erasure coding policy", ioe);
+      // expected
+    }
+
     assertFalse("Could not leave SM",
         dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE));
   }


---------------------------------------------------------------------
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-6987. Log app attempt during InvalidStateTransition. Contributed by Jonathan Eagles

Posted by ae...@apache.org.
YARN-6987. Log app attempt during InvalidStateTransition. Contributed by Jonathan Eagles


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

Branch: refs/heads/HDFS-7240
Commit: 3325ef653d6f364a82dd32485d9ef6d987380ce3
Parents: 6b09c32
Author: Jason Lowe <jl...@yahoo-inc.com>
Authored: Mon Aug 14 14:40:08 2017 -0500
Committer: Jason Lowe <jl...@yahoo-inc.com>
Committed: Mon Aug 14 14:40:08 2017 -0500

----------------------------------------------------------------------
 .../hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java       | 3 ++-
 .../server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java    | 3 ++-
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3325ef65/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.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/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index fa2f20c..03be793 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -888,7 +888,8 @@ public class RMAppImpl implements RMApp, Recoverable {
         /* keep the master in sync with the state machine */
         this.stateMachine.doTransition(event.getType(), event);
       } catch (InvalidStateTransitionException e) {
-        LOG.error("Can't handle this event at current state", e);
+        LOG.error("App: " + appID
+            + " can't handle this event at current state", e);
         /* TODO fail the application on the failed transition */
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3325ef65/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.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/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index 254768b..7d453bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -911,7 +911,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         /* keep the master in sync with the state machine */
         this.stateMachine.doTransition(event.getType(), event);
       } catch (InvalidStateTransitionException e) {
-        LOG.error("Can't handle this event at current state", e);
+        LOG.error("App attempt: " + appAttemptID
+            + " can't handle this event at current state", e);
         /* TODO fail the application on the failed transition */
       }
 


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