소스 검색

Merge branch 'trunk' into HDFS-7240

Anu Engineer 7 년 전
부모
커밋
215a94253c
100개의 변경된 파일2398개의 추가작업 그리고 473개의 파일을 삭제
  1. 2 2
      BUILDING.txt
  2. 14 0
      NOTICE.txt
  3. 1 1
      dev-support/docker/Dockerfile
  4. 6 1
      hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml
  5. 12 0
      hadoop-common-project/hadoop-auth/pom.xml
  6. 4 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicy.java
  7. 83 76
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
  8. 12 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
  9. 3 1
      hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
  10. 8 0
      hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm
  11. 40 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
  12. 1 1
      hadoop-common-project/hadoop-kms/pom.xml
  13. 15 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  14. 21 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  15. 1 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  16. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  17. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
  18. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
  19. 36 35
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
  20. 25 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  21. 18 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
  22. 33 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java
  23. 0 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java
  24. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
  25. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
  26. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffList.java
  27. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffListByArrayList.java
  28. 545 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryDiffList.java
  29. 6 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
  30. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java
  31. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ReadOnlyList.java
  32. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  33. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
  34. 7 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.html
  35. 58 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.js
  36. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  37. 33 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
  38. 10 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java
  39. 25 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
  40. 133 32
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java
  41. 128 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
  42. 10 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCClientRetries.java
  43. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
  44. 264 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestDirectoryDiffList.java
  45. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
  46. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
  47. 4 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java
  48. 31 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobHistoryUtils.java
  49. 24 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/jobhistory/TestJobHistoryUtils.java
  50. 22 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
  51. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  52. 14 2
      hadoop-project/pom.xml
  53. 1 0
      hadoop-tools/hadoop-azure-datalake/pom.xml
  54. 4 3
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/http/ExceptionDiags.java
  55. 4 3
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/http/RestClientBindings.java
  56. 5 4
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/http/SwiftRestClient.java
  57. 4 4
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystem.java
  58. 5 5
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
  59. 4 3
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeInputStream.java
  60. 4 4
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeOutputStream.java
  61. 4 4
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/util/SwiftTestUtils.java
  62. 4 4
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/util/SwiftUtils.java
  63. 4 4
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/SwiftFileSystemBaseTest.java
  64. 4 4
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestLogResources.java
  65. 4 4
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestReadPastBuffer.java
  66. 4 4
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSeek.java
  67. 4 4
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemBasicOps.java
  68. 4 4
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemConcurrency.java
  69. 4 4
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemContract.java
  70. 4 3
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftObjectPath.java
  71. 5 5
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/http/TestSwiftRestClient.java
  72. 4 3
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/scale/TestWriteManySmallFiles.java
  73. 1 1
      hadoop-tools/hadoop-sls/pom.xml
  74. 17 6
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
  75. 13 8
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
  76. 6 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/conf/SLSConfiguration.java
  77. 22 6
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java
  78. 19 2
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ContainerSimulator.java
  79. 3 1
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
  80. 3 1
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java
  81. 14 4
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java
  82. 13 1
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java
  83. 3 0
      hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java
  84. 10 1
      hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/nodemanager/TestNMSimulator.java
  85. 10 5
      hadoop-tools/hadoop-sls/src/test/resources/inputsls.json
  86. 2 0
      hadoop-tools/hadoop-sls/src/test/resources/syn.json
  87. 4 2
      hadoop-tools/hadoop-sls/src/test/resources/syn_generic.json
  88. 2 1
      hadoop-tools/hadoop-sls/src/test/resources/syn_stream.json
  89. 140 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java
  90. 88 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintParser.java
  91. 12 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
  92. 6 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java
  93. 63 55
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/TestApiServer.java
  94. 21 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
  95. 2 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
  96. 9 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java
  97. 43 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java
  98. 7 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java
  99. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
  100. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceSet.java

+ 2 - 2
BUILDING.txt

@@ -331,10 +331,10 @@ If the build process fails with an out of memory error, you should be able to fi
 it by increasing the memory used by maven which can be done via the environment
 variable MAVEN_OPTS.
 
-Here is an example setting to allocate between 256 MB and 1 GB of heap space to
+Here is an example setting to allocate between 256 MB and 1.5 GB of heap space to
 Maven
 
-export MAVEN_OPTS="-Xms256m -Xmx1g"
+export MAVEN_OPTS="-Xms256m -Xmx1536m"
 
 ----------------------------------------------------------------------------------
 

+ 14 - 0
NOTICE.txt

@@ -591,3 +591,17 @@ The binary distribution of this product bundles binaries of
 swagger-annotations (https://github.com/swagger-api/swagger-core),
 which has the following notices:
  * Copyright 2016 SmartBear Software
+
+The binary distribution of this product bundles binaries of
+metrics-core 3.2.4
+which has the following notices:
+ * Copyright 2010-2013 Coda Hale and Yammer, Inc.
+
+   This product includes software developed by Coda Hale and Yammer, Inc.
+
+   This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+   LongAdder), which was released with the following comments:
+
+   Written by Doug Lea with assistance from members of JCP JSR-166
+   Expert Group and released to the public domain, as explained at
+   http://creativecommons.org/publicdomain/zero/1.0/

+ 1 - 1
dev-support/docker/Dockerfile

@@ -173,7 +173,7 @@ RUN apt-get -y install nodejs && \
 ###
 # Avoid out of memory errors in builds
 ###
-ENV MAVEN_OPTS -Xms256m -Xmx1g
+ENV MAVEN_OPTS -Xms256m -Xmx1536m
 
 
 ###

+ 6 - 1
hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml

@@ -47,7 +47,7 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
-      <scope>compile</scope>
+      <scope>provided</scope>
       <exclusions>
         <exclusion>
           <groupId>javax.servlet</groupId>
@@ -118,6 +118,11 @@
       <artifactId>hadoop-azure</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-azure-datalake</artifactId>
+      <scope>compile</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-openstack</artifactId>

+ 12 - 0
hadoop-common-project/hadoop-auth/pom.xml

@@ -110,8 +110,20 @@
           <groupId>org.bouncycastle</groupId>
           <artifactId>bcprov-jdk15on</artifactId>
         </exclusion>
+        <!-- HACK.  Transitive dependency for nimbus-jose-jwt.  Needed for
+        packaging.  Please re-check this version when updating
+        nimbus-jose-jwt.  Please read HADOOP-14903 for more details.
+        -->
+        <exclusion>
+          <groupId>net.minidev</groupId>
+          <artifactId>json-smart</artifactId>
+        </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>net.minidev</groupId>
+      <artifactId>json-smart</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.zookeeper</groupId>
       <artifactId>zookeeper</artifactId>

+ 4 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicy.java

@@ -89,9 +89,10 @@ public interface RetryPolicy {
    *          {@link Idempotent} or {@link AtMostOnce} and so can reasonably be
    *          retried on failover when we don't know if the previous attempt
    *          reached the server or not
-   * @return <code>true</code> if the method should be retried,
-   *         <code>false</code> if the method should not be retried but
-   *         shouldn't fail with an exception (only for void methods)
+   * @return {@link RetryAction} with {@code RetryDecision.FAIL} if the method
+   *          should not be retried, {@code RetryDecision.RETRY} if the method
+   *          should be retried or {@code RetryDecision.FAILOVER_AND_RETRY}
+   *          if failover has to be performed before retry.
    * @throws Exception The re-thrown exception <code>e</code> indicating that
    *           the method failed and should not be retried further
    */

+ 83 - 76
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java

@@ -715,9 +715,9 @@ public class NetUtils {
    * return an IOException with the input exception as the cause and also
    * include the host details. The new exception provides the stack trace of the
    * place where the exception is thrown and some extra diagnostics information.
-   * If the exception is BindException or ConnectException or
-   * UnknownHostException or SocketTimeoutException, return a new one of the
-   * same type; Otherwise return an IOException.
+   * If the exception is of type BindException, ConnectException,
+   * UnknownHostException, SocketTimeoutException or has a String constructor,
+   * return a new one of the same type; Otherwise return an IOException.
    *
    * @param destHost target host (nullable)
    * @param destPort target port
@@ -731,83 +731,90 @@ public class NetUtils {
                                           final String localHost,
                                           final int localPort,
                                           final IOException exception) {
-    if (exception instanceof BindException) {
-      return wrapWithMessage(exception,
-          "Problem binding to ["
-              + localHost
-              + ":"
-              + localPort
-              + "] "
-              + exception
-              + ";"
-              + see("BindException"));
-    } else if (exception instanceof ConnectException) {
-      // Check if client was trying to connect to an unspecified IPv4 address
-      // (0.0.0.0) or IPv6 address(0:0:0:0:0:0:0:0 or ::)
-      if ((destHost != null && (destHost.equals("0.0.0.0") ||
-          destHost.equals("0:0:0:0:0:0:0:0") || destHost.equals("::")))
-          || destPort == 0) {
-        return wrapWithMessage(exception, "Your endpoint configuration" +
-            " is wrong;" + see("UnsetHostnameOrPort"));
-      } else {
-        // connection refused; include the host:port in the error
+    try {
+      if (exception instanceof BindException) {
         return wrapWithMessage(exception,
-            "Call From "
+            "Problem binding to ["
                 + localHost
-                + " to "
-                + destHost
                 + ":"
-                + destPort
-                + " failed on connection exception: "
+                + localPort
+                + "] "
                 + exception
                 + ";"
-                + see("ConnectionRefused"));
+                + see("BindException"));
+      } else if (exception instanceof ConnectException) {
+        // Check if client was trying to connect to an unspecified IPv4 address
+        // (0.0.0.0) or IPv6 address(0:0:0:0:0:0:0:0 or ::)
+        if ((destHost != null && (destHost.equals("0.0.0.0") ||
+            destHost.equals("0:0:0:0:0:0:0:0") || destHost.equals("::")))
+            || destPort == 0) {
+          return wrapWithMessage(exception, "Your endpoint configuration" +
+              " is wrong;" + see("UnsetHostnameOrPort"));
+        } else {
+          // connection refused; include the host:port in the error
+          return wrapWithMessage(exception,
+              "Call From "
+                  + localHost
+                  + " to "
+                  + destHost
+                  + ":"
+                  + destPort
+                  + " failed on connection exception: "
+                  + exception
+                  + ";"
+                  + see("ConnectionRefused"));
+        }
+      } else if (exception instanceof UnknownHostException) {
+        return wrapWithMessage(exception,
+            "Invalid host name: "
+                + getHostDetailsAsString(destHost, destPort, localHost)
+                + exception
+                + ";"
+                + see("UnknownHost"));
+      } else if (exception instanceof SocketTimeoutException) {
+        return wrapWithMessage(exception,
+            "Call From "
+                + localHost + " to " + destHost + ":" + destPort
+                + " failed on socket timeout exception: " + exception
+                + ";"
+                + see("SocketTimeout"));
+      } else if (exception instanceof NoRouteToHostException) {
+        return wrapWithMessage(exception,
+            "No Route to Host from  "
+                + localHost + " to " + destHost + ":" + destPort
+                + " failed on socket timeout exception: " + exception
+                + ";"
+                + see("NoRouteToHost"));
+      } else if (exception instanceof EOFException) {
+        return wrapWithMessage(exception,
+            "End of File Exception between "
+                + getHostDetailsAsString(destHost, destPort, localHost)
+                + ": " + exception
+                + ";"
+                + see("EOFException"));
+      } else if (exception instanceof SocketException) {
+        // Many of the predecessor exceptions are subclasses of SocketException,
+        // so must be handled before this
+        return wrapWithMessage(exception,
+            "Call From "
+                + localHost + " to " + destHost + ":" + destPort
+                + " failed on socket exception: " + exception
+                + ";"
+                + see("SocketException"));
+      } else {
+        // Return instance of same type if Exception has a String constructor
+        return wrapWithMessage(exception,
+            "DestHost:destPort " + destHost + ":" + destPort
+                + " , LocalHost:localPort " + localHost
+                + ":" + localPort + ". Failed on local exception: " +
+                exception);
+
       }
-    } else if (exception instanceof UnknownHostException) {
-      return wrapWithMessage(exception,
-          "Invalid host name: "
-              + getHostDetailsAsString(destHost, destPort, localHost)
-              + exception
-              + ";"
-              + see("UnknownHost"));
-    } else if (exception instanceof SocketTimeoutException) {
-      return wrapWithMessage(exception,
-          "Call From "
-              + localHost + " to " + destHost + ":" + destPort
-              + " failed on socket timeout exception: " + exception
-              + ";"
-              + see("SocketTimeout"));
-    } else if (exception instanceof NoRouteToHostException) {
-      return wrapWithMessage(exception,
-          "No Route to Host from  "
-              + localHost + " to " + destHost + ":" + destPort
-              + " failed on socket timeout exception: " + exception
-              + ";"
-              + see("NoRouteToHost"));
-    } else if (exception instanceof EOFException) {
-      return wrapWithMessage(exception,
-          "End of File Exception between "
-              + getHostDetailsAsString(destHost,  destPort, localHost)
-              + ": " + exception
-              + ";"
-              + see("EOFException"));
-    } else if (exception instanceof SocketException) {
-      // Many of the predecessor exceptions are subclasses of SocketException,
-      // so must be handled before this
-      return wrapWithMessage(exception,
-          "Call From "
-              + localHost + " to " + destHost + ":" + destPort
-              + " failed on socket exception: " + exception
-              + ";"
-              + see("SocketException"));
-    }
-    else {
+    } catch (IOException ex) {
       return (IOException) new IOException("Failed on local exception: "
-             + exception
-             + "; Host Details : "
-             + getHostDetailsAsString(destHost, destPort, localHost))
+          + exception + "; Host Details : "
+          + getHostDetailsAsString(destHost, destPort, localHost))
           .initCause(exception);
-
     }
   }
 
@@ -817,16 +824,16 @@ public class NetUtils {
   
   @SuppressWarnings("unchecked")
   private static <T extends IOException> T wrapWithMessage(
-      T exception, String msg) {
+      T exception, String msg) throws T {
     Class<? extends Throwable> clazz = exception.getClass();
     try {
       Constructor<? extends Throwable> ctor = clazz.getConstructor(String.class);
       Throwable t = ctor.newInstance(msg);
       return (T)(t.initCause(exception));
     } catch (Throwable e) {
-      LOG.warn("Unable to wrap exception of type " +
-          clazz + ": it has no (String) constructor", e);
-      return exception;
+      LOG.warn("Unable to wrap exception of type {}: it has no (String) "
+          + "constructor", clazz, e);
+      throw exception;
     }
   }
 

+ 12 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java

@@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectReader;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
@@ -138,11 +139,17 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
   public void authenticate(URL url, AuthenticatedURL.Token token)
       throws IOException, AuthenticationException {
     if (!hasDelegationToken(url, token)) {
-      // check and renew TGT to handle potential expiration
-      UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
-      LOG.debug("No delegation token found for url={}, token={}, authenticating"
-          + " with {}", url, token, authenticator.getClass());
-      authenticator.authenticate(url, token);
+      try {
+        // check and renew TGT to handle potential expiration
+        UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
+        LOG.debug("No delegation token found for url={}, token={}, "
+            + "authenticating with {}", url, token, authenticator.getClass());
+        authenticator.authenticate(url, token);
+      } catch (IOException ex) {
+        throw NetUtils.wrapException(url.getHost(), url.getPort(),
+            null, 0, ex);
+      }
+
     } else {
       LOG.debug("Authenticated from delegation token. url={}, token={}",
           url, token);

+ 3 - 1
hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md

@@ -167,7 +167,9 @@ Because the DataNode data transfer protocol does not use the Hadoop RPC framewor
 
 When you execute the `hdfs datanode` command as root, the server process binds privileged ports at first, then drops privilege and runs as the user account specified by `HDFS_DATANODE_SECURE_USER`. This startup process uses [the jsvc program](https://commons.apache.org/proper/commons-daemon/jsvc.html "Link to Apache Commons Jsvc") installed to `JSVC_HOME`. You must specify `HDFS_DATANODE_SECURE_USER` and `JSVC_HOME` as environment variables on start up (in `hadoop-env.sh`).
 
-As of version 2.6.0, SASL can be used to authenticate the data transfer protocol. In this configuration, it is no longer required for secured clusters to start the DataNode as root using `jsvc` and bind to privileged ports. To enable SASL on data transfer protocol, set `dfs.data.transfer.protection` in hdfs-site.xml, set a non-privileged port for `dfs.datanode.address`, set `dfs.http.policy` to `HTTPS_ONLY` and make sure the `HDFS_DATANODE_SECURE_USER` environment variable is not defined. Note that it is not possible to use SASL on data transfer protocol if `dfs.datanode.address` is set to a privileged port. This is required for backwards-compatibility reasons.
+As of version 2.6.0, SASL can be used to authenticate the data transfer protocol. In this configuration, it is no longer required for secured clusters to start the DataNode as root using `jsvc` and bind to privileged ports. To enable SASL on data transfer protocol, set `dfs.data.transfer.protection` in hdfs-site.xml. A SASL enabled DataNode can be started in secure mode in following two ways:
+1. Set a non-privileged port for `dfs.datanode.address`.
+1. Set `dfs.http.policy` to `HTTPS_ONLY` or set `dfs.datanode.http.address` to a privileged port and make sure the `HDFS_DATANODE_SECURE_USER` and `JSVC_HOME` environment variables are specified properly as environment variables on start up (in `hadoop-env.sh`).
 
 In order to migrate an existing cluster that used root authentication to start using SASL instead, first ensure that version 2.6.0 or later has been deployed to all cluster nodes as well as any external applications that need to connect to the cluster. Only versions 2.6.0 and later of the HDFS client can connect to a DataNode that uses SASL for authentication of data transfer protocol, so it is vital that all callers have the correct version before migrating. After version 2.6.0 or later has been deployed everywhere, update configuration of any external applications to enable SASL. If an HDFS client is enabled for SASL, then it can connect successfully to a DataNode running with either root authentication or SASL authentication. Changing configuration for all clients guarantees that subsequent configuration changes on DataNodes will not disrupt the applications. Finally, each individual DataNode can be migrated by changing its configuration and restarting. It is acceptable to have a mix of some DataNodes running with root authentication and some DataNodes running with SASL authentication temporarily during this migration period, because an HDFS client enabled for SASL can connect to both.
 

+ 8 - 0
hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm

@@ -192,6 +192,14 @@ The following instructions assume that 1. ~ 4. steps of [the above instructions]
             </property>
         </configuration>
 
+        <configuration>
+            <property>
+                <name>mapreduce.application.classpath</name>
+                <value>$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/*:$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/lib/*</value>
+            </property>
+        </configuration>
+
+
     `etc/hadoop/yarn-site.xml`:
 
         <configuration>

+ 40 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java

@@ -32,6 +32,7 @@ import java.net.SocketException;
 import java.net.SocketTimeoutException;
 import java.net.URI;
 import java.net.UnknownHostException;
+import java.nio.charset.CharacterCodingException;
 import java.util.Arrays;
 import java.util.Enumeration;
 import java.util.List;
@@ -40,6 +41,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.security.KerberosAuthException;
 import org.apache.hadoop.security.NetUtilsTestResolver;
 import org.junit.Assume;
 import org.junit.Before;
@@ -262,6 +264,44 @@ public class TestNetUtils {
     assertInException(wrapped, "/EOFException");
   }
 
+  @Test
+  public void testWrapKerbAuthException() throws Throwable {
+    IOException e = new KerberosAuthException("socket timeout on connection");
+    IOException wrapped = verifyExceptionClass(e, KerberosAuthException.class);
+    assertInException(wrapped, "socket timeout on connection");
+    assertInException(wrapped, "localhost");
+    assertInException(wrapped, "DestHost:destPort ");
+    assertInException(wrapped, "LocalHost:localPort");
+    assertRemoteDetailsIncluded(wrapped);
+    assertInException(wrapped, "KerberosAuthException");
+  }
+
+  @Test
+  public void testWrapIOEWithNoStringConstructor() throws Throwable {
+    IOException e = new CharacterCodingException();
+    IOException wrapped = verifyExceptionClass(e, IOException.class);
+    assertInException(wrapped, "Failed on local exception");
+    assertNotInException(wrapped, NetUtils.HADOOP_WIKI);
+    assertInException(wrapped, "Host Details ");
+    assertRemoteDetailsIncluded(wrapped);
+  }
+
+  @Test
+  public void testWrapIOEWithPrivateStringConstructor() throws Throwable {
+    class TestIOException extends CharacterCodingException{
+      private  TestIOException(String cause){
+      }
+      TestIOException(){
+      }
+    }
+    IOException e = new TestIOException();
+    IOException wrapped = verifyExceptionClass(e, IOException.class);
+    assertInException(wrapped, "Failed on local exception");
+    assertNotInException(wrapped, NetUtils.HADOOP_WIKI);
+    assertInException(wrapped, "Host Details ");
+    assertRemoteDetailsIncluded(wrapped);
+  }
+
   @Test
   public void testWrapSocketException() throws Throwable {
     IOException wrapped = verifyExceptionClass(new SocketException("failed"),

+ 1 - 1
hadoop-common-project/hadoop-kms/pom.xml

@@ -159,7 +159,7 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>com.codahale.metrics</groupId>
+      <groupId>io.dropwizard.metrics</groupId>
       <artifactId>metrics-core</artifactId>
       <scope>compile</scope>
     </dependency>

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -139,6 +139,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
@@ -2118,6 +2119,20 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   /**
    * Get the difference between two snapshots, or between a snapshot and the
    * current tree of a directory.
+   * @see ClientProtocol#getSnapshotDiffReport
+   */
+  public SnapshotDiffReport getSnapshotDiffReport(String snapshotDir,
+      String fromSnapshot, String toSnapshot) throws IOException {
+    checkOpen();
+    try (TraceScope ignored = tracer.newScope("getSnapshotDiffReport")) {
+      return namenode
+          .getSnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+  /**
+   * Get the difference between two snapshots of a directory iteratively.
    * @see ClientProtocol#getSnapshotDiffReportListing
    */
   public SnapshotDiffReportListing getSnapshotDiffReportListing(

+ 21 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -2020,8 +2020,13 @@ public class DistributedFileSystem extends FileSystem
       @Override
       public RemoteIterator<SnapshotDiffReportListing> doCall(final Path p)
           throws IOException {
-        return new SnapshotDiffReportListingIterator(
-            getPathName(p), fromSnapshot, toSnapshot);
+        if (!isValidSnapshotName(fromSnapshot) || !isValidSnapshotName(
+            toSnapshot)) {
+          throw new UnsupportedOperationException("Remote Iterator is"
+              + "supported for snapshotDiffReport between two snapshots");
+        }
+        return new SnapshotDiffReportListingIterator(getPathName(p),
+            fromSnapshot, toSnapshot);
       }
 
       @Override
@@ -2081,9 +2086,23 @@ public class DistributedFileSystem extends FileSystem
     }
   }
 
+  private boolean isValidSnapshotName(String snapshotName) {
+    // If any of the snapshots specified in the getSnapshotDiffReport call
+    // is null or empty, it points to the current tree.
+    return (snapshotName != null && !snapshotName.isEmpty());
+  }
+
   private SnapshotDiffReport getSnapshotDiffReportInternal(
       final String snapshotDir, final String fromSnapshot,
       final String toSnapshot) throws IOException {
+    // In case the diff needs to be computed between a snapshot and the current
+    // tree, we should not do iterative diffReport computation as the iterative
+    // approach might fail if in between the rpc calls the current tree
+    // changes in absence of the global fsn lock.
+    if (!isValidSnapshotName(fromSnapshot) || !isValidSnapshotName(
+        toSnapshot)) {
+      return dfs.getSnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot);
+    }
     byte[] startPath = DFSUtilClient.EMPTY_BYTES;
     int index = -1;
     SnapshotDiffReportGenerator snapshotDiffReport;

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -1308,8 +1308,7 @@ public interface ClientProtocol {
       String fromSnapshot, String toSnapshot) throws IOException;
 
   /**
-   * Get the difference between two snapshots, or between a snapshot and the
-   * current tree of a directory.
+   * Get the difference between two snapshots of a directory iteratively.
    *
    * @param snapshotRoot
    *          full path of the directory where snapshots are taken

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -1157,6 +1157,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.disk.balancer.max.disk.errors";
   public static final int DFS_DISK_BALANCER_MAX_DISK_ERRORS_DEFAULT = 5;
 
+  public static final String DFS_DISK_BALANCER_PLAN_VALID_INTERVAL =
+      "dfs.disk.balancer.plan.valid.interval";
+  public static final String DFS_DISK_BALANCER_PLAN_VALID_INTERVAL_DEFAULT =
+      "1d";
+
 
   public static final String DFS_DISK_BALANCER_BLOCK_TOLERANCE =
       "dfs.disk.balancer.block.tolerance.percent";

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java

@@ -147,10 +147,10 @@ public class HAUtil {
   }
   
   /**
-   * Get the NN ID of the other node in an HA setup.
+   * Get the NN ID of the other nodes in an HA setup.
    * 
    * @param conf the configuration of this node
-   * @return the NN ID of the other node in this nameservice
+   * @return a list of NN IDs of other nodes in this nameservice
    */
   public static List<String> getNameNodeIdOfOtherNodes(Configuration conf, String nsId) {
     Preconditions.checkArgument(nsId != null,
@@ -183,26 +183,26 @@ public class HAUtil {
   }
 
   /**
-   * Given the configuration for this node, return a Configuration object for
-   * the other node in an HA setup.
+   * Given the configuration for this node, return a list of configurations
+   * for the other nodes in an HA setup.
    * 
    * @param myConf the configuration of this node
-   * @return the configuration of the other node in an HA setup
+   * @return a list of configuration of other nodes in an HA setup
    */
   public static List<Configuration> getConfForOtherNodes(
       Configuration myConf) {
     
     String nsId = DFSUtil.getNamenodeNameServiceId(myConf);
-    List<String> otherNn = getNameNodeIdOfOtherNodes(myConf, nsId);
+    List<String> otherNodes = getNameNodeIdOfOtherNodes(myConf, nsId);
 
     // Look up the address of the other NNs
-    List<Configuration> confs = new ArrayList<Configuration>(otherNn.size());
+    List<Configuration> confs = new ArrayList<Configuration>(otherNodes.size());
     myConf = new Configuration(myConf);
     // unset independent properties
     for (String idpKey : HA_SPECIAL_INDEPENDENT_KEYS) {
       myConf.unset(idpKey);
     }
-    for (String nn : otherNn) {
+    for (String nn : otherNodes) {
       Configuration confForOtherNode = new Configuration(myConf);
       NameNode.initializeGenericKeys(confForOtherNode, nsId, nn);
       confs.add(confForOtherNode);

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java

@@ -496,7 +496,9 @@ public class QuorumJournalManager implements JournalManager {
 
         // If it's bounded by durable Txns, endTxId could not be larger
         // than committedTxnId. This ensures the consistency.
-        if (onlyDurableTxns && inProgressOk) {
+        // We don't do the following for finalized log segments, since all
+        // edits in those are guaranteed to be committed.
+        if (onlyDurableTxns && inProgressOk && remoteLog.isInProgress()) {
           endTxId = Math.min(endTxId, committedTxnId);
           if (endTxId < remoteLog.getStartTxId()) {
             LOG.warn("Found endTxId (" + endTxId + ") that is less than " +

+ 36 - 35
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java

@@ -22,18 +22,17 @@ import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.security.SecureRandom;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.io.WritableUtils;
@@ -41,11 +40,12 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.Timer;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.util.Timer;
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Multiset;
 
 /**
  * BlockTokenSecretManager can be instantiated in 2 modes, master mode
@@ -153,8 +153,9 @@ public class BlockTokenSecretManager extends
 
   /** Initialize block keys */
   private synchronized void generateKeys() {
-    if (!isMaster)
+    if (!isMaster) {
       return;
+    }
     /*
      * Need to set estimated expiry dates for currentKey and nextKey so that if
      * NN crashes, DN can still expire those keys. NN will stop using the newly
@@ -179,10 +180,10 @@ public class BlockTokenSecretManager extends
 
   /** Export block keys, only to be used in master mode */
   public synchronized ExportedBlockKeys exportKeys() {
-    if (!isMaster)
+    if (!isMaster) {
       return null;
-    if (LOG.isDebugEnabled())
-      LOG.debug("Exporting access keys");
+    }
+    LOG.debug("Exporting access keys");
     return new ExportedBlockKeys(true, keyUpdateInterval, tokenLifetime,
         currentKey, allKeys.values().toArray(new BlockKey[0]));
   }
@@ -203,16 +204,17 @@ public class BlockTokenSecretManager extends
    */
   public synchronized void addKeys(ExportedBlockKeys exportedKeys)
       throws IOException {
-    if (isMaster || exportedKeys == null)
+    if (isMaster || exportedKeys == null) {
       return;
+    }
     LOG.info("Setting block keys");
     removeExpiredKeys();
     this.currentKey = exportedKeys.getCurrentKey();
     BlockKey[] receivedKeys = exportedKeys.getAllKeys();
     for (int i = 0; i < receivedKeys.length; i++) {
-      if (receivedKeys[i] == null)
-        continue;
-      this.allKeys.put(receivedKeys[i].getKeyId(), receivedKeys[i]);
+      if (receivedKeys[i] != null) {
+        this.allKeys.put(receivedKeys[i].getKeyId(), receivedKeys[i]);
+      }
     }
   }
 
@@ -231,8 +233,9 @@ public class BlockTokenSecretManager extends
    * Update block keys, only to be used in master mode
    */
   synchronized boolean updateKeys() throws IOException {
-    if (!isMaster)
+    if (!isMaster) {
       return false;
+    }
 
     LOG.info("Updating block keys");
     removeExpiredKeys();
@@ -283,10 +286,10 @@ public class BlockTokenSecretManager extends
       ExtendedBlock block, BlockTokenIdentifier.AccessMode mode,
       StorageType[] storageTypes, String[] storageIds) throws InvalidToken {
     checkAccess(id, userId, block, mode);
-    if (storageTypes != null && storageTypes.length > 0) {
+    if (ArrayUtils.isNotEmpty(storageTypes)) {
       checkAccess(id.getStorageTypes(), storageTypes, "StorageTypes");
     }
-    if (storageIds != null && storageIds.length > 0) {
+    if (ArrayUtils.isNotEmpty(storageIds)) {
       checkAccess(id.getStorageIds(), storageIds, "StorageIDs");
     }
   }
@@ -296,26 +299,26 @@ public class BlockTokenSecretManager extends
       throws InvalidToken {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Checking access for user=" + userId + ", block=" + block
-          + ", access mode=" + mode + " using " + id.toString());
+          + ", access mode=" + mode + " using " + id);
     }
     if (userId != null && !userId.equals(id.getUserId())) {
-      throw new InvalidToken("Block token with " + id.toString()
+      throw new InvalidToken("Block token with " + id
           + " doesn't belong to user " + userId);
     }
     if (!id.getBlockPoolId().equals(block.getBlockPoolId())) {
-      throw new InvalidToken("Block token with " + id.toString()
+      throw new InvalidToken("Block token with " + id
           + " doesn't apply to block " + block);
     }
     if (id.getBlockId() != block.getBlockId()) {
-      throw new InvalidToken("Block token with " + id.toString()
+      throw new InvalidToken("Block token with " + id
           + " doesn't apply to block " + block);
     }
     if (isExpired(id.getExpiryDate())) {
-      throw new InvalidToken("Block token with " + id.toString()
+      throw new InvalidToken("Block token with " + id
           + " is expired.");
     }
     if (!id.getAccessModes().contains(mode)) {
-      throw new InvalidToken("Block token with " + id.toString()
+      throw new InvalidToken("Block token with " + id
           + " doesn't have " + mode + " permission");
     }
   }
@@ -329,26 +332,23 @@ public class BlockTokenSecretManager extends
    */
   public static <T> void checkAccess(T[] candidates, T[] requested, String msg)
       throws InvalidToken {
-    if (requested.length == 0) {
+    if (ArrayUtils.isEmpty(requested)) {
       throw new InvalidToken("The request has no " + msg + ". "
           + "This is probably a configuration error.");
     }
-    if (candidates.length == 0) {
+    if (ArrayUtils.isEmpty(candidates)) {
       return;
     }
 
-    List unseenCandidates = new ArrayList<T>();
-    unseenCandidates.addAll(Arrays.asList(candidates));
+    Multiset<T> c = HashMultiset.create(Arrays.asList(candidates));
+
     for (T req : requested) {
-      final int index = unseenCandidates.indexOf(req);
-      if (index == -1) {
+      if (!c.remove(req)) {
         throw new InvalidToken("Block token with " + msg + " "
             + Arrays.toString(candidates)
             + " not valid for access with " + msg + " "
             + Arrays.toString(requested));
       }
-      Collections.swap(unseenCandidates, index, unseenCandidates.size()-1);
-      unseenCandidates.remove(unseenCandidates.size()-1);
     }
   }
 
@@ -367,7 +367,7 @@ public class BlockTokenSecretManager extends
     }
     checkAccess(id, userId, block, mode, storageTypes, storageIds);
     if (!Arrays.equals(retrievePassword(id), token.getPassword())) {
-      throw new InvalidToken("Block token with " + id.toString()
+      throw new InvalidToken("Block token with " + id
           + " doesn't have the correct token password");
     }
   }
@@ -416,12 +416,13 @@ public class BlockTokenSecretManager extends
     synchronized (this) {
       key = currentKey;
     }
-    if (key == null)
+    if (key == null) {
       throw new IllegalStateException("currentKey hasn't been initialized.");
+    }
     identifier.setExpiryDate(timer.now() + tokenLifetime);
     identifier.setKeyId(key.getKeyId());
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Generating block token for " + identifier.toString());
+      LOG.debug("Generating block token for " + identifier);
     }
     return createPassword(identifier.getBytes(), key.getKey());
   }
@@ -438,7 +439,7 @@ public class BlockTokenSecretManager extends
   public byte[] retrievePassword(BlockTokenIdentifier identifier)
       throws InvalidToken {
     if (isExpired(identifier.getExpiryDate())) {
-      throw new InvalidToken("Block token with " + identifier.toString()
+      throw new InvalidToken("Block token with " + identifier
           + " is expired.");
     }
     BlockKey key = null;
@@ -447,7 +448,7 @@ public class BlockTokenSecretManager extends
     }
     if (key == null) {
       throw new InvalidToken("Can't re-compute password for "
-          + identifier.toString() + ", since the required block key (keyID="
+          + identifier + ", since the required block key (keyID="
           + identifier.getKeyId() + ") doesn't exist.");
     }
     return createPassword(identifier.getBytes(), key.getKey());

+ 25 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -1506,22 +1506,35 @@ public class DataNode extends ReconfigurableBase
       throw new RuntimeException(errMessage);
     }
 
-    SaslPropertiesResolver saslPropsResolver = dnConf.getSaslPropsResolver();
-    if (resources != null && saslPropsResolver == null) {
-      return;
-    }
     if (dnConf.getIgnoreSecurePortsForTesting()) {
       return;
     }
-    if (saslPropsResolver != null &&
-        DFSUtil.getHttpPolicy(conf) == HttpConfig.Policy.HTTPS_ONLY &&
-        resources == null) {
-      return;
+
+    if (resources != null) {
+      final boolean httpSecured = resources.isHttpPortPrivileged()
+          || DFSUtil.getHttpPolicy(conf) == HttpConfig.Policy.HTTPS_ONLY;
+      final boolean rpcSecured = resources.isRpcPortPrivileged()
+          || resources.isSaslEnabled();
+
+      // Allow secure DataNode to startup if:
+      // 1. Http is secure.
+      // 2. Rpc is secure
+      if (rpcSecured && httpSecured) {
+        return;
+      }
+    } else {
+      // Handle cases when SecureDataNodeStarter#getSecureResources is not
+      // invoked
+      SaslPropertiesResolver saslPropsResolver = dnConf.getSaslPropsResolver();
+      if (saslPropsResolver != null &&
+          DFSUtil.getHttpPolicy(conf) == HttpConfig.Policy.HTTPS_ONLY) {
+        return;
+      }
     }
-    throw new RuntimeException("Cannot start secure DataNode without " +
-      "configuring either privileged resources or SASL RPC data transfer " +
-      "protection and SSL for HTTP.  Using privileged resources in " +
-      "combination with SASL RPC data transfer protection is not supported.");
+
+    throw new RuntimeException("Cannot start secure DataNode due to incorrect "
+        + "config. See https://cwiki.apache.org/confluence/display/HADOOP/"
+        + "Secure+DataNode for details.");
   }
   
   public static String generateUuid() {

+ 18 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java

@@ -91,6 +91,8 @@ public class DiskBalancer {
   private String planFile;
   private DiskBalancerWorkStatus.Result currentResult;
   private long bandwidth;
+  private long planValidityInterval;
+  private final Configuration config;
 
   /**
    * Constructs a Disk Balancer object. This object takes care of reading a
@@ -102,6 +104,7 @@ public class DiskBalancer {
    */
   public DiskBalancer(String dataNodeUUID,
                       Configuration conf, BlockMover blockMover) {
+    this.config = conf;
     this.currentResult = Result.NO_PLAN;
     this.blockMover = blockMover;
     this.dataset = this.blockMover.getDataset();
@@ -117,6 +120,10 @@ public class DiskBalancer {
     this.bandwidth = conf.getInt(
         DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_THROUGHPUT,
         DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_THROUGHPUT_DEFAULT);
+    this.planValidityInterval = conf.getTimeDuration(
+        DFSConfigKeys.DFS_DISK_BALANCER_PLAN_VALID_INTERVAL,
+        DFSConfigKeys.DFS_DISK_BALANCER_PLAN_VALID_INTERVAL_DEFAULT,
+        TimeUnit.MILLISECONDS);
   }
 
   /**
@@ -417,15 +424,17 @@ public class DiskBalancer {
     long now = Time.now();
     long planTime = plan.getTimeStamp();
 
-    // TODO : Support Valid Plan hours as a user configurable option.
-    if ((planTime +
-        (TimeUnit.HOURS.toMillis(
-            DiskBalancerConstants.DISKBALANCER_VALID_PLAN_HOURS))) < now) {
-      String hourString = "Plan was generated more than " +
-          Integer.toString(DiskBalancerConstants.DISKBALANCER_VALID_PLAN_HOURS)
-          + " hours ago.";
-      LOG.error("Disk Balancer - " + hourString);
-      throw new DiskBalancerException(hourString,
+    if ((planTime + planValidityInterval) < now) {
+      String planValidity = config.get(
+          DFSConfigKeys.DFS_DISK_BALANCER_PLAN_VALID_INTERVAL,
+          DFSConfigKeys.DFS_DISK_BALANCER_PLAN_VALID_INTERVAL_DEFAULT);
+      if (planValidity.matches("[0-9]$")) {
+        planValidity += "ms";
+      }
+      String errorString = "Plan was generated more than " + planValidity
+          + " ago";
+      LOG.error("Disk Balancer - " + errorString);
+      throw new DiskBalancerException(errorString,
           DiskBalancerException.Result.OLD_PLAN_SUBMITTED);
     }
   }

+ 33 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java

@@ -25,9 +25,9 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
 
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
@@ -43,11 +43,21 @@ public class SecureDataNodeStarter implements Daemon {
    * Stash necessary resources needed for datanode operation in a secure env.
    */
   public static class SecureResources {
+    private final boolean isSaslEnabled;
+    private final boolean isRpcPortPrivileged;
+    private final boolean isHttpPortPrivileged;
+
     private final ServerSocket streamingSocket;
     private final ServerSocketChannel httpServerSocket;
-    public SecureResources(ServerSocket streamingSocket, ServerSocketChannel httpServerSocket) {
+
+    public SecureResources(ServerSocket streamingSocket, ServerSocketChannel
+        httpServerSocket, boolean saslEnabled, boolean rpcPortPrivileged,
+        boolean httpPortPrivileged) {
       this.streamingSocket = streamingSocket;
       this.httpServerSocket = httpServerSocket;
+      this.isSaslEnabled = saslEnabled;
+      this.isRpcPortPrivileged = rpcPortPrivileged;
+      this.isHttpPortPrivileged = httpPortPrivileged;
     }
 
     public ServerSocket getStreamingSocket() { return streamingSocket; }
@@ -55,6 +65,18 @@ public class SecureDataNodeStarter implements Daemon {
     public ServerSocketChannel getHttpServerChannel() {
       return httpServerSocket;
     }
+
+    public boolean isSaslEnabled() {
+      return isSaslEnabled;
+    }
+
+    public boolean isRpcPortPrivileged() {
+      return isRpcPortPrivileged;
+    }
+
+    public boolean isHttpPortPrivileged() {
+      return isHttpPortPrivileged;
+    }
   }
   
   private String [] args;
@@ -90,8 +112,12 @@ public class SecureDataNodeStarter implements Daemon {
   public static SecureResources getSecureResources(Configuration conf)
       throws Exception {
     HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);
-    boolean isSecure = UserGroupInformation.isSecurityEnabled();
+    boolean isSaslEnabled =
+        DataTransferSaslUtil.getSaslPropertiesResolver(conf) != null;
+    boolean isRpcPrivileged;
+    boolean isHttpPrivileged = false;
 
+    System.err.println("isSaslEnabled:" + isSaslEnabled);
     // Obtain secure port for data streaming to datanode
     InetSocketAddress streamingAddr  = DataNode.getStreamingAddr(conf);
     int socketWriteTimeout = conf.getInt(
@@ -118,12 +144,7 @@ public class SecureDataNodeStarter implements Daemon {
               + "context. Needed " + streamingAddr.getPort() + ", got "
               + ss.getLocalPort());
     }
-
-    if (!SecurityUtil.isPrivilegedPort(ss.getLocalPort()) && isSecure) {
-      throw new RuntimeException(
-        "Cannot start secure datanode with unprivileged RPC ports");
-    }
-
+    isRpcPrivileged = SecurityUtil.isPrivilegedPort(ss.getLocalPort());
     System.err.println("Opened streaming server at " + streamingAddr);
 
     // Bind a port for the web server. The code intends to bind HTTP server to
@@ -151,16 +172,14 @@ public class SecureDataNodeStarter implements Daemon {
       System.err.println("Successfully obtained privileged resources (streaming port = "
           + ss + " ) (http listener port = " + localAddr.getPort() +")");
 
-      if (localAddr.getPort() > 1023 && isSecure) {
-        throw new RuntimeException(
-            "Cannot start secure datanode with unprivileged HTTP ports");
-      }
+      isHttpPrivileged = SecurityUtil.isPrivilegedPort(localAddr.getPort());
       System.err.println("Opened info server at " + infoSocAddr);
     } else {
       httpChannel = null;
     }
 
-    return new SecureResources(ss, httpChannel);
+    return new SecureResources(ss, httpChannel, isSaslEnabled,
+        isRpcPrivileged, isHttpPrivileged);
   }
 
   private static BindException appendMessageToBindException(BindException e,

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java

@@ -33,11 +33,6 @@ public final class DiskBalancerConstants {
   public static final int DISKBALANCER_MIN_VERSION = 1;
   public static final int DISKBALANCER_MAX_VERSION = 1;
 
-  /**
-   * We treat a plan as stale if it was generated before the hours
-   * defined by the constant below. Defaults to 24 hours.
-   */
-  public static final int DISKBALANCER_VALID_PLAN_HOURS = 24;
   // never constructed.
   private DiskBalancerConstants() {
   }

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java

@@ -456,17 +456,17 @@ final class FSDirEncryptionZoneOp {
         }
       }
 
-      final CryptoProtocolVersion version = encryptionZone.getVersion();
-      final CipherSuite suite = encryptionZone.getSuite();
-      final String keyName = encryptionZone.getKeyName();
       XAttr fileXAttr = FSDirXAttrOp.unprotectedGetXAttrByPrefixedName(
           iip, CRYPTO_XATTR_FILE_ENCRYPTION_INFO);
-
       if (fileXAttr == null) {
         NameNode.LOG.warn("Could not find encryption XAttr for file " +
             iip.getPath() + " in encryption zone " + encryptionZone.getPath());
         return null;
       }
+
+      final CryptoProtocolVersion version = encryptionZone.getVersion();
+      final CipherSuite suite = encryptionZone.getSuite();
+      final String keyName = encryptionZone.getKeyName();
       try {
         HdfsProtos.PerFileEncryptionInfoProto fileProto =
             HdfsProtos.PerFileEncryptionInfoProto.parseFrom(

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java

@@ -573,7 +573,9 @@ public class FSPermissionChecker implements AccessControlEnforcer {
         && mode.getGroupAction().implies(access)) {
       return;
     }
-    if (mode.getOtherAction().implies(access)) {
+    if (!getUser().equals(pool.getOwnerName())
+        && !isMemberOfGroup(pool.getGroupName())
+        && mode.getOtherAction().implies(access)) {
       return;
     }
     throw new AccessControlException("Permission denied while accessing pool "

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffList.java

@@ -17,8 +17,11 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+
 import java.util.Collections;
 import java.util.Iterator;
+import java.util.List;
 
 /**
  * This interface defines the methods used to store and manage InodeDiffs.
@@ -82,6 +85,12 @@ public interface DiffList<T extends Comparable<Integer>> extends Iterable<T> {
       public Iterator<T> iterator() {
         return diffs.iterator();
       }
+
+      @Override
+      public List<T> getMinListForRange(int startIndex, int endIndex,
+          INodeDirectory dir) {
+        return diffs.getMinListForRange(startIndex, endIndex, dir);
+      }
     };
   }
 
@@ -137,4 +146,13 @@ public interface DiffList<T extends Comparable<Integer>> extends Iterable<T> {
    */
   int binarySearch(int key);
 
+  /**
+   * Returns the list of minimal list of elements need to combine to generate
+   * cumulative sum from startIndex to endIndex.
+   * @param startIndex
+   * @param endIndex
+   * @return list of T
+   */
+  List<T> getMinListForRange(int startIndex, int endIndex, INodeDirectory dir);
+
 }

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffListByArrayList.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Iterator;
@@ -77,4 +79,10 @@ public class DiffListByArrayList<T extends Comparable<Integer>>
   public Iterator<T> iterator() {
     return list.iterator();
   }
+
+  @Override
+  public List<T> getMinListForRange(int startIndex, int endIndex,
+      INodeDirectory dir) {
+    return list.subList(startIndex, endIndex);
+  }
 }

+ 545 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryDiffList.java

@@ -0,0 +1,545 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.hdfs.server.namenode.snapshot;
+
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.
+    DirectoryWithSnapshotFeature.DirectoryDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.
+    DirectoryWithSnapshotFeature.ChildrenDiff;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Random;
+import java.util.Objects;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * SkipList is an implementation of a data structure for storing a sorted list
+ * of Directory Diff elements, using a hierarchy of linked lists that connect
+ * increasingly sparse subsequences(defined by skip interval here) of the diffs.
+ * The elements contained in the tree must be mutually comparable.
+ * <p>
+ * Consider  a case where we have 10 snapshots for a directory starting from s0
+ * to s9 each associated with certain change records in terms of inodes deleted
+ * and created after a particular snapshot and before the next snapshot. The
+ * sequence will look like this:
+ * <p>
+ * s0->s1->s2->s3->s4->s5->s6->s7->s8->s9.
+ * <p>
+ * Assuming a skip interval of 3, which means a new diff will be added at a
+ * level higher than the current level after we have  ore than 3 snapshots.
+ * Next level promotion happens after 9 snapshots and so on.
+ * <p>
+ * level 2:   s08------------------------------->s9
+ * level 1:   S02------->s35-------->s68-------->s9
+ * level 0:  s0->s1->s2->s3->s4->s5->s6->s7->s8->s9
+ * <p>
+ * s02 will be created by combining diffs for s0, s1, s2 once s3 gets created.
+ * Similarly, s08 will be created by combining s02, s35 and s68 once s9 gets
+ * created.So, for constructing the children list fot s0, we have  to combine
+ * s08, s9 and reverse apply to the live fs.
+ * <p>
+ * Similarly, for constructing the children list for s2, s2, s35, s68 and s9
+ * need to get combined(or added) and reverse applied to current fs.
+ * <p>
+ * This approach will improve the snapshot deletion and snapshot diff
+ * calculation.
+ * <p>
+ * Once a snapshot gets deleted, the list needs to be balanced.
+ */
+public class DirectoryDiffList implements DiffList<DirectoryDiff> {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DirectoryDiffList.class);
+
+  private static class SkipDiff {
+    /**
+     * The references to the subsequent nodes.
+     */
+    private SkipListNode skipTo;
+    /**
+     * combined diff over a skip Interval.
+     */
+    private ChildrenDiff diff;
+
+    SkipDiff(ChildrenDiff diff) {
+      this.diff = diff;
+    }
+
+    public ChildrenDiff getDiff() {
+      return diff;
+    }
+
+    public SkipListNode getSkipTo() {
+      return skipTo;
+    }
+
+    public void setSkipTo(SkipListNode node) {
+      skipTo = node;
+    }
+
+    public void setDiff(ChildrenDiff diff) {
+      this.diff = diff;
+    }
+
+    @Override
+    public String toString() {
+      return "->" + skipTo + (diff == null? " (diff==null)": "");
+    }
+  }
+
+  /**
+   * SkipListNode is an implementation of a DirectoryDiff List node,
+   * which stores a Directory Diff and references to subsequent nodes.
+   */
+  private final static class SkipListNode implements Comparable<Integer> {
+
+    /**
+     * The data element stored in this node.
+     */
+    private DirectoryDiff diff;
+
+    /**
+     * List containing combined children diffs over a skip interval.
+     */
+    private List<SkipDiff> skipDiffList;
+
+    /**
+     * Constructs a new instance of SkipListNode with the specified data element
+     * and level.
+     *
+     * @param diff The element to be stored in the node.
+     */
+    SkipListNode(DirectoryDiff diff, int level) {
+      this.diff = diff;
+      skipDiffList = new ArrayList<>(level + 1);
+    }
+
+    /**
+     * Returns the level of this SkipListNode.
+     */
+    public int level() {
+      return skipDiffList.size() - 1;
+    }
+
+    void trim() {
+      for (int level = level();
+           level > 0 && getSkipNode(level) == null; level--) {
+        skipDiffList.remove(level);
+      }
+    }
+
+    public DirectoryDiff getDiff() {
+      return diff;
+    }
+
+    /**
+     * Compare diffs with snapshot ID.
+     */
+    @Override
+    public int compareTo(Integer that) {
+      return diff.compareTo(that);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      SkipListNode that = (SkipListNode) o;
+      return Objects.equals(diff, that.diff);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(diff);
+    }
+
+    public void setSkipDiff(ChildrenDiff cDiff, int level) {
+      if (level < skipDiffList.size()) {
+        skipDiffList.get(level).setDiff(cDiff);
+      } else {
+        skipDiffList.add(new SkipDiff(cDiff));
+      }
+    }
+
+    public void setSkipTo(SkipListNode node, int level) {
+      for (int i = skipDiffList.size(); i <= level; i++) {
+        skipDiffList.add(new SkipDiff(null));
+      }
+      skipDiffList.get(level).setSkipTo(node);
+    }
+
+    public ChildrenDiff getChildrenDiff(int level) {
+      if (level == 0) {
+        return diff.getChildrenDiff();
+      } else {
+        return skipDiffList.get(level).getDiff();
+      }
+    }
+
+    SkipListNode getSkipNode(int level) {
+      if (level >= skipDiffList.size()) {
+        return null;
+      } else {
+        return skipDiffList.get(level).getSkipTo();
+      }
+    }
+
+    @Override
+    public String toString() {
+      return diff != null ? "" + diff.getSnapshotId() : "?";
+    }
+  }
+
+  /**
+   * The reference to the first node of the list.
+   * The list will grow linearly once a new Directory diff gets added.
+   * All the list inteface defined methods provide a linear view of the list.
+   */
+  private List<SkipListNode> skipNodeList;
+
+  /**
+   * The max no of skipLevels.
+   */
+  private final int maxSkipLevels;
+
+  /**
+   * The no of diffs after which the level promotion happens.
+   */
+  private final int skipInterval;
+
+  /**
+   * The head node to the list.
+   */
+  private SkipListNode head;
+
+  /**
+   * Constructs a new, empty instance of SkipList.
+   */
+  public DirectoryDiffList(int capacity, int interval, int skipLevel) {
+    skipNodeList = new ArrayList<>(capacity);
+    head = new SkipListNode(null, 0);
+    this.maxSkipLevels = skipLevel;
+    this.skipInterval = interval;
+  }
+
+  /**
+   * Adds the specified data element to the beginning of the SkipList,
+   * if the element is not already present.
+   * @param diff the element to be inserted
+   */
+  @Override
+  public void addFirst(DirectoryDiff diff) {
+    final int nodeLevel = randomLevel(skipInterval, maxSkipLevels);
+    final SkipListNode[] nodePath = new SkipListNode[nodeLevel + 1];
+
+    Arrays.fill(nodePath, head);
+    for (int level = head.level() + 1; level <= nodeLevel; level++) {
+      head.skipDiffList.add(new SkipDiff(null));
+    }
+
+    final SkipListNode newNode = new SkipListNode(diff, nodeLevel);
+    for (int level = 0; level <= nodeLevel; level++) {
+      if (level > 0) {
+        // Case : S0 is added at the beginning and it has 3 levels
+        //  suppose the list is like:
+        //  level 1: head ------------------->s5------------->NULL
+        //  level 0:head->    s1->s2->s3->s4->s5->s6->s7->s8->s9
+        //  in this case:
+        //  level 2: head -> s0 -------------------------------->NULL
+        //  level 1: head -> s0'---------------->s5------------->NULL
+        //  level 0:head->   s0->s1->s2->s3->s4->s5->s6->s7->s8->s9
+        //  At level 1, we need to combine s0, s1, s2, s3, s4 and s5 and store
+        //  as s0'. At level 2, s0 of next is pointing to null;
+        //  Note: in this case, the diff of element being added is included
+        //  while combining the diffs.
+        final SkipListNode nextNode = head.getSkipNode(level);
+        if (nextNode != null) {
+          ChildrenDiff combined = combineDiff(newNode, nextNode, level);
+          if (combined != null) {
+            newNode.setSkipDiff(combined, level);
+          }
+        }
+      }
+      //insert to the linked list
+      newNode.setSkipTo(nodePath[level].getSkipNode(level), level);
+      nodePath[level].setSkipTo(newNode, level);
+    }
+    skipNodeList.add(0, newNode);
+  }
+
+  private SkipListNode[] findPreviousNodes(SkipListNode node, int nodeLevel) {
+    final SkipListNode[] nodePath = new SkipListNode[nodeLevel + 1];
+    SkipListNode cur = head;
+    final int headLevel = head.level();
+    for (int level = headLevel < nodeLevel ? headLevel : nodeLevel;
+         level >= 0; level--) {
+      while (cur.getSkipNode(level) != node) {
+        cur = cur.getSkipNode(level);
+      }
+      nodePath[level] = cur;
+    }
+    for (int level = headLevel + 1; level <= nodeLevel; level++) {
+      nodePath[level] = head;
+    }
+    return nodePath;
+  }
+
+  /**
+   * Adds the specified data element to the end of the SkipList,
+   * if the element is not already present.
+   * @param diff the element to be inserted
+   */
+  @Override
+  public boolean addLast(DirectoryDiff diff) {
+    final int nodeLevel = randomLevel(skipInterval, maxSkipLevels);
+    final int headLevel = head.level();
+    final SkipListNode[] nodePath = findPreviousNodes(null, nodeLevel);
+    for (int level = headLevel + 1; level <= nodeLevel; level++) {
+      head.skipDiffList.add(new SkipDiff(null));
+      nodePath[level] = head;
+    }
+
+    final SkipListNode current = new SkipListNode(diff, nodeLevel);
+    for (int level = 0; level <= nodeLevel; level++) {
+      if (level > 0 && nodePath[level] != head) {
+        //  suppose the list is like:
+        //  level 2: head ->  s1----------------------------->NULL
+        //  level 1: head ->  s1---->s3'------>s5------------->NULL
+        //  level 0:head->    s1->s2->s3->s4->s5->s6->s7->s8->s9
+
+        // case : s10 is added at the end the let the level for this node = 4
+        //  in this case,
+        //  level 2: head ->  s1''------------------------------------>s10
+        //  level 1: head ->  s1'---->s3'------>s5'-------------------->s10
+        //  level 0:head->    s1->s2->s3->s4->s5->s6->s7->s8->s9---->s10
+        //  At level 1, we combine s5, s6, s7, s8, s9 and store as s5'
+        //  At level 2, we combine s1', s3', s5' and form s1'' and store at s1.
+        // Note : the last element(elemnt being added) diff is not added while
+        // combining the diffs.
+        ChildrenDiff combined = combineDiff(nodePath[level], current, level);
+        if (combined != null) {
+          nodePath[level].setSkipDiff(combined, level);
+        }
+      }
+      nodePath[level].setSkipTo(current, level);
+      current.setSkipTo(null, level);
+    }
+    return skipNodeList.add(current);
+  }
+
+  private static ChildrenDiff combineDiff(SkipListNode from, SkipListNode to,
+      int level) {
+    ChildrenDiff combined = null;
+    SkipListNode cur = from;
+    for (int i = level - 1; i >= 0; i--) {
+      while (cur != to) {
+        final SkipListNode next = cur.getSkipNode(i);
+        if (next == null) {
+          break;
+        }
+        if (combined == null) {
+          combined = new ChildrenDiff();
+        }
+        combined.combinePosterior(cur.getChildrenDiff(i), null);
+        cur = next;
+      }
+    }
+    return combined;
+  }
+
+  /**
+   * Returns the data element at the specified index in this SkipList.
+   *
+   * @param index The index of the element to be returned.
+   * @return The element at the specified index in this SkipList.
+   */
+  @Override
+  public DirectoryDiff get(int index) {
+    return skipNodeList.get(index).getDiff();
+  }
+
+  /**
+   * Removes the element at the specified position in this list.
+   *
+   * @param index the index of the element to be removed
+   * @return the removed DirectoryDiff
+   */
+  @Override
+  public DirectoryDiff remove(int index) {
+    SkipListNode node = getNode(index);
+    int headLevel = head.level();
+    int nodeLevel = node.level();
+    final SkipListNode[] nodePath = findPreviousNodes(node, nodeLevel);
+    for (int level = 0; level <= nodeLevel; level++) {
+      if (nodePath[level] != head && level > 0) {
+        // if the last snapshot is deleted, for all the skip level nodes
+        // pointing to the last one, the combined children diff at each level
+        // > 0 should be made null and skip pointers will be updated to null.
+        // if the snapshot being deleted is not the last one, we have to merge
+        // the diff of deleted node at each level to the previous skip level
+        // node at that level and the skip pointers will be updated to point to
+        // the skip nodes of the deleted node.
+        if (index == size() - 1) {
+          nodePath[level].setSkipDiff(null, level);
+        } else {
+          /* Ideally at level 0, the deleted diff will be combined with
+           * the previous diff , and deleted inodes will be cleaned up
+           * by passing a deleted processor here while combining the diffs.
+           * Level 0 merge with previous diff will be handled inside the
+           * {@link AbstractINodeDiffList#deleteSnapshotDiff} function.
+           */
+          if (node.getChildrenDiff(level) != null) {
+            nodePath[level].getChildrenDiff(level)
+                .combinePosterior(node.getChildrenDiff(level), null);
+          }
+        }
+      }
+      nodePath[level].setSkipTo(node.getSkipNode(level), level);
+    }
+    if (nodeLevel == headLevel) {
+      head.trim();
+    }
+    return skipNodeList.remove(index).getDiff();
+  }
+
+  /**
+   * Returns true if this SkipList contains no data elements. In other words,
+   * returns true if the size of this SkipList is zero.
+   *
+   * @return True if this SkipList contains no elements.
+   */
+  @Override
+  public boolean isEmpty() {
+    return skipNodeList.isEmpty();
+  }
+
+  /**
+   * Returns the number of data elements in this SkipList.
+   *
+   * @return The number of elements in this SkipList.
+   */
+  @Override
+  public int size() {
+    return skipNodeList.size();
+  }
+
+  /**
+   * Iterator is an iterator over the SkipList. This should
+   * always provide a linear view of the list.
+   */
+  @Override
+  public Iterator<DirectoryDiff> iterator() {
+    final Iterator<SkipListNode> i = skipNodeList.iterator();
+    return new Iterator<DirectoryDiff>() {
+
+      @Override
+      public boolean hasNext() {
+        return i.hasNext();
+      }
+
+      @Override
+      public DirectoryDiff next() {
+        return i.next().getDiff();
+      }
+    };
+  }
+
+  @Override
+  public int binarySearch(int key) {
+    return Collections.binarySearch(skipNodeList, key);
+  }
+
+  private SkipListNode getNode(int index) {
+    return skipNodeList.get(index);
+  }
+
+  /**
+   * Returns the level of the skipList node.
+   *
+   * @param skipInterval The max interval after which the next level promotion
+   *                     should happen.
+   * @param maxLevel     Maximum no of skip levels
+   * @return A value in the range 0 to maxLevel-1.
+   */
+  static int randomLevel(int skipInterval, int maxLevel) {
+    final Random r = ThreadLocalRandom.current();
+    for (int level = 0; level < maxLevel; level++) {
+      // skip to the next level with probability 1/skipInterval
+      if (r.nextInt(skipInterval) > 0) {
+        return level;
+      }
+    }
+    return maxLevel;
+  }
+
+  /**
+   * This function returns the minimal set of diffs required to combine in
+   * order to generate all the changes occurred between fromIndex and
+   * toIndex.
+   *
+   * @param fromIndex index from where the summation has to start(inclusive)
+   * @param toIndex   index till where the summation has to end(exclusive)
+   * @return list of Directory Diff
+   */
+  @Override
+  public List<DirectoryDiff> getMinListForRange(int fromIndex, int toIndex,
+      INodeDirectory dir) {
+    final List<DirectoryDiff> subList = new ArrayList<>();
+    final int toSnapshotId = get(toIndex - 1).getSnapshotId();
+    for (SkipListNode current = getNode(fromIndex); current != null;) {
+      SkipListNode next = null;
+      ChildrenDiff childrenDiff = null;
+      for (int level = current.level(); level >= 0; level--) {
+        next = current.getSkipNode(level);
+        if (next != null && next.getDiff().compareTo(toSnapshotId) <= 0) {
+          childrenDiff = current.getChildrenDiff(level);
+          break;
+        }
+      }
+      final DirectoryDiff curDiff = current.getDiff();
+      subList.add(childrenDiff == null ? curDiff :
+          new DirectoryDiff(curDiff.getSnapshotId(), dir, childrenDiff));
+
+      if (current.getDiff().compareTo(toSnapshotId) == 0) {
+        break;
+      }
+      current = next;
+    }
+    return subList;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder b = new StringBuilder(getClass().getSimpleName());
+    b.append(" head: ").append(head).append(head.skipDiffList);
+    for (SkipListNode n : skipNodeList) {
+      b.append("\n  ").append(n).append(n.skipDiffList);
+    }
+    return b.toString();
+  }
+}

+ 6 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java

@@ -168,12 +168,15 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     private boolean isSnapshotRoot = false;
     
     private DirectoryDiff(int snapshotId, INodeDirectory dir) {
-      super(snapshotId, null, null);
+      this(snapshotId, dir, new ChildrenDiff());
+    }
 
+    public DirectoryDiff(int snapshotId, INodeDirectory dir,
+        ChildrenDiff diff) {
+      super(snapshotId, null, null);
       this.childrenSize = dir.getChildrenList(Snapshot.CURRENT_STATE_ID).size();
-      this.diff = new ChildrenDiff();
+      this.diff = diff;
     }
-
     /** Constructor used by FSImage loading */
     DirectoryDiff(int snapshotId, INodeDirectoryAttributes snapshotINode,
         DirectoryDiff posteriorDiff, int childrenSize, List<INode> createdList,

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java

@@ -337,8 +337,8 @@ public class CryptoAdmin extends Configured implements Tool {
         action = ReencryptAction.CANCEL;
       }
 
-      final HdfsAdmin admin =
-          new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
+      Path p = new Path(path);
+      final HdfsAdmin admin = new HdfsAdmin(p.toUri(), conf);
       try {
         admin.reencryptEncryptionZone(new Path(path), action);
         System.out.println("re-encrypt command successfully submitted for "

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ReadOnlyList.java

@@ -106,6 +106,11 @@ public interface ReadOnlyList<E> extends Iterable<E> {
         public E get(int i) {
           return list.get(i);
         }
+
+        @Override
+        public String toString() {
+          return list.toString();
+        }
       };
     }
 
@@ -234,6 +239,19 @@ public interface ReadOnlyList<E> extends Iterable<E> {
         public <T> T[] toArray(T[] a) {
           throw new UnsupportedOperationException();
         }
+
+        @Override
+        public String toString() {
+          if (list.isEmpty()) {
+            return "[]";
+          }
+          final Iterator<E> i = list.iterator();
+          final StringBuilder b = new StringBuilder("[").append(i.next());
+          for(; i.hasNext();) {
+            b.append(", ").append(i.next());
+          }
+          return b + "]";
+        }
       };
     }
   }

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -4630,6 +4630,17 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.disk.balancer.plan.valid.interval</name>
+    <value>1d</value>
+    <description>
+      Maximum number of hours the disk balancer plan is valid.
+      This setting supports multiple time unit suffixes as described
+      in dfs.heartbeat.interval. If no suffix is specified then milliseconds
+      is assumed.
+    </description>
+  </property>
+
 
   <property>
     <name>dfs.disk.balancer.enabled</name>

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js

@@ -331,14 +331,14 @@
           $('#table-datanodes').dataTable( {
             'lengthMenu': [ [25, 50, 100, -1], [25, 50, 100, "All"] ],
             'columns': [
-              { 'orderDataType': 'ng-value', 'searchable': true },
-              { 'orderDataType': 'ng-value', 'searchable': true },
-              { 'orderDataType': 'ng-value', 'type': 'num' },
-              { 'orderDataType': 'ng-value', 'type': 'num' },
-              { 'orderDataType': 'ng-value', 'type': 'num' },
-              { 'type': 'num' },
-              { 'orderDataType': 'ng-value', 'type': 'num'},
-              { 'type': 'string' }
+              { 'orderDataType': 'ng-value', 'searchable': true , "defaultContent": "" },
+              { 'orderDataType': 'ng-value', 'searchable': true , "defaultContent": ""},
+              { 'orderDataType': 'ng-value', 'type': 'num' , "defaultContent": 0},
+              { 'orderDataType': 'ng-value', 'type': 'num' , "defaultContent": 0},
+              { 'orderDataType': 'ng-value', 'type': 'num' , "defaultContent": 0},
+              { 'type': 'num' , "defaultContent": 0},
+              { 'orderDataType': 'ng-value', 'type': 'num' , "defaultContent": 0},
+              { 'type': 'string' , "defaultContent": ""}
             ]});
           renderHistogram(data);
           $('#ui-tabs a[href="#tab-datanode"]').tab('show');

+ 7 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.html

@@ -35,6 +35,7 @@
   <ul class="nav navbar-nav" id="ui-tabs">
     <li><a href="#tab-overview">Overview</a></li>
     <li><a href="#tab-namenode">Subclusters</a></li>
+    <li><a href="#tab-router">Routers</a></li>
     <li><a href="#tab-datanode">Datanodes</a></li>
     <li><a href="#tab-mounttable">Mount table</a></li>
     <li class="dropdown">
@@ -62,6 +63,7 @@
 <div class="tab-content">
   <div class="tab-pane" id="tab-overview"></div>
   <div class="tab-pane" id="tab-namenode"></div>
+  <div class="tab-pane" id="tab-router"></div>
   <div class="tab-pane" id="tab-datanode"></div>
   <div class="tab-pane" id="tab-mounttable"></div>
 </div>
@@ -245,11 +247,13 @@
 </small>
 </script>
 
+<!-- Routers -->
+<script type="text/x-dust-template" id="tmpl-router">
 <div class="page-header"><h1>Routers Information</h1></div>
 <div>
   <ul class="dfshealth-node-legend">
     <li class="dfshealth-node-icon dfshealth-node-alive">Active</li>
-    <li class="dfshealth-node-icon dfshealth-node-decommisioned">Safe mode</li>
+    <li class="dfshealth-node-icon dfshealth-node-decommissioned">Safe mode</li>
     <li class="dfshealth-node-icon dfshealth-node-down">Unavailable</li>
   </ul>
 </div>
@@ -293,8 +297,8 @@
   <ul class="dfshealth-node-legend">
     <li class="dfshealth-node-icon dfshealth-node-alive">In service</li>
     <li class="dfshealth-node-icon dfshealth-node-down">Down</li>
-    <li class="dfshealth-node-icon dfshealth-node-decommisioned">Decommisioned</li>
-    <li class="dfshealth-node-icon dfshealth-node-down-decommisioned">Decommissioned &amp; dead</li>
+    <li class="dfshealth-node-icon dfshealth-node-decommissioned">Decommissioned</li>
+    <li class="dfshealth-node-icon dfshealth-node-down-decommissioned">Decommissioned &amp; dead</li>
   </ul>
 </div>
 <div class="page-header"><h1><small>In operation</small></h1></div>

+ 58 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/router/federationhealth.js

@@ -20,6 +20,7 @@
 
   dust.loadSource(dust.compile($('#tmpl-federationhealth').html(), 'federationhealth'));
   dust.loadSource(dust.compile($('#tmpl-namenode').html(), 'namenode-info'));
+  dust.loadSource(dust.compile($('#tmpl-router').html(), 'router-info'));
   dust.loadSource(dust.compile($('#tmpl-datanode').html(), 'datanode-info'));
   dust.loadSource(dust.compile($('#tmpl-mounttable').html(), 'mounttable'));
 
@@ -116,13 +117,13 @@
           n.iconState = "down";
           if (n.isSafeMode === true) {
             n.title = capitalise(n.state) + " (safe mode)"
-            n.iconState = "decommisioned";
+            n.iconState = "decommissioned";
           } else if (n.state === "ACTIVE") {
             n.title = capitalise(n.state);
             n.iconState = "alive";
           } else if (nodes[i].state === "STANDBY") {
             n.title = capitalise(n.state);
-            n.iconState = "down-decommisioned";
+            n.iconState = "down-decommissioned";
           } else if (nodes[i].state === "UNAVAILABLE") {
             n.title = capitalise(n.state);
             n.iconState = "down";
@@ -133,6 +134,48 @@
         }
       }
 
+      r.Nameservices = node_map_to_array(JSON.parse(r.Nameservices));
+      augment_namenodes(r.Nameservices);
+      r.Namenodes = node_map_to_array(JSON.parse(r.Namenodes));
+      augment_namenodes(r.Namenodes);
+      return r;
+    }
+
+    $.get(
+      '/jmx?qry=Hadoop:service=Router,name=FederationState',
+      guard_with_startup_progress(function (resp) {
+        var data = workaround(resp.beans[0]);
+        var base = dust.makeBase(HELPERS);
+        dust.render('namenode-info', base.push(data), function(err, out) {
+          $('#tab-namenode').html(out);
+          $('#ui-tabs a[href="#tab-namenode"]').tab('show');
+        });
+      })).error(ajax_error_handler);
+  }
+
+  function load_router_info() {
+    var HELPERS = {
+      'helper_lastcontact_tostring' : function (chunk, ctx, bodies, params) {
+        var value = dust.helpers.tap(params.value, chunk, ctx);
+        return chunk.write('' + new Date(Date.now()-1000*Number(value)));
+      }
+    };
+
+    function workaround(r) {
+      function node_map_to_array(nodes) {
+        var res = [];
+        for (var n in nodes) {
+          var p = nodes[n];
+          p.name = n;
+          res.push(p);
+        }
+        return res;
+      }
+
+      function capitalise(string) {
+          return string.charAt(0).toUpperCase() + string.slice(1).toLowerCase();
+      }
+
       function augment_routers(nodes) {
         for (var i = 0, e = nodes.length; i < e; ++i) {
           var n = nodes[i];
@@ -146,10 +189,10 @@
             n.iconState = "alive";
           } else if (n.status === "SAFEMODE") {
             n.title = capitalise(n.status);
-            n.iconState = "down-decommisioned";
+            n.iconState = "down-decommissioned";
           } else if (n.status === "STOPPING") {
             n.title = capitalise(n.status);
-            n.iconState = "decommisioned";
+            n.iconState = "decommissioned";
           } else if (n.status === "SHUTDOWN") {
             n.title = capitalise(n.status);
             n.iconState = "down";
@@ -157,10 +200,6 @@
         }
       }
 
-      r.Nameservices = node_map_to_array(JSON.parse(r.Nameservices));
-      augment_namenodes(r.Nameservices);
-      r.Namenodes = node_map_to_array(JSON.parse(r.Namenodes));
-      augment_namenodes(r.Namenodes);
       r.Routers = node_map_to_array(JSON.parse(r.Routers));
       augment_routers(r.Routers);
       return r;
@@ -171,9 +210,9 @@
       guard_with_startup_progress(function (resp) {
         var data = workaround(resp.beans[0]);
         var base = dust.makeBase(HELPERS);
-        dust.render('namenode-info', base.push(data), function(err, out) {
-          $('#tab-namenode').html(out);
-          $('#ui-tabs a[href="#tab-namenode"]').tab('show');
+        dust.render('router-info', base.push(data), function(err, out) {
+          $('#tab-router').html(out);
+          $('#ui-tabs a[href="#tab-router"]').tab('show');
         });
       })).error(ajax_error_handler);
   }
@@ -219,7 +258,7 @@
           if (n.adminState === "In Service") {
             n.state = "alive";
           } else if (nodes[i].adminState === "Decommission In Progress") {
-            n.state = "decommisioning";
+            n.state = "decommissioning";
           } else if (nodes[i].adminState === "Decommissioned") {
             n.state = "decommissioned";
           }
@@ -314,17 +353,20 @@
   function load_page() {
     var hash = window.location.hash;
     switch(hash) {
-      case "#tab-mounttable":
-        load_mount_table();
+      case "#tab-overview":
+        load_overview();
         break;
       case "#tab-namenode":
         load_namenode_info();
         break;
+      case "#tab-router":
+        load_router_info();
+        break;
       case "#tab-datanode":
         load_datanode_info();
         break;
-      case "#tab-overview":
-        load_overview();
+      case "#tab-mounttable":
+        load_mount_table();
         break;
       default:
         window.location.hash = "tab-overview";

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -2750,7 +2750,8 @@ public class MiniDFSCluster implements AutoCloseable {
     final DataNode dn = dataNodes.get(dataNodeIndex).datanode;
     final FsDatasetSpi<?> dataSet = DataNodeTestUtils.getFSDataset(dn);
     if (!(dataSet instanceof SimulatedFSDataset)) {
-      throw new IOException("injectBlocks is valid only for SimilatedFSDataset");
+      throw new IOException("injectBlocks is valid only for" +
+          " SimulatedFSDataset");
     }
     if (bpid == null) {
       bpid = getNamesystem().getBlockPoolId();
@@ -2771,7 +2772,8 @@ public class MiniDFSCluster implements AutoCloseable {
     final DataNode dn = dataNodes.get(dataNodeIndex).datanode;
     final FsDatasetSpi<?> dataSet = DataNodeTestUtils.getFSDataset(dn);
     if (!(dataSet instanceof SimulatedFSDataset)) {
-      throw new IOException("injectBlocks is valid only for SimilatedFSDataset");
+      throw new IOException("injectBlocks is valid only for" +
+          " SimulatedFSDataset");
     }
     String bpid = getNamesystem(nameNodeIndex).getBlockPoolId();
     SimulatedFSDataset sdataset = (SimulatedFSDataset) dataSet;

+ 33 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java

@@ -1418,6 +1418,39 @@ public class TestEncryptionZones {
         dfsAdmin.getEncryptionZoneForPath(snap3Zone).getPath().toString());
   }
 
+  /**
+   * Test correctness of encryption zones on a existing snapshot path.
+   * Specifically, test the file in encryption zones with no encryption info
+   */
+  @Test
+  public void testSnapshotWithFile() throws Exception {
+    final int len = 8196;
+    final Path zoneParent = new Path("/zones");
+    final Path zone = new Path(zoneParent, "zone");
+    final Path zoneFile = new Path(zone, "zoneFile");
+    fsWrapper.mkdir(zone, FsPermission.getDirDefault(), true);
+    DFSTestUtil.createFile(fs, zoneFile, len, (short) 1, 0xFEED);
+    String contents = DFSTestUtil.readFile(fs, zoneFile);
+
+    // Create the snapshot which contains the file
+    dfsAdmin.allowSnapshot(zoneParent);
+    final Path snap1 = fs.createSnapshot(zoneParent, "snap1");
+
+    // Now delete the file and create encryption zone
+    fsWrapper.delete(zoneFile, false);
+    dfsAdmin.createEncryptionZone(zone, TEST_KEY, NO_TRASH);
+    assertEquals("Got unexpected ez path", zone.toString(),
+        dfsAdmin.getEncryptionZoneForPath(zone).getPath());
+
+    // The file in snapshot shouldn't have any encryption info
+    final Path snapshottedZoneFile = new Path(
+        snap1 + "/" + zone.getName() + "/" + zoneFile.getName());
+    FileEncryptionInfo feInfo = getFileEncryptionInfo(snapshottedZoneFile);
+    assertNull("Expected null ez info", feInfo);
+    assertEquals("Contents of snapshotted file have changed unexpectedly",
+        contents, DFSTestUtil.readFile(fs, snapshottedZoneFile));
+  }
+
   /**
    * Verify symlinks can be created in encryption zones and that
    * they function properly when the target is in the same

+ 10 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.http.HttpConfig.Policy;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -167,12 +168,20 @@ public class TestSaslDataTransfer extends SaslDataTransferTestCase {
   public void testDataNodeAbortsIfNotHttpsOnly() throws Exception {
     HdfsConfiguration clusterConf = createSecureConfig("authentication");
     clusterConf.set(DFS_HTTP_POLICY_KEY,
-      HttpConfig.Policy.HTTP_AND_HTTPS.name());
+        HttpConfig.Policy.HTTP_AND_HTTPS.name());
     exception.expect(RuntimeException.class);
     exception.expectMessage("Cannot start secure DataNode");
     startCluster(clusterConf);
   }
 
+  @Test
+  public void testDataNodeStartIfHttpsQopPrivacy() throws Exception {
+    HdfsConfiguration clusterConf = createSecureConfig("privacy");
+    clusterConf.set(DFS_HTTP_POLICY_KEY,
+        Policy.HTTPS_ONLY.name());
+    startCluster(clusterConf);
+  }
+
   @Test
   public void testNoSaslAndSecurePortsIgnored() throws Exception {
     HdfsConfiguration clusterConf = createSecureConfig("");

+ 25 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java

@@ -933,7 +933,31 @@ public class TestQuorumJournalManager {
     
     verifyEdits(streams, 25, 50);
   }
-  
+
+  @Test
+  public void testInProgressRecovery() throws Exception {
+    // Test the case when in-progress edit log tailing is on, and
+    // new active performs recovery when the old active crashes
+    // without closing the last log segment.
+    // See HDFS-13145 for more details.
+
+    // Write two batches of edits. After these, the commitId on the
+    // journals should be 5, and endTxnId should be 8.
+    EditLogOutputStream stm = qjm.startLogSegment(1,
+        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
+    writeTxns(stm, 1, 5);
+    writeTxns(stm, 6, 3);
+
+    // Do recovery from a separate QJM, just like in failover.
+    QuorumJournalManager qjm2 = createSpyingQJM();
+    qjm2.recoverUnfinalizedSegments();
+    checkRecovery(cluster, 1, 8);
+
+    // When selecting input stream, we should see all txns up to 8.
+    List<EditLogInputStream> streams = new ArrayList<>();
+    qjm2.selectInputStreams(streams, 1, true, true);
+    verifyEdits(streams, 1, 8);
+  }
   
   private QuorumJournalManager createSpyingQJM()
       throws IOException, URISyntaxException {

+ 133 - 32
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java

@@ -100,6 +100,19 @@ public class TestBalancerWithMultipleNameNodes {
       replication = 1;
       this.parameters = parameters;
     }
+
+    Suite(MiniDFSCluster cluster, final int nNameNodes, final int nDataNodes,
+          BalancerParameters parameters, Configuration conf, short
+              replicationFactor) throws IOException {
+      this.conf = conf;
+      this.cluster = cluster;
+      clients = new ClientProtocol[nNameNodes];
+      for(int i = 0; i < nNameNodes; i++) {
+        clients[i] = cluster.getNameNode(i).getRpcServer();
+      }
+      replication = replicationFactor;
+      this.parameters = parameters;
+    }
   }
 
   /* create a file with a length of <code>fileLen</code> */
@@ -154,7 +167,7 @@ public class TestBalancerWithMultipleNameNodes {
 
   static void runBalancer(Suite s,
       final long totalUsed, final long totalCapacity) throws Exception {
-    final double avg = totalUsed*100.0/totalCapacity;
+    double avg = totalUsed*100.0/totalCapacity;
 
     LOG.info("BALANCER 0: totalUsed=" + totalUsed
         + ", totalCapacity=" + totalCapacity
@@ -180,6 +193,9 @@ public class TestBalancerWithMultipleNameNodes {
     for(boolean balanced = false; !balanced; i++) {
       final long[] used = new long[s.cluster.getDataNodes().size()];
       final long[] cap = new long[used.length];
+      final long[][] bpUsed = new long[s.clients.length][s.cluster
+          .getDataNodes().size()];
+
 
       for(int n = 0; n < s.clients.length; n++) {
         final DatanodeInfo[] datanodes = s.clients[n].getDatanodeReport(
@@ -199,25 +215,52 @@ public class TestBalancerWithMultipleNameNodes {
             Assert.assertEquals(used[d], datanodes[d].getDfsUsed());
             Assert.assertEquals(cap[d], datanodes[d].getCapacity());
           }
+          bpUsed[n][d] = datanodes[d].getBlockPoolUsed();
         }
       }
 
+
+
       balanced = true;
       for(int d = 0; d < used.length; d++) {
-        final double p = used[d]*100.0/cap[d];
-        balanced = p <= avg + s.parameters.getThreshold();
-        if (!balanced) {
-          if (i % 100 == 0) {
-            LOG.warn("datanodes " + d + " is not yet balanced: "
-                + "used=" + used[d] + ", cap=" + cap[d] + ", avg=" + avg);
-            LOG.warn("TestBalancer.sum(used)=" + TestBalancer.sum(used)
-                + ", TestBalancer.sum(cap)=" + TestBalancer.sum(cap));
+        double p;
+        if(s.parameters.getBalancingPolicy() == BalancingPolicy.Pool.INSTANCE) {
+          for (int k = 0; k < s.parameters.getBlockPools().size(); k++) {
+            avg = TestBalancer.sum(bpUsed[k])*100/totalCapacity;
+            p = bpUsed[k][d] * 100.0 / cap[d];
+            balanced = p <= avg + s.parameters.getThreshold();
+            if (!balanced) {
+              if (i % 100 == 0) {
+                LOG.warn("datanodes " + d + " is not yet balanced: "
+                    + "block pool used=" + bpUsed[d][k] + ", cap=" + cap[d] +
+                    ", avg=" + avg);
+                LOG.warn("sum(blockpoolUsed)=" + TestBalancer.sum(bpUsed[k])
+                    + ", sum(cap)=" + TestBalancer.sum(cap));
+              }
+              sleep(100);
+              break;
+            }
+          }
+          if (!balanced) {
+            break;
+          }
+        } else {
+          p = used[d] * 100.0 / cap[d];
+          balanced = p <= avg + s.parameters.getThreshold();
+          if (!balanced) {
+            if (i % 100 == 0) {
+              LOG.warn("datanodes " + d + " is not yet balanced: "
+                  + "used=" + used[d] + ", cap=" + cap[d] + ", avg=" + avg);
+              LOG.warn("sum(used)=" + TestBalancer.sum(used)
+                  + ", sum(cap)=" + TestBalancer.sum(cap));
+            }
+            sleep(100);
+            break;
           }
-          sleep(100);
-          break;
         }
       }
     }
+
     LOG.info("BALANCER 6");
     // cluster is balanced, verify that only selected blockpools were touched
     Map<Integer, DatanodeStorageReport[]> postBalancerPoolUsages =
@@ -425,19 +468,26 @@ public class TestBalancerWithMultipleNameNodes {
    * It then adds an empty node and start balancing.
    *
    * @param nNameNodes Number of NameNodes
-   * @param capacities Capacities of the datanodes
    * @param racks Rack names
-   * @param newCapacity the capacity of the new DataNode
    * @param newRack the rack for the new DataNode
    * @param conf Configuration
+   * @param nNameNodestoBalance noOfNameNodestoBalance
+   * @param balancerParameters BalancerParameters
    */ 
-  private void runTest(final int nNameNodes, long[] capacities, String[] racks,
-      long newCapacity, String newRack, Configuration conf) throws Exception {
-    final int nDataNodes = capacities.length;
+  private void runTest(final int nNameNodes, String[] racks,
+                       String[] newRack, Configuration conf,
+                       int nNameNodestoBalance,
+                       BalancerParameters balancerParameters)
+      throws Exception {
+    final int nDataNodes = racks.length;
+    final long[] capacities = new long[nDataNodes];
+    Arrays.fill(capacities, CAPACITY);
     LOG.info("nNameNodes=" + nNameNodes + ", nDataNodes=" + nDataNodes);
     Assert.assertEquals(nDataNodes, racks.length);
 
-    LOG.info("RUN_TEST -1");
+    LOG.info("RUN_TEST -1: start a cluster with nNameNodes=" + nNameNodes
+        + ", nDataNodes=" + nDataNodes);
+
     final MiniDFSCluster cluster = new MiniDFSCluster
         .Builder(new Configuration(conf))
         .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(nNameNodes))
@@ -451,44 +501,63 @@ public class TestBalancerWithMultipleNameNodes {
     try {
       cluster.waitActive();
       LOG.info("RUN_TEST 1");
-      final Suite s =
-          new Suite(cluster, nNameNodes, nDataNodes,
-              BalancerParameters.DEFAULT, conf);
+
+      Suite s;
+
+      Set<String> blockpools = new HashSet<>();
+      if(balancerParameters == null) {
+        s = new Suite(cluster, nNameNodes, nDataNodes,
+            BalancerParameters.DEFAULT, conf);
+
+      } else {
+        for (int i=0; i< nNameNodestoBalance; i++) {
+          blockpools.add(cluster.getNamesystem(i).getBlockPoolId());
+        }
+        BalancerParameters.Builder b =
+            new BalancerParameters.Builder();
+        b.setBalancingPolicy(balancerParameters.getBalancingPolicy());
+        b.setBlockpools(blockpools);
+        BalancerParameters params = b.build();
+        s = new Suite(cluster, nNameNodes, nDataNodes, params, conf, (short)2);
+      }
       long totalCapacity = TestBalancer.sum(capacities);
 
-      LOG.info("RUN_TEST 2");
+      LOG.info("RUN_TEST 2: create files");
       // fill up the cluster to be 30% full
-      final long totalUsed = totalCapacity*3/10;
+      final long totalUsed = (totalCapacity * s.replication)*3/10;
       final long size = (totalUsed/nNameNodes)/s.replication;
       for(int n = 0; n < nNameNodes; n++) {
         createFile(s, n, size);
       }
 
-      LOG.info("RUN_TEST 3");
+      LOG.info("RUN_TEST 3: " + newRack.length + " new datanodes");
       // start up an empty node with the same capacity and on the same rack
-      cluster.startDataNodes(conf, 1, true, null,
-          new String[]{newRack}, new long[]{newCapacity});
+      final long[] newCapacity = new long[newRack.length];
+      Arrays.fill(newCapacity, CAPACITY);
+      cluster.startDataNodes(conf, newCapacity.length, true, null,
+          newRack, newCapacity);
 
-      totalCapacity += newCapacity;
+      totalCapacity += TestBalancer.sum(newCapacity);
 
-      LOG.info("RUN_TEST 4");
+      LOG.info("RUN_TEST 4: run Balancer");
       // run RUN_TEST and validate results
       runBalancer(s, totalUsed, totalCapacity);
       LOG.info("RUN_TEST 5");
     } finally {
       cluster.shutdown();
     }
-    LOG.info("RUN_TEST 6");
+    LOG.info("RUN_TEST 6: done");
   }
+
   
   /** Test a cluster with even distribution, 
-   * then a new empty node is added to the cluster
+   * then a new empty node is added to the cluster.
    */
   @Test
-  public void testBalancer() throws Exception {
+  public void testTwoOneOne() throws Exception {
     final Configuration conf = createConf();
-    runTest(2, new long[]{CAPACITY}, new String[]{RACK0},
-        CAPACITY/2, RACK0, conf);
+    runTest(2, new String[]{RACK0}, new String[] {RACK0}, conf,
+        2, null);
   }
 
   /** Test unevenly distributed cluster */
@@ -517,4 +586,36 @@ public class TestBalancerWithMultipleNameNodes {
         5 * CAPACITY / 100, 10 * CAPACITY / 100 }, new long[] { CAPACITY,
         CAPACITY, CAPACITY }, new String[] { RACK0, RACK1, RACK2 }, conf);
   }
+
+  /** Even distribution with 2 Namenodes, 4 Datanodes and 2 new Datanodes. */
+  @Test(timeout = 600000)
+  public void testTwoFourTwo() throws Exception {
+    final Configuration conf = createConf();
+    runTest(2, new String[]{RACK0, RACK0, RACK1, RACK1},
+        new String[]{RACK2, RACK2}, conf, 2, null);
+  }
+
+  @Test(timeout=600000)
+  public void testBalancingBlockpoolsWithBlockPoolPolicy() throws Exception {
+    final Configuration conf = createConf();
+    BalancerParameters balancerParameters = new BalancerParameters.Builder()
+        .setBalancingPolicy(BalancingPolicy.Pool.INSTANCE).build();
+    runTest(2, new String[]{RACK0, RACK0, RACK1, RACK1},
+        new String[]{RACK2, RACK2}, conf, 2,
+        balancerParameters);
+  }
+
+  @Test(timeout = 600000)
+  public void test1OutOf2BlockpoolsWithBlockPoolPolicy()
+      throws
+      Exception {
+    final Configuration conf = createConf();
+    BalancerParameters balancerParameters = new BalancerParameters.Builder()
+        .setBalancingPolicy(BalancingPolicy.Pool.INSTANCE).build();
+    runTest(2, new String[]{RACK0, RACK0, RACK1, RACK1},
+        new String[]{RACK2, RACK2}, conf, 1,
+        balancerParameters);
+  }
+
+
 }

+ 128 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.diskbalancer.command;
 
 
+import static java.lang.Thread.sleep;
 import static org.apache.hadoop.hdfs.tools.DiskBalancerCLI.CANCEL;
 import static org.apache.hadoop.hdfs.tools.DiskBalancerCLI.EXECUTE;
 import static org.apache.hadoop.hdfs.tools.DiskBalancerCLI.HELP;
@@ -58,6 +59,7 @@ import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode
 import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -187,6 +189,132 @@ public class TestDiskBalancerCommand {
     }
   }
 
+
+
+  @Test(timeout = 600000)
+  public void testDiskBalancerExecuteOptionPlanValidityWithException() throws
+      Exception {
+    final int numDatanodes = 1;
+
+    final Configuration hdfsConf = new HdfsConfiguration();
+    hdfsConf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+    hdfsConf.set(DFSConfigKeys.DFS_DISK_BALANCER_PLAN_VALID_INTERVAL, "0d");
+
+    /* new cluster with imbalanced capacity */
+    final MiniDFSCluster miniCluster = DiskBalancerTestUtil.
+        newImbalancedCluster(
+        hdfsConf,
+        numDatanodes,
+        CAPACITIES,
+        DEFAULT_BLOCK_SIZE,
+        FILE_LEN);
+
+    try {
+      /* get full path of plan */
+      final String planFileFullName = runAndVerifyPlan(miniCluster, hdfsConf);
+
+      /* run execute command */
+      final String cmdLine = String.format(
+          "hdfs diskbalancer -%s %s",
+          EXECUTE,
+          planFileFullName);
+
+      LambdaTestUtils.intercept(
+          RemoteException.class,
+          "DiskBalancerException",
+          "Plan was generated more than 0d ago",
+          () -> {
+            runCommand(cmdLine, hdfsConf, miniCluster);
+          });
+    }  finally{
+      if (miniCluster != null) {
+        miniCluster.shutdown();
+      }
+    }
+  }
+
+  @Test(timeout = 600000)
+  public void testDiskBalancerExecutePlanValidityWithOutUnitException()
+      throws
+      Exception {
+    final int numDatanodes = 1;
+
+    final Configuration hdfsConf = new HdfsConfiguration();
+    hdfsConf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+    hdfsConf.set(DFSConfigKeys.DFS_DISK_BALANCER_PLAN_VALID_INTERVAL, "0");
+
+    /* new cluster with imbalanced capacity */
+    final MiniDFSCluster miniCluster = DiskBalancerTestUtil.
+        newImbalancedCluster(
+            hdfsConf,
+            numDatanodes,
+            CAPACITIES,
+            DEFAULT_BLOCK_SIZE,
+            FILE_LEN);
+
+    try {
+      /* get full path of plan */
+      final String planFileFullName = runAndVerifyPlan(miniCluster, hdfsConf);
+
+      /* run execute command */
+      final String cmdLine = String.format(
+          "hdfs diskbalancer -%s %s",
+          EXECUTE,
+          planFileFullName);
+
+      LambdaTestUtils.intercept(
+          RemoteException.class,
+          "DiskBalancerException",
+          "Plan was generated more than 0ms ago",
+          () -> {
+            runCommand(cmdLine, hdfsConf, miniCluster);
+          });
+    }  finally{
+      if (miniCluster != null) {
+        miniCluster.shutdown();
+      }
+    }
+  }
+
+
+  @Test(timeout = 600000)
+  public void testDiskBalancerExecuteOptionPlanValidity() throws Exception {
+    final int numDatanodes = 1;
+
+    final Configuration hdfsConf = new HdfsConfiguration();
+    hdfsConf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+    hdfsConf.set(DFSConfigKeys.DFS_DISK_BALANCER_PLAN_VALID_INTERVAL, "600s");
+
+    /* new cluster with imbalanced capacity */
+    final MiniDFSCluster miniCluster = DiskBalancerTestUtil.
+        newImbalancedCluster(
+            hdfsConf,
+            numDatanodes,
+            CAPACITIES,
+            DEFAULT_BLOCK_SIZE,
+            FILE_LEN);
+
+    try {
+      /* get full path of plan */
+      final String planFileFullName = runAndVerifyPlan(miniCluster, hdfsConf);
+
+      /* run execute command */
+      final String cmdLine = String.format(
+          "hdfs diskbalancer -%s %s",
+          EXECUTE,
+          planFileFullName);
+
+      // Plan is valid for 600 seconds, sleeping for 10seconds, so now
+      // diskbalancer should execute the plan
+      sleep(10000);
+      runCommand(cmdLine, hdfsConf, miniCluster);
+    }  finally{
+      if (miniCluster != null) {
+        miniCluster.shutdown();
+      }
+    }
+  }
+
   private String runAndVerifyPlan(
       final MiniDFSCluster miniCluster,
       final Configuration hdfsConf) throws Exception {

+ 10 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCClientRetries.java

@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -64,8 +65,16 @@ public class TestRouterRPCClientRetries {
         .rpc()
         .build();
 
+    // reduce IPC client connection retry times and interval time
+    Configuration clientConf = new Configuration(false);
+    clientConf.setInt(
+        CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 1);
+    clientConf.setInt(
+        CommonConfigurationKeys.IPC_CLIENT_CONNECT_RETRY_INTERVAL_KEY, 100);
+
     cluster.addRouterOverrides(routerConf);
-    cluster.startCluster();
+    // override some settings for the client
+    cluster.startCluster(clientConf);
     cluster.startRouters();
     cluster.waitClusterUp();
 

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java

@@ -420,6 +420,9 @@ public class TestCacheDirectives {
         setMode(new FsPermission((short)0777)));
     proto.addCachePool(new CachePoolInfo("pool4").
         setMode(new FsPermission((short)0)));
+    proto.addCachePool(new CachePoolInfo("pool5").
+        setMode(new FsPermission((short)0007))
+        .setOwnerName(unprivilegedUser.getShortUserName()));
 
     CacheDirectiveInfo alpha = new CacheDirectiveInfo.Builder().
         setPath(new Path("/alpha")).
@@ -488,6 +491,18 @@ public class TestCacheDirectives {
 
     long deltaId = addAsUnprivileged(delta);
 
+    try {
+      addAsUnprivileged(new CacheDirectiveInfo.Builder().
+          setPath(new Path("/epsilon")).
+          setPool("pool5").
+          build());
+      fail("expected an error when adding to a pool with " +
+          "mode 007 (no permissions for pool owner).");
+    } catch (AccessControlException e) {
+      GenericTestUtils.
+          assertExceptionContains("Permission denied while accessing pool", e);
+    }
+
     // We expect the following to succeed, because DistributedFileSystem
     // qualifies the path.
     long relativeId = addAsUnprivileged(

+ 264 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestDirectoryDiffList.java

@@ -0,0 +1,264 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.snapshot;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.ChildrenDiff;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.IntFunction;
+
+/**
+ * This class tests the DirectoryDiffList API's.
+ */
+public class TestDirectoryDiffList{
+  static final int NUM_SNAPSHOTS = 100;
+  static {
+    SnapshotTestHelper.disableLogs();
+  }
+
+  private static final Configuration CONF = new Configuration();
+  private static MiniDFSCluster cluster;
+  private static FSNamesystem fsn;
+  private static FSDirectory fsdir;
+  private static DistributedFileSystem hdfs;
+
+  @Before
+  public void setUp() throws Exception {
+    cluster =
+        new MiniDFSCluster.Builder(CONF).numDataNodes(0).format(true).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    fsdir = fsn.getFSDirectory();
+    hdfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  static void assertList(List<INode> expected, List<INode> computed) {
+    Assert.assertEquals(expected.size(), computed.size());
+    for (int index = 0; index < expected.size(); index++) {
+      Assert.assertEquals(expected.get(index), computed.get(index));
+    }
+  }
+
+  static void verifyChildrenList(DirectoryDiffList skip, INodeDirectory dir) {
+    final int n = skip.size();
+    for (int i = 0; i < skip.size(); i++) {
+      final List<INode> expected = ReadOnlyList.Util.asList(
+          dir.getChildrenList(dir.getDiffs().asList().get(i).getSnapshotId()));
+      final List<INode> computed = getChildrenList(skip, i, n, dir);
+      try {
+        assertList(expected, computed);
+      } catch (AssertionError ae) {
+        throw new AssertionError(
+            "i = " + i + "\ncomputed = " + computed + "\nexpected = "
+                + expected + "\n" + skip, ae);
+      }
+    }
+  }
+
+  static void verifyChildrenList(
+      DiffList<DirectoryDiff> array, DirectoryDiffList skip,
+      INodeDirectory dir, List<INode> childrenList) {
+    final int n = array.size();
+    Assert.assertEquals(n, skip.size());
+    for (int i = 0; i < n - 1; i++) {
+      for (int j = i + 1; j < n - 1; j++) {
+        final List<INode> expected = getCombined(array, i, j, dir)
+            .apply2Previous(childrenList);
+        final List<INode> computed = getCombined(skip, i, j, dir)
+            .apply2Previous(childrenList);
+        try {
+          assertList(expected, computed);
+        } catch (AssertionError ae) {
+          throw new AssertionError(
+              "i = " + i + ", j = " + j + "\ncomputed = " + computed
+                  + "\nexpected = " + expected + "\n" + skip, ae);
+        }
+      }
+    }
+  }
+
+  private static ChildrenDiff getCombined(
+      DiffList<DirectoryDiff> list, int from, int to, INodeDirectory dir) {
+    final List<DirectoryDiff> minList = list.getMinListForRange(from, to, dir);
+    final ChildrenDiff combined = new ChildrenDiff();
+    for (DirectoryDiff d : minList) {
+      combined.combinePosterior(d.getChildrenDiff(), null);
+    }
+    return combined;
+  }
+
+  static List<INode> getChildrenList(
+      DiffList<DirectoryDiff> list, int from, int to, INodeDirectory dir) {
+    final ChildrenDiff combined = getCombined(list, from, to, dir);
+    return combined.apply2Current(ReadOnlyList.Util.asList(dir.getChildrenList(
+        Snapshot.CURRENT_STATE_ID)));
+  }
+
+  static Path getChildPath(Path parent, int i) {
+    return new Path(parent, "c" + i);
+  }
+
+  @Test
+  public void testAddLast() throws Exception {
+    testAddLast(NUM_SNAPSHOTS);
+  }
+
+  static void testAddLast(int n) throws Exception {
+    final Path root = new Path("/testAddLast" + n);
+    DirectoryDiffList.LOG.info("run " + root);
+
+    final DirectoryDiffList skipList = new DirectoryDiffList(0, 3, 5);
+    final DiffList<DirectoryDiff> arrayList = new DiffListByArrayList<>(0);
+    INodeDirectory dir = addDiff(n, skipList, arrayList, root);
+    // verify that the both the children list obtained from hdfs and
+    // DirectoryDiffList are same
+    verifyChildrenList(skipList, dir);
+    verifyChildrenList(arrayList, skipList, dir, Collections.emptyList());
+  }
+
+
+  @Test
+  public void testAddFirst() throws Exception {
+    testAddFirst(NUM_SNAPSHOTS);
+  }
+
+  static void testAddFirst(int n) throws Exception {
+    final Path root = new Path("/testAddFirst" + n);
+    DirectoryDiffList.LOG.info("run " + root);
+
+    hdfs.mkdirs(root);
+    for (int i = 1; i < n; i++) {
+      final Path child = getChildPath(root, i);
+      hdfs.mkdirs(child);
+    }
+    INodeDirectory dir = fsdir.getINode(root.toString()).asDirectory();
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
+    for (int i = 1; i < n; i++) {
+      final Path child = getChildPath(root, n - i);
+      hdfs.delete(child, false);
+      hdfs.createSnapshot(root, "s" + i);
+    }
+    DiffList<DirectoryDiff> diffs = dir.getDiffs().asList();
+    List<INode> childrenList = ReadOnlyList.Util.asList(dir.getChildrenList(
+        diffs.get(0).getSnapshotId()));
+    final DirectoryDiffList skipList = new DirectoryDiffList(0, 3, 5);
+    final DiffList<DirectoryDiff> arrayList = new DiffListByArrayList<>(0);
+    for (int i = diffs.size() - 1; i >= 0; i--) {
+      final DirectoryDiff d = diffs.get(i);
+      skipList.addFirst(d);
+      arrayList.addFirst(d);
+    }
+    // verify that the both the children list obtained from hdfs and
+    // DirectoryDiffList are same
+    verifyChildrenList(skipList, dir);
+    verifyChildrenList(arrayList, skipList, dir, childrenList);
+  }
+
+  static INodeDirectory addDiff(int n, DiffList skipList, DiffList arrayList,
+      final Path root) throws Exception {
+    hdfs.mkdirs(root);
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
+    for (int i = 1; i < n; i++) {
+      final Path child = getChildPath(root, i);
+      hdfs.mkdirs(child);
+      hdfs.createSnapshot(root, "s" + i);
+    }
+    INodeDirectory dir = fsdir.getINode(root.toString()).asDirectory();
+    DiffList<DirectoryDiff> diffs = dir.getDiffs().asList();
+    for (DirectoryDiff d : diffs) {
+      skipList.addLast(d);
+      arrayList.addLast(d);
+    }
+    return dir;
+  }
+
+  @Test
+  public void testRemoveFromTail() throws Exception {
+    final int n = NUM_SNAPSHOTS;
+    testRemove("FromTail", n, i -> n - 1 - i);
+  }
+
+  @Test
+  public void testReomveFromHead() throws Exception {
+    testRemove("FromHead", NUM_SNAPSHOTS, i -> 0);
+  }
+
+  @Test
+  public void testRemoveRandom() throws Exception {
+    final int n = NUM_SNAPSHOTS;
+    testRemove("Random", n, i -> ThreadLocalRandom.current().nextInt(n - i));
+  }
+
+  static void testRemove(String name, int n, IntFunction<Integer> indexFunction)
+      throws Exception {
+    final Path root = new Path("/testRemove" + name + n);
+    DirectoryDiffList.LOG.info("run " + root);
+
+    final DirectoryDiffList skipList = new DirectoryDiffList(0, 3, 5);
+    final DiffList<DirectoryDiff> arrayList = new DiffListByArrayList<>(0);
+    final INodeDirectory dir = addDiff(n, skipList, arrayList, root);
+    Assert.assertEquals(n, arrayList.size());
+    Assert.assertEquals(n, skipList.size());
+
+    for(int i = 0; i < n; i++) {
+      final int index = indexFunction.apply(i);
+      final DirectoryDiff diff = remove(index, skipList, arrayList);
+      hdfs.deleteSnapshot(root, "s" + diff.getSnapshotId());
+      verifyChildrenList(skipList, dir);
+      verifyChildrenList(arrayList, skipList, dir, Collections.emptyList());
+    }
+  }
+
+  static DirectoryDiff remove(int i, DirectoryDiffList skip,
+      DiffList<DirectoryDiff> array) {
+    DirectoryDiffList.LOG.info("remove " + i);
+    final DirectoryDiff expected = array.remove(i);
+    final DirectoryDiff computed = skip.remove(i);
+    assertDirectoryDiff(expected, computed);
+    return expected;
+  }
+
+  static void assertDirectoryDiff(DirectoryDiff expected,
+      DirectoryDiff computed) {
+    Assert.assertEquals(expected.getSnapshotId(), computed.getSnapshotId());
+  }
+}

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java

@@ -1539,4 +1539,17 @@ public class TestSnapshotDiffReport {
         new DiffReportEntry(DiffType.DELETE,
             DFSUtil.string2Bytes("dir3/file3")));
   }
+
+  @Test
+  public void testSnapshotDiffReportRemoteIterator2() throws Exception {
+    final Path root = new Path("/");
+    hdfs.mkdirs(root);
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
+    try {
+      hdfs.snapshotDiffReportListingRemoteIterator(root, "s0", "");
+    } catch (Exception e) {
+      Assert.assertTrue(e.getMessage().contains("Remote Iterator is"
+          + "supported for snapshotDiffReport between two snapshots"));
+    }
+  }
 }

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java

@@ -231,8 +231,8 @@ public class JobHistoryEventHandler extends AbstractService
     try {
       doneDirPrefixPath =
           FileContext.getFileContext(conf).makeQualified(new Path(userDoneDirStr));
-      mkdir(doneDirFS, doneDirPrefixPath, new FsPermission(
-          JobHistoryUtils.HISTORY_INTERMEDIATE_USER_DIR_PERMISSIONS));
+      mkdir(doneDirFS, doneDirPrefixPath, JobHistoryUtils.
+          getConfiguredHistoryIntermediateUserDoneDirPermissions(conf));
     } catch (IOException e) {
       LOG.error("Error creating user intermediate history done directory: [ "
           + doneDirPrefixPath + "]", e);

+ 4 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java

@@ -93,6 +93,10 @@ public class JHAdminConfig {
    **/
   public static final String MR_HISTORY_INTERMEDIATE_DONE_DIR =
     MR_HISTORY_PREFIX + "intermediate-done-dir";
+  public static final String MR_HISTORY_INTERMEDIATE_USER_DONE_DIR_PERMISSIONS =
+      MR_HISTORY_PREFIX + "intermediate-user-done-dir.permissions";
+  public static final short
+      DEFAULT_MR_HISTORY_INTERMEDIATE_USER_DONE_DIR_PERMISSIONS = 0770;
   
   /** Size of the job list cache.*/
   public static final String MR_HISTORY_JOBLIST_CACHE_SIZE =

+ 31 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JobHistoryUtils.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -89,13 +90,7 @@ public class JobHistoryUtils {
    */
   public static final FsPermission HISTORY_INTERMEDIATE_DONE_DIR_PERMISSIONS = 
     FsPermission.createImmutable((short) 01777);
-  
-  /**
-   * Permissions for the user directory under the intermediate done directory.
-   */
-  public static final FsPermission HISTORY_INTERMEDIATE_USER_DIR_PERMISSIONS = 
-    FsPermission.createImmutable((short) 0770);
-  
+
   public static final FsPermission HISTORY_INTERMEDIATE_FILE_PERMISSIONS = 
     FsPermission.createImmutable((short) 0770); // rwx------
   
@@ -208,6 +203,35 @@ public class JobHistoryUtils {
     }
     return ensurePathInDefaultFileSystem(doneDirPrefix, conf);
   }
+
+  /**
+   * Gets the configured directory permissions for the user directories in the
+   * directory of the intermediate done history files. The user and the group
+   * both need full permissions, this is enforced by this method.
+   * @param conf The configuration object
+   * @return FsPermission of the user directories
+   */
+  public static FsPermission
+        getConfiguredHistoryIntermediateUserDoneDirPermissions(
+            Configuration conf) {
+    String userDoneDirPermissions = conf.get(
+        JHAdminConfig.MR_HISTORY_INTERMEDIATE_USER_DONE_DIR_PERMISSIONS);
+    if (userDoneDirPermissions == null) {
+      return new FsPermission(
+          JHAdminConfig.DEFAULT_MR_HISTORY_INTERMEDIATE_USER_DONE_DIR_PERMISSIONS);
+    }
+    FsPermission permission = new FsPermission(userDoneDirPermissions);
+    if (permission.getUserAction() != FsAction.ALL ||
+        permission.getGroupAction() != FsAction.ALL) {
+      permission = new FsPermission(FsAction.ALL, FsAction.ALL,
+          permission.getOtherAction(), permission.getStickyBit());
+      LOG.warn("Unsupported permission configured in " +
+          JHAdminConfig.MR_HISTORY_INTERMEDIATE_USER_DONE_DIR_PERMISSIONS +
+          ", the user and the group permission must be 7 (rwx). " +
+          "The permission was set to " + permission.toString());
+    }
+    return permission;
+  }
   
   /**
    * Gets the configured directory prefix for Done history files.

+ 24 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/jobhistory/TestJobHistoryUtils.java

@@ -24,6 +24,10 @@ import java.io.IOException;
 import java.util.Calendar;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
@@ -31,6 +35,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.junit.Assert;
 import org.junit.Test;
 
+import static org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils.getConfiguredHistoryIntermediateUserDoneDirPermissions;
+
+
 public class TestJobHistoryUtils {
 
   final static String TEST_DIR = new File(System.getProperty("test.build.data"))
@@ -140,4 +147,21 @@ public class TestJobHistoryUtils {
     fc.mkdir(path, FsPermission.getDirDefault(), true);
     return path;
   }
+
+  @Test
+  public void testGetConfiguredHistoryIntermediateUserDoneDirPermissions() {
+    Configuration conf = new Configuration();
+    Map<String, FsPermission> parameters = ImmutableMap.of(
+      "775", new FsPermission(0775),
+      "123", new FsPermission(0773),
+      "-rwx", new FsPermission(0770) ,
+      "+rwx", new FsPermission(0777)
+    );
+    for (Map.Entry<String, FsPermission> entry : parameters.entrySet()) {
+      conf.set(JHAdminConfig.MR_HISTORY_INTERMEDIATE_USER_DONE_DIR_PERMISSIONS,
+          entry.getKey());
+      Assert.assertEquals(entry.getValue(),
+          getConfiguredHistoryIntermediateUserDoneDirPermissions(conf));
+    }
+  }
 }

+ 22 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java

@@ -37,6 +37,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.RpcNoSuchMethodException;
 import org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -171,7 +173,7 @@ class JobResourceUploader {
 
     if (!conf.getBoolean(MRJobConfig.MR_AM_STAGING_DIR_ERASURECODING_ENABLED,
         MRJobConfig.DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED)) {
-      disableErasureCodingForPath(jtFs, submitJobDir);
+      disableErasureCodingForPath(submitJobDir);
     }
 
     // Get the resources that have been added via command line arguments in the
@@ -874,13 +876,26 @@ class JobResourceUploader {
     return finalPath;
   }
 
-  private void disableErasureCodingForPath(FileSystem fs, Path path)
+  private void disableErasureCodingForPath(Path path)
       throws IOException {
-    if (jtFs instanceof DistributedFileSystem) {
-      LOG.info("Disabling Erasure Coding for path: " + path);
-      DistributedFileSystem dfs = (DistributedFileSystem) jtFs;
-      dfs.setErasureCodingPolicy(path,
-          SystemErasureCodingPolicies.getReplicationPolicy().getName());
+    try {
+      if (jtFs instanceof DistributedFileSystem) {
+        LOG.info("Disabling Erasure Coding for path: " + path);
+        DistributedFileSystem dfs = (DistributedFileSystem) jtFs;
+        dfs.setErasureCodingPolicy(path,
+            SystemErasureCodingPolicies.getReplicationPolicy().getName());
+      }
+    } catch (RemoteException e) {
+      if (!RpcNoSuchMethodException.class.getName().equals(e.getClassName())) {
+        throw e;
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(
+              "Ignore disabling erasure coding for path {} because method "
+                  + "disableErasureCodingForPath doesn't exist, probably "
+                  + "talking to a lower version HDFS.", path.toString(), e);
+        }
+      }
     }
   }
 }

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -1730,6 +1730,15 @@
   <description></description>
 </property>
 
+<property>
+  <name>mapreduce.jobhistory.intermediate-user-done-dir.permissions</name>
+  <value>770</value>
+  <description>The permissions of the user directories in
+  ${mapreduce.jobhistory.intermediate-done-dir}. The user and the group
+  permission must be 7, this is enforced.
+  </description>
+</property>
+
 <property>
   <name>mapreduce.jobhistory.done-dir</name>
   <value>${yarn.app.mapreduce.am.staging-dir}/history/done</value>

+ 14 - 2
hadoop-project/pom.xml

@@ -542,6 +542,12 @@
         <version>${project.version}</version>
       </dependency>
 
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-azure-datalake</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
       <dependency>
         <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-aws</artifactId>
@@ -886,6 +892,12 @@
         <groupId>com.amazonaws</groupId>
         <artifactId>aws-java-sdk-bundle</artifactId>
         <version>${aws-java-sdk.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>io.netty</groupId>
+            <artifactId>*</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>org.apache.mina</groupId>
@@ -1140,9 +1152,9 @@
         <version>${hsqldb.version}</version>
       </dependency>
       <dependency>
-        <groupId>com.codahale.metrics</groupId>
+        <groupId>io.dropwizard.metrics</groupId>
         <artifactId>metrics-core</artifactId>
-        <version>3.0.1</version>
+        <version>3.2.4</version>
       </dependency>
       <dependency>
         <groupId>org.apache.hadoop</groupId>

+ 1 - 0
hadoop-tools/hadoop-azure-datalake/pom.xml

@@ -116,6 +116,7 @@
     <dependency>
     <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>com.squareup.okhttp</groupId>

+ 4 - 3
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/http/ExceptionDiags.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift.http;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.lang.reflect.Constructor;
@@ -33,7 +33,8 @@ import java.net.UnknownHostException;
  * available in branch-1 too.
  */
 public class ExceptionDiags {
-  private static final Log LOG = LogFactory.getLog(ExceptionDiags.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ExceptionDiags.class);
 
   /** text to point users elsewhere: {@value} */
   private static final String FOR_MORE_DETAILS_SEE

+ 4 - 3
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/http/RestClientBindings.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift.http;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.swift.exceptions.SwiftConfigurationException;
 
@@ -39,7 +39,8 @@ import static org.apache.hadoop.fs.swift.http.SwiftProtocolConstants.*;
  * and then builds an appropriate Properties file.
  */
 public final class RestClientBindings {
-  private static final Log LOG = LogFactory.getLog(RestClientBindings.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RestClientBindings.class);
 
   public static final String E_INVALID_NAME = "Invalid swift hostname '%s':" +
           " hostname must in form container.service";

+ 5 - 4
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/http/SwiftRestClient.java

@@ -40,8 +40,8 @@ import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.DefaultHttpRequestRetryHandler;
 import org.apache.http.impl.client.HttpClientBuilder;
 import org.apache.http.message.BasicHeader;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.swift.auth.ApiKeyAuthenticationRequest;
 import org.apache.hadoop.fs.swift.auth.ApiKeyCredentials;
@@ -97,7 +97,8 @@ import static org.apache.hadoop.fs.swift.http.SwiftProtocolConstants.*;
  * details.
  */
 public final class SwiftRestClient {
-  private static final Log LOG = LogFactory.getLog(SwiftRestClient.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SwiftRestClient.class);
 
   /**
    * Header that says "use newest version" -ensures that
@@ -1725,7 +1726,7 @@ public final class SwiftRestClient {
       for (Header header : req.getAllHeaders()) {
         builder.append(header.toString());
       }
-      LOG.debug(builder);
+      LOG.debug(builder.toString());
     }
     HttpResponse resp = client.execute(req);
     if (LOG.isDebugEnabled()) {

+ 4 - 4
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystem.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift.snative;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
@@ -54,8 +54,8 @@ public class SwiftNativeFileSystem extends FileSystem {
 
   /** filesystem prefix: {@value} */
   public static final String SWIFT = "swift";
-  private static final Log LOG =
-          LogFactory.getLog(SwiftNativeFileSystem.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SwiftNativeFileSystem.class);
 
   /**
    * path to user work directory for storing temporary files

+ 5 - 5
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java

@@ -22,8 +22,8 @@ import com.fasterxml.jackson.databind.type.CollectionType;
 import org.apache.http.Header;
 import org.apache.http.HttpStatus;
 import org.apache.http.message.BasicHeader;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
@@ -65,8 +65,8 @@ import java.util.regex.Pattern;
 public class SwiftNativeFileSystemStore {
   private static final Pattern URI_PATTERN = Pattern.compile("\"\\S+?\"");
   private static final String PATTERN = "EEE, d MMM yyyy hh:mm:ss zzz";
-  private static final Log LOG =
-          LogFactory.getLog(SwiftNativeFileSystemStore.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SwiftNativeFileSystemStore.class);
   private URI uri;
   private SwiftRestClient swiftRestClient;
 
@@ -720,7 +720,7 @@ public class SwiftNativeFileSystemStore {
     if (LOG.isDebugEnabled()) {
       LOG.debug(message + ": listing of " + objectPath);
       for (FileStatus fileStatus : statuses) {
-        LOG.debug(fileStatus.getPath());
+        LOG.debug(fileStatus.getPath().toString());
       }
     }
   }

+ 4 - 3
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeInputStream.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift.snative;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -43,7 +43,8 @@ import java.io.IOException;
  */
 class SwiftNativeInputStream extends FSInputStream {
 
-  private static final Log LOG = LogFactory.getLog(SwiftNativeInputStream.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SwiftNativeInputStream.class);
 
   /**
    *  range requested off the server: {@value}

+ 4 - 4
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeOutputStream.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift.snative;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.swift.exceptions.SwiftConnectionClosedException;
@@ -45,8 +45,8 @@ import java.io.OutputStream;
 class SwiftNativeOutputStream extends OutputStream {
   public static final int ATTEMPT_LIMIT = 3;
   private long filePartSize;
-  private static final Log LOG =
-          LogFactory.getLog(SwiftNativeOutputStream.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SwiftNativeOutputStream.class);
   private Configuration conf;
   private String key;
   private File backupFile;

+ 4 - 4
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/util/SwiftTestUtils.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift.util;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -40,8 +40,8 @@ import java.util.Properties;
  */
 public class SwiftTestUtils extends org.junit.Assert {
 
-  private static final Log LOG =
-    LogFactory.getLog(SwiftTestUtils.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SwiftTestUtils.class);
 
   public static final String TEST_FS_SWIFT = "test.fs.swift.name";
   public static final String IO_FILE_BUFFER_SIZE = "io.file.buffer.size";

+ 4 - 4
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/util/SwiftUtils.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.fs.swift.util;
 
-import org.apache.commons.logging.Log;
+import org.slf4j.Logger;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -98,7 +98,7 @@ public final class SwiftUtils {
    * @param text text message
    * @param args args arguments to the print statement
    */
-  public static void debug(Log log, String text, Object... args) {
+  public static void debug(Logger log, String text, Object... args) {
     if (log.isDebugEnabled()) {
       log.debug(String.format(text, args));
     }
@@ -110,7 +110,7 @@ public final class SwiftUtils {
    * @param text text message
    * @param ex exception
    */
-  public static void debugEx(Log log, String text, Exception ex) {
+  public static void debugEx(Logger log, String text, Exception ex) {
     if (log.isDebugEnabled()) {
       log.debug(text + ex, ex);
     }
@@ -124,7 +124,7 @@ public final class SwiftUtils {
    * @param text text message
    * @param args args arguments to the print statement
    */
-  public static void trace(Log log, String text, Object... args) {
+  public static void trace(Logger log, String text, Object... args) {
     if (log.isTraceEnabled()) {
       log.trace(String.format(text, args));
     }

+ 4 - 4
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/SwiftFileSystemBaseTest.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -53,8 +53,8 @@ import static org.apache.hadoop.fs.swift.util.SwiftTestUtils.noteAction;
 public class SwiftFileSystemBaseTest extends Assert implements
                                                     SwiftTestConstants {
 
-  protected static final Log LOG =
-          LogFactory.getLog(SwiftFileSystemBaseTest.class);
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(SwiftFileSystemBaseTest.class);
   protected SwiftNativeFileSystem fs;
   protected static SwiftNativeFileSystem lastFs;
   protected byte[] data = SwiftTestUtils.dataset(getBlockSize() * 2, 0, 255);

+ 4 - 4
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestLogResources.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.junit.Test;
 
 import java.net.URL;
@@ -28,8 +28,8 @@ import java.net.URL;
  * This test just debugs which log resources are being picked up
  */
 public class TestLogResources implements SwiftTestConstants {
-  protected static final Log LOG =
-    LogFactory.getLog(TestLogResources.class);
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(TestLogResources.class);
 
   private void printf(String format, Object... args) {
     String msg = String.format(format, args);

+ 4 - 4
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestReadPastBuffer.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
@@ -44,8 +44,8 @@ import org.junit.Test;
  * to get confused.
  */
 public class TestReadPastBuffer extends SwiftFileSystemBaseTest {
-  protected static final Log LOG =
-    LogFactory.getLog(TestReadPastBuffer.class);
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(TestReadPastBuffer.class);
   public static final int SWIFT_READ_BLOCKSIZE = 4096;
   public static final int SEEK_FILE_LEN = SWIFT_READ_BLOCKSIZE * 2;
 

+ 4 - 4
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSeek.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
@@ -48,8 +48,8 @@ import java.io.IOException;
  * to get confused.
  */
 public class TestSeek extends SwiftFileSystemBaseTest {
-  protected static final Log LOG =
-    LogFactory.getLog(TestSeek.class);
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(TestSeek.class);
   public static final int SMALL_SEEK_FILE_LEN = 256;
 
   private Path testPath;

+ 4 - 4
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemBasicOps.java

@@ -19,8 +19,8 @@
 package org.apache.hadoop.fs.swift;
 
 import org.junit.Assert;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
@@ -46,8 +46,8 @@ import static org.apache.hadoop.fs.swift.util.SwiftTestUtils.writeTextFile;
  */
 public class TestSwiftFileSystemBasicOps extends SwiftFileSystemBaseTest {
 
-  private static final Log LOG =
-          LogFactory.getLog(TestSwiftFileSystemBasicOps.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestSwiftFileSystemBasicOps.class);
 
   @Test(timeout = SWIFT_TEST_TIMEOUT)
   public void testLsRoot() throws Throwable {

+ 4 - 4
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemConcurrency.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.swift.util.SwiftTestUtils;
@@ -37,8 +37,8 @@ import java.util.concurrent.TimeUnit;
  * Consider it "best effort"
  */
 public class TestSwiftFileSystemConcurrency extends SwiftFileSystemBaseTest {
-  protected static final Log LOG =
-    LogFactory.getLog(TestSwiftFileSystemConcurrency.class);
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(TestSwiftFileSystemConcurrency.class);
   private Exception thread1Ex, thread2Ex;
   public static final String TEST_RACE_CONDITION_ON_DELETE_DIR =
     "/test/testraceconditionondirdeletetest";

+ 4 - 4
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemContract.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
@@ -47,8 +47,8 @@ import java.net.URISyntaxException;
  */
 public class TestSwiftFileSystemContract
         extends FileSystemContractBaseTest {
-  private static final Log LOG =
-          LogFactory.getLog(TestSwiftFileSystemContract.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestSwiftFileSystemContract.class);
 
   /**
    * Override this if the filesystem is not case sensitive

+ 4 - 3
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftObjectPath.java

@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.fs.swift;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.swift.http.RestClientBindings;
 import org.apache.hadoop.fs.swift.http.SwiftRestClient;
@@ -36,7 +36,8 @@ import static org.junit.Assert.assertTrue;
  * Unit tests for SwiftObjectPath class.
  */
 public class TestSwiftObjectPath implements SwiftTestConstants {
-  private static final Log LOG = LogFactory.getLog(TestSwiftObjectPath.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestSwiftObjectPath.class);
 
   /**
    * What an endpoint looks like. This is derived from a (valid)

+ 5 - 5
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/http/TestSwiftRestClient.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift.http;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.swift.SwiftTestConstants;
@@ -39,8 +39,8 @@ import java.io.IOException;
 import java.net.URI;
 
 public class TestSwiftRestClient implements SwiftTestConstants {
-  private static final Log LOG =
-          LogFactory.getLog(TestSwiftRestClient.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestSwiftRestClient.class);
 
   private Configuration conf;
   private boolean runTests;
@@ -110,7 +110,7 @@ public class TestSwiftRestClient implements SwiftTestConstants {
       //expected
     }
     for (DurationStats stats: client.getOperationStatistics()) {
-      LOG.info(stats);
+      LOG.info(stats.toString());
     }
   }
 

+ 4 - 3
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/scale/TestWriteManySmallFiles.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.fs.swift.scale;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.swift.util.Duration;
@@ -29,7 +29,8 @@ import org.junit.Test;
 
 public class TestWriteManySmallFiles extends SwiftScaleTestBase {
 
-  public static final Log LOG = LogFactory.getLog(TestWriteManySmallFiles.class);
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TestWriteManySmallFiles.class);
 
   @Test(timeout = SWIFT_BULK_IO_TEST_TIMEOUT)
   public void testScaledWriteThenRead() throws Throwable {

+ 1 - 1
hadoop-tools/hadoop-sls/pom.xml

@@ -49,7 +49,7 @@
       <artifactId>hadoop-rumen</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.codahale.metrics</groupId>
+      <groupId>io.dropwizard.metrics</groupId>
       <artifactId>metrics-core</artifactId>
       <scope>compile</scope>
     </dependency>

+ 17 - 6
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.tools.rumen.LoggedTaskAttempt;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.ReservationId;
@@ -284,9 +285,12 @@ public class SLSRunner extends Configured implements Tool {
 
   private void startNM() throws YarnException, IOException {
     // nm configuration
-    int heartbeatInterval =
-        getConf().getInt(SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS,
-            SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    int heartbeatInterval = getConf().getInt(
+        SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS,
+        SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS_DEFAULT);
+    float resourceUtilizationRatio = getConf().getFloat(
+        SLSConfiguration.NM_RESOURCE_UTILIZATION_RATIO,
+        SLSConfiguration.NM_RESOURCE_UTILIZATION_RATIO_DEFAULT);
     // nm information (fetch from topology file, or from sls/rumen json file)
     Set<String> nodeSet = new HashSet<String>();
     if (nodeFile.isEmpty()) {
@@ -324,7 +328,7 @@ public class SLSRunner extends Configured implements Tool {
       // we randomize the heartbeat start time from zero to 1 interval
       NMSimulator nm = new NMSimulator();
       nm.init(hostName, nodeManagerResource, random.nextInt(heartbeatInterval),
-          heartbeatInterval, rm);
+          heartbeatInterval, rm, resourceUtilizationRatio);
       nmMap.put(nm.getNode().getNodeID(), nm);
       runner.schedule(nm);
       rackSet.add(nm.getNode().getRackName());
@@ -499,9 +503,15 @@ public class SLSRunner extends Configured implements Tool {
       }
       count = Math.max(count, 1);
 
+      ExecutionType executionType = ExecutionType.GUARANTEED;
+      if (jsonTask.containsKey(SLSConfiguration.TASK_EXECUTION_TYPE)) {
+        executionType = ExecutionType.valueOf(
+            jsonTask.get(SLSConfiguration.TASK_EXECUTION_TYPE).toString());
+      }
       for (int i = 0; i < count; i++) {
         containers.add(
-            new ContainerSimulator(res, duration, hostname, priority, type));
+            new ContainerSimulator(res, duration, hostname, priority, type,
+                executionType));
       }
     }
 
@@ -670,7 +680,8 @@ public class SLSRunner extends Configured implements Tool {
             .newInstance((int) task.getMemory(), (int) task.getVcores());
         containerList.add(
             new ContainerSimulator(containerResource, containerLifeTime,
-                hostname, task.getPriority(), task.getType()));
+                hostname, task.getPriority(), task.getType(),
+                task.getExecutionType()));
       }
 
 

+ 13 - 8
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java

@@ -47,6 +47,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ReservationId;
@@ -259,13 +261,16 @@ public abstract class AMSimulator extends TaskRunner.Task {
     }
   }
 
-  protected ResourceRequest createResourceRequest(
-          Resource resource, String host, int priority, int numContainers) {
+  protected ResourceRequest createResourceRequest(Resource resource,
+      ExecutionType executionType, String host, int priority, int
+      numContainers) {
     ResourceRequest request = recordFactory
         .newRecordInstance(ResourceRequest.class);
     request.setCapability(resource);
     request.setResourceName(host);
     request.setNumContainers(numContainers);
+    request.setExecutionTypeRequest(
+        ExecutionTypeRequest.newInstance(executionType));
     Priority prio = recordFactory.newRecordInstance(Priority.class);
     prio.setPriority(priority);
     request.setPriority(prio);
@@ -400,8 +405,8 @@ public abstract class AMSimulator extends TaskRunner.Task {
           rackLocalRequestMap.get(rackname).setNumContainers(
               rackLocalRequestMap.get(rackname).getNumContainers() + 1);
         } else {
-          ResourceRequest request =
-              createResourceRequest(cs.getResource(), rackname, priority, 1);
+          ResourceRequest request = createResourceRequest(cs.getResource(),
+              cs.getExecutionType(), rackname, priority, 1);
           rackLocalRequestMap.put(rackname, request);
         }
         // check node local
@@ -410,15 +415,15 @@ public abstract class AMSimulator extends TaskRunner.Task {
           nodeLocalRequestMap.get(hostname).setNumContainers(
               nodeLocalRequestMap.get(hostname).getNumContainers() + 1);
         } else {
-          ResourceRequest request =
-              createResourceRequest(cs.getResource(), hostname, priority, 1);
+          ResourceRequest request = createResourceRequest(cs.getResource(),
+              cs.getExecutionType(), hostname, priority, 1);
           nodeLocalRequestMap.put(hostname, request);
         }
       }
       // any
       if (anyRequest == null) {
-        anyRequest = createResourceRequest(
-                cs.getResource(), ResourceRequest.ANY, priority, 1);
+        anyRequest = createResourceRequest(cs.getResource(),
+            cs.getExecutionType(), ResourceRequest.ANY, priority, 1);
       } else {
         anyRequest.setNumContainers(anyRequest.getNumContainers() + 1);
       }

+ 6 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/conf/SLSConfiguration.java

@@ -62,6 +62,8 @@ public class SLSConfiguration {
   public static final String AM_PREFIX = PREFIX + "am.";
   public static final String AM_HEARTBEAT_INTERVAL_MS = AM_PREFIX
                                                   + "heartbeat.interval.ms";
+  public static final String NM_RESOURCE_UTILIZATION_RATIO = NM_PREFIX
+      + "resource.utilization.ratio";
   public static final int AM_HEARTBEAT_INTERVAL_MS_DEFAULT = 1000;
   public static final String AM_TYPE = AM_PREFIX + "type";
   public static final String AM_TYPE_PREFIX = AM_TYPE + ".";
@@ -74,6 +76,8 @@ public class SLSConfiguration {
       "container.vcores";
   public static final int AM_CONTAINER_VCORES_DEFAULT = 1;
 
+  public static final float NM_RESOURCE_UTILIZATION_RATIO_DEFAULT = -1F;
+
   // container
   public static final String CONTAINER_PREFIX = PREFIX + "container.";
   public static final String CONTAINER_MEMORY_MB = CONTAINER_PREFIX
@@ -116,5 +120,7 @@ public class SLSConfiguration {
   public static final String TASK_DURATION_MS = TASK_CONTAINER + DURATION_MS;
   public static final String TASK_PRIORITY = TASK_CONTAINER + "priority";
   public static final String TASK_TYPE = TASK_CONTAINER + "type";
+  public static final String TASK_EXECUTION_TYPE = TASK_CONTAINER
+      + "execution.type";
 
 }

+ 22 - 6
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
@@ -72,18 +73,20 @@ public class NMSimulator extends TaskRunner.Task {
   private ResourceManager rm;
   // heart beat response id
   private int responseId = 0;
+  private float resourceUtilizationRatio;
   private final static Logger LOG = LoggerFactory.getLogger(NMSimulator.class);
   
-  public void init(String nodeIdStr, Resource nodeResource,
-          int dispatchTime, int heartBeatInterval, ResourceManager rm)
-          throws IOException, YarnException {
+  public void init(String nodeIdStr, Resource nodeResource, int dispatchTime,
+      int heartBeatInterval, ResourceManager pRm,
+      float pResourceUtilizationRatio)
+      throws IOException, YarnException {
     super.init(dispatchTime, dispatchTime + 1000000L * heartBeatInterval,
-            heartBeatInterval);
+        heartBeatInterval);
     // create resource
     String rackHostName[] = SLSUtils.getRackHostName(nodeIdStr);
     this.node = NodeInfo.newNodeInfo(rackHostName[0], rackHostName[1],
         Resources.clone(nodeResource));
-    this.rm = rm;
+    this.rm = pRm;
     // init data structures
     completedContainerList =
             Collections.synchronizedList(new ArrayList<ContainerId>());
@@ -100,9 +103,10 @@ public class NMSimulator extends TaskRunner.Task {
     req.setNodeId(node.getNodeID());
     req.setResource(node.getTotalCapability());
     req.setHttpPort(80);
-    RegisterNodeManagerResponse response = rm.getResourceTrackerService()
+    RegisterNodeManagerResponse response = this.rm.getResourceTrackerService()
             .registerNodeManager(req);
     masterKey = response.getNMTokenMasterKey();
+    this.resourceUtilizationRatio = pResourceUtilizationRatio;
   }
 
   @Override
@@ -133,6 +137,18 @@ public class NMSimulator extends TaskRunner.Task {
     ns.setKeepAliveApplications(new ArrayList<ApplicationId>());
     ns.setResponseId(responseId++);
     ns.setNodeHealthStatus(NodeHealthStatus.newInstance(true, "", 0));
+
+    //set node & containers utilization
+    if (resourceUtilizationRatio > 0 && resourceUtilizationRatio <=1) {
+      int pMemUsed = Math.round(node.getTotalCapability().getMemorySize()
+          * resourceUtilizationRatio);
+      float cpuUsed = node.getTotalCapability().getVirtualCores()
+          * resourceUtilizationRatio;
+      ResourceUtilization resourceUtilization = ResourceUtilization.newInstance(
+          pMemUsed, pMemUsed, cpuUsed);
+      ns.setContainersUtilization(resourceUtilization);
+      ns.setNodeUtilization(resourceUtilization);
+    }
     beatRequest.setNodeStatus(ns);
     NodeHeartbeatResponse beatResponse =
         rm.getResourceTrackerService().nodeHeartbeat(beatRequest);

+ 19 - 2
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ContainerSimulator.java

@@ -24,6 +24,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Resource;
 
 @Private
@@ -43,21 +44,33 @@ public class ContainerSimulator implements Delayed {
   private int priority;
   // type 
   private String type;
+  // execution type
+  private ExecutionType executionType = ExecutionType.GUARANTEED;
 
   /**
-   * invoked when AM schedules containers to allocate
+   * invoked when AM schedules containers to allocate.
    */
   public ContainerSimulator(Resource resource, long lifeTime,
       String hostname, int priority, String type) {
+    this(resource, lifeTime, hostname, priority, type,
+        ExecutionType.GUARANTEED);
+  }
+
+  /**
+   * invoked when AM schedules containers to allocate.
+   */
+  public ContainerSimulator(Resource resource, long lifeTime,
+      String hostname, int priority, String type, ExecutionType executionType) {
     this.resource = resource;
     this.lifeTime = lifeTime;
     this.hostname = hostname;
     this.priority = priority;
     this.type = type;
+    this.executionType = executionType;
   }
 
   /**
-   * invoke when NM schedules containers to run
+   * invoke when NM schedules containers to run.
    */
   public ContainerSimulator(ContainerId id, Resource resource, long endTime,
       long lifeTime) {
@@ -114,4 +127,8 @@ public class ContainerSimulator implements Delayed {
   public void setPriority(int p) {
     priority = p;
   }
+
+  public ExecutionType getExecutionType() {
+    return executionType;
+  }
 }

+ 3 - 1
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java

@@ -327,7 +327,9 @@ public class SLSCapacityScheduler extends CapacityScheduler implements
   @Override
   public void serviceStop() throws Exception {
     try {
-      schedulerMetrics.tearDown();
+      if (metricsON) {
+        schedulerMetrics.tearDown();
+      }
     } catch (Exception e) {
       e.printStackTrace();
     }

+ 3 - 1
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java

@@ -311,7 +311,9 @@ public class SLSFairScheduler extends FairScheduler
   @Override
   public void serviceStop() throws Exception {
     try {
-      schedulerMetrics.tearDown();
+      if (metricsON) {
+        schedulerMetrics.tearDown();
+      }
     } catch (Exception e) {
       e.printStackTrace();
     }

+ 14 - 4
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.tools.rumen.ReduceTaskAttemptInfo;
 import org.apache.hadoop.tools.rumen.TaskAttemptInfo;
 import org.apache.hadoop.tools.rumen.TaskInfo;
 import org.apache.hadoop.tools.rumen.Pre21JobHistoryConstants.Values;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.sls.appmaster.MRAMSimulator;
 
 import java.util.ArrayList;
@@ -92,14 +93,16 @@ public class SynthJob implements JobStory {
     private long maxMemory;
     private long maxVcores;
     private int priority;
+    private ExecutionType executionType;
 
     private SynthTask(String type, long time, long maxMemory, long maxVcores,
-        int priority){
+        int priority, ExecutionType executionType){
       this.type = type;
       this.time = time;
       this.maxMemory = maxMemory;
       this.maxVcores = maxVcores;
       this.priority = priority;
+      this.executionType = executionType;
     }
 
     public String getType(){
@@ -122,11 +125,15 @@ public class SynthJob implements JobStory {
       return priority;
     }
 
+    public ExecutionType getExecutionType() {
+      return executionType;
+    }
+
     @Override
     public String toString(){
       return String.format("[task]\ttype: %1$-10s\ttime: %2$3s\tmemory: "
-              + "%3$4s\tvcores: %4$2s%n", getType(), getTime(), getMemory(),
-          getVcores());
+              + "%3$4s\tvcores: %4$2s\texecution_type: %5$-10s%n", getType(),
+          getTime(), getMemory(), getVcores(), getExecutionType().toString());
     }
   }
 
@@ -181,6 +188,9 @@ public class SynthJob implements JobStory {
       long vcores = task.max_vcores.getLong();
       vcores = vcores < MIN_VCORES ? MIN_VCORES  : vcores;
       int priority = task.priority;
+      ExecutionType executionType = task.executionType == null
+          ? ExecutionType.GUARANTEED
+          : ExecutionType.valueOf(task.executionType);
 
       // Save task information by type
       taskByType.put(taskType, new ArrayList<>());
@@ -192,7 +202,7 @@ public class SynthJob implements JobStory {
         long time = task.time.getLong();
         totalSlotTime += time;
         SynthTask t = new SynthTask(taskType, time, memory, vcores,
-            priority);
+            priority, executionType);
         tasks.add(t);
         taskByType.get(taskType).add(t);
       }

+ 13 - 1
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.tools.rumen.JobStory;
 import org.apache.hadoop.tools.rumen.JobStoryProducer;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.sls.appmaster.MRAMSimulator;
 import org.codehaus.jackson.annotate.JsonCreator;
@@ -199,6 +200,7 @@ public class SynthTraceJobProducer implements JobStoryProducer {
         map.max_vcores = new Sample((double) jobDef.map_max_vcores_avg,
             jobDef.map_max_vcores_stddev);
         map.priority = DEFAULT_MAPPER_PRIORITY;
+        map.executionType = jobDef.map_execution_type;
 
         jobDef.tasks.add(map);
         TaskDefinition reduce = new TaskDefinition();
@@ -210,6 +212,7 @@ public class SynthTraceJobProducer implements JobStoryProducer {
         reduce.max_vcores = new Sample((double) jobDef.reduce_max_vcores_avg,
             jobDef.reduce_max_vcores_stddev);
         reduce.priority = DEFAULT_REDUCER_PRIORITY;
+        reduce.executionType = jobDef.reduce_execution_type;
 
         jobDef.tasks.add(reduce);
       } catch (JsonMappingException e) {
@@ -425,6 +428,12 @@ public class SynthTraceJobProducer implements JobStoryProducer {
     @JsonProperty("reduce_max_vcores_stddev")
     double reduce_max_vcores_stddev;
 
+    //container execution type
+    @JsonProperty("map_execution_type")
+    String map_execution_type = ExecutionType.GUARANTEED.name();
+    @JsonProperty("reduce_execution_type")
+    String reduce_execution_type = ExecutionType.GUARANTEED.name();
+
     public void init(JDKRandomGenerator rand){
       deadline_factor.init(rand);
       duration.init(rand);
@@ -464,12 +473,15 @@ public class SynthTraceJobProducer implements JobStoryProducer {
     Sample max_vcores;
     @JsonProperty("priority")
     int priority;
+    @JsonProperty("execution_type")
+    String executionType = ExecutionType.GUARANTEED.name();
 
     @Override
     public String toString(){
       return "\nTaskDefinition " + type
           + " Count[" + count + "] Time[" + time + "] Memory[" + max_memory
-          + "] Vcores[" + max_vcores + "] Priority[" + priority + "]";
+          + "] Vcores[" + max_vcores + "] Priority[" + priority
+          + "] ExecutionType[" + executionType + "]";
     }
   }
 

+ 3 - 0
hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.yarn.sls;
 
 import org.apache.commons.math3.random.JDKRandomGenerator;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.codehaus.jackson.map.JsonMappingException;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.apache.hadoop.conf.Configuration;
@@ -31,6 +32,7 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.util.Arrays;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import static org.codehaus.jackson.JsonParser.Feature.INTERN_FIELD_NAMES;
@@ -254,6 +256,7 @@ public class TestSynthJobGeneration {
       assertTrue(t.getTime() > 0);
       assertTrue(t.getMemory() > 0);
       assertTrue(t.getVcores() > 0);
+      assertEquals(ExecutionType.GUARANTEED, t.getExecutionType());
     }
   }
 }

+ 10 - 1
hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/nodemanager/TestNMSimulator.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.yarn.sls.nodemanager;
 
+import com.google.common.base.Supplier;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -76,7 +78,7 @@ public class TestNMSimulator {
     // Register one node
     NMSimulator node1 = new NMSimulator();
     node1.init("/rack1/node1", Resources.createResource(GB * 10, 10), 0, 1000,
-        rm);
+        rm, -1f);
     node1.middleStep();
 
     int numClusterNodes = rm.getResourceScheduler().getNumClusterNodes();
@@ -89,6 +91,13 @@ public class TestNMSimulator {
       numClusterNodes = rm.getResourceScheduler().getNumClusterNodes();
     }
 
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override public Boolean get() {
+        return rm.getResourceScheduler().getRootQueueMetrics()
+            .getAvailableMB() > 0;
+      }
+    }, 500, 10000);
+
     Assert.assertEquals(1, rm.getResourceScheduler().getNumClusterNodes());
     Assert.assertEquals(GB * 10,
         rm.getResourceScheduler().getRootQueueMetrics().getAvailableMB());

+ 10 - 5
hadoop-tools/hadoop-sls/src/test/resources/inputsls.json

@@ -11,21 +11,24 @@
       "container.start.ms": 6664,
       "container.end.ms": 23707,
       "container.priority": 20,
-      "container.type": "map"
+      "container.type": "map",
+      "container.execution.type": "GUARANTEED"
     },
     {
       "container.host": "/default-rack/node3",
       "container.start.ms": 6665,
       "container.end.ms": 21593,
       "container.priority": 20,
-      "container.type": "map"
+      "container.type": "map",
+      "container.execution.type": "GUARANTEED"
     },
     {
       "container.host": "/default-rack/node2",
       "container.start.ms": 68770,
       "container.end.ms": 86613,
       "container.priority": 20,
-      "container.type": "map"
+      "container.type": "map",
+      "container.execution.type": "GUARANTEED"
     }
   ]
 }
@@ -42,14 +45,16 @@
       "container.start.ms": 111822,
       "container.end.ms": 133985,
       "container.priority": 20,
-      "container.type": "map"
+      "container.type": "map",
+      "container.execution.type": "GUARANTEED"
     },
     {
       "container.host": "/default-rack/node2",
       "container.start.ms": 111788,
       "container.end.ms": 131377,
       "container.priority": 20,
-      "container.type": "map"
+      "container.type": "map",
+      "container.execution.type": "GUARANTEED"
     }
   ]
 }

+ 2 - 0
hadoop-tools/hadoop-sls/src/test/resources/syn.json

@@ -27,8 +27,10 @@
           "rtime_stddev": 4,
           "map_max_memory_avg": 1024,
           "map_max_memory_stddev": 0.001,
+          "map_execution_type": "GUARANTEED",
           "reduce_max_memory_avg": 2048,
           "reduce_max_memory_stddev": 0.001,
+          "reduce_execution_type": "GUARANTEED",
           "map_max_vcores_avg": 1,
           "map_max_vcores_stddev": 0.001,
           "reduce_max_vcores_avg": 2,

+ 4 - 2
hadoop-tools/hadoop-sls/src/test/resources/syn_generic.json

@@ -26,7 +26,8 @@
               "count": { "val": 5, "std": 1},
               "time": {"val": 10, "std": 2},
               "max_memory": {"val": 1024},
-              "max_vcores": {"val": 1}
+              "max_vcores": {"val": 1},
+              "execution_type": "GUARANTEED"
             },
             {
               "type": "reduce",
@@ -34,7 +35,8 @@
               "count": { "val": 5, "std": 1},
               "time": {"val": 20, "std": 4},
               "max_memory": {"val": 2048},
-              "max_vcores": {"val": 2}
+              "max_vcores": {"val": 2},
+              "execution_type": "GUARANTEED"
             }
           ]
         }

+ 2 - 1
hadoop-tools/hadoop-sls/src/test/resources/syn_stream.json

@@ -26,7 +26,8 @@
               "count": { "val": 2},
               "time": {"val": 60000},
               "max_memory": {"val": 4096},
-              "max_vcores": {"val": 4}
+              "max_vcores": {"val": 4},
+              "execution_type": "GUARANTEED"
             }
           ]
         }

+ 140 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/resource/PlacementConstraint.java

@@ -23,6 +23,8 @@ import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.Iterator;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -45,6 +47,11 @@ public class PlacementConstraint {
     this.constraintExpr = constraintExpr;
   }
 
+  @Override
+  public String toString() {
+    return this.constraintExpr.toString();
+  }
+
   /**
    * Get the constraint expression of the placement constraint.
    *
@@ -225,6 +232,42 @@ public class PlacementConstraint {
       return result;
     }
 
+    @Override
+    public String toString() {
+      int max = getMaxCardinality();
+      int min = getMinCardinality();
+      List<String> targetExprList = getTargetExpressions().stream()
+          .map(TargetExpression::toString).collect(Collectors.toList());
+      List<String> targetConstraints = new ArrayList<>();
+      for (String targetExpr : targetExprList) {
+        if (min == 0 && max == 0) {
+          // anti-affinity
+          targetConstraints.add(new StringBuilder()
+              .append("notin").append(",")
+              .append(getScope()).append(",")
+              .append(targetExpr)
+              .toString());
+        } else if (min == 1 && max == Integer.MAX_VALUE) {
+          // affinity
+          targetConstraints.add(new StringBuilder()
+              .append("in").append(",")
+              .append(getScope()).append(",")
+              .append(targetExpr)
+              .toString());
+        } else {
+          // cardinality
+          targetConstraints.add(new StringBuilder()
+              .append("cardinality").append(",")
+              .append(getScope()).append(",")
+              .append(targetExpr).append(",")
+              .append(min).append(",")
+              .append(max)
+              .toString());
+        }
+      }
+      return String.join(":", targetConstraints);
+    }
+
     @Override
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
@@ -325,6 +368,23 @@ public class PlacementConstraint {
           : that.targetValues == null;
     }
 
+    @Override
+    public String toString() {
+      StringBuffer sb = new StringBuffer();
+      if (TargetType.ALLOCATION_TAG == this.targetType) {
+        // following by a comma separated tags
+        sb.append(String.join(",", getTargetValues()));
+      } else if (TargetType.NODE_ATTRIBUTE == this.targetType) {
+        // following by a comma separated key value pairs
+        if (this.getTargetValues() != null) {
+          String attributeName = this.getTargetKey();
+          String attributeValues = String.join(":", this.getTargetValues());
+          sb.append(attributeName + "=[" + attributeValues + "]");
+        }
+      }
+      return sb.toString();
+    }
+
     @Override
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
@@ -345,7 +405,16 @@ public class PlacementConstraint {
      * TargetOperator enum helps to specify type.
      */
     enum TargetOperator {
-      IN, NOT_IN
+      IN("in"), NOT_IN("notin");
+
+      private String operator;
+      TargetOperator(String op) {
+        this.operator = op;
+      }
+
+      String getOperator() {
+        return this.operator;
+      }
     }
 
     private TargetOperator op;
@@ -414,6 +483,17 @@ public class PlacementConstraint {
       return result;
     }
 
+    @Override
+    public String toString() {
+      List<String> targetExprs = getTargetExpressions().stream().map(
+          targetExpression -> new StringBuilder()
+              .append(op.getOperator()).append(",")
+              .append(scope).append(",")
+              .append(targetExpression.toString())
+              .toString()).collect(Collectors.toList());
+      return String.join(":", targetExprs);
+    }
+
     @Override
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
@@ -517,6 +597,17 @@ public class PlacementConstraint {
           + (allocationTags != null ? allocationTags.hashCode() : 0);
       return result;
     }
+
+    @Override
+    public String toString() {
+      StringBuffer sb = new StringBuffer();
+      sb.append("cardinality").append(",").append(getScope()).append(",");
+      for (String tag : getAllocationTags()) {
+        sb.append(tag).append(",");
+      }
+      sb.append(minCardinality).append(",").append(maxCardinality);
+      return sb.toString();
+    }
   }
 
   /**
@@ -580,6 +671,22 @@ public class PlacementConstraint {
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
     }
+
+    @Override
+    public String toString() {
+      StringBuffer sb = new StringBuffer();
+      sb.append("and(");
+      Iterator<AbstractConstraint> it = getChildren().iterator();
+      while (it.hasNext()) {
+        AbstractConstraint child = it.next();
+        sb.append(child.toString());
+        if (it.hasNext()) {
+          sb.append(":");
+        }
+      }
+      sb.append(")");
+      return sb.toString();
+    }
   }
 
   /**
@@ -606,6 +713,22 @@ public class PlacementConstraint {
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
     }
+
+    @Override
+    public String toString() {
+      StringBuffer sb = new StringBuffer();
+      sb.append("or(");
+      Iterator<AbstractConstraint> it = getChildren().iterator();
+      while (it.hasNext()) {
+        AbstractConstraint child = it.next();
+        sb.append(child.toString());
+        if (it.hasNext()) {
+          sb.append(":");
+        }
+      }
+      sb.append(")");
+      return sb.toString();
+    }
   }
 
   /**
@@ -636,6 +759,22 @@ public class PlacementConstraint {
     public <T> T accept(Visitor<T> visitor) {
       return visitor.visit(this);
     }
+
+    @Override
+    public String toString() {
+      StringBuffer sb = new StringBuffer();
+      sb.append("DelayedOr(");
+      Iterator<TimedPlacementConstraint> it = getChildren().iterator();
+      while (it.hasNext()) {
+        TimedPlacementConstraint child = it.next();
+        sb.append(child.toString());
+        if (it.hasNext()) {
+          sb.append(",");
+        }
+      }
+      sb.append(")");
+      return sb.toString();
+    }
   }
 
   /**

+ 88 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintParser.java

@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.Set;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint.Or;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.SingleConstraint;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression;
 import org.apache.hadoop.yarn.util.constraint.PlacementConstraintParseException;
@@ -51,42 +52,50 @@ public class TestPlacementConstraintParser {
   @Test
   public void testTargetExpressionParser()
       throws PlacementConstraintParseException {
+    String expressionStr;
     ConstraintParser parser;
     AbstractConstraint constraint;
     SingleConstraint single;
 
     // Anti-affinity with single target tag
-    // NOTIN,NDOE,foo
-    parser = new TargetConstraintParser("NOTIN, NODE, foo");
+    // NOTIN,NODE,foo
+    expressionStr = "NOTIN, NODE, foo";
+    parser = new TargetConstraintParser(expressionStr);
     constraint = parser.parse();
     Assert.assertTrue(constraint instanceof SingleConstraint);
     single = (SingleConstraint) constraint;
     Assert.assertEquals("node", single.getScope());
     Assert.assertEquals(0, single.getMinCardinality());
     Assert.assertEquals(0, single.getMaxCardinality());
+    verifyConstraintToString(expressionStr, constraint);
 
     // lower cases is also valid
-    parser = new TargetConstraintParser("notin, node, foo");
+    expressionStr = "notin, node, foo";
+    parser = new TargetConstraintParser(expressionStr);
     constraint = parser.parse();
     Assert.assertTrue(constraint instanceof SingleConstraint);
     single = (SingleConstraint) constraint;
     Assert.assertEquals("node", single.getScope());
     Assert.assertEquals(0, single.getMinCardinality());
     Assert.assertEquals(0, single.getMaxCardinality());
+    verifyConstraintToString(expressionStr, constraint);
 
     // Affinity with single target tag
     // IN,NODE,foo
-    parser = new TargetConstraintParser("IN, NODE, foo");
+    expressionStr = "IN, NODE, foo";
+    parser = new TargetConstraintParser(expressionStr);
     constraint = parser.parse();
     Assert.assertTrue(constraint instanceof SingleConstraint);
     single = (SingleConstraint) constraint;
     Assert.assertEquals("node", single.getScope());
     Assert.assertEquals(1, single.getMinCardinality());
     Assert.assertEquals(Integer.MAX_VALUE, single.getMaxCardinality());
+    verifyConstraintToString(expressionStr, constraint);
 
     // Anti-affinity with multiple target tags
     // NOTIN,NDOE,foo,bar,exp
-    parser = new TargetConstraintParser("NOTIN, NODE, foo, bar, exp");
+    expressionStr = "NOTIN, NODE, foo, bar, exp";
+    parser = new TargetConstraintParser(expressionStr);
     constraint = parser.parse();
     Assert.assertTrue(constraint instanceof SingleConstraint);
     single = (SingleConstraint) constraint;
@@ -98,6 +107,7 @@ public class TestPlacementConstraintParser {
         single.getTargetExpressions().iterator().next();
     Assert.assertEquals("ALLOCATION_TAG", exp.getTargetType().toString());
     Assert.assertEquals(3, exp.getTargetValues().size());
+    verifyConstraintToString(expressionStr, constraint);
 
     // Invalid OP
     parser = new TargetConstraintParser("XYZ, NODE, foo");
@@ -112,12 +122,14 @@ public class TestPlacementConstraintParser {
   @Test
   public void testCardinalityConstraintParser()
       throws PlacementConstraintParseException {
+    String expressionExpr;
     ConstraintParser parser;
     AbstractConstraint constraint;
     SingleConstraint single;
 
     // cardinality,NODE,foo,0,1
-    parser = new CardinalityConstraintParser("cardinality, NODE, foo, 0, 1");
+    expressionExpr = "cardinality, NODE, foo, 0, 1";
+    parser = new CardinalityConstraintParser(expressionExpr);
     constraint = parser.parse();
     Assert.assertTrue(constraint instanceof SingleConstraint);
     single = (SingleConstraint) constraint;
@@ -130,10 +142,11 @@ public class TestPlacementConstraintParser {
     Assert.assertEquals("ALLOCATION_TAG", exp.getTargetType().toString());
     Assert.assertEquals(1, exp.getTargetValues().size());
     Assert.assertEquals("foo", exp.getTargetValues().iterator().next());
+    verifyConstraintToString(expressionExpr, constraint);
 
     // cardinality,NODE,foo,bar,moo,0,1
-    parser = new CardinalityConstraintParser(
-        "cardinality,RACK,foo,bar,moo,0,1");
+    expressionExpr = "cardinality,RACK,foo,bar,moo,0,1";
+    parser = new CardinalityConstraintParser(expressionExpr);
     constraint = parser.parse();
     Assert.assertTrue(constraint instanceof SingleConstraint);
     single = (SingleConstraint) constraint;
@@ -147,6 +160,7 @@ public class TestPlacementConstraintParser {
     Set<String> expectedTags = Sets.newHashSet("foo", "bar", "moo");
     Assert.assertTrue(Sets.difference(expectedTags, exp.getTargetValues())
         .isEmpty());
+    verifyConstraintToString(expressionExpr, constraint);
 
     // Invalid scope string
     try {
@@ -174,25 +188,29 @@ public class TestPlacementConstraintParser {
   @Test
   public void testAndConstraintParser()
       throws PlacementConstraintParseException {
+    String expressionExpr;
     ConstraintParser parser;
     AbstractConstraint constraint;
     And and;
 
-    parser = new ConjunctionConstraintParser(
-        "AND(NOTIN,NODE,foo:NOTIN,NODE,bar)");
+    expressionExpr = "AND(NOTIN,NODE,foo:NOTIN,NODE,bar)";
+    parser = new ConjunctionConstraintParser(expressionExpr);
     constraint = parser.parse();
     Assert.assertTrue(constraint instanceof And);
     and = (And) constraint;
     Assert.assertEquals(2, and.getChildren().size());
+    verifyConstraintToString(expressionExpr, constraint);
 
-    parser = new ConjunctionConstraintParser(
-        "AND(NOTIN,NODE,foo:cardinality,NODE,foo,0,1)");
+    expressionExpr = "AND(NOTIN,NODE,foo:cardinality,NODE,foo,0,1)";
+    parser = new ConjunctionConstraintParser(expressionExpr);
     constraint = parser.parse();
     Assert.assertTrue(constraint instanceof And);
     Assert.assertEquals(2, and.getChildren().size());
+    verifyConstraintToString(expressionExpr, constraint);
 
-    parser = new ConjunctionConstraintParser(
-        "AND(NOTIN,NODE,foo:AND(NOTIN,NODE,foo:cardinality,NODE,foo,0,1))");
+    expressionExpr =
+        "AND(NOTIN,NODE,foo:AND(NOTIN,NODE,foo:cardinality,NODE,foo,0,1))";
+    parser = new ConjunctionConstraintParser(expressionExpr);
     constraint = parser.parse();
     Assert.assertTrue(constraint instanceof And);
     and = (And) constraint;
@@ -200,6 +218,43 @@ public class TestPlacementConstraintParser {
     Assert.assertTrue(and.getChildren().get(1) instanceof And);
     and = (And) and.getChildren().get(1);
     Assert.assertEquals(2, and.getChildren().size());
+    verifyConstraintToString(expressionExpr, constraint);
+  }
+
+  @Test
+  public void testOrConstraintParser()
+      throws PlacementConstraintParseException {
+    String expressionExpr;
+    ConstraintParser parser;
+    AbstractConstraint constraint;
+    Or or;
+
+    expressionExpr = "OR(NOTIN,NODE,foo:NOTIN,NODE,bar)";
+    parser = new ConjunctionConstraintParser(expressionExpr);
+    constraint = parser.parse();
+    Assert.assertTrue(constraint instanceof Or);
+    or = (Or) constraint;
+    Assert.assertEquals(2, or.getChildren().size());
+    verifyConstraintToString(expressionExpr, constraint);
+
+    expressionExpr = "OR(NOTIN,NODE,foo:cardinality,NODE,foo,0,1)";
+    parser = new ConjunctionConstraintParser(expressionExpr);
+    constraint = parser.parse();
+    Assert.assertTrue(constraint instanceof Or);
+    Assert.assertEquals(2, or.getChildren().size());
+    verifyConstraintToString(expressionExpr, constraint);
+
+    expressionExpr =
+        "OR(NOTIN,NODE,foo:OR(NOTIN,NODE,foo:cardinality,NODE,foo,0,1))";
+    parser = new ConjunctionConstraintParser(expressionExpr);
+    constraint = parser.parse();
+    Assert.assertTrue(constraint instanceof Or);
+    or = (Or) constraint;
+    Assert.assertTrue(or.getChildren().get(0) instanceof SingleConstraint);
+    Assert.assertTrue(or.getChildren().get(1) instanceof Or);
+    or = (Or) or.getChildren().get(1);
+    Assert.assertEquals(2, or.getChildren().size());
+    verifyConstraintToString(expressionExpr, constraint);
   }
 
   @Test
@@ -369,4 +424,23 @@ public class TestPlacementConstraintParser {
     expectedPc2 = targetNotIn("node", allocationTag("foo")).build();
     Assert.assertEquals(expectedPc2, actualPc2);
   }
+
+  // We verify the toString result by parsing it again
+  // instead of raw string comparing. This is because internally
+  // we are not storing tags strictly to its original order, so
+  // the toString result might have different ordering with the
+  // input expression.
+  private void verifyConstraintToString(String inputExpr,
+      AbstractConstraint constraint) {
+    String constrainExpr = constraint.toString();
+    System.out.println("Input:    " + inputExpr
+        .toLowerCase().replaceAll(" ", ""));
+    System.out.println("ToString: " + constrainExpr);
+    try {
+      PlacementConstraintParser.parseExpression(constrainExpr);
+    } catch (PlacementConstraintParseException e) {
+      Assert.fail("The parser is unable to parse the expression: "
+          + constrainExpr + ", caused by: " + e.getMessage());
+    }
+  }
 }

+ 12 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java

@@ -59,7 +59,7 @@ import java.util.Map;
 
 import static org.apache.hadoop.yarn.service.api.records.ServiceState.ACCEPTED;
 import static org.apache.hadoop.yarn.service.conf.RestApiConstants.*;
-import static org.apache.hadoop.yarn.service.exceptions.LauncherExitCodes.EXIT_SUCCESS;
+import static org.apache.hadoop.yarn.service.exceptions.LauncherExitCodes.*;
 
 /**
  * The rest API endpoints for users to manage services on YARN.
@@ -240,7 +240,7 @@ public class ApiServer {
   private Response stopService(String appName, boolean destroy,
       final UserGroupInformation ugi) throws IOException,
       InterruptedException, YarnException, FileNotFoundException {
-    ugi.doAs(new PrivilegedExceptionAction<Integer>() {
+    int result = ugi.doAs(new PrivilegedExceptionAction<Integer>() {
       @Override
       public Integer run() throws IOException, YarnException,
           FileNotFoundException {
@@ -249,11 +249,12 @@ public class ApiServer {
         sc.init(YARN_CONFIG);
         sc.start();
         result = sc.actionStop(appName, destroy);
+        if (result == EXIT_SUCCESS) {
+          LOG.info("Successfully stopped service {}", appName);
+        }
         if (destroy) {
           result = sc.actionDestroy(appName);
           LOG.info("Successfully deleted service {}", appName);
-        } else {
-          LOG.info("Successfully stopped service {}", appName);
         }
         sc.close();
         return result;
@@ -264,8 +265,13 @@ public class ApiServer {
       serviceStatus.setDiagnostics("Successfully destroyed service " +
           appName);
     } else {
-      serviceStatus.setDiagnostics("Successfully stopped service " +
-          appName);
+      if (result == EXIT_COMMAND_ARGUMENT_ERROR) {
+        serviceStatus
+            .setDiagnostics("Service " + appName + " is already stopped");
+        return formatResponse(Status.BAD_REQUEST, serviceStatus);
+      } else {
+        serviceStatus.setDiagnostics("Successfully stopped service " + appName);
+      }
     }
     return formatResponse(Status.OK, serviceStatus);
   }

+ 6 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/ServiceClientTest.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.client.ServiceClient;
 import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
+import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
 
 /**
  * A mock version of ServiceClient - This class is design
@@ -49,9 +50,9 @@ public class ServiceClientTest extends ServiceClient {
   }
 
   @Override
-  public ApplicationId actionCreate(Service service) {
-    String serviceName = service.getName();
-    ServiceApiUtil.validateNameFormat(serviceName, getConfig());
+  public ApplicationId actionCreate(Service service) throws IOException {
+    ServiceApiUtil.validateAndResolveService(service,
+        new SliderFileSystem(conf), getConfig());
     return ApplicationId.newInstance(System.currentTimeMillis(), 1);
   }
 
@@ -88,6 +89,8 @@ public class ServiceClientTest extends ServiceClient {
     }
     if (serviceName.equals("jenkins")) {
       return EXIT_SUCCESS;
+    } else if (serviceName.equals("jenkins-second-stop")) {
+      return EXIT_COMMAND_ARGUMENT_ERROR;
     } else {
       throw new ApplicationNotFoundException("");
     }

+ 63 - 55
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/TestApiServer.java

@@ -69,15 +69,14 @@ public class TestApiServer {
         this.apiServer.getClass().isAnnotationPresent(Path.class));
     final Path path = this.apiServer.getClass()
         .getAnnotation(Path.class);
-    assertEquals("The path has /v1 annotation", path.value(),
-        "/v1");
+    assertEquals("The path has /v1 annotation", "/v1", path.value());
   }
 
   @Test
   public void testGetVersion() {
     final Response actual = apiServer.getVersion();
-    assertEquals("Version number is", actual.getStatus(),
-        Response.ok().build().getStatus());
+    assertEquals("Version number is", Response.ok().build().getStatus(),
+        actual.getStatus());
   }
 
   @Test
@@ -85,8 +84,9 @@ public class TestApiServer {
     Service service = new Service();
     // Test for invalid argument
     final Response actual = apiServer.createService(request, service);
-    assertEquals("Create service is ", actual.getStatus(),
-        Response.status(Status.BAD_REQUEST).build().getStatus());
+    assertEquals("Create service is ",
+        Response.status(Status.BAD_REQUEST).build().getStatus(),
+        actual.getStatus());
   }
 
   @Test
@@ -109,52 +109,55 @@ public class TestApiServer {
     components.add(c);
     service.setComponents(components);
     final Response actual = apiServer.createService(request, service);
-    assertEquals("Create service is ", actual.getStatus(),
-        Response.status(Status.ACCEPTED).build().getStatus());
+    assertEquals("Create service is ",
+        Response.status(Status.ACCEPTED).build().getStatus(),
+        actual.getStatus());
   }
 
   @Test
   public void testBadGetService() {
     final Response actual = apiServer.getService(request, "no-jenkins");
-    assertEquals("Get service is ", actual.getStatus(),
-        Response.status(Status.NOT_FOUND).build().getStatus());
+    assertEquals("Get service is ",
+        Response.status(Status.NOT_FOUND).build().getStatus(),
+        actual.getStatus());
   }
 
   @Test
   public void testBadGetService2() {
     final Response actual = apiServer.getService(request, null);
-    assertEquals("Get service is ", actual.getStatus(),
-        Response.status(Status.NOT_FOUND)
-            .build().getStatus());
+    assertEquals("Get service is ",
+        Response.status(Status.NOT_FOUND).build().getStatus(),
+        actual.getStatus());
   }
 
   @Test
   public void testGoodGetService() {
     final Response actual = apiServer.getService(request, "jenkins");
-    assertEquals("Get service is ", actual.getStatus(),
-        Response.status(Status.OK).build().getStatus());
+    assertEquals("Get service is ",
+        Response.status(Status.OK).build().getStatus(), actual.getStatus());
   }
 
   @Test
   public void testBadDeleteService() {
     final Response actual = apiServer.deleteService(request, "no-jenkins");
-    assertEquals("Delete service is ", actual.getStatus(),
-        Response.status(Status.BAD_REQUEST).build().getStatus());
+    assertEquals("Delete service is ",
+        Response.status(Status.BAD_REQUEST).build().getStatus(),
+        actual.getStatus());
   }
 
   @Test
   public void testBadDeleteService2() {
     final Response actual = apiServer.deleteService(request, null);
-    assertEquals("Delete service is ", actual.getStatus(),
-        Response.status(Status.BAD_REQUEST)
-            .build().getStatus());
+    assertEquals("Delete service is ",
+        Response.status(Status.BAD_REQUEST).build().getStatus(),
+        actual.getStatus());
   }
 
   @Test
   public void testGoodDeleteService() {
     final Response actual = apiServer.deleteService(request, "jenkins");
-    assertEquals("Delete service is ", actual.getStatus(),
-        Response.status(Status.OK).build().getStatus());
+    assertEquals("Delete service is ",
+        Response.status(Status.OK).build().getStatus(), actual.getStatus());
   }
 
   @Test
@@ -179,8 +182,8 @@ public class TestApiServer {
     service.setComponents(components);
     final Response actual = apiServer.updateService(request, "jenkins",
         service);
-    assertEquals("update service is ", actual.getStatus(),
-        Response.status(Status.OK).build().getStatus());
+    assertEquals("update service is ",
+        Response.status(Status.OK).build().getStatus(), actual.getStatus());
   }
 
   @Test
@@ -206,8 +209,9 @@ public class TestApiServer {
     System.out.println("before stop");
     final Response actual = apiServer.updateService(request, "no-jenkins",
         service);
-    assertEquals("flex service is ", actual.getStatus(),
-        Response.status(Status.BAD_REQUEST).build().getStatus());
+    assertEquals("flex service is ",
+        Response.status(Status.BAD_REQUEST).build().getStatus(),
+        actual.getStatus());
   }
 
   @Test
@@ -232,8 +236,8 @@ public class TestApiServer {
     service.setComponents(components);
     final Response actual = apiServer.updateService(request, "jenkins",
         service);
-    assertEquals("flex service is ", actual.getStatus(),
-        Response.status(Status.OK).build().getStatus());
+    assertEquals("flex service is ",
+        Response.status(Status.OK).build().getStatus(), actual.getStatus());
   }
 
   @Test
@@ -258,9 +262,9 @@ public class TestApiServer {
     service.setComponents(components);
     final Response actual = apiServer.updateService(request, "no-jenkins",
         service);
-    assertEquals("start service is ", actual.getStatus(),
-        Response.status(Status.BAD_REQUEST).build()
-            .getStatus());
+    assertEquals("start service is ",
+        Response.status(Status.BAD_REQUEST).build().getStatus(),
+        actual.getStatus());
   }
 
   @Test
@@ -285,8 +289,8 @@ public class TestApiServer {
     service.setComponents(components);
     final Response actual = apiServer.updateService(request, "jenkins",
         service);
-    assertEquals("start service is ", actual.getStatus(),
-        Response.status(Status.OK).build().getStatus());
+    assertEquals("start service is ",
+        Response.status(Status.OK).build().getStatus(), actual.getStatus());
   }
 
   @Test
@@ -312,35 +316,39 @@ public class TestApiServer {
     System.out.println("before stop");
     final Response actual = apiServer.updateService(request, "no-jenkins",
         service);
-    assertEquals("stop service is ", actual.getStatus(),
-        Response.status(Status.BAD_REQUEST).build().getStatus());
+    assertEquals("stop service is ",
+        Response.status(Status.BAD_REQUEST).build().getStatus(),
+        actual.getStatus());
   }
 
   @Test
   public void testGoodStopServices() {
     Service service = new Service();
-    service.setState(ServiceState.STARTED);
+    service.setState(ServiceState.STOPPED);
     service.setName("jenkins");
-    Artifact artifact = new Artifact();
-    artifact.setType(TypeEnum.DOCKER);
-    artifact.setId("jenkins:latest");
-    Resource resource = new Resource();
-    resource.setCpus(1);
-    resource.setMemory("2048");
-    List<Component> components = new ArrayList<Component>();
-    Component c = new Component();
-    c.setName("jenkins");
-    c.setNumberOfContainers(-1L);
-    c.setArtifact(artifact);
-    c.setLaunchCommand("");
-    c.setResource(resource);
-    components.add(c);
-    service.setComponents(components);
     System.out.println("before stop");
     final Response actual = apiServer.updateService(request, "jenkins",
         service);
-    assertEquals("stop service is ", actual.getStatus(),
-        Response.status(Status.OK).build().getStatus());
+    assertEquals("stop service is ",
+        Response.status(Status.OK).build().getStatus(), actual.getStatus());
+  }
+
+  @Test
+  public void testBadSecondStopServices() throws Exception {
+    Service service = new Service();
+    service.setState(ServiceState.STOPPED);
+    service.setName("jenkins-second-stop");
+    // simulates stop on an already stopped service
+    System.out.println("before second stop");
+    final Response actual = apiServer.updateService(request,
+        "jenkins-second-stop", service);
+    assertEquals("stop service should have thrown 400 Bad Request: ",
+        Response.status(Status.BAD_REQUEST).build().getStatus(),
+        actual.getStatus());
+    ServiceStatus serviceStatus = (ServiceStatus) actual.getEntity();
+    assertEquals("Stop service should have failed with service already stopped",
+        "Service jenkins-second-stop is already stopped",
+        serviceStatus.getDiagnostics());
   }
 
   @Test
@@ -366,9 +374,9 @@ public class TestApiServer {
     System.out.println("before stop");
     final Response actual = apiServer.updateService(request, "no-jenkins",
         service);
-    assertEquals("update service is ", actual.getStatus(),
+    assertEquals("update service is ",
         Response.status(Status.BAD_REQUEST)
-            .build().getStatus());
+            .build().getStatus(), actual.getStatus());
   }
 
   @Test

+ 21 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -96,6 +97,7 @@ import java.util.concurrent.TimeUnit;
 
 import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
 import static org.apache.hadoop.registry.client.api.RegistryConstants.*;
+import static org.apache.hadoop.yarn.api.records.ContainerExitStatus.KILLED_AFTER_APP_COMPLETION;
 import static org.apache.hadoop.yarn.service.api.ServiceApiConstants.*;
 import static org.apache.hadoop.yarn.service.component.ComponentEventType.*;
 
@@ -253,6 +255,13 @@ public class ServiceScheduler extends CompositeService {
   public void serviceStop() throws Exception {
     LOG.info("Stopping service scheduler");
 
+    // Mark component-instances/containers as STOPPED
+    if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
+      for (ContainerId containerId : getLiveInstances().keySet()) {
+        serviceTimelinePublisher.componentInstanceFinished(containerId,
+            KILLED_AFTER_APP_COMPLETION, diagnostics.toString());
+      }
+    }
     if (executorService != null) {
       executorService.shutdownNow();
     }
@@ -352,17 +361,22 @@ public class ServiceScheduler extends CompositeService {
         amRMClient.releaseAssignedContainer(container.getId());
       }
     }
-
+    ApplicationId appId = ApplicationId.fromString(app.getId());
     existingRecords.forEach((encodedContainerId, record) -> {
       String componentName = record.get(YarnRegistryAttributes.YARN_COMPONENT);
       if (componentName != null) {
         Component component = componentsByName.get(componentName);
-        ComponentInstance compInstance = component.getComponentInstance(
-            record.description);
-        ContainerId containerId = ContainerId.fromString(record.get(
-            YarnRegistryAttributes.YARN_ID));
-        unRecoveredInstances.put(containerId, compInstance);
-        component.removePendingInstance(compInstance);
+        if (component != null) {
+          ComponentInstance compInstance = component.getComponentInstance(
+              record.description);
+          ContainerId containerId = ContainerId.fromString(record.get(
+              YarnRegistryAttributes.YARN_ID));
+          if (containerId.getApplicationAttemptId().getApplicationId()
+              .equals(appId)) {
+            unRecoveredInstances.put(containerId, compInstance);
+            component.removePendingInstance(compInstance);
+          }
+        }
       }
     });
 

+ 2 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java

@@ -210,7 +210,6 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
   public ApplicationId actionCreate(Service service)
       throws IOException, YarnException {
     String serviceName = service.getName();
-    ServiceApiUtil.validateNameFormat(serviceName, getConfig());
     ServiceApiUtil.validateAndResolveService(service, fs, getConfig());
     verifyNoLiveAppInRM(serviceName, "create");
     Path appDir = checkAppNotExistOnHdfs(service);
@@ -363,7 +362,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     if (terminatedStates.contains(report.getYarnApplicationState())) {
       LOG.info("Service {} is already in a terminated state {}", serviceName,
           report.getYarnApplicationState());
-      return EXIT_SUCCESS;
+      return EXIT_COMMAND_ARGUMENT_ERROR;
     }
     if (preRunningStates.contains(report.getYarnApplicationState())) {
       String msg = serviceName + " is at " + report.getYarnApplicationState()
@@ -779,7 +778,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     Service service = ServiceApiUtil.loadService(fs, serviceName);
     ServiceApiUtil.validateAndResolveService(service, fs, getConfig());
     // see if it is actually running and bail out;
-    verifyNoLiveAppInRM(serviceName, "thaw");
+    verifyNoLiveAppInRM(serviceName, "start");
     ApplicationId appId = submitApp(service);
     service.setId(appId.toString());
     // write app definition on to hdfs

+ 9 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java

@@ -196,7 +196,15 @@ public class MockServiceAM extends ServiceMaster {
 
           @Override
           public RegisterApplicationMasterResponse registerApplicationMaster(
-              String appHostName, int appHostPort, String appTrackingUrl) {
+              String appHostName, int appHostPort, String appTrackingUrl,
+              Map placementConstraintsMap) throws YarnException, IOException {
+            return this.registerApplicationMaster(appHostName, appHostPort,
+                appTrackingUrl);
+          }
+
+          @Override
+            public RegisterApplicationMasterResponse registerApplicationMaster(
+                String appHostName, int appHostPort, String appTrackingUrl) {
             RegisterApplicationMasterResponse response = mock(
                 RegisterApplicationMasterResponse.class);
             when(response.getResourceTypes()).thenReturn(

+ 43 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java

@@ -210,6 +210,49 @@ public class TestServiceAM extends ServiceTestUtils{
             .getState());
   }
 
+  // Test to verify that the AM doesn't wait for containers of a different app
+  // even though it corresponds to the same service.
+  @Test(timeout = 200000)
+  public void testContainersFromDifferentApp()
+      throws Exception {
+    ApplicationId applicationId = ApplicationId.newInstance(
+        System.currentTimeMillis(), 1);
+    Service exampleApp = new Service();
+    exampleApp.setId(applicationId.toString());
+    exampleApp.setName("testContainersFromDifferentApp");
+    String comp1Name = "comp1";
+    String comp1InstName = "comp1-0";
+
+    org.apache.hadoop.yarn.service.api.records.Component compA =
+        createComponent(comp1Name, 1, "sleep");
+    exampleApp.addComponent(compA);
+
+    MockServiceAM am = new MockServiceAM(exampleApp);
+    ContainerId containerId = am.createContainerId(1);
+    // saves the container in the registry
+    am.feedRegistryComponent(containerId, comp1Name, comp1InstName);
+
+    ApplicationId changedAppId = ApplicationId.newInstance(
+        System.currentTimeMillis(), 2);
+    exampleApp.setId(changedAppId.toString());
+    am.init(conf);
+    am.start();
+    // 1 pending instance since the container in registry belongs to a different
+    // app.
+    Assert.assertEquals(1,
+        am.getComponent(comp1Name).getPendingInstances().size());
+
+    am.feedContainerToComp(exampleApp, 1, comp1Name);
+    GenericTestUtils.waitFor(() -> am.getCompInstance(comp1Name, comp1InstName)
+        .getContainerStatus() != null, 2000, 200000);
+
+    Assert.assertEquals("container state",
+        org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+        am.getCompInstance(comp1Name, comp1InstName).getContainerStatus()
+            .getState());
+    am.stop();
+  }
+
   @Test
   public void testScheduleWithMultipleResourceTypes()
       throws TimeoutException, InterruptedException, IOException {

+ 7 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/resource/TestPlacementConstraintTransformations.java

@@ -65,6 +65,10 @@ public class TestPlacementConstraintTransformations {
 
     SingleConstraint single = (SingleConstraint) sConstraintExpr;
     TargetConstraint target = (TargetConstraint) tConstraintExpr;
+
+    // Make sure the expression string is consistent
+    // before and after transforming
+    Assert.assertEquals(single.toString(), target.toString());
     Assert.assertEquals(single.getScope(), target.getScope());
     Assert.assertEquals(TargetOperator.IN, target.getOp());
     Assert.assertEquals(single.getTargetExpressions(),
@@ -101,6 +105,9 @@ public class TestPlacementConstraintTransformations {
     Assert.assertTrue(sConstraintExpr instanceof SingleConstraint);
 
     SingleConstraint single = (SingleConstraint) sConstraintExpr;
+    // Make sure the consistent expression string is consistent
+    // before and after transforming
+    Assert.assertEquals(single.toString(), cardinality.toString());
     Assert.assertEquals(cardinality.getScope(), single.getScope());
     Assert.assertEquals(cardinality.getMinCardinality(),
         single.getMinCardinality());

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml

@@ -122,7 +122,7 @@
     </dependency>
 
     <dependency>
-      <groupId>com.codahale.metrics</groupId>
+      <groupId>io.dropwizard.metrics</groupId>
       <artifactId>metrics-core</artifactId>
     </dependency>
     <!--

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceSet.java

@@ -136,6 +136,10 @@ public class ResourceSet {
   }
 
   public void resourceLocalizationFailed(LocalResourceRequest request) {
+    // Skip null request when localization failed for running container
+    if (request == null) {
+      return;
+    }
     pendingResources.remove(request);
     resourcesFailedToBeLocalized.add(request);
   }

이 변경점에서 너무 많은 파일들이 변경되어 몇몇 파일들은 표시되지 않았습니다.