Browse Source

Merge branch 'trunk' into HDFS-7240

Anu Engineer 7 years ago
parent
commit
a5dfae69f5
100 changed files with 2110 additions and 1326 deletions
  1. 5 21
      hadoop-common-project/hadoop-common/pom.xml
  2. 3 0
      hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
  3. 11 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
  4. 3 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationFilterInitializer.java
  5. 0 124
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationWithProxyUserFilter.java
  6. 11 0
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  7. 22 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
  8. 0 494
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpengo.java
  9. 2 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java
  10. 10 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java
  11. 0 79
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationWithProxyUserFilter.java
  12. 46 22
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
  13. 5 1
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
  14. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java
  15. 29 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  16. 6 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
  17. 8 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
  18. 4 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  19. 34 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
  20. 14 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  21. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
  22. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
  23. 5 21
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  24. 14 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
  25. 33 26
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
  26. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java
  27. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPoolId.java
  28. 20 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
  29. 13 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaManager.java
  30. 12 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
  31. 18 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipState.java
  32. 26 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java
  33. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/RouterState.java
  34. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
  35. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  36. 4 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  37. 0 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  38. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
  39. 50 81
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
  40. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
  41. 4 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotDiffInfo.java
  42. 7 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotDiffListingInfo.java
  43. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
  44. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  45. 5 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
  46. 93 38
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java
  47. 23 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  48. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  49. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
  50. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
  51. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
  52. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md
  53. 139 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md
  54. 92 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
  55. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
  56. 10 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithDefaultECPolicy.java
  57. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithRandomECPolicy.java
  58. 29 19
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java
  59. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFileWithRandomECPolicy.java
  60. 8 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java
  61. 23 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
  62. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
  63. 17 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java
  64. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReconstruction.java
  65. 25 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
  66. 69 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java
  67. 157 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestConnectionManager.java
  68. 34 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
  69. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuota.java
  70. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuotaManager.java
  71. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterSafemode.java
  72. 43 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestMountTable.java
  73. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
  74. 43 36
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
  75. 50 79
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  76. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java
  77. 38 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
  78. 183 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
  79. 25 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
  80. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
  81. 3 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
  82. 0 16
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  83. 100 0
      hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/paralleltests/CreateDirsMojo.java
  84. 1 1
      hadoop-project/src/site/markdown/index.md.vm
  85. 5 21
      hadoop-tools/hadoop-aws/pom.xml
  86. 11 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
  87. 25 13
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
  88. 1 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
  89. 2 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
  90. 39 2
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
  91. 47 6
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
  92. 0 5
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
  93. 38 15
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
  94. 1 1
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
  95. 27 21
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/ThrottledInputStream.java
  96. 30 8
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
  97. 27 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  98. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
  99. 8 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md
  100. 7 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml

+ 5 - 21
hadoop-common-project/hadoop-common/pom.xml

@@ -979,30 +979,13 @@
       <build>
         <plugins>
           <plugin>
-            <artifactId>maven-antrun-plugin</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-maven-plugins</artifactId>
             <executions>
               <execution>
-                <id>create-parallel-tests-dirs</id>
-                <phase>test-compile</phase>
-                <configuration>
-                  <target>
-                    <script language="javascript"><![CDATA[
-                      var baseDirs = [
-                          "${test.build.data}",
-                          "${test.build.dir}",
-                          "${hadoop.tmp.dir}" ];
-                      for (var i in baseDirs) {
-                        for (var j = 1; j <= ${testsThreadCount}; ++j) {
-                          var mkdir = project.createTask("mkdir");
-                          mkdir.setDir(new java.io.File(baseDirs[i], j));
-                          mkdir.perform();
-                        }
-                      }
-                    ]]></script>
-                  </target>
-                </configuration>
+                <id>parallel-tests-createdir</id>
                 <goals>
-                  <goal>run</goal>
+                  <goal>parallel-tests-createdir</goal>
                 </goals>
               </execution>
             </executions>
@@ -1015,6 +998,7 @@
               <reuseForks>false</reuseForks>
               <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
               <systemPropertyVariables>
+                <testsThreadCount>${testsThreadCount}</testsThreadCount>
                 <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
                 <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
                 <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>

+ 3 - 0
hadoop-common-project/hadoop-common/src/main/conf/log4j.properties

@@ -340,3 +340,6 @@ log4j.appender.FILE.layout.ConversionPattern=%d{ISO8601} [%t] %-5p \
 #log4j.appender.FSSTATEDUMP.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
 #log4j.appender.FSSTATEDUMP.MaxFileSize=${hadoop.log.maxfilesize}
 #log4j.appender.FSSTATEDUMP.MaxBackupIndex=${hadoop.log.maxbackupindex}
+
+# Log levels of third-party libraries
+log4j.logger.org.apache.commons.beanutils=WARN

+ 11 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java

@@ -134,6 +134,14 @@ public final class HttpServer2 implements FilterContainer {
       "hadoop.http.socket.backlog.size";
   public static final int HTTP_SOCKET_BACKLOG_SIZE_DEFAULT = 128;
   public static final String HTTP_MAX_THREADS_KEY = "hadoop.http.max.threads";
+  public static final String HTTP_ACCEPTOR_COUNT_KEY =
+      "hadoop.http.acceptor.count";
+  // -1 to use default behavior of setting count based on CPU core count
+  public static final int HTTP_ACCEPTOR_COUNT_DEFAULT = -1;
+  public static final String HTTP_SELECTOR_COUNT_KEY =
+      "hadoop.http.selector.count";
+  // -1 to use default behavior of setting count based on CPU core count
+  public static final int HTTP_SELECTOR_COUNT_DEFAULT = -1;
   public static final String HTTP_TEMP_DIR_KEY = "hadoop.http.temp.dir";
 
   public static final String FILTER_INITIALIZER_PROPERTY
@@ -465,7 +473,9 @@ public final class HttpServer2 implements FilterContainer {
 
     private ServerConnector createHttpChannelConnector(
         Server server, HttpConfiguration httpConfig) {
-      ServerConnector conn = new ServerConnector(server);
+      ServerConnector conn = new ServerConnector(server,
+          conf.getInt(HTTP_ACCEPTOR_COUNT_KEY, HTTP_ACCEPTOR_COUNT_DEFAULT),
+          conf.getInt(HTTP_SELECTOR_COUNT_KEY, HTTP_SELECTOR_COUNT_DEFAULT));
       ConnectionFactory connFactory = new HttpConnectionFactory(httpConfig);
       conn.addConnectionFactory(connFactory);
       configureChannelConnector(conn);

+ 3 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationFilterInitializer.java

@@ -29,9 +29,8 @@ import java.util.HashMap;
 import java.util.Map;
 
 /**
- * Initializes {@link AuthenticationWithProxyUserFilter}
- * which provides support for Kerberos HTTP SPNEGO authentication
- * and proxy user authentication.
+ * Initializes hadoop-auth AuthenticationFilter which provides support for
+ * Kerberos HTTP SPNEGO authentication.
  * <p/>
  * It enables anonymous access, simple/speudo and Kerberos HTTP SPNEGO
  * authentication  for Hadoop JobTracker, NameNode, DataNodes and
@@ -59,10 +58,8 @@ public class AuthenticationFilterInitializer extends FilterInitializer {
   public void initFilter(FilterContainer container, Configuration conf) {
     Map<String, String> filterConfig = getFilterConfigMap(conf, PREFIX);
 
-    // extend AuthenticationFilter's feature to
-    // support proxy user operation.
     container.addFilter("authentication",
-                        AuthenticationWithProxyUserFilter.class.getName(),
+                        AuthenticationFilter.class.getName(),
                         filterConfig);
   }
 

+ 0 - 124
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationWithProxyUserFilter.java

@@ -1,124 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <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.security;
-
-import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
-import org.apache.hadoop.security.authorize.AuthorizationException;
-import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.apache.http.NameValuePair;
-import org.apache.http.client.utils.URLEncodedUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.servlet.FilterChain;
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletRequestWrapper;
-import javax.servlet.http.HttpServletResponse;
-import java.io.IOException;
-import java.nio.charset.Charset;
-import java.util.List;
-
-/**
- * Extend the function of {@link AuthenticationFilter} to
- * support authorizing proxy user. If the query string
- * contains doAs parameter, then check the proxy user,
- * otherwise do the next filter.
- */
-public class AuthenticationWithProxyUserFilter extends AuthenticationFilter {
-
-  public static final Logger LOG =
-      LoggerFactory.getLogger(AuthenticationWithProxyUserFilter.class);
-
-  /**
-   * Constant used in URL's query string to perform a proxy user request, the
-   * value of the <code>DO_AS</code> parameter is the user the request will be
-   * done on behalf of.
-   */
-  private static final String DO_AS = "doAs";
-
-  private static final Charset UTF8_CHARSET = Charset.forName("UTF-8");
-
-
-  /**
-   * This method provide the ability to do pre/post tasks
-   * in filter chain. Override this method to authorize
-   * proxy user between AuthenticationFilter and next filter.
-   * @param filterChain the filter chain object.
-   * @param request the request object.
-   * @param response the response object.
-   *
-   * @throws IOException
-   * @throws ServletException
-   */
-  @Override
-  protected void doFilter(FilterChain filterChain, HttpServletRequest request,
-      HttpServletResponse response) throws IOException, ServletException {
-
-    final String proxyUser = getDoAs(request);
-    if (proxyUser != null) {
-
-      // Change the remote user after proxy user is authorized.
-      final HttpServletRequest finalReq = request;
-      request = new HttpServletRequestWrapper(finalReq) {
-
-        private String getRemoteOrProxyUser() throws AuthorizationException {
-          UserGroupInformation realUser =
-              UserGroupInformation.createRemoteUser(finalReq.getRemoteUser());
-          UserGroupInformation proxyUserInfo =
-              UserGroupInformation.createProxyUser(proxyUser, realUser);
-          ProxyUsers.authorize(proxyUserInfo, finalReq.getRemoteAddr());
-          return proxyUserInfo.getUserName();
-        }
-
-        @Override
-        public String getRemoteUser() {
-          try {
-            return getRemoteOrProxyUser();
-          } catch (AuthorizationException ex) {
-            LOG.error("Unable to verify proxy user: " + ex.getMessage(), ex);
-          }
-          return null;
-        }
-      };
-
-    }
-    filterChain.doFilter(request, response);
-  }
-
-  /**
-   * Get proxy user from query string.
-   * @param request the request object
-   * @return proxy user
-   */
-  public static String getDoAs(HttpServletRequest request) {
-    String queryString = request.getQueryString();
-    if (queryString == null) {
-      return null;
-    }
-    List<NameValuePair> list = URLEncodedUtils.parse(queryString, UTF8_CHARSET);
-    if (list != null) {
-      for (NameValuePair nv : list) {
-        if (DO_AS.equalsIgnoreCase(nv.getName())) {
-          return nv.getValue();
-        }
-      }
-    }
-    return null;
-  }
-}

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

@@ -1547,6 +1547,17 @@
   </description>
 </property>
 
+<property>
+  <name>fs.s3a.etag.checksum.enabled</name>
+  <value>false</value>
+  <description>
+    Should calls to getFileChecksum() return the etag value of the remote
+    object.
+    WARNING: if enabled, distcp operations between HDFS and S3 will fail unless
+    -skipcrccheck is set.
+  </description>
+</property>
+
 <!-- Azure file system properties -->
 <property>
   <name>fs.wasb.impl</name>

+ 22 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java

@@ -147,7 +147,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
 
   @BeforeClass public static void setup() throws Exception {
     Configuration conf = new Configuration();
-    conf.setInt(HttpServer2.HTTP_MAX_THREADS_KEY, 10);
+    conf.setInt(HttpServer2.HTTP_MAX_THREADS_KEY, MAX_THREADS);
     server = createTestServer(conf);
     server.addServlet("echo", "/echo", EchoServlet.class);
     server.addServlet("echomap", "/echomap", EchoMapServlet.class);
@@ -195,6 +195,27 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     ready.await();
     start.countDown();
   }
+
+  /**
+   * Test that the number of acceptors and selectors can be configured by
+   * trying to configure more of them than would be allowed based on the
+   * maximum thread count.
+   */
+  @Test
+  public void testAcceptorSelectorConfigurability() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(HttpServer2.HTTP_MAX_THREADS_KEY, MAX_THREADS);
+    conf.setInt(HttpServer2.HTTP_ACCEPTOR_COUNT_KEY, MAX_THREADS - 2);
+    conf.setInt(HttpServer2.HTTP_SELECTOR_COUNT_KEY, MAX_THREADS - 2);
+    HttpServer2 badserver = createTestServer(conf);
+    try {
+      badserver.start();
+      // Should not succeed
+      fail();
+    } catch (IOException ioe) {
+      assertTrue(ioe.getCause() instanceof IllegalStateException);
+    }
+  }
   
   @Test public void testEcho() throws Exception {
     assertEquals("a:b\nc:d\n", 

+ 0 - 494
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpengo.java

@@ -1,494 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <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.http;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.minikdc.MiniKdc;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AuthenticationFilterInitializer;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
-import org.apache.hadoop.security.authentication.KerberosTestUtils;
-import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
-import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
-import org.apache.hadoop.security.authentication.server.AuthenticationToken;
-import org.apache.hadoop.security.authentication.util.Signer;
-import org.apache.hadoop.security.authentication.util.SignerSecretProvider;
-import org.apache.hadoop.security.authentication.util.StringSignerSecretProviderCreator;
-import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.ietf.jgss.GSSException;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.Assert;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileWriter;
-import java.io.Writer;
-import java.net.HttpURLConnection;
-import java.net.URI;
-import java.net.URL;
-import java.security.AccessController;
-import java.security.PrivilegedExceptionAction;
-import java.util.HashSet;
-import java.util.Properties;
-import java.util.Set;
-import javax.security.auth.Subject;
-import javax.servlet.ServletContext;
-
-import static org.junit.Assert.assertTrue;
-
-/**
- * This class is tested for http server with SPENGO authentication.
- */
-public class TestHttpServerWithSpengo {
-
-  static final Logger LOG =
-      LoggerFactory.getLogger(TestHttpServerWithSpengo.class);
-
-  private static final String SECRET_STR = "secret";
-  private static final String HTTP_USER = "HTTP";
-  private static final String PREFIX = "hadoop.http.authentication.";
-  private static final long TIMEOUT = 20000;
-
-  private static File httpSpnegoKeytabFile = new File(
-      KerberosTestUtils.getKeytabFile());
-  private static String httpSpnegoPrincipal =
-      KerberosTestUtils.getServerPrincipal();
-  private static String realm = KerberosTestUtils.getRealm();
-
-  private static File testRootDir = new File("target",
-      TestHttpServerWithSpengo.class.getName() + "-root");
-  private static MiniKdc testMiniKDC;
-  private static File secretFile = new File(testRootDir, SECRET_STR);
-
-  private static UserGroupInformation authUgi;
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    try {
-      testMiniKDC = new MiniKdc(MiniKdc.createConf(), testRootDir);
-      testMiniKDC.start();
-      testMiniKDC.createPrincipal(
-          httpSpnegoKeytabFile, HTTP_USER + "/localhost", "keytab-user");
-    } catch (Exception e) {
-      assertTrue("Couldn't setup MiniKDC", false);
-    }
-
-    System.setProperty("sun.security.krb5.debug", "true");
-    Configuration conf = new Configuration();
-    SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
-    UserGroupInformation.setConfiguration(conf);
-    authUgi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
-        "keytab-user", httpSpnegoKeytabFile.toString());
-    Writer w = new FileWriter(secretFile);
-    w.write("secret");
-    w.close();
-  }
-
-  @AfterClass
-  public static void tearDown() {
-    if (testMiniKDC != null) {
-      testMiniKDC.stop();
-    }
-  }
-
-  /**
-   * groupA
-   *  - userA
-   * groupB
-   *  - userA, userB
-   * groupC
-   *  - userC
-   * SPNEGO filter has been enabled.
-   * userA has the privilege to impersonate users in groupB.
-   * userA has admin access to all default servlets, but userB
-   * and userC don't have. So "/logs" can only be accessed by userA.
-   * @throws Exception
-   */
-  @Test
-  public void testAuthenticationWithProxyUser() throws Exception {
-
-    Configuration spengoConf = getSpengoConf(new Configuration());
-
-    //setup logs dir
-    System.setProperty("hadoop.log.dir", testRootDir.getAbsolutePath());
-
-    // Setup user group
-    UserGroupInformation.createUserForTesting("userA",
-        new String[]{"groupA", "groupB"});
-    UserGroupInformation.createUserForTesting("userB",
-        new String[]{"groupB"});
-    UserGroupInformation.createUserForTesting("userC",
-        new String[]{"groupC"});
-
-    // Make userA impersonate users in groupB
-    spengoConf.set("hadoop.proxyuser.userA.hosts", "*");
-    spengoConf.set("hadoop.proxyuser.userA.groups", "groupB");
-    ProxyUsers.refreshSuperUserGroupsConfiguration(spengoConf);
-
-    HttpServer2 httpServer = null;
-    try {
-      // Create http server to test.
-      httpServer = getCommonBuilder()
-          .setConf(spengoConf)
-          .setACL(new AccessControlList("userA groupA"))
-          .build();
-      httpServer.start();
-
-      // Get signer to encrypt token
-      Signer signer = getSignerToEncrypt();
-
-      // setup auth token for userA
-      AuthenticatedURL.Token token = getEncryptedAuthToken(signer, "userA");
-
-      String serverURL = "http://" +
-          NetUtils.getHostPortString(httpServer.getConnectorAddress(0)) + "/";
-
-      // The default authenticator is kerberos.
-      AuthenticatedURL authUrl = new AuthenticatedURL();
-
-      // userA impersonates userB, it's allowed.
-      for (String servlet :
-          new String[]{"stacks", "jmx", "conf"}) {
-        HttpURLConnection conn = authUrl
-            .openConnection(new URL(serverURL + servlet + "?doAs=userB"),
-                token);
-        Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
-      }
-
-      // userA cannot impersonate userC, but for /stacks, /jmx and /conf,
-      // they doesn't require users to authorize by default, so they
-      // can be accessed.
-      for (String servlet :
-          new String[]{"stacks", "jmx", "conf"}){
-        HttpURLConnection conn = authUrl
-            .openConnection(new URL(serverURL + servlet + "?doAs=userC"),
-                token);
-        Assert.assertEquals(HttpURLConnection.HTTP_OK,
-            conn.getResponseCode());
-      }
-
-      // "/logs" and "/logLevel" require admin authorization,
-      // only userA has the access.
-      for (String servlet :
-          new String[]{"logLevel", "logs"}) {
-        HttpURLConnection conn = authUrl
-            .openConnection(new URL(serverURL + servlet + "?doAs=userC"),
-                token);
-        Assert.assertEquals(HttpURLConnection.HTTP_FORBIDDEN,
-            conn.getResponseCode());
-      }
-
-      // "/logs" and "/logLevel" require admin authorization,
-      // only userA has the access.
-      for (String servlet :
-          new String[]{"logLevel", "logs"}) {
-        HttpURLConnection conn = authUrl
-            .openConnection(new URL(serverURL + servlet), token);
-        Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
-      }
-
-      // Setup token for userB
-      token = getEncryptedAuthToken(signer, "userB");
-
-      // userB cannot access these servlets.
-      for (String servlet :
-          new String[]{"logLevel", "logs"}) {
-        HttpURLConnection conn = authUrl
-            .openConnection(new URL(serverURL + servlet), token);
-        Assert.assertEquals(HttpURLConnection.HTTP_FORBIDDEN,
-            conn.getResponseCode());
-      }
-
-    } finally {
-      if (httpServer != null) {
-        httpServer.stop();
-      }
-    }
-  }
-
-  @Test
-  public void testSessionCookie() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set(HttpServer2.FILTER_INITIALIZER_PROPERTY,
-        AuthenticationFilterInitializer.class.getName());
-    conf.set(PREFIX + "type", "kerberos");
-    conf.setBoolean(PREFIX + "simple.anonymous.allowed", false);
-    conf.set(PREFIX + "signer.secret.provider",
-        TestSignerSecretProvider.class.getName());
-
-    conf.set(PREFIX + "kerberos.keytab",
-        httpSpnegoKeytabFile.getAbsolutePath());
-    conf.set(PREFIX + "kerberos.principal", httpSpnegoPrincipal);
-    conf.set(PREFIX + "cookie.domain", realm);
-    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
-        true);
-
-    //setup logs dir
-    System.setProperty("hadoop.log.dir", testRootDir.getAbsolutePath());
-
-    HttpServer2 httpServer = null;
-    // Create http server to test.
-    httpServer = getCommonBuilder()
-        .setConf(conf)
-        .build();
-    httpServer.start();
-
-    // Get signer to encrypt token
-    final Signer signer = new Signer(new TestSignerSecretProvider());
-    final AuthenticatedURL authUrl = new AuthenticatedURL();
-
-    final URL url = new URL("http://" + NetUtils.getHostPortString(
-        httpServer.getConnectorAddress(0)) + "/conf");
-
-    // this illustrates an inconsistency with AuthenticatedURL.  the
-    // authenticator is only called when the token is not set.  if the
-    // authenticator fails then it must throw an AuthenticationException to
-    // the caller, yet the caller may see 401 for subsequent requests
-    // that require re-authentication like token expiration.
-    final UserGroupInformation simpleUgi =
-        UserGroupInformation.createRemoteUser("simple-user");
-
-    authUgi.doAs(new PrivilegedExceptionAction<Void>() {
-      @Override
-      public Void run() throws Exception {
-        TestSignerSecretProvider.rollSecret();
-        HttpURLConnection conn = null;
-        AuthenticatedURL.Token token = new AuthenticatedURL.Token();
-
-        // initial request should trigger authentication and set the token.
-        conn = authUrl.openConnection(url, token);
-        Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
-        Assert.assertTrue(token.isSet());
-        String cookie = token.toString();
-
-        // token should not change.
-        conn = authUrl.openConnection(url, token);
-        Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
-        Assert.assertTrue(token.isSet());
-        Assert.assertEquals(cookie, token.toString());
-
-        // roll secret to invalidate token.
-        TestSignerSecretProvider.rollSecret();
-        conn = authUrl.openConnection(url, token);
-        // this may or may not happen.  under normal circumstances the
-        // jdk will silently renegotiate and the client never sees a 401.
-        // however in some cases the jdk will give up doing spnego.  since
-        // the token is already set, the authenticator isn't invoked (which
-        // would do the spnego if the jdk doesn't), which causes the client
-        // to see a 401.
-        if (conn.getResponseCode() == HttpURLConnection.HTTP_UNAUTHORIZED) {
-          // if this happens, the token should be cleared which means the
-          // next request should succeed and receive a new token.
-          Assert.assertFalse(token.isSet());
-          conn = authUrl.openConnection(url, token);
-        }
-
-        // token should change.
-        Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
-        Assert.assertTrue(token.isSet());
-        Assert.assertNotEquals(cookie, token.toString());
-        cookie = token.toString();
-
-        // token should not change.
-        for (int i=0; i < 3; i++) {
-          conn = authUrl.openConnection(url, token);
-          Assert.assertEquals("attempt"+i,
-              HttpURLConnection.HTTP_OK, conn.getResponseCode());
-          Assert.assertTrue(token.isSet());
-          Assert.assertEquals(cookie, token.toString());
-        }
-
-        // blow out the kerberos creds test only auth token is used.
-        Subject s = Subject.getSubject(AccessController.getContext());
-        Set<Object> oldCreds = new HashSet<>(s.getPrivateCredentials());
-        s.getPrivateCredentials().clear();
-
-        // token should not change.
-        for (int i=0; i < 3; i++) {
-          try {
-            conn = authUrl.openConnection(url, token);
-            Assert.assertEquals("attempt"+i,
-                HttpURLConnection.HTTP_OK, conn.getResponseCode());
-          } catch (AuthenticationException ae) {
-            Assert.fail("attempt"+i+" "+ae);
-          }
-          Assert.assertTrue(token.isSet());
-          Assert.assertEquals(cookie, token.toString());
-        }
-
-        // invalidate token.  connections should fail now and token should be
-        // unset.
-        TestSignerSecretProvider.rollSecret();
-        conn = authUrl.openConnection(url, token);
-        Assert.assertEquals(
-            HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode());
-        Assert.assertFalse(token.isSet());
-        Assert.assertEquals("", token.toString());
-
-        // restore the kerberos creds, should work again.
-        s.getPrivateCredentials().addAll(oldCreds);
-        conn = authUrl.openConnection(url, token);
-        Assert.assertEquals(
-            HttpURLConnection.HTTP_OK, conn.getResponseCode());
-        Assert.assertTrue(token.isSet());
-        cookie = token.toString();
-
-        // token should not change.
-        for (int i=0; i < 3; i++) {
-          conn = authUrl.openConnection(url, token);
-          Assert.assertEquals("attempt"+i,
-              HttpURLConnection.HTTP_OK, conn.getResponseCode());
-          Assert.assertTrue(token.isSet());
-          Assert.assertEquals(cookie, token.toString());
-        }
-        return null;
-      }
-    });
-
-    simpleUgi.doAs(new PrivilegedExceptionAction<Void>() {
-      @Override
-      public Void run() throws Exception {
-        TestSignerSecretProvider.rollSecret();
-        AuthenticatedURL authUrl = new AuthenticatedURL();
-        AuthenticatedURL.Token token = new AuthenticatedURL.Token();
-        HttpURLConnection conn = null;
-
-        // initial connect with unset token will trigger authenticator which
-        // should fail since we have no creds and leave token unset.
-        try {
-          authUrl.openConnection(url, token);
-          Assert.fail("should fail with no credentials");
-        } catch (AuthenticationException ae) {
-          Assert.assertNotNull(ae.getCause());
-          Assert.assertEquals(GSSException.class,
-              ae.getCause().getCause().getClass());
-          GSSException gsse = (GSSException)ae.getCause().getCause();
-          Assert.assertEquals(GSSException.NO_CRED, gsse.getMajor());
-        } catch (Throwable t) {
-          Assert.fail("Unexpected exception" + t);
-        }
-        Assert.assertFalse(token.isSet());
-
-        // create a valid token and save its value.
-        token = getEncryptedAuthToken(signer, "valid");
-        String cookie = token.toString();
-
-        // server should accept token.  after the request the token should
-        // be set to the same value (ie. server didn't reissue cookie)
-        conn = authUrl.openConnection(url, token);
-        Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
-        Assert.assertTrue(token.isSet());
-        Assert.assertEquals(cookie, token.toString());
-
-        conn = authUrl.openConnection(url, token);
-        Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
-        Assert.assertTrue(token.isSet());
-        Assert.assertEquals(cookie, token.toString());
-
-        // change the secret to effectively invalidate the cookie.  see above
-        // regarding inconsistency.  the authenticator has no way to know the
-        // token is bad, so the client will encounter a 401 instead of
-        // AuthenticationException.
-        TestSignerSecretProvider.rollSecret();
-        conn = authUrl.openConnection(url, token);
-        Assert.assertEquals(
-            HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode());
-        Assert.assertFalse(token.isSet());
-        Assert.assertEquals("", token.toString());
-        return null;
-      }
-    });
-  }
-
-  public static class TestSignerSecretProvider extends SignerSecretProvider {
-    static int n = 0;
-    static byte[] secret;
-
-    static void rollSecret() {
-      secret = ("secret[" + (n++) + "]").getBytes();
-    }
-
-    public TestSignerSecretProvider() {
-    }
-
-    @Override
-    public void init(Properties config, ServletContext servletContext,
-            long tokenValidity) throws Exception {
-      rollSecret();
-    }
-
-    @Override
-    public byte[] getCurrentSecret() {
-      return secret;
-    }
-
-    @Override
-    public byte[][] getAllSecrets() {
-      return new byte[][]{secret};
-    }
-  }
-
-  private AuthenticatedURL.Token getEncryptedAuthToken(Signer signer,
-      String user) throws Exception {
-    AuthenticationToken token =
-        new AuthenticationToken(user, user, "kerberos");
-    token.setExpires(System.currentTimeMillis() + TIMEOUT);
-    return new AuthenticatedURL.Token(signer.sign(token.toString()));
-  }
-
-  private Signer getSignerToEncrypt() throws Exception {
-    SignerSecretProvider secretProvider =
-        StringSignerSecretProviderCreator.newStringSignerSecretProvider();
-    Properties secretProviderProps = new Properties();
-    secretProviderProps.setProperty(
-        AuthenticationFilter.SIGNATURE_SECRET, SECRET_STR);
-    secretProvider.init(secretProviderProps, null, TIMEOUT);
-    return new Signer(secretProvider);
-  }
-
-  private Configuration getSpengoConf(Configuration conf) {
-    conf = new Configuration();
-    conf.set(HttpServer2.FILTER_INITIALIZER_PROPERTY,
-        AuthenticationFilterInitializer.class.getName());
-    conf.set(PREFIX + "type", "kerberos");
-    conf.setBoolean(PREFIX + "simple.anonymous.allowed", false);
-    conf.set(PREFIX + "signature.secret.file",
-        secretFile.getAbsolutePath());
-    conf.set(PREFIX + "kerberos.keytab",
-        httpSpnegoKeytabFile.getAbsolutePath());
-    conf.set(PREFIX + "kerberos.principal", httpSpnegoPrincipal);
-    conf.set(PREFIX + "cookie.domain", realm);
-    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
-        true);
-    return conf;
-  }
-
-  private HttpServer2.Builder getCommonBuilder() throws Exception {
-    return new HttpServer2.Builder().setName("test")
-        .addEndpoint(new URI("http://localhost:0"))
-        .setFindPort(true);
-  }
-}

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java

@@ -358,7 +358,7 @@ public class TestLogLevel extends KerberosSecurityTestcase {
     } catch (SSLException e) {
       GenericTestUtils.assertExceptionContains("Error while authenticating "
           + "with endpoint", e);
-      GenericTestUtils.assertExceptionContains("Unrecognized SSL message", e
+      GenericTestUtils.assertExceptionContains("recognized SSL message", e
           .getCause());
     }
   }
@@ -379,7 +379,7 @@ public class TestLogLevel extends KerberosSecurityTestcase {
     } catch (SSLException e) {
       GenericTestUtils.assertExceptionContains("Error while authenticating "
           + "with endpoint", e);
-      GenericTestUtils.assertExceptionContains("Unrecognized SSL message", e
+      GenericTestUtils.assertExceptionContains("recognized SSL message", e
           .getCause());
     }
   }

+ 10 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.security;
 
 import static org.junit.Assert.*;
 import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.FilterContainer;
 import org.junit.Test;
@@ -26,6 +27,9 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import java.io.File;
+import java.io.FileWriter;
+import java.io.Writer;
 import java.util.Map;
 
 public class TestAuthenticationFilter {
@@ -40,7 +44,7 @@ public class TestAuthenticationFilter {
     
     FilterContainer container = Mockito.mock(FilterContainer.class);
     Mockito.doAnswer(
-        new Answer() {
+      new Answer() {
         @Override
         public Object answer(InvocationOnMock invocationOnMock)
           throws Throwable {
@@ -48,6 +52,8 @@ public class TestAuthenticationFilter {
 
           assertEquals("authentication", args[0]);
 
+          assertEquals(AuthenticationFilter.class.getName(), args[1]);
+
           Map<String, String> conf = (Map<String, String>) args[2];
           assertEquals("/", conf.get("cookie.path"));
 
@@ -62,8 +68,9 @@ public class TestAuthenticationFilter {
           assertEquals("bar", conf.get("foo"));
 
           return null;
-        }}
-        ).when(container).addFilter(Mockito.<String>anyObject(),
+        }
+      }
+    ).when(container).addFilter(Mockito.<String>anyObject(),
                                 Mockito.<String>anyObject(),
                                 Mockito.<Map<String, String>>anyObject());
 

+ 0 - 79
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationWithProxyUserFilter.java

@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.hadoop.security;
-
-import org.junit.Test;
-import static org.junit.Assert.*;
-import org.apache.hadoop.http.HttpServer2;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.http.FilterContainer;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-import java.util.Map;
-
-/**
- * This class is tested for {@link AuthenticationWithProxyUserFilter}
- * to verify configurations of this filter.
- */
-public class TestAuthenticationWithProxyUserFilter {
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testConfiguration() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set("hadoop.http.authentication.foo", "bar");
-
-    conf.set(HttpServer2.BIND_ADDRESS, "barhost");
-
-    FilterContainer container = Mockito.mock(FilterContainer.class);
-    Mockito.doAnswer(
-      new Answer() {
-        @Override
-        public Object answer(InvocationOnMock invocationOnMock)
-          throws Throwable {
-          Object[] args = invocationOnMock.getArguments();
-
-          assertEquals("authentication", args[0]);
-
-          assertEquals(
-              AuthenticationWithProxyUserFilter.class.getName(), args[1]);
-
-          Map<String, String> conf = (Map<String, String>) args[2];
-          assertEquals("/", conf.get("cookie.path"));
-
-          assertEquals("simple", conf.get("type"));
-          assertEquals("36000", conf.get("token.validity"));
-          assertNull(conf.get("cookie.domain"));
-          assertEquals("true", conf.get("simple.anonymous.allowed"));
-          assertEquals("HTTP/barhost@LOCALHOST",
-                       conf.get("kerberos.principal"));
-          assertEquals(System.getProperty("user.home") +
-                       "/hadoop.keytab", conf.get("kerberos.keytab"));
-          assertEquals("bar", conf.get("foo"));
-
-          return null;
-        }
-      }
-    ).when(container).addFilter(Mockito.<String>anyObject(),
-                                Mockito.<String>anyObject(),
-                                Mockito.<Map<String, String>>anyObject());
-
-    new AuthenticationFilterInitializer().initFilter(container, conf);
-  }
-
-}

+ 46 - 22
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java

@@ -286,7 +286,7 @@ public abstract class GenericTestUtils {
   public static void assertExists(File f) {
     Assert.assertTrue("File " + f + " should exist", f.exists());
   }
-    
+
   /**
    * List all of the files in 'dir' that match the regex 'pattern'.
    * Then check that this list is identical to 'expectedMatches'.
@@ -294,7 +294,7 @@ public abstract class GenericTestUtils {
    */
   public static void assertGlobEquals(File dir, String pattern,
       String ... expectedMatches) throws IOException {
-    
+
     Set<String> found = Sets.newTreeSet();
     for (File f : FileUtil.listFiles(dir)) {
       if (f.getName().matches(pattern)) {
@@ -349,7 +349,7 @@ public abstract class GenericTestUtils {
               StringUtils.stringifyException(t)),
           t);
     }
-  }  
+  }
 
   /**
    * Wait for the specified test to return true. The test will be performed
@@ -499,18 +499,18 @@ public abstract class GenericTestUtils {
    */
   public static class DelayAnswer implements Answer<Object> {
     private final Log LOG;
-    
+
     private final CountDownLatch fireLatch = new CountDownLatch(1);
     private final CountDownLatch waitLatch = new CountDownLatch(1);
     private final CountDownLatch resultLatch = new CountDownLatch(1);
-    
+
     private final AtomicInteger fireCounter = new AtomicInteger(0);
     private final AtomicInteger resultCounter = new AtomicInteger(0);
-    
+
     // Result fields set after proceed() is called.
     private volatile Throwable thrown;
     private volatile Object returnValue;
-    
+
     public DelayAnswer(Log log) {
       this.LOG = log;
     }
@@ -521,7 +521,7 @@ public abstract class GenericTestUtils {
     public void waitForCall() throws InterruptedException {
       fireLatch.await();
     }
-  
+
     /**
      * Tell the method to proceed.
      * This should only be called after waitForCall()
@@ -529,7 +529,7 @@ public abstract class GenericTestUtils {
     public void proceed() {
       waitLatch.countDown();
     }
-  
+
     @Override
     public Object answer(InvocationOnMock invocation) throws Throwable {
       LOG.info("DelayAnswer firing fireLatch");
@@ -558,7 +558,7 @@ public abstract class GenericTestUtils {
         resultLatch.countDown();
       }
     }
-    
+
     /**
      * After calling proceed(), this will wait until the call has
      * completed and a result has been returned to the caller.
@@ -566,7 +566,7 @@ public abstract class GenericTestUtils {
     public void waitForResult() throws InterruptedException {
       resultLatch.await();
     }
-    
+
     /**
      * After the call has gone through, return any exception that
      * was thrown, or null if no exception was thrown.
@@ -574,7 +574,7 @@ public abstract class GenericTestUtils {
     public Throwable getThrown() {
       return thrown;
     }
-    
+
     /**
      * After the call has gone through, return the call's return value,
      * or null in case it was void or an exception was thrown.
@@ -582,20 +582,20 @@ public abstract class GenericTestUtils {
     public Object getReturnValue() {
       return returnValue;
     }
-    
+
     public int getFireCount() {
       return fireCounter.get();
     }
-    
+
     public int getResultCount() {
       return resultCounter.get();
     }
   }
-  
+
   /**
    * An Answer implementation that simply forwards all calls through
    * to a delegate.
-   * 
+   *
    * This is useful as the default Answer for a mock object, to create
    * something like a spy on an RPC proxy. For example:
    * <code>
@@ -606,14 +606,14 @@ public abstract class GenericTestUtils {
    *    ...
    * </code>
    */
-  public static class DelegateAnswer implements Answer<Object> { 
+  public static class DelegateAnswer implements Answer<Object> {
     private final Object delegate;
     private final Log log;
-    
+
     public DelegateAnswer(Object delegate) {
       this(null, delegate);
     }
-    
+
     public DelegateAnswer(Log log, Object delegate) {
       this.log = log;
       this.delegate = delegate;
@@ -653,7 +653,7 @@ public abstract class GenericTestUtils {
       this.minSleepTime = minSleepTime;
       this.maxSleepTime = maxSleepTime;
     }
-    
+
     @Override
     public Object answer(InvocationOnMock invocation) throws Throwable {
       boolean interrupted = false;
@@ -683,11 +683,11 @@ public abstract class GenericTestUtils {
         " but got:\n" + output,
         Pattern.compile(pattern).matcher(output).find());
   }
-  
+
   public static void assertValueNear(long expected, long actual, long allowedError) {
     assertValueWithinRange(expected - allowedError, expected + allowedError, actual);
   }
-  
+
   public static void assertValueWithinRange(long expectedMin, long expectedMax,
       long actual) {
     Assert.assertTrue("Expected " + actual + " to be in range (" + expectedMin + ","
@@ -842,4 +842,28 @@ public abstract class GenericTestUtils {
       failf(format, args);
     }
   }
+
+  /**
+   * Retreive the max number of parallel test threads when running under maven.
+   * @return int number of threads
+   */
+  public static int getTestsThreadCount() {
+    String propString = System.getProperty("testsThreadCount", "1");
+    int threadCount = 1;
+    if (propString != null) {
+      String trimProp = propString.trim();
+      if (trimProp.endsWith("C")) {
+        double multiplier = Double.parseDouble(
+            trimProp.substring(0, trimProp.length()-1));
+        double calculated = multiplier * ((double) Runtime
+            .getRuntime()
+            .availableProcessors());
+        threadCount = calculated > 0d ? Math.max((int) calculated, 1) : 0;
+      } else {
+        threadCount = Integer.parseInt(trimProp);
+      }
+    }
+    return threadCount;
+  }
+
 }

+ 5 - 1
hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java

@@ -2667,7 +2667,11 @@ public class TestKMS {
                   kp.createKey("kbb", new KeyProvider.Options(conf));
                   Assert.fail();
                 } catch (Exception ex) {
-                  Assert.assertTrue(ex.getMessage(), ex.getMessage().contains("Forbidden"));
+                  GenericTestUtils.assertExceptionContains("Error while " +
+                      "authenticating with endpoint", ex);
+                  GenericTestUtils.assertExceptionContains("Forbidden", ex
+                      .getCause().getCause());
+
                 }
                 return null;
               }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java

@@ -88,6 +88,7 @@ public class DFSOpsCountStatistics extends StorageStatistics {
     SET_TIMES(CommonStatisticNames.OP_SET_TIMES),
     SET_XATTR("op_set_xattr"),
     GET_SNAPSHOT_DIFF("op_get_snapshot_diff"),
+    GET_SNAPSHOTTABLE_DIRECTORY_LIST("op_get_snapshottable_directory_list"),
     TRUNCATE(CommonStatisticNames.OP_TRUNCATE),
     UNSET_STORAGE_POLICY("op_unset_storage_policy");
 

+ 29 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -56,6 +56,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   private List<String> dependentHostNames = new LinkedList<>();
   private String upgradeDomain;
   public static final DatanodeInfo[] EMPTY_ARRAY = {};
+  private int numBlocks;
 
   // Datanode administrative states
   public enum AdminStates {
@@ -106,6 +107,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.upgradeDomain = from.getUpgradeDomain();
     this.lastBlockReportTime = from.getLastBlockReportTime();
     this.lastBlockReportMonotonic = from.getLastBlockReportMonotonic();
+    this.numBlocks = from.getNumBlocks();
   }
 
   protected DatanodeInfo(DatanodeID nodeID) {
@@ -123,6 +125,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.adminState = null;
     this.lastBlockReportTime = 0L;
     this.lastBlockReportMonotonic = 0L;
+    this.numBlocks = 0;
   }
 
   protected DatanodeInfo(DatanodeID nodeID, String location) {
@@ -139,7 +142,8 @@ public class DatanodeInfo extends DatanodeID implements Node {
       final long lastUpdate, final long lastUpdateMonotonic,
       final int xceiverCount, final String networkLocation,
       final AdminStates adminState, final String upgradeDomain,
-      final long lastBlockReportTime, final long lastBlockReportMonotonic) {
+      final long lastBlockReportTime, final long lastBlockReportMonotonic,
+                       final int blockCount) {
     super(ipAddr, hostName, datanodeUuid, xferPort, infoPort, infoSecurePort,
         ipcPort);
     this.capacity = capacity;
@@ -157,6 +161,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.upgradeDomain = upgradeDomain;
     this.lastBlockReportTime = lastBlockReportTime;
     this.lastBlockReportMonotonic = lastBlockReportMonotonic;
+    this.numBlocks = blockCount;
   }
 
   /** Network location name. */
@@ -246,6 +251,13 @@ public class DatanodeInfo extends DatanodeID implements Node {
    */
   public long getLastUpdateMonotonic() { return lastUpdateMonotonic;}
 
+  /**
+   * @return Num of Blocks
+   */
+  public int getNumBlocks() {
+    return numBlocks;
+  }
+
   /**
    * Set lastUpdate monotonic time
    */
@@ -301,6 +313,11 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.xceiverCount = xceiverCount;
   }
 
+  /** Sets number of blocks. */
+  public void setNumBlocks(int blockCount) {
+    this.numBlocks = blockCount;
+  }
+
   /** network location */
   @Override
   public String getNetworkLocation() {return location;}
@@ -351,6 +368,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     float cacheUsedPercent = getCacheUsedPercent();
     float cacheRemainingPercent = getCacheRemainingPercent();
     String lookupName = NetUtils.getHostNameOfIP(getName());
+    int blockCount = getNumBlocks();
 
     buffer.append("Name: ").append(getName());
     if (lookupName != null) {
@@ -406,6 +424,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
         .append(
             lastBlockReportTime != 0 ? new Date(lastBlockReportTime) : "Never")
         .append("\n");
+    buffer.append("Num of Blocks: ").append(blockCount).append("\n");
     return buffer.toString();
   }
 
@@ -680,6 +699,8 @@ public class DatanodeInfo extends DatanodeID implements Node {
     private long nonDfsUsed = 0L;
     private long lastBlockReportTime = 0L;
     private long lastBlockReportMonotonic = 0L;
+    private int numBlocks;
+
 
     public DatanodeInfoBuilder setFrom(DatanodeInfo from) {
       this.capacity = from.getCapacity();
@@ -697,6 +718,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
       this.upgradeDomain = from.getUpgradeDomain();
       this.lastBlockReportTime = from.getLastBlockReportTime();
       this.lastBlockReportMonotonic = from.getLastBlockReportMonotonic();
+      this.numBlocks = from.getNumBlocks();
       setNodeID(from);
       return this;
     }
@@ -823,13 +845,18 @@ public class DatanodeInfo extends DatanodeID implements Node {
       this.lastBlockReportMonotonic = time;
       return this;
     }
+    public DatanodeInfoBuilder setNumBlocks(int blockCount) {
+      this.numBlocks = blockCount;
+      return this;
+    }
 
     public DatanodeInfo build() {
       return new DatanodeInfo(ipAddr, hostName, datanodeUuid, xferPort,
           infoPort, infoSecurePort, ipcPort, capacity, dfsUsed, nonDfsUsed,
           remaining, blockPoolUsed, cacheCapacity, cacheUsed, lastUpdate,
           lastUpdateMonotonic, xceiverCount, location, adminState,
-          upgradeDomain, lastBlockReportTime, lastBlockReportMonotonic);
+          upgradeDomain, lastBlockReportTime, lastBlockReportMonotonic,
+          numBlocks);
     }
   }
 }

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java

@@ -188,11 +188,11 @@ public class LocatedBlocks {
 
   @Override
   public String toString() {
-    return getClass().getSimpleName() + "{" + "\n  fileLength=" + fileLength
-        + "\n  underConstruction=" + underConstruction
-        + "\n  blocks=" + blocks
-        + "\n  lastLocatedBlock=" + lastLocatedBlock
-        + "\n  isLastBlockComplete=" + isLastBlockComplete
-        + "\n  ecPolicy=" + ecPolicy + "}";
+    return getClass().getSimpleName() + "{" + ";  fileLength=" + fileLength
+        + ";  underConstruction=" + underConstruction
+        + ";  blocks=" + blocks
+        + ";  lastLocatedBlock=" + lastLocatedBlock
+        + ";  isLastBlockComplete=" + isLastBlockComplete
+        + ";  ecPolicy=" + ecPolicy + "}";
   }
 }

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java

@@ -79,6 +79,14 @@ public class SnapshottableDirectoryStatus {
     this.parentFullPath = parentFullPath;
   }
 
+  public SnapshottableDirectoryStatus(HdfsFileStatus dirStatus,
+      int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
+    this.dirStatus = dirStatus;
+    this.snapshotNumber = snapshotNumber;
+    this.snapshotQuota = snapshotQuota;
+    this.parentFullPath = parentFullPath;
+  }
+
   /**
    * @return Number of snapshots that have been taken for the directory
    */

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -333,6 +333,7 @@ public class PBHelperClient {
         .setAdminState(convert(info.getAdminState()))
         .setLastBlockReportTime(info.getLastBlockReportTime())
         .setLastBlockReportMonotonic(info.getLastBlockReportMonotonic())
+        .setNumBlocks(info.getNumBlocks())
         .build();
     return builder.build();
   }
@@ -704,7 +705,9 @@ public class PBHelperClient {
             .setLastBlockReportTime(di.hasLastBlockReportTime() ?
                 di.getLastBlockReportTime() : 0)
             .setLastBlockReportMonotonic(di.hasLastBlockReportMonotonic() ?
-                di.getLastBlockReportMonotonic() : 0);
+                di.getLastBlockReportMonotonic() : 0)
+            .setNumBlocks(di.getNumBlocks());
+
     if (di.hasNonDfsUsed()) {
       dinfo.setNonDfsUsed(di.getNonDfsUsed());
     } else {

+ 34 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -742,4 +743,37 @@ class JsonUtilClient {
     }
     return DFSUtilClient.string2Bytes(str);
   }
+
+  public static SnapshottableDirectoryStatus[] toSnapshottableDirectoryList(
+      final Map<?, ?> json) {
+    if (json == null) {
+      return null;
+    }
+    List<?> list = (List<?>) json.get("SnapshottableDirectoryList");
+    if (list == null) {
+      return null;
+    }
+    SnapshottableDirectoryStatus[] statuses =
+        new SnapshottableDirectoryStatus[list.size()];
+    for (int i = 0; i < list.size(); i++) {
+      statuses[i] = toSnapshottableDirectoryStatus((Map<?, ?>) list.get(i));
+    }
+    return statuses;
+  }
+
+  private static SnapshottableDirectoryStatus toSnapshottableDirectoryStatus(
+      Map<?, ?> json) {
+    if (json == null) {
+      return null;
+    }
+    int snapshotNumber = getInt(json, "snapshotNumber", 0);
+    int snapshotQuota = getInt(json, "snapshotQuota", 0);
+    byte[] parentFullPath = toByteArray((String) json.get("parentFullPath"));
+    HdfsFileStatus dirStatus =
+        toFileStatus((Map<?, ?>) json.get("dirStatus"), false);
+    SnapshottableDirectoryStatus snapshottableDirectoryStatus =
+        new SnapshottableDirectoryStatus(dirStatus, snapshotNumber,
+            snapshotQuota, parentFullPath);
+    return snapshottableDirectoryStatus;
+  }
 }

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -99,6 +99,7 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FileEncryptionInfoProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -1358,6 +1359,19 @@ public class WebHdfsFileSystem extends FileSystem
     }.run();
   }
 
+  public SnapshottableDirectoryStatus[] getSnapshottableDirectoryList()
+      throws IOException {
+    storageStatistics
+        .incrementOpCounter(OpType.GET_SNAPSHOTTABLE_DIRECTORY_LIST);
+    final HttpOpParam.Op op = GetOpParam.Op.GETSNAPSHOTTABLEDIRECTORYLIST;
+    return new FsPathResponseRunner<SnapshottableDirectoryStatus[]>(op, null) {
+      @Override
+      SnapshottableDirectoryStatus[] decodeResponse(Map<?, ?> json) {
+        return JsonUtilClient.toSnapshottableDirectoryList(json);
+      }
+    }.run();
+  }
+
   @Override
   public boolean setReplication(final Path p, final short replication
   ) throws IOException {

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java

@@ -48,7 +48,8 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     CHECKACCESS(false, HttpURLConnection.HTTP_OK),
     LISTSTATUS_BATCH(false, HttpURLConnection.HTTP_OK),
     GETSERVERDEFAULTS(false, HttpURLConnection.HTTP_OK),
-    GETSNAPSHOTDIFF(false, HttpURLConnection.HTTP_OK);
+    GETSNAPSHOTDIFF(false, HttpURLConnection.HTTP_OK),
+    GETSNAPSHOTTABLEDIRECTORYLIST(false, HttpURLConnection.HTTP_OK);
 
     final boolean redirect;
     final int expectedHttpResponseCode;

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto

@@ -134,6 +134,7 @@ message DatanodeInfoProto {
   optional string upgradeDomain = 14;
   optional uint64 lastBlockReportTime = 15 [default = 0];
   optional uint64 lastBlockReportMonotonic = 16 [default = 0];
+  optional uint32 numBlocks = 17 [default = 0];
 }
 
 /**

+ 5 - 21
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -629,30 +629,13 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <build>
         <plugins>
           <plugin>
-            <artifactId>maven-antrun-plugin</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-maven-plugins</artifactId>
             <executions>
               <execution>
-                <id>create-parallel-tests-dirs</id>
-                <phase>test-compile</phase>
-                <configuration>
-                  <target>
-                    <script language="javascript"><![CDATA[
-                      var baseDirs = [
-                          "${test.build.data}",
-                          "${test.build.dir}",
-                          "${hadoop.tmp.dir}" ];
-                      for (var i in baseDirs) {
-                        for (var j = 1; j <= ${testsThreadCount}; ++j) {
-                          var mkdir = project.createTask("mkdir");
-                          mkdir.setDir(new java.io.File(baseDirs[i], j));
-                          mkdir.perform();
-                        }
-                      }
-                    ]]></script>
-                  </target>
-                </configuration>
+                <id>parallel-tests-createdir</id>
                 <goals>
-                  <goal>run</goal>
+                  <goal>parallel-tests-createdir</goal>
                 </goals>
               </execution>
             </executions>
@@ -665,6 +648,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
               <reuseForks>false</reuseForks>
               <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
               <systemPropertyVariables>
+                <testsThreadCount>${testsThreadCount}</testsThreadCount>
                 <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
                 <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
                 <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>

+ 14 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.federation.resolver;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.isParentEntry;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -238,9 +239,17 @@ public class MountTableResolver
       Entry<String, PathLocation> entry = it.next();
       PathLocation loc = entry.getValue();
       String src = loc.getSourcePath();
-      if (src.startsWith(path)) {
-        LOG.debug("Removing {}", src);
-        it.remove();
+      if (src != null) {
+        if(isParentEntry(src, path)) {
+          LOG.debug("Removing {}", src);
+          it.remove();
+        }
+      } else {
+        String dest = loc.getDefaultLocation().getDest();
+        if (dest.startsWith(path)) {
+          LOG.debug("Removing default cache {}", dest);
+          it.remove();
+        }
       }
     }
 
@@ -287,6 +296,7 @@ public class MountTableResolver
         if (!oldEntries.contains(srcPath)) {
           // Add node, it does not exist
           this.tree.put(srcPath, entry);
+          invalidateLocationCache(srcPath);
           LOG.info("Added new mount point {} to resolver", srcPath);
         } else {
           // Node exists, check for updates
@@ -530,7 +540,7 @@ public class MountTableResolver
     readLock.lock();
     try {
       Entry<String, MountTable> entry = this.tree.floorEntry(path);
-      while (entry != null && !path.startsWith(entry.getKey())) {
+      while (entry != null && !isParentEntry(path, entry.getKey())) {
         entry = this.tree.lowerEntry(entry.getKey());
       }
       if (entry == null) {

+ 33 - 26
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java

@@ -32,6 +32,7 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -303,6 +304,38 @@ public class ConnectionManager {
     return JSON.toString(info);
   }
 
+  @VisibleForTesting
+  Map<ConnectionPoolId, ConnectionPool> getPools() {
+    return this.pools;
+  }
+
+  /**
+   * Clean the unused connections for this pool.
+   *
+   * @param pool Connection pool to cleanup.
+   */
+  @VisibleForTesting
+  void cleanup(ConnectionPool pool) {
+    if (pool.getNumConnections() > pool.getMinSize()) {
+      // Check if the pool hasn't been active in a while or not 50% are used
+      long timeSinceLastActive = Time.now() - pool.getLastActiveTime();
+      int total = pool.getNumConnections();
+      int active = pool.getNumActiveConnections();
+      if (timeSinceLastActive > connectionCleanupPeriodMs ||
+          active < MIN_ACTIVE_RATIO * total) {
+        // Remove and close 1 connection
+        List<ConnectionContext> conns = pool.removeConnections(1);
+        for (ConnectionContext conn : conns) {
+          conn.close();
+        }
+        LOG.debug("Removed connection {} used {} seconds ago. " +
+                "Pool has {}/{} connections", pool.getConnectionPoolId(),
+            TimeUnit.MILLISECONDS.toSeconds(timeSinceLastActive),
+            pool.getNumConnections(), pool.getMaxSize());
+      }
+    }
+  }
+
   /**
    * Removes stale connections not accessed recently from the pool. This is
    * invoked periodically.
@@ -350,32 +383,6 @@ public class ConnectionManager {
         }
       }
     }
-
-    /**
-     * Clean the unused connections for this pool.
-     *
-     * @param pool Connection pool to cleanup.
-     */
-    private void cleanup(ConnectionPool pool) {
-      if (pool.getNumConnections() > pool.getMinSize()) {
-        // Check if the pool hasn't been active in a while or not 50% are used
-        long timeSinceLastActive = Time.now() - pool.getLastActiveTime();
-        int total = pool.getNumConnections();
-        int active = getNumActiveConnections();
-        if (timeSinceLastActive > connectionCleanupPeriodMs ||
-            active < MIN_ACTIVE_RATIO * total) {
-          // Remove and close 1 connection
-          List<ConnectionContext> conns = pool.removeConnections(1);
-          for (ConnectionContext conn : conns) {
-            conn.close();
-          }
-          LOG.debug("Removed connection {} used {} seconds ago. " +
-              "Pool has {}/{} connections", pool.getConnectionPoolId(),
-              TimeUnit.MILLISECONDS.toSeconds(timeSinceLastActive),
-              pool.getNumConnections(), pool.getMaxSize());
-        }
-      }
-    }
   }
 
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java

@@ -159,7 +159,7 @@ public class ConnectionPool {
     for (int i=0; i<size; i++) {
       int index = (threadIndex + i) % size;
       conn = tmpConnections.get(index);
-      if (conn != null && !conn.isUsable()) {
+      if (conn != null && conn.isUsable()) {
         return conn;
       }
     }

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPoolId.java

@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -99,6 +100,11 @@ public class ConnectionPoolId implements Comparable<ConnectionPoolId> {
     return ret;
   }
 
+  @VisibleForTesting
+  UserGroupInformation getUgi() {
+    return this.ugi;
+  }
+
   /**
    * Get the token identifiers for this connection.
    * @return List with the token identifiers.

+ 20 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java

@@ -26,6 +26,7 @@ import java.net.URL;
 import java.net.URLConnection;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
@@ -186,4 +187,23 @@ public final class FederationUtil {
         ActiveNamenodeResolver.class);
     return newInstance(conf, stateStore, StateStoreService.class, clazz);
   }
+
+  /**
+   * Check if the given path is the child of parent path.
+   * @param path Path to be check.
+   * @param parent Parent path.
+   * @return True if parent path is parent entry for given path.
+   */
+  public static boolean isParentEntry(final String path, final String parent) {
+    if (!path.startsWith(parent)) {
+      return false;
+    }
+
+    if (path.equals(parent)) {
+      return true;
+    }
+
+    return path.charAt(parent.length()) == Path.SEPARATOR_CHAR
+        || parent.equals(Path.SEPARATOR);
+  }
 }

+ 13 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaManager.java

@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.isParentEntry;
+
+import java.util.HashSet;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
@@ -94,7 +97,16 @@ public class RouterQuotaManager {
       String from = parentPath;
       String to = parentPath + Character.MAX_VALUE;
       SortedMap<String, RouterQuotaUsage> subMap = this.cache.subMap(from, to);
-      return subMap.keySet();
+
+      Set<String> validPaths = new HashSet<>();
+      if (subMap != null) {
+        for (String path : subMap.keySet()) {
+          if (isParentEntry(path, parentPath)) {
+            validPaths.add(path);
+          }
+        }
+      }
+      return validPaths;
     } finally {
       readLock.unlock();
     }

+ 12 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java

@@ -32,6 +32,10 @@ import org.apache.hadoop.util.Time;
  * </ul>
  */
 public abstract class BaseRecord implements Comparable<BaseRecord> {
+  public static final String ERROR_MSG_CREATION_TIME_NEGATIVE =
+      "The creation time for the record cannot be negative.";
+  public static final String ERROR_MSG_MODIFICATION_TIME_NEGATIVE =
+      "The modification time for the record cannot be negative.";
 
   /**
    * Set the modification time for the record.
@@ -193,11 +197,15 @@ public abstract class BaseRecord implements Comparable<BaseRecord> {
 
   /**
    * Validates the record. Called when the record is created, populated from the
-   * state store, and before committing to the state store.
-   * @return If the record is valid.
+   * state store, and before committing to the state store. If validate failed,
+   * there throws an exception.
    */
-  public boolean validate() {
-    return getDateCreated() > 0 && getDateModified() > 0;
+  public void validate() {
+    if (getDateCreated() <= 0) {
+      throw new IllegalArgumentException(ERROR_MSG_CREATION_TIME_NEGATIVE);
+    } else if (getDateModified() <= 0) {
+      throw new IllegalArgumentException(ERROR_MSG_MODIFICATION_TIME_NEGATIVE);
+    }
   }
 
   @Override

+ 18 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipState.java

@@ -37,6 +37,14 @@ import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerialize
  */
 public abstract class MembershipState extends BaseRecord
     implements FederationNamenodeContext {
+  public static final String ERROR_MSG_NO_NS_SPECIFIED =
+      "Invalid registration, no nameservice specified ";
+  public static final String ERROR_MSG_NO_WEB_ADDR_SPECIFIED =
+      "Invalid registration, no web address specified ";
+  public static final String ERROR_MSG_NO_RPC_ADDR_SPECIFIED =
+      "Invalid registration, no rpc address specified ";
+  public static final String ERROR_MSG_NO_BP_SPECIFIED =
+      "Invalid registration, no block pool specified ";
 
   /** Expiration time in ms for this entry. */
   private static long expirationMs;
@@ -226,26 +234,25 @@ public abstract class MembershipState extends BaseRecord
    * is missing required information.
    */
   @Override
-  public boolean validate() {
-    boolean ret = super.validate();
+  public void validate() {
+    super.validate();
     if (getNameserviceId() == null || getNameserviceId().length() == 0) {
-      //LOG.error("Invalid registration, no nameservice specified " + this);
-      ret = false;
+      throw new IllegalArgumentException(
+          ERROR_MSG_NO_NS_SPECIFIED + this);
     }
     if (getWebAddress() == null || getWebAddress().length() == 0) {
-      //LOG.error("Invalid registration, no web address specified " + this);
-      ret = false;
+      throw new IllegalArgumentException(
+          ERROR_MSG_NO_WEB_ADDR_SPECIFIED + this);
     }
     if (getRpcAddress() == null || getRpcAddress().length() == 0) {
-      //LOG.error("Invalid registration, no rpc address specified " + this);
-      ret = false;
+      throw new IllegalArgumentException(
+          ERROR_MSG_NO_RPC_ADDR_SPECIFIED + this);
     }
     if (!isBadState() &&
         (getBlockPoolId().isEmpty() || getBlockPoolId().length() == 0)) {
-      //LOG.error("Invalid registration, no block pool specified " + this);
-      ret = false;
+      throw new IllegalArgumentException(
+          ERROR_MSG_NO_BP_SPECIFIED + this);
     }
-    return ret;
   }
 
 

+ 26 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java

@@ -51,7 +51,18 @@ import org.slf4j.LoggerFactory;
 public abstract class MountTable extends BaseRecord {
 
   private static final Logger LOG = LoggerFactory.getLogger(MountTable.class);
-
+  public static final String ERROR_MSG_NO_SOURCE_PATH =
+      "Invalid entry, no source path specified ";
+  public static final String ERROR_MSG_MUST_START_WITH_BACK_SLASH =
+      "Invalid entry, all mount points must start with / ";
+  public static final String ERROR_MSG_NO_DEST_PATH_SPECIFIED =
+      "Invalid entry, no destination paths specified ";
+  public static final String ERROR_MSG_INVAILD_DEST_NS =
+      "Invalid entry, invalid destination nameservice ";
+  public static final String ERROR_MSG_INVAILD_DEST_PATH =
+      "Invalid entry, invalid destination path ";
+  public static final String ERROR_MSG_ALL_DEST_MUST_START_WITH_BACK_SLASH =
+      "Invalid entry, all destination must start with / ";
 
   /** Comparator for paths which considers the /. */
   public static final Comparator<String> PATH_COMPARATOR =
@@ -342,36 +353,35 @@ public abstract class MountTable extends BaseRecord {
   }
 
   @Override
-  public boolean validate() {
-    boolean ret = super.validate();
+  public void validate() {
+    super.validate();
     if (this.getSourcePath() == null || this.getSourcePath().length() == 0) {
-      LOG.error("Invalid entry, no source path specified ", this);
-      ret = false;
+      throw new IllegalArgumentException(
+          ERROR_MSG_NO_SOURCE_PATH + this);
     }
     if (!this.getSourcePath().startsWith("/")) {
-      LOG.error("Invalid entry, all mount points must start with / ", this);
-      ret = false;
+      throw new IllegalArgumentException(
+          ERROR_MSG_MUST_START_WITH_BACK_SLASH + this);
     }
     if (this.getDestinations() == null || this.getDestinations().size() == 0) {
-      LOG.error("Invalid entry, no destination paths specified ", this);
-      ret = false;
+      throw new IllegalArgumentException(
+          ERROR_MSG_NO_DEST_PATH_SPECIFIED + this);
     }
     for (RemoteLocation loc : getDestinations()) {
       String nsId = loc.getNameserviceId();
       if (nsId == null || nsId.length() == 0) {
-        LOG.error("Invalid entry, invalid destination nameservice ", this);
-        ret = false;
+        throw new IllegalArgumentException(
+            ERROR_MSG_INVAILD_DEST_NS + this);
       }
       if (loc.getDest() == null || loc.getDest().length() == 0) {
-        LOG.error("Invalid entry, invalid destination path ", this);
-        ret = false;
+        throw new IllegalArgumentException(
+            ERROR_MSG_INVAILD_DEST_PATH + this);
       }
       if (!loc.getDest().startsWith("/")) {
-        LOG.error("Invalid entry, all destination must start with / ", this);
-        ret = false;
+        throw new IllegalArgumentException(
+            ERROR_MSG_ALL_DEST_MUST_START_WITH_BACK_SLASH + this);
       }
     }
-    return ret;
   }
 
   @Override

+ 4 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/RouterState.java

@@ -127,14 +127,13 @@ public abstract class RouterState extends BaseRecord {
   }
 
   @Override
-  public boolean validate() {
-    boolean ret = super.validate();
+  public void validate() {
+    super.validate();
     if ((getAddress() == null || getAddress().length() == 0) &&
         getStatus() != RouterServiceState.INITIALIZING) {
-      LOG.error("Invalid router entry, no address specified {}", this);
-      ret = false;
+      throw new IllegalArgumentException(
+          "Invalid router entry, no address specified " + this);
     }
-    return ret;
   }
 
   @Override

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java

@@ -588,8 +588,7 @@ class FSDirRenameOp {
     private INode srcChild;
     private INode oldDstChild;
 
-    RenameOperation(FSDirectory fsd, INodesInPath srcIIP, INodesInPath dstIIP)
-        throws QuotaExceededException {
+    RenameOperation(FSDirectory fsd, INodesInPath srcIIP, INodesInPath dstIIP) {
       this.fsd = fsd;
       this.srcIIP = srcIIP;
       this.dstIIP = dstIIP;

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -4338,6 +4338,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       for (int i=0; i<arr.length; i++) {
         arr[i] = new DatanodeInfoBuilder().setFrom(results.get(i))
             .build();
+        arr[i].setNumBlocks(results.get(i).numBlocks());
       }
     } finally {
       readUnlock(operationName);

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

@@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFea
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
-import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -353,7 +352,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
     // replace the instance in the created list of the diff list
     DirectoryWithSnapshotFeature sf = this.getDirectoryWithSnapshotFeature();
     if (sf != null) {
-      sf.getDiffs().replaceChild(ListType.CREATED, oldChild, newChild);
+      sf.getDiffs().replaceCreatedChild(oldChild, newChild);
     }
     
     // update the inodeMap
@@ -746,8 +745,8 @@ public class INodeDirectory extends INodeWithAdditionalFields
       final INode newChild) throws QuotaExceededException {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     assert sf != null : "Directory does not have snapshot feature";
-    sf.getDiffs().removeChild(ListType.DELETED, oldChild);
-    sf.getDiffs().replaceChild(ListType.CREATED, oldChild, newChild);
+    sf.getDiffs().removeDeletedChild(oldChild);
+    sf.getDiffs().replaceCreatedChild(oldChild, newChild);
     addChild(newChild, true, Snapshot.CURRENT_STATE_ID);
   }
   
@@ -761,8 +760,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
       int latestSnapshotId) throws QuotaExceededException {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     assert sf != null : "Directory does not have snapshot feature";
-    boolean removeDeletedChild = sf.getDiffs().removeChild(ListType.DELETED,
-        deletedChild);
+    boolean removeDeletedChild = sf.getDiffs().removeDeletedChild(deletedChild);
     int sid = removeDeletedChild ? Snapshot.CURRENT_STATE_ID : latestSnapshotId;
     final boolean added = addChild(deletedChild, true, sid);
     // update quota usage if adding is successfully and the old child has not

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

@@ -126,10 +126,6 @@ public abstract class INodeReference extends INode {
     return referred;
   }
 
-  public final void setReferredINode(INode referred) {
-    this.referred = referred;
-  }
-  
   @Override
   public final boolean isReference() {
     return true;

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

@@ -44,7 +44,6 @@ import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
 import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
-import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.Time;
@@ -396,7 +395,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
           .getId());
       for (INode child : children) {
         final byte[] name = child.getLocalNameBytes();
-        boolean toProcess = diff.searchIndex(ListType.DELETED, name) < 0;
+        boolean toProcess = !diff.containsDeleted(name);
         if (!toProcess && child instanceof INodeReference.WithName) {
           byte[][] renameTargetPath = findRenameTargetPath(
               snapshotDir, (WithName) child,
@@ -476,7 +475,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
         }
         iterate = true;
         level = level + 1;
-        boolean toProcess = diff.searchIndex(ListType.DELETED, name) < 0;
+        boolean toProcess = !diff.containsDeleted(name);
         if (!toProcess && child instanceof INodeReference.WithName) {
           byte[][] renameTargetPath = findRenameTargetPath(snapshotDir,
               (WithName) child, Snapshot.getSnapshotId(later));

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

@@ -17,38 +17,22 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
-import org.apache.hadoop.hdfs.server.namenode.AclStorage;
-import org.apache.hadoop.hdfs.server.namenode.ContentCounts;
-import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
-import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
-import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
-import org.apache.hadoop.hdfs.server.namenode.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.INodeReference;
-import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
+import org.apache.hadoop.hdfs.server.namenode.*;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 import org.apache.hadoop.hdfs.util.Diff;
 import org.apache.hadoop.hdfs.util.Diff.Container;
-import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.Diff.UndoInfo;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
-
-import com.google.common.base.Preconditions;
 import org.apache.hadoop.security.AccessControlException;
 
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.*;
+
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
 
 /**
@@ -70,56 +54,43 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
 
     /**
-     * Replace the given child from the created/deleted list.
+     * Replace the given child from the created list.
      * @return true if the child is replaced; false if the child is not found.
      */
-    private boolean replace(final ListType type,
-        final INode oldChild, final INode newChild) {
-      final List<INode> list = getList(type);
+    private boolean replaceCreated(final INode oldChild, final INode newChild) {
+      final List<INode> list = getCreatedUnmodifiable();
       final int i = search(list, oldChild.getLocalNameBytes());
       if (i < 0 || list.get(i).getId() != oldChild.getId()) {
         return false;
       }
 
-      final INode removed = list.set(i, newChild);
+      final INode removed = setCreated(i, newChild);
       Preconditions.checkState(removed == oldChild);
       return true;
     }
 
-    private boolean removeChild(ListType type, final INode child) {
-      final List<INode> list = getList(type);
-      final int i = searchIndex(type, child.getLocalNameBytes());
-      if (i >= 0 && list.get(i) == child) {
-        list.remove(i);
-        return true;
-      }
-      return false;
-    }
-
     /** clear the created list */
     private void destroyCreatedList(INode.ReclaimContext reclaimContext,
         final INodeDirectory currentINode) {
-      final List<INode> createdList = getList(ListType.CREATED);
-      for (INode c : createdList) {
+      for (INode c : getCreatedUnmodifiable()) {
         c.destroyAndCollectBlocks(reclaimContext);
         // c should be contained in the children list, remove it
         currentINode.removeChild(c);
       }
-      createdList.clear();
+      clearCreated();
     }
 
     /** clear the deleted list */
     private void destroyDeletedList(INode.ReclaimContext reclaimContext) {
-      final List<INode> deletedList = getList(ListType.DELETED);
-      for (INode d : deletedList) {
+      for (INode d : getDeletedUnmodifiable()) {
         d.destroyAndCollectBlocks(reclaimContext);
       }
-      deletedList.clear();
+      clearDeleted();
     }
 
     /** Serialize {@link #created} */
     private void writeCreated(DataOutput out) throws IOException {
-      final List<INode> created = getList(ListType.CREATED);
+      final List<INode> created = getCreatedUnmodifiable();
       out.writeInt(created.size());
       for (INode node : created) {
         // For INode in created list, we only need to record its local name
@@ -132,7 +103,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     /** Serialize {@link #deleted} */
     private void writeDeleted(DataOutput out,
         ReferenceMap referenceMap) throws IOException {
-      final List<INode> deleted = getList(ListType.DELETED);
+      final List<INode> deleted = getDeletedUnmodifiable();
       out.writeInt(deleted.size());
       for (INode node : deleted) {
         FSImageSerialization.saveINode2Image(node, out, true, referenceMap);
@@ -148,7 +119,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
 
     /** Get the list of INodeDirectory contained in the deleted list */
     private void getDirsInDeleted(List<INodeDirectory> dirList) {
-      for (INode node : getList(ListType.DELETED)) {
+      for (INode node : getDeletedUnmodifiable()) {
         if (node.isDirectory()) {
           dirList.add(node.asDirectory());
         }
@@ -347,24 +318,24 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
 
     /** Replace the given child in the created/deleted list, if there is any. */
-    public boolean replaceChild(final ListType type, final INode oldChild,
+    public boolean replaceCreatedChild(final INode oldChild,
         final INode newChild) {
       final DiffList<DirectoryDiff> diffList = asList();
       for(int i = diffList.size() - 1; i >= 0; i--) {
         final ChildrenDiff diff = diffList.get(i).diff;
-        if (diff.replace(type, oldChild, newChild)) {
+        if (diff.replaceCreated(oldChild, newChild)) {
           return true;
         }
       }
       return false;
     }
 
-    /** Remove the given child in the created/deleted list, if there is any. */
-    public boolean removeChild(final ListType type, final INode child) {
+    /** Remove the given child from the deleted list, if there is any. */
+    public boolean removeDeletedChild(final INode child) {
       final DiffList<DirectoryDiff> diffList = asList();
       for(int i = diffList.size() - 1; i >= 0; i--) {
         final ChildrenDiff diff = diffList.get(i).diff;
-        if (diff.removeChild(type, child)) {
+        if (diff.removeDeleted(child)) {
           return true;
         }
       }
@@ -380,11 +351,9 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     public int findSnapshotDeleted(final INode child) {
       final DiffList<DirectoryDiff> diffList = asList();
       for(int i = diffList.size() - 1; i >= 0; i--) {
-        final ChildrenDiff diff = diffList.get(i).diff;
-        final int d = diff.searchIndex(ListType.DELETED,
-            child.getLocalNameBytes());
-        if (d >= 0 && diff.getList(ListType.DELETED).get(d) == child) {
-          return diffList.get(i).getSnapshotId();
+        final DirectoryDiff diff = diffList.get(i);
+        if (diff.getChildrenDiff().containsDeleted(child)) {
+          return diff.getSnapshotId();
         }
       }
       return NO_SNAPSHOT_ID;
@@ -442,7 +411,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         DirectoryDiffList diffList = sf.getDiffs();
         DirectoryDiff priorDiff = diffList.getDiffById(prior);
         if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
-          List<INode> dList = priorDiff.diff.getList(ListType.DELETED);
+          List<INode> dList = priorDiff.diff.getDeletedUnmodifiable();
           excludedNodes = cloneDiffList(dList);
         }
         
@@ -512,8 +481,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         }
 
         for (INode child : dir.getChildrenList(prior)) {
-          if (priorChildrenDiff != null && priorChildrenDiff.search(
-              ListType.DELETED, child.getLocalNameBytes()) != null) {
+          if (priorChildrenDiff != null && priorChildrenDiff.getDeleted(
+              child.getLocalNameBytes()) != null) {
             continue;
           }
           queue.addLast(child);
@@ -558,12 +527,14 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
       boolean setModTime, int latestSnapshotId) throws QuotaExceededException {
     ChildrenDiff diff = diffs.checkAndAddLatestSnapshotDiff(latestSnapshotId,
         parent).diff;
-    int undoInfo = diff.create(inode);
-
-    final boolean added = parent.addChild(inode, setModTime,
-        Snapshot.CURRENT_STATE_ID);
-    if (!added) {
-      diff.undoCreate(inode, undoInfo);
+    final int undoInfo = diff.create(inode);
+    boolean added = false;
+    try {
+      added = parent.addChild(inode, setModTime, Snapshot.CURRENT_STATE_ID);
+    } finally {
+      if (!added) {
+        diff.undoCreate(inode, undoInfo);
+      }
     }
     return added;
   }
@@ -587,12 +558,14 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     // any snapshot.
     ChildrenDiff diff = diffs.checkAndAddLatestSnapshotDiff(latestSnapshotId,
         parent).diff;
-    UndoInfo<INode> undoInfo = diff.delete(child);
-
-    final boolean removed = parent.removeChild(child);
-    if (!removed && undoInfo != null) {
-      // remove failed, undo
-      diff.undoDelete(child, undoInfo);
+    final UndoInfo<INode> undoInfo = diff.delete(child);
+    boolean removed = false;
+    try {
+      removed = parent.removeChild(child);
+    } finally {
+      if (!removed) {
+        diff.undoDelete(child, undoInfo);
+      }
     }
     return removed;
   }
@@ -648,7 +621,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
       BlockStoragePolicySuite bsps, byte storagePolicyId) {
     final QuotaCounts counts = new QuotaCounts.Builder().build();
     for(DirectoryDiff d : diffs) {
-      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
+      for(INode deleted : d.getChildrenDiff().getDeletedUnmodifiable()) {
         final byte childPolicyId = deleted.getStoragePolicyIDForQuota(
             storagePolicyId);
         counts.add(deleted.computeQuotaUsage(bsps, childPolicyId, false,
@@ -664,7 +637,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     ContentSummaryComputationContext summary = 
         new ContentSummaryComputationContext(bsps);
     for(DirectoryDiff d : diffs) {
-      for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
+      for(INode deleted : d.getChildrenDiff().getDeletedUnmodifiable()) {
         deleted.computeContentSummary(Snapshot.CURRENT_STATE_ID, summary);
       }
     }
@@ -747,10 +720,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
       if (prior != NO_SNAPSHOT_ID) {
         DirectoryDiff priorDiff = this.getDiffs().getDiffById(prior);
         if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
-          List<INode> cList = priorDiff.diff.getList(ListType.CREATED);
-          List<INode> dList = priorDiff.diff.getList(ListType.DELETED);
-          priorCreated = cloneDiffList(cList);
-          priorDeleted = cloneDiffList(dList);
+          priorCreated = cloneDiffList(priorDiff.diff.getCreatedUnmodifiable());
+          priorDeleted = cloneDiffList(priorDiff.diff.getDeletedUnmodifiable());
         }
       }
 
@@ -770,8 +741,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           // cleanSubtreeRecursively call.
           if (priorCreated != null) {
             // we only check the node originally in prior's created list
-            for (INode cNode : priorDiff.getChildrenDiff().getList(
-                ListType.CREATED)) {
+            for (INode cNode : priorDiff.diff.getCreatedUnmodifiable()) {
               if (priorCreated.containsKey(cNode)) {
                 cNode.cleanSubtree(reclaimContext, snapshot, NO_SNAPSHOT_ID);
               }
@@ -786,8 +756,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           // For files moved from posterior's deleted list, we also need to
           // delete its snapshot copy associated with the posterior snapshot.
           
-          for (INode dNode : priorDiff.getChildrenDiff().getList(
-              ListType.DELETED)) {
+          for (INode dNode : priorDiff.diff.getDeletedUnmodifiable()) {
             if (priorDeleted == null || !priorDeleted.containsKey(dNode)) {
               cleanDeletedINode(reclaimContext, dNode, snapshot, prior);
             }

+ 2 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java

@@ -78,7 +78,6 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeat
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.Root;
 import org.apache.hadoop.hdfs.server.namenode.XAttrFeature;
-import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.EnumCounters;
 
 import com.google.common.base.Preconditions;
@@ -581,10 +580,9 @@ public class FSImageFormatPBSnapshot {
                     buildINodeDirectory(copy, parent.getSaverContext()));
           }
           // process created list and deleted list
-          List<INode> created = diff.getChildrenDiff()
-              .getList(ListType.CREATED);
+          List<INode> created = diff.getChildrenDiff().getCreatedUnmodifiable();
           db.setCreatedListSize(created.size());
-          List<INode> deleted = diff.getChildrenDiff().getList(ListType.DELETED);
+          List<INode> deleted = diff.getChildrenDiff().getDeletedUnmodifiable();
           for (INode d : deleted) {
             if (d.isReference()) {
               refList.add(d.asReference());

+ 4 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotDiffInfo.java

@@ -32,7 +32,6 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.ChildrenDiff;
-import org.apache.hadoop.hdfs.util.Diff.ListType;
 
 import com.google.common.base.Preconditions;
 import com.google.common.primitives.SignedBytes;
@@ -141,7 +140,7 @@ class SnapshotDiffInfo {
     dirDiffMap.put(dir, diff);
     diffMap.put(dir, relativePath);
     // detect rename
-    for (INode created : diff.getList(ListType.CREATED)) {
+    for (INode created : diff.getCreatedUnmodifiable()) {
       if (created.isReference()) {
         RenameEntry entry = getEntry(created.getId());
         if (entry.getTargetPath() == null) {
@@ -149,7 +148,7 @@ class SnapshotDiffInfo {
         }
       }
     }
-    for (INode deleted : diff.getList(ListType.DELETED)) {
+    for (INode deleted : diff.getDeletedUnmodifiable()) {
       if (deleted instanceof INodeReference.WithName) {
         RenameEntry entry = getEntry(deleted.getId());
         entry.setSource(deleted, relativePath);
@@ -221,11 +220,9 @@ class SnapshotDiffInfo {
   private List<DiffReportEntry> generateReport(ChildrenDiff dirDiff,
       byte[][] parentPath, boolean fromEarlier, Map<Long, RenameEntry> renameMap) {
     List<DiffReportEntry> list = new ChunkedArrayList<>();
-    List<INode> created = dirDiff.getList(ListType.CREATED);
-    List<INode> deleted = dirDiff.getList(ListType.DELETED);
     byte[][] fullPath = new byte[parentPath.length + 1][];
     System.arraycopy(parentPath, 0, fullPath, 0, parentPath.length);
-    for (INode cnode : created) {
+    for (INode cnode : dirDiff.getCreatedUnmodifiable()) {
       RenameEntry entry = renameMap.get(cnode.getId());
       if (entry == null || !entry.isRename()) {
         fullPath[fullPath.length - 1] = cnode.getLocalNameBytes();
@@ -233,7 +230,7 @@ class SnapshotDiffInfo {
             : DiffType.DELETE, fullPath));
       }
     }
-    for (INode dnode : deleted) {
+    for (INode dnode : dirDiff.getDeletedUnmodifiable()) {
       RenameEntry entry = renameMap.get(dnode.getId());
       if (entry != null && entry.isRename()) {
         list.add(new DiffReportEntry(DiffType.RENAME,

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

@@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.ChildrenDiff;
-import org.apache.hadoop.hdfs.util.Diff.ListType;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.util.ChunkedArrayList;
@@ -96,10 +95,10 @@ class SnapshotDiffListingInfo {
       }
     }
 
-    if (lastIndex == -1 || lastIndex < diff.getList(ListType.CREATED).size()) {
+    final List<INode> clist =  diff.getCreatedUnmodifiable();
+    if (lastIndex == -1 || lastIndex < clist.size()) {
       ListIterator<INode> iterator = lastIndex != -1 ?
-          diff.getList(ListType.CREATED).listIterator(lastIndex)
-          : diff.getList(ListType.CREATED).listIterator();
+          clist.listIterator(lastIndex): clist.listIterator();
       while (iterator.hasNext()) {
         if (getTotalEntries() < maxEntries) {
           INode created = iterator.next();
@@ -115,11 +114,11 @@ class SnapshotDiffListingInfo {
       setLastIndex(-1);
     }
 
-    if (lastIndex == -1 || lastIndex >= diff.getList(ListType.CREATED).size()) {
-      int size = diff.getList(ListType.DELETED).size();
+    if (lastIndex == -1 || lastIndex >= clist.size()) {
+      final List<INode> dlist =  diff.getDeletedUnmodifiable();
+      int size = dlist.size();
       ListIterator<INode> iterator = lastIndex != -1 ?
-          diff.getList(ListType.DELETED).listIterator(lastIndex - size)
-          : diff.getList(ListType.DELETED).listIterator();
+          dlist.listIterator(lastIndex - size): dlist.listIterator();
       while (iterator.hasNext()) {
         if (getTotalEntries() < maxEntries) {
           final INode d = iterator.next();

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

@@ -38,7 +38,6 @@ import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
 import org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff;
-import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import com.google.common.base.Preconditions;
@@ -145,8 +144,7 @@ public class SnapshotFSImageFormat {
     // the INode in the created list should be a reference to another INode
     // in posterior SnapshotDiffs or one of the current children
     for (DirectoryDiff postDiff : parent.getDiffs()) {
-      final INode d = postDiff.getChildrenDiff().search(ListType.DELETED,
-          createdNodeName);
+      final INode d = postDiff.getChildrenDiff().getDeleted(createdNodeName);
       if (d != null) {
         return d;
       } // else go to the next SnapshotDiff

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -80,6 +80,7 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
@@ -1207,6 +1208,12 @@ public class NamenodeWebHdfsMethods {
       final String js = JsonUtil.toJsonString(diffReport);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
+    case GETSNAPSHOTTABLEDIRECTORYLIST: {
+      SnapshottableDirectoryStatus[] snapshottableDirectoryList =
+          cp.getSnapshottableDirListing();
+      final String js = JsonUtil.toJsonString(snapshottableDirectoryList);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }

+ 5 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java

@@ -358,17 +358,15 @@ public class ECAdmin extends Configured implements Tool {
       try {
         dfs.setErasureCodingPolicy(p, ecPolicyName);
         if (ecPolicyName == null){
-          System.out.println("Set default erasure coding policy" +
-              " on " + path);
-        } else {
-          System.out.println("Set erasure coding policy " + ecPolicyName +
-              " on " + path);
+          ecPolicyName = "default";
         }
+        System.out.println("Set " + ecPolicyName + " erasure coding policy on" +
+            " " + path);
         RemoteIterator<FileStatus> dirIt = dfs.listStatusIterator(p);
         if (dirIt.hasNext()) {
           System.out.println("Warning: setting erasure coding policy on a " +
-              "non-empty directory will not automatically convert existing" +
-              " files to " + ecPolicyName);
+              "non-empty directory will not automatically convert existing " +
+              "files to " + ecPolicyName + " erasure coding policy");
         }
       } catch (Exception e) {
         System.err.println(AdminHelper.prettifyException(e));

+ 93 - 38
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java

@@ -73,10 +73,6 @@ import com.google.common.base.Preconditions;
  * @param <E> The element type, which must implement {@link Element} interface.
  */
 public class Diff<K, E extends Diff.Element<K>> {
-  public enum ListType {
-    CREATED, DELETED
-  }
-
   /** An interface for the elements in a {@link Diff}. */
   public static interface Element<K> extends Comparable<K> {
     /** @return the key of this object. */
@@ -156,26 +152,73 @@ public class Diff<K, E extends Diff.Element<K>> {
     this.deleted = deleted;
   }
 
-  /** @return the created list, which is never null. */
-  public List<E> getList(final ListType type) {
-    final List<E> list = type == ListType.CREATED? created: deleted;
-    return list == null? Collections.<E>emptyList(): list;
+  public List<E> getCreatedUnmodifiable() {
+    return created != null? Collections.unmodifiableList(created)
+        : Collections.emptyList();
+  }
+
+  public E setCreated(int index, E element) {
+    final E old = created.set(index, element);
+    if (old.compareTo(element.getKey()) != 0) {
+      throw new AssertionError("Element mismatched: element=" + element
+          + " but old=" + old);
+    }
+    return old;
+  }
+
+  public void clearCreated() {
+    if (created != null) {
+      created.clear();
+    }
+  }
+
+  public List<E> getDeletedUnmodifiable() {
+    return deleted != null? Collections.unmodifiableList(deleted)
+        : Collections.emptyList();
+  }
+
+  public boolean containsDeleted(final K key) {
+    if (deleted != null) {
+      return search(deleted, key) >= 0;
+    }
+    return false;
   }
 
-  public int searchIndex(final ListType type, final K name) {
-    return search(getList(type), name);
+  public boolean containsDeleted(final E element) {
+    return getDeleted(element.getKey()) == element;
   }
 
   /**
    * @return null if the element is not found;
-   *         otherwise, return the element in the created/deleted list.
+   *         otherwise, return the element in the deleted list.
    */
-  public E search(final ListType type, final K name) {
-    final List<E> list = getList(type); 
-    final int c = search(list, name);
-    return c < 0 ? null : list.get(c);
+  public E getDeleted(final K key) {
+    if (deleted != null) {
+      final int c = search(deleted, key);
+      if (c >= 0) {
+        return deleted.get(c);
+      }
+    }
+    return null;
   }
-  
+
+  public boolean removeDeleted(final E element) {
+    if (deleted != null) {
+      final int i = search(deleted, element.getKey());
+      if (i >= 0 && deleted.get(i) == element) {
+        deleted.remove(i);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public void clearDeleted() {
+    if (deleted != null) {
+      deleted.clear();
+    }
+  }
+
   /** @return true if no changes contained in the diff */
   public boolean isEmpty() {
     return (created == null || created.isEmpty())
@@ -183,34 +226,44 @@ public class Diff<K, E extends Diff.Element<K>> {
   }
   
   /**
-   * Insert the given element to the created/deleted list.
+   * Add the given element to the created list,
+   * provided the element does not exist, i.e. i < 0.
+   *
    * @param i the insertion point defined
    *          in {@link Collections#binarySearch(List, Object)}
+   * @throws AssertionError if i >= 0.
    */
-  private void insert(final ListType type, final E element, final int i) {
-    List<E> list = type == ListType.CREATED? created: deleted; 
+  private void addCreated(final E element, final int i) {
     if (i >= 0) {
       throw new AssertionError("Element already exists: element=" + element
-          + ", " + type + "=" + list);
+          + ", created=" + created);
     }
-    if (list == null) {
-      list = new ArrayList<E>(DEFAULT_ARRAY_INITIAL_CAPACITY);
-      if (type == ListType.CREATED) {
-        created = list;
-      } else if (type == ListType.DELETED){
-        deleted = list;
-      }
+    if (created == null) {
+      created = new ArrayList<>(DEFAULT_ARRAY_INITIAL_CAPACITY);
+    }
+    created.add(-i - 1, element);
+  }
+
+  /** Similar to {@link #addCreated(Element, int)} but for the deleted list. */
+  private void addDeleted(final E element, final int i) {
+    if (i >= 0) {
+      throw new AssertionError("Element already exists: element=" + element
+          + ", deleted=" + deleted);
     }
-    list.add(-i - 1, element);
+    if (deleted == null) {
+      deleted = new ArrayList<>(DEFAULT_ARRAY_INITIAL_CAPACITY);
+    }
+    deleted.add(-i - 1, element);
   }
 
+
   /**
    * Create an element in current state.
    * @return the c-list insertion point for undo.
    */
   public int create(final E element) {
     final int c = search(created, element.getKey());
-    insert(ListType.CREATED, element, c);
+    addCreated(element, c);
     return c;
   }
 
@@ -236,7 +289,7 @@ public class Diff<K, E extends Diff.Element<K>> {
     } else {
       // not in c-list, it must be in previous
       d = search(deleted, element.getKey());
-      insert(ListType.DELETED, element, d);
+      addDeleted(element, d);
     }
     return new UndoInfo<E>(c, previous, d);
   }
@@ -277,8 +330,8 @@ public class Diff<K, E extends Diff.Element<K>> {
       d = search(deleted, oldElement.getKey());
       if (d < 0) {
         // Case 2.3: neither in c-list nor d-list
-        insert(ListType.CREATED, newElement, c);
-        insert(ListType.DELETED, oldElement, d);
+        addCreated(newElement, c);
+        addDeleted(oldElement, d);
       }
     }
     return new UndoInfo<E>(c, previous, d);
@@ -348,7 +401,7 @@ public class Diff<K, E extends Diff.Element<K>> {
    */
   public List<E> apply2Previous(final List<E> previous) {
     return apply2Previous(previous,
-        getList(ListType.CREATED), getList(ListType.DELETED));
+        getCreatedUnmodifiable(), getDeletedUnmodifiable());
   }
 
   private static <K, E extends Diff.Element<K>> List<E> apply2Previous(
@@ -408,7 +461,7 @@ public class Diff<K, E extends Diff.Element<K>> {
    */
   public List<E> apply2Current(final List<E> current) {
     return apply2Previous(current,
-        getList(ListType.DELETED), getList(ListType.CREATED));
+        getDeletedUnmodifiable(), getCreatedUnmodifiable());
   }
   
   /**
@@ -443,8 +496,10 @@ public class Diff<K, E extends Diff.Element<K>> {
    */
   public void combinePosterior(final Diff<K, E> posterior,
       final Processor<E> deletedProcesser) {
-    final Iterator<E> createdIterator = posterior.getList(ListType.CREATED).iterator();
-    final Iterator<E> deletedIterator = posterior.getList(ListType.DELETED).iterator();
+    final Iterator<E> createdIterator
+        = posterior.getCreatedUnmodifiable().iterator();
+    final Iterator<E> deletedIterator
+        = posterior.getDeletedUnmodifiable().iterator();
 
     E c = createdIterator.hasNext()? createdIterator.next(): null;
     E d = deletedIterator.hasNext()? deletedIterator.next(): null;
@@ -479,7 +534,7 @@ public class Diff<K, E extends Diff.Element<K>> {
   @Override
   public String toString() {
     return getClass().getSimpleName()
-        +  "{created=" + getList(ListType.CREATED)
-        + ", deleted=" + getList(ListType.DELETED) + "}";
+        +  "{created=" + getCreatedUnmodifiable()
+        + ", deleted=" + getDeletedUnmodifiable() + "}";
   }
 }

+ 23 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -134,6 +134,9 @@ public class JsonUtil {
     }
     if (status.isErasureCoded()) {
       m.put("ecBit", true);
+      if (status.getErasureCodingPolicy() != null) {
+        m.put("ecPolicy", status.getErasureCodingPolicy().getName());
+      }
     }
     if (status.isSnapshotEnabled()) {
       m.put("snapshotEnabled", status.isSnapshotEnabled());
@@ -527,4 +530,24 @@ public class JsonUtil {
     }
     return m;
   }
+
+  public static String toJsonString(
+      SnapshottableDirectoryStatus[] snapshottableDirectoryList) {
+    Object[] a = new Object[snapshottableDirectoryList.length];
+    for (int i = 0; i < snapshottableDirectoryList.length; i++) {
+      a[i] = toJsonMap(snapshottableDirectoryList[i]);
+    }
+    return toJsonString("SnapshottableDirectoryList", a);
+  }
+
+  private static Object toJsonMap(
+      SnapshottableDirectoryStatus snapshottableDirectoryStatus) {
+    final Map<String, Object> m = new TreeMap<String, Object>();
+    m.put("snapshotNumber", snapshottableDirectoryStatus.getSnapshotNumber());
+    m.put("snapshotQuota", snapshottableDirectoryStatus.getSnapshotQuota());
+    m.put("parentFullPath", DFSUtilClient
+        .bytes2String(snapshottableDirectoryStatus.getParentFullPath()));
+    m.put("dirStatus", toJsonMap(snapshottableDirectoryStatus.getDirStatus()));
+    return m;
+  }
 }

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

@@ -4651,9 +4651,9 @@
     <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
+      Maximum amount of time 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>

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html

@@ -43,6 +43,9 @@
       <ul class="dropdown-menu">
         <li><a href="explorer.html">Browse the file system</a></li>
         <li><a href="logs">Logs</a></li>
+        <li><a href="jmx">Metrics</a></li>
+        <li><a href="conf">Configuration</a></li>
+        <li><a href="stacks">Process Thread Dump</a></li>
       </ul>
     </li>
   </ul>

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html

@@ -44,6 +44,9 @@
           <ul class="dropdown-menu">
             <li><a href="#">Browse the file system</a></li>
             <li><a href="logs">Logs</a></li>
+            <li><a href="jmx">Metrics</a></li>
+            <li><a href="conf">Configuration</a></li>
+            <li><a href="stacks">Process Thread Dump</a></li>
           </ul>
         </li>
       </ul>

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md

@@ -79,6 +79,10 @@ Execute command takes a plan command executes it against the datanode that plan
 
 This executes the plan by reading datanode’s address from the plan file.
 
+| COMMAND\_OPTION    | Description |
+|:---- |:---- |
+| `-skipDateCheck` |  Skip date check and force execute the plan.|
+
 ### Query
 
 Query command gets the current status of the diskbalancer from a datanode.
@@ -122,7 +126,7 @@ There is a set of diskbalancer settings that can be controlled via hdfs-site.xml
 |`dfs.disk.balancer.max.disk.errors`| sets the value of maximum number of errors we can ignore for a specific move between two disks before it is abandoned. For example, if a plan has 3 pair of disks to copy between , and the first disk set encounters more than 5 errors, then we abandon the first copy and start the second copy in the plan. The default value of max errors is set to 5.|
 |`dfs.disk.balancer.block.tolerance.percent`| The tolerance percent specifies when we have reached a good enough value for any copy step. For example, if you specify 10% then getting close to 10% of the target value is good enough.|
 |`dfs.disk.balancer.plan.threshold.percent`| The percentage threshold value for volume Data Density in a plan. If the absolute value of volume Data Density which is out of threshold value in a node, it means that the volumes corresponding to the disks should do the balancing in the plan. The default value is 10.|
-
+|`dfs.disk.balancer.plan.valid.interval`| Maximum amount of time disk balancer plan is valid. Supports the following suffixes (case insensitive): ms(millis), s(sec), m(min), h(hour), d(day) to specify the time (such as 2s, 2m, 1h, etc.). If no suffix is specified then milliseconds is assumed. Default value is 1d|
  Debugging
 ---------
 

+ 7 - 5
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md

@@ -29,7 +29,9 @@ Architecture
 ------------
 
 A natural extension to this partitioned federation is to add a layer of software responsible for federating the namespaces.
-This extra layer allows users to access any subcluster transparently, lets subclusters manage their own block pools independently, and supports rebalancing of data across subclusters.
+This extra layer allows users to access any subcluster transparently, lets subclusters manage their own block pools independently, and will support rebalancing of data across subclusters later
+(see more info in [HDFS-13123](https://issues.apache.org/jira/browse/HDFS-13123)). The subclusters in RBF are not required to be the independent HDFS clusters, a normal federation cluster
+(with multiple block pools) or a mixed cluster with federation and independent cluster is also allowed.
 To accomplish these goals, the federation layer directs block accesses to the proper subcluster, maintains the state of the namespaces, and provides mechanisms for data rebalancing.
 This layer must be scalable, highly available, and fault tolerant.
 
@@ -324,8 +326,8 @@ The connection to the State Store and the internal caching at the Router.
 | Property | Default | Description|
 |:---- |:---- |:---- |
 | dfs.federation.router.store.enable | `true` | If `true`, the Router connects to the State Store. |
-| dfs.federation.router.store.serializer | `StateStoreSerializerPBImpl` | Class to serialize State Store records. |
-| dfs.federation.router.store.driver.class | `StateStoreZooKeeperImpl` | Class to implement the State Store. |
+| dfs.federation.router.store.serializer | `org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreSerializerPBImpl` | Class to serialize State Store records. |
+| dfs.federation.router.store.driver.class | `org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreZooKeeperImpl` | Class to implement the State Store. |
 | dfs.federation.router.store.connection.test | 60000 | How often to check for the connection to the State Store in milliseconds. |
 | dfs.federation.router.cache.ttl | 60000 | How often to refresh the State Store caches in milliseconds. |
 | dfs.federation.router.store.membership.expiration | 300000 | Expiration time in milliseconds for a membership record. |
@@ -336,8 +338,8 @@ Forwarding client requests to the right subcluster.
 
 | Property | Default | Description|
 |:---- |:---- |:---- |
-| dfs.federation.router.file.resolver.client.class | MountTableResolver | Class to resolve files to subclusters. |
-| dfs.federation.router.namenode.resolver.client.class | MembershipNamenodeResolver | Class to resolve the namenode for a subcluster. |
+| dfs.federation.router.file.resolver.client.class | `org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver` | Class to resolve files to subclusters. |
+| dfs.federation.router.namenode.resolver.client.class | `org.apache.hadoop.hdfs.server.federation.resolver.MembershipNamenodeResolver` | Class to resolve the namenode for a subcluster. |
 
 ### Namenode monitoring
 

+ 139 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md

@@ -180,6 +180,145 @@ Recall that one cannot rename files or directories across namenodes or clusters
 
 This will NOT work in the new world if `/user` and `/data` are actually stored on different namenodes within a cluster.
 
+Multi-Filesystem I/0 with Nfly Mount Points
+-----------------
+
+HDFS and other distributed filesystems provide data resilience via some sort of
+redundancy such as block replication or more sophisticated distributed encoding.
+However, modern setups may be comprised of multiple Hadoop clusters, enterprise
+filers, hosted on and off premise. Nfly mount points make it possible for a
+single logical file to be synchronously replicated by multiple filesystems.
+It's designed for a relatively small files up to a gigabyte. In general it's a
+function of a single core/single network link performance since the logic
+resides in a single client JVM using ViewFs such as FsShell or a
+MapReduce task.
+
+### Basic Configuration
+
+Consider the following example to understand the basic configuration of Nfly.
+Suppose we want to keep the directory `ads` replicated on three filesystems
+represented by URIs: `uri1`, `uri2` and `uri3`.
+
+```xml
+  <property>
+    <name>fs.viewfs.mounttable.global.linkNfly../ads</name>
+    <value>uri1,uri2,uri3</value>
+  </property>
+```
+Note 2 consecutive `..` in the property name. They arise because of empty
+settings for advanced tweaking of the mount point which we will show in
+subsequent sections. The property value is a comma-separated list of URIs.
+
+URIs may point to different clusters in different regions
+`hdfs://datacenter-east/ads`, `s3a://models-us-west/ads`, `hdfs://datacenter-west/ads`
+or in the simplest case to different directories under the same filesystem,
+e.g., `file:/tmp/ads1`, `file:/tmp/ads2`, `file:/tmp/ads3`
+
+All *modifications* performed under the global path `viewfs://global/ads` are
+propagated to all destination URIs if the underlying system is available.
+
+For instance if we create a file via hadoop shell
+```bash
+hadoop fs -touchz viewfs://global/ads/z1
+```
+
+We will find it via local filesystem in the latter configuration
+```bash
+ls -al /tmp/ads*/z1
+-rw-r--r--  1 user  wheel  0 Mar 11 12:17 /tmp/ads1/z1
+-rw-r--r--  1 user  wheel  0 Mar 11 12:17 /tmp/ads2/z1
+-rw-r--r--  1 user  wheel  0 Mar 11 12:17 /tmp/ads3/z1
+```
+
+A read from the global path is processed by the first filesystem that does not
+result in an exception. The order in which filesystems are accessed depends on
+whether they are available at this moment or and whether a topological order
+exists.
+
+### Advanced Configuration
+
+Mount points `linkNfly` can be further configured using parameters passed as a
+comma-separated list of key=value pairs. Following parameters are currently
+supported.
+
+`minReplication=int` determines the minimum number of destinations that have to
+process a write modification without exceptions, if below nfly write is failed.
+It is an configuration error to have minReplication higher than the number of
+target URIs. The default is 2.
+
+If minReplication is lower than the number of target URIs we may have some
+target URIs without latest writes. It can be compensated by employing more
+expensive read operations controlled by the following settings
+
+`readMostRecent=boolean` if set to `true` causes Nfly client to check the path
+under all target URIs instead of just the first one based on the topology order.
+Among all available at the moment the one with the most recent modification time
+is processed.
+
+`repairOnRead=boolean` if set to `true` causes Nfly to copy most recent replica
+to stale targets such that subsequent reads can be done cheaply again from the
+closest replica.
+
+### Network Topology
+
+Nfly seeks to satisfy reads from the "closest" target URI.
+
+To this end, Nfly extends the notion of
+<a href="hadoop-project-dist/hadoop-common/RackAwareness.html">Rack Awareness</a>
+to the authorities of target URIs.
+
+Nfly applies NetworkTopology to resolve authorities of the URIs. Most commonly
+a script based mapping is used in a heterogeneous setup. We could have a script
+providing the following topology mapping
+
+| URI                           | Topology                 |
+|-------------------------------|------------------------- |
+| `hdfs://datacenter-east/ads`  | /us-east/onpremise-hdfs  |
+| `s3a://models-us-west/ads`    | /us-west/aws             |
+| `hdfs://datacenter-west/ads`  | /us-west/onpremise-hdfs  |
+
+
+If a target URI does not have the authority part as in `file:/` Nfly injects
+client's local node name.
+
+### Example Nfly Configuration
+
+```xml
+  <property>
+    <name>fs.viewfs.mounttable.global.linkNfly.minReplication=3,readMostRecent=true,repairOnRead=false./ads</name>
+    <value>hdfs://datacenter-east/ads,hdfs://datacenter-west/ads,s3a://models-us-west/ads,file:/tmp/ads</value>
+  </property>
+```
+
+### How Nfly File Creation works
+
+```java
+FileSystem fs = FileSystem.get("viewfs://global/", ...);
+FSDataOutputStream out = fs.create("viewfs://global/ads/f1");
+out.write(...);
+out.close();
+```
+The code above would result in the following execution.
+
+1. create an invisible file `_nfly_tmp_f1` under each target URI i.e.,
+`hdfs://datacenter-east/ads/_nfly_tmp_f1`, `hdfs://datacenter-west/ads/_nfly_tmp_f1`, etc.
+This is done by calling `create` on underlying filesystems and returns a
+`FSDataOutputStream` object `out` that wraps all four output streams.
+
+2. Thus each subsequent write on `out` can be forwarded to each wrapped stream.
+
+3. On `out.close` all streams are closed, and the files are renamed from `_nfly_tmp_f1` to `f1`.
+All files receive the same *modification time* corresponding to the client
+system time as of beginning of this step.
+
+4. If at least `minReplication` destinations have gone through steps 1-3 without
+failures Nfly considers the transaction logically committed; Otherwise it tries
+to clean up the temporary files in a best-effort attempt.
+
+Note that because 4 is a best-effort step and the client JVM could crash and never
+resume its work, it's a good idea to provision some sort of cron job to purge such
+`_nfly_tmp` files.
+
 ### FAQ
 
 1.  **As I move from non-federated world to the federated world, I will have to keep track of namenodes for different volumes; how do I do that?**

+ 92 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md

@@ -50,6 +50,7 @@ The HTTP REST API supports the complete [FileSystem](../../api/org/apache/hadoop
     * [`CHECKACCESS`](#Check_access) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).access)
     * [`GETALLSTORAGEPOLICY`](#Get_all_Storage_Policies) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getAllStoragePolicies)
     * [`GETSTORAGEPOLICY`](#Get_Storage_Policy) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getStoragePolicy)
+    * [`GETSNAPSHOTDIFF`](#Get_Snapshot_Diff)
 *   HTTP PUT
     * [`CREATE`](#Create_and_Write_to_a_File) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).create)
     * [`MKDIRS`](#Make_a_Directory) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).mkdirs)
@@ -1266,6 +1267,21 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).deleteSna
 
 See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).renameSnapshot
 
+### Get Snapshot Diff
+
+* Submit a HTTP GET request.
+
+        curl -i GET "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETSNAPSHOTDIFF
+                           &oldsnapshotname=<SNAPSHOTNAME>&snapshotname=<SNAPSHOTNAME>"
+
+    The client receives a response with a [`SnapshotDiffReport` JSON object](#SnapshotDiffReport_JSON_Schema):
+
+        HTTP/1.1 200 OK
+        Content-Type: application/json
+        Transfer-Encoding: chunked
+
+        {"SnapshotDiffReport":{"diffList":[],"fromSnapshot":"s3","snapshotRoot":"/foo","toSnapshot":"s4"}}
+
 Delegation Token Operations
 ---------------------------
 
@@ -2043,6 +2059,82 @@ A `BlockStoragePolicies` JSON object represents an array of `BlockStoragePolicy`
 }
 ```
 
+### SnapshotDiffReport JSON Schema
+
+```json
+{
+  "name": "SnapshotDiffReport",
+  "type": "object",
+  "properties":
+  {
+    "SnapshotDiffReport":
+    {
+      "type"        : "object",
+      "properties"  :
+      {
+        "diffList":
+        {
+          "description": "An array of DiffReportEntry",
+          "type"        : "array",
+          "items"       : diffReportEntries,
+          "required"    : true
+        },
+        "fromSnapshot":
+        {
+          "description": "Source snapshot",
+          "type"        : "string",
+          "required"    : true
+        },
+        "snapshotRoot":
+        {
+          "description" : "String representation of snapshot root path",
+          "type"        : "string",
+          "required"    : true
+        },
+        "toSnapshot":
+        {
+          "description" : "Destination snapshot",
+          "type"        : "string",
+          "required"    : true
+        }
+      }
+    }
+  }
+}
+```
+
+#### DiffReport Entries
+
+JavaScript syntax is used to define `diffReportEntries` so that it can be referred in `SnapshotDiffReport` JSON schema.
+
+```javascript
+var diffReportEntries =
+{
+  "type": "object",
+  "properties":
+  {
+    "sourcePath":
+    {
+      "description" : "Source path name relative to snapshot root",
+      "type"        : "string",
+      "required"    : true
+    },
+    "targetPath":
+    {
+      "description" : "Target path relative to snapshot root used for renames",
+      "type"        : "string",
+      "required"    : true
+    },
+    "type":
+    {
+      "description" : "Type of diff report entry",
+      "enum"        : ["CREATE", "MODIFY", "DELETE", "RENAME"],
+      "required"    : true
+    }
+  }
+}
+```
+
 HTTP Query Parameter Dictionary
 -------------------------------
 

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java

@@ -46,6 +46,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
@@ -591,4 +592,18 @@ public class StripedFileTestUtil {
             .getPolicies();
     return policies.get(1 + rand.nextInt(policies.size() - 1));
   }
+
+  /**
+   * Get all Erasure Coding Policies for Parameterized tests.
+   * @return Collection<Object[]>
+   */
+  public static Collection<Object[]> getECPolicies() {
+    ArrayList<Object[]> params = new ArrayList<>();
+    List<ErasureCodingPolicy> policies =
+        SystemErasureCodingPolicies.getPolicies();
+    for (ErasureCodingPolicy policy: policies) {
+      params.add(new Object[]{policy});
+    }
+    return params;
+  }
 }

+ 10 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithDefaultECPolicy.java

@@ -34,7 +34,10 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
 
-public class TestFileStatusWithECPolicy {
+/**
+ * This test ensures the statuses of EC files with the default policy.
+ */
+public class TestFileStatusWithDefaultECPolicy {
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
   private DFSClient client;
@@ -50,8 +53,7 @@ public class TestFileStatusWithECPolicy {
     cluster.waitActive();
     fs = cluster.getFileSystem();
     client = fs.getClient();
-    fs.enableErasureCodingPolicy(
-        StripedFileTestUtil.getDefaultECPolicy().getName());
+    fs.enableErasureCodingPolicy(getEcPolicy().getName());
   }
 
   @After
@@ -62,6 +64,10 @@ public class TestFileStatusWithECPolicy {
     }
   }
 
+  public ErasureCodingPolicy getEcPolicy() {
+    return StripedFileTestUtil.getDefaultECPolicy();
+  }
+
   @Test
   public void testFileStatusWithECPolicy() throws Exception {
     // test directory doesn't have an EC policy
@@ -76,8 +82,7 @@ public class TestFileStatusWithECPolicy {
     ContractTestUtils.assertNotErasureCoded(fs, file);
     fs.delete(file, true);
 
-    final ErasureCodingPolicy ecPolicy1 =
-        StripedFileTestUtil.getDefaultECPolicy();
+    final ErasureCodingPolicy ecPolicy1 = getEcPolicy();
     // set EC policy on dir
     fs.setErasureCodingPolicy(dir, ecPolicy1.getName());
     ContractTestUtils.assertErasureCoded(fs, dir);

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithRandomECPolicy.java

@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This test extends TestFileStatusWithDefaultECPolicy to use a random
+ * (non-default) EC policy.
+ */
+public class TestFileStatusWithRandomECPolicy extends
+    TestFileStatusWithDefaultECPolicy {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      TestFileStatusWithRandomECPolicy.class);
+
+  private ErasureCodingPolicy ecPolicy;
+
+  public TestFileStatusWithRandomECPolicy() {
+    // If you want to debug this test with a specific ec policy, please use
+    // SystemErasureCodingPolicies class.
+    // e.g. ecPolicy = SystemErasureCodingPolicies.getByID(RS_3_2_POLICY_ID);
+    ecPolicy = StripedFileTestUtil.getRandomNonDefaultECPolicy();
+    LOG.info("run {} with {}.",
+        TestFileStatusWithRandomECPolicy.class
+            .getSuperclass().getSimpleName(), ecPolicy.getName());
+  }
+
+  @Override
+  public ErasureCodingPolicy getEcPolicy() {
+    return ecPolicy;
+  }
+}

+ 29 - 19
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeTrue;
 
 import java.io.File;
 import java.io.IOException;
@@ -67,14 +68,13 @@ import org.junit.Test;
 public class TestReconstructStripedFile {
   public static final Log LOG = LogFactory.getLog(TestReconstructStripedFile.class);
 
-  private final ErasureCodingPolicy ecPolicy =
-      StripedFileTestUtil.getDefaultECPolicy();
-  private final int dataBlkNum = ecPolicy.getNumDataUnits();
-  private final int parityBlkNum = ecPolicy.getNumParityUnits();
-  private final int cellSize = ecPolicy.getCellSize();
-  private final int blockSize = cellSize * 3;
-  private final int groupSize = dataBlkNum + parityBlkNum;
-  private final int dnNum = groupSize + parityBlkNum;
+  private ErasureCodingPolicy ecPolicy;
+  private int dataBlkNum;
+  private int parityBlkNum;
+  private int cellSize;
+  private int blockSize;
+  private int groupSize;
+  private int dnNum;
 
   static {
     GenericTestUtils.setLogLevel(DFSClient.LOG, Level.ALL);
@@ -95,8 +95,20 @@ public class TestReconstructStripedFile {
   private Map<DatanodeID, Integer> dnMap = new HashMap<>();
   private final Random random = new Random();
 
+  public ErasureCodingPolicy getEcPolicy() {
+    return StripedFileTestUtil.getDefaultECPolicy();
+  }
+
   @Before
   public void setup() throws IOException {
+    ecPolicy = getEcPolicy();
+    dataBlkNum = ecPolicy.getNumDataUnits();
+    parityBlkNum = ecPolicy.getNumParityUnits();
+    cellSize = ecPolicy.getCellSize();
+    blockSize = cellSize * 3;
+    groupSize = dataBlkNum + parityBlkNum;
+    dnNum = groupSize + parityBlkNum;
+
     conf = new Configuration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     conf.setInt(
@@ -114,10 +126,8 @@ public class TestReconstructStripedFile {
     cluster.waitActive();
 
     fs = cluster.getFileSystem();
-    fs.enableErasureCodingPolicy(
-        StripedFileTestUtil.getDefaultECPolicy().getName());
-    fs.getClient().setErasureCodingPolicy("/",
-        StripedFileTestUtil.getDefaultECPolicy().getName());
+    fs.enableErasureCodingPolicy(ecPolicy.getName());
+    fs.getClient().setErasureCodingPolicy("/", ecPolicy.getName());
 
     List<DataNode> datanodes = cluster.getDataNodes();
     for (int i = 0; i < dnNum; i++) {
@@ -432,7 +442,7 @@ public class TestReconstructStripedFile {
 
     BlockECReconstructionInfo invalidECInfo = new BlockECReconstructionInfo(
         new ExtendedBlock("bp-id", 123456), dataDNs, dnStorageInfo, liveIndices,
-        StripedFileTestUtil.getDefaultECPolicy());
+        ecPolicy);
     List<BlockECReconstructionInfo> ecTasks = new ArrayList<>();
     ecTasks.add(invalidECInfo);
     dataNode.getErasureCodingWorker().processErasureCodingTasks(ecTasks);
@@ -461,7 +471,8 @@ public class TestReconstructStripedFile {
         .numDataNodes(numDataNodes).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
-    ErasureCodingPolicy policy = StripedFileTestUtil.getDefaultECPolicy();
+    ErasureCodingPolicy policy = ecPolicy;
+    fs.enableErasureCodingPolicy(policy.getName());
     fs.getClient().setErasureCodingPolicy("/", policy.getName());
 
     final int fileLen = cellSize * ecPolicy.getNumDataUnits();
@@ -470,7 +481,8 @@ public class TestReconstructStripedFile {
     }
 
     // Inject data-loss by tear down desired number of DataNodes.
-    assertTrue(policy.getNumParityUnits() >= deadDN);
+    assumeTrue("Ignore case where num dead DNs > num parity units",
+        policy.getNumParityUnits() >= deadDN);
     List<DataNode> dataNodes = new ArrayList<>(cluster.getDataNodes());
     Collections.shuffle(dataNodes);
     for (DataNode dn : dataNodes.subList(0, deadDN)) {
@@ -516,10 +528,8 @@ public class TestReconstructStripedFile {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(dnNum).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
-    fs.enableErasureCodingPolicy(
-        StripedFileTestUtil.getDefaultECPolicy().getName());
-    fs.getClient().setErasureCodingPolicy("/",
-        StripedFileTestUtil.getDefaultECPolicy().getName());
+    fs.enableErasureCodingPolicy(ecPolicy.getName());
+    fs.getClient().setErasureCodingPolicy("/", ecPolicy.getName());
 
     final int fileLen = cellSize * ecPolicy.getNumDataUnits() * 2;
     writeFile(fs, "/ec-xmits-weight", fileLen);

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFileWithRandomECPolicy.java

@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This test extends TestReconstructStripedFile to use a random
+ * (non-default) EC policy.
+ */
+public class TestReconstructStripedFileWithRandomECPolicy extends
+    TestReconstructStripedFile {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      TestReconstructStripedFileWithRandomECPolicy.class);
+
+  private ErasureCodingPolicy ecPolicy;
+
+  public TestReconstructStripedFileWithRandomECPolicy() {
+    // If you want to debug this test with a specific ec policy, please use
+    // SystemErasureCodingPolicies class.
+    // e.g. ecPolicy = SystemErasureCodingPolicies.getByID(RS_3_2_POLICY_ID);
+    ecPolicy = StripedFileTestUtil.getRandomNonDefaultECPolicy();
+    LOG.info("run {} with {}.",
+        TestReconstructStripedFileWithRandomECPolicy.class
+            .getSuperclass().getSimpleName(), ecPolicy.getName());
+  }
+
+  @Override
+  public ErasureCodingPolicy getEcPolicy() {
+    return ecPolicy;
+  }
+}

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java

@@ -416,13 +416,19 @@ public class TestBalancerWithMultipleNameNodes {
     }
 
     conf.set(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, "0.0f");
+    // Adjust the capacity of each DN since it will redistribute blocks
+    // nNameNodes times in the following operations.
+    long[] newCapacities = new long[nDataNodes];
+    for (int i = 0; i < nDataNodes; i++) {
+      newCapacities[i] = capacities[i] * nNameNodes;
+    }
     {
       LOG.info("UNEVEN 10");
       final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
           .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(nNameNodes))
           .numDataNodes(nDataNodes)
           .racks(racks)
-          .simulatedCapacities(capacities)
+          .simulatedCapacities(newCapacities)
           .format(false)
           .build();
       LOG.info("UNEVEN 11");
@@ -450,7 +456,7 @@ public class TestBalancerWithMultipleNameNodes {
           LOG.info("UNEVEN 13: n=" + n);
         }
     
-        final long totalCapacity = TestBalancer.sum(capacities);
+        final long totalCapacity = TestBalancer.sum(newCapacities);
         final long totalUsed = nNameNodes*usedSpacePerNN;
         LOG.info("UNEVEN 14");
         runBalancer(s, totalUsed, totalCapacity);

+ 23 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java

@@ -25,29 +25,42 @@ import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.mockito.internal.util.reflection.Whitebox;
 
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.ByteArrayOutputStream;
 import java.nio.ByteBuffer;
+import java.util.Collection;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 /**
- * Test {@link BlockInfoStriped}
+ * Test {@link BlockInfoStriped}.
  */
+@RunWith(Parameterized.class)
 public class TestBlockInfoStriped {
   private static final long BASE_ID = -1600;
   private final Block baseBlock = new Block(BASE_ID);
-  private final ErasureCodingPolicy testECPolicy
-      = StripedFileTestUtil.getDefaultECPolicy();
-  private final int totalBlocks = testECPolicy.getNumDataUnits() +
-      testECPolicy.getNumParityUnits();
-  private final BlockInfoStriped info = new BlockInfoStriped(baseBlock,
-      testECPolicy);
+  private final ErasureCodingPolicy testECPolicy;
+  private final int totalBlocks;
+  private final BlockInfoStriped info;
+
+  public TestBlockInfoStriped(ErasureCodingPolicy policy) {
+    testECPolicy = policy;
+    totalBlocks = testECPolicy.getNumDataUnits()
+        + testECPolicy.getNumParityUnits();
+    info = new BlockInfoStriped(baseBlock, testECPolicy);
+  }
+
+  @Parameterized.Parameters(name = "{index}: {0}")
+  public static Collection<Object[]> policies() {
+    return StripedFileTestUtil.getECPolicies();
+  }
 
   private Block[] createReportedBlocks(int num) {
     Block[] blocks = new Block[num];
@@ -61,7 +74,7 @@ public class TestBlockInfoStriped {
   public Timeout globalTimeout = new Timeout(300000);
 
   /**
-   * Test adding storage and reported block
+   * Test adding storage and reported block.
    */
   @Test
   public void testAddStorage() {
@@ -108,8 +121,8 @@ public class TestBlockInfoStriped {
     }
 
     // the same block is reported from another storage
-    DatanodeStorageInfo[] storageInfos2 = DFSTestUtil.createDatanodeStorageInfos(
-        totalBlocks * 2);
+    DatanodeStorageInfo[] storageInfos2 =
+        DFSTestUtil.createDatanodeStorageInfos(totalBlocks * 2);
     // only add the second half of info2
     for (i = totalBlocks; i < storageInfos2.length; i++) {
       info.addStorage(storageInfos2[i], blocks[i % totalBlocks]);

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -160,6 +161,10 @@ public class TestBlockStatsMXBean {
 
   @Test
   public void testStorageTypeStatsWhenStorageFailed() throws Exception {
+    // The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
+    // volume failures which is currently not supported on Windows.
+    assumeNotWindows();
+
     DFSTestUtil.createFile(cluster.getFileSystem(),
         new Path("/blockStatsFile1"), 1024, (short) 1, 0L);
     Map<StorageType, StorageTypeStats> storageTypeStatsMap = cluster

+ 17 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java

@@ -18,22 +18,37 @@
 
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import java.util.Collection;
 import java.util.Iterator;
 
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 
+/**
+ * Test {@link LowRedundancyBlocks}.
+ */
+@RunWith(Parameterized.class)
 public class TestLowRedundancyBlockQueues {
 
-  private final ErasureCodingPolicy ecPolicy =
-      StripedFileTestUtil.getDefaultECPolicy();
+  private final ErasureCodingPolicy ecPolicy;
+
+  public TestLowRedundancyBlockQueues(ErasureCodingPolicy policy) {
+    ecPolicy = policy;
+  }
+
+  @Parameterized.Parameters(name = "{index}: {0}")
+  public static Collection<Object[]> policies() {
+    return StripedFileTestUtil.getECPolicies();
+  }
 
   private BlockInfo genBlockInfo(long id) {
     return new BlockInfoContiguous(new Block(id), (short) 3);

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReconstruction.java

@@ -456,14 +456,14 @@ public class TestPendingReconstruction {
             "STORAGE_ID", "TEST");
         bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[1],
             "STORAGE_ID", "TEST");
+        BlockManagerTestUtil.computeAllPendingWork(bm);
+        BlockManagerTestUtil.updateState(bm);
+        assertEquals(bm.getPendingReconstructionBlocksCount(), 1L);
+        BlockInfo storedBlock = bm.getStoredBlock(block.getBlock().getLocalBlock());
+        assertEquals(bm.pendingReconstruction.getNumReplicas(storedBlock), 2);
       } finally {
         cluster.getNamesystem().writeUnlock();
       }
-      BlockManagerTestUtil.computeAllPendingWork(bm);
-      BlockManagerTestUtil.updateState(bm);
-      assertEquals(bm.getPendingReconstructionBlocksCount(), 1L);
-      BlockInfo storedBlock = bm.getStoredBlock(block.getBlock().getLocalBlock());
-      assertEquals(bm.pendingReconstruction.getNumReplicas(storedBlock), 2);
 
       // 4. delete the file
       fs.delete(filePath, true);

+ 25 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java

@@ -152,7 +152,8 @@ public class TestDataNodeVolumeFailure {
   @Test(timeout = 120000)
   public void testVolumeFailure() throws Exception {
     System.out.println("Data dir: is " +  dataDir.getPath());
-
+   
+    
     // Data dir structure is dataDir/data[1-4]/[current,tmp...]
     // data1,2 is for datanode 1, data2,3 - datanode2 
     String filename = "/test.txt";
@@ -167,7 +168,7 @@ public class TestDataNodeVolumeFailure {
    
     // fail the volume
     // delete/make non-writable one of the directories (failed volume)
-    data_fail = cluster.getInstanceStorageDir(1, 0);
+    data_fail = new File(dataDir, "data3");
     failedDir = MiniDFSCluster.getFinalizedDir(data_fail,
         cluster.getNamesystem().getBlockPoolId());
     if (failedDir.exists() &&
@@ -234,7 +235,7 @@ public class TestDataNodeVolumeFailure {
     DFSTestUtil.createFile(fs, file1, 1024, (short) 2, 1L);
     DFSTestUtil.waitReplication(fs, file1, (short) 2);
 
-    File dn0Vol1 = cluster.getInstanceStorageDir(0, 0);
+    File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
     DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
     DataNode dn0 = cluster.getDataNodes().get(0);
     DataNodeTestUtils.waitForDiskError(dn0,
@@ -292,9 +293,13 @@ public class TestDataNodeVolumeFailure {
   @Test(timeout=10000)
   public void testDataNodeShutdownAfterNumFailedVolumeExceedsTolerated()
       throws Exception {
+    // The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
+    // volume failures which is currently not supported on Windows.
+    assumeNotWindows();
+
     // make both data directories to fail on dn0
-    final File dn0Vol1 = cluster.getInstanceStorageDir(0, 0);
-    final File dn0Vol2 = cluster.getInstanceStorageDir(0, 1);
+    final File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
+    final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
     DataNodeTestUtils.injectDataDirFailure(dn0Vol1, dn0Vol2);
     DataNode dn0 = cluster.getDataNodes().get(0);
     DataNodeTestUtils.waitForDiskError(dn0,
@@ -313,8 +318,12 @@ public class TestDataNodeVolumeFailure {
   @Test
   public void testVolumeFailureRecoveredByHotSwappingVolume()
       throws Exception {
-    final File dn0Vol1 = cluster.getInstanceStorageDir(0, 0);
-    final File dn0Vol2 = cluster.getInstanceStorageDir(0, 1);
+    // The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
+    // volume failures which is currently not supported on Windows.
+    assumeNotWindows();
+
+    final File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
+    final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
     final DataNode dn0 = cluster.getDataNodes().get(0);
     final String oldDataDirs = dn0.getConf().get(
         DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
@@ -353,9 +362,13 @@ public class TestDataNodeVolumeFailure {
   @Test
   public void testTolerateVolumeFailuresAfterAddingMoreVolumes()
       throws Exception {
-    final File dn0Vol1 = cluster.getInstanceStorageDir(0, 0);
-    final File dn0Vol2 = cluster.getInstanceStorageDir(0, 1);
-    final File dn0VolNew = new File(cluster.getDataDirectory(), "data_new");
+    // The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
+    // volume failures which is currently not supported on Windows.
+    assumeNotWindows();
+
+    final File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
+    final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
+    final File dn0VolNew = new File(dataDir, "data_new");
     final DataNode dn0 = cluster.getDataNodes().get(0);
     final String oldDataDirs = dn0.getConf().get(
         DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
@@ -400,8 +413,8 @@ public class TestDataNodeVolumeFailure {
     DFSTestUtil.waitReplication(fs, file1, (short)3);
 
     // Fail the first volume on both datanodes
-    File dn1Vol1 = cluster.getInstanceStorageDir(0, 0);
-    File dn2Vol1 = cluster.getInstanceStorageDir(1, 0);
+    File dn1Vol1 = new File(dataDir, "data"+(2*0+1));
+    File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
     DataNodeTestUtils.injectDataDirFailure(dn1Vol1, dn2Vol1);
 
     Path file2 = new Path("/test2");

+ 69 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.federation.resolver;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
@@ -82,6 +83,7 @@ public class TestMountTableResolver {
     Configuration conf = new Configuration();
     conf.setInt(
         FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE, TEST_MAX_CACHE_SIZE);
+    conf.setStrings(DFS_ROUTER_DEFAULT_NAMESERVICE, "0");
     mountTable = new MountTableResolver(conf);
 
     // Root mount point
@@ -178,6 +180,29 @@ public class TestMountTableResolver {
     }
   }
 
+  @Test
+  public void testGetMountPoint() throws IOException {
+    // Check get the mount table entry for a path
+    MountTable mtEntry;
+    mtEntry = mountTable.getMountPoint("/");
+    assertTrue(mtEntry.getSourcePath().equals("/"));
+
+    mtEntry = mountTable.getMountPoint("/user");
+    assertTrue(mtEntry.getSourcePath().equals("/user"));
+
+    mtEntry = mountTable.getMountPoint("/user/a");
+    assertTrue(mtEntry.getSourcePath().equals("/user/a"));
+
+    mtEntry = mountTable.getMountPoint("/user/a/");
+    assertTrue(mtEntry.getSourcePath().equals("/user/a"));
+
+    mtEntry = mountTable.getMountPoint("/user/a/11");
+    assertTrue(mtEntry.getSourcePath().equals("/user/a"));
+
+    mtEntry = mountTable.getMountPoint("/user/a1");
+    assertTrue(mtEntry.getSourcePath().equals("/user"));
+  }
+
   @Test
   public void testGetMountPoints() throws IOException {
 
@@ -456,4 +481,48 @@ public class TestMountTableResolver {
     long cacheSize = mountTable.getCacheSize();
     assertTrue(cacheSize <= TEST_MAX_CACHE_SIZE);
   }
+
+  @Test
+  public void testLocationCache() throws Exception {
+    List<MountTable> entries = new ArrayList<>();
+
+    // Add entry and test location cache
+    Map<String, String> map1 = getMountTableEntry("1", "/testlocationcache");
+    MountTable entry1 = MountTable.newInstance("/testlocationcache", map1);
+    entries.add(entry1);
+
+    Map<String, String> map2 = getMountTableEntry("2",
+            "/anothertestlocationcache");
+    MountTable entry2 = MountTable.newInstance("/anothertestlocationcache",
+            map2);
+    entries.add(entry2);
+    mountTable.refreshEntries(entries);
+    assertEquals("1->/testlocationcache/",
+            mountTable.getDestinationForPath("/testlocationcache").toString());
+    assertEquals("2->/anothertestlocationcache/",
+            mountTable.getDestinationForPath("/anothertestlocationcache")
+                    .toString());
+
+    // Remove the entry1
+    entries.remove(entry1);
+    mountTable.refreshEntries(entries);
+
+    // Add the default location and test location cache
+    assertEquals("0->/testlocationcache",
+            mountTable.getDestinationForPath("/testlocationcache").toString());
+
+    // Add the entry again but mount to another ns
+    Map<String, String> map3 = getMountTableEntry("3", "/testlocationcache");
+    MountTable entry3 = MountTable.newInstance("/testlocationcache", map3);
+    entries.add(entry3);
+    mountTable.refreshEntries(entries);
+
+    // Ensure location cache update correctly
+    assertEquals("3->/testlocationcache/",
+            mountTable.getDestinationForPath("/testlocationcache").toString());
+
+    // Cleanup before exit
+    mountTable.removeEntry("/testlocationcache");
+    mountTable.removeEntry("/anothertestlocationcache");
+  }
 }

+ 157 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestConnectionManager.java

@@ -0,0 +1,157 @@
+/**
+ * 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.federation.router;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test functionalities of {@link ConnectionManager}, which manages a pool
+ * of connections to NameNodes.
+ */
+public class TestConnectionManager {
+  private Configuration conf;
+  private ConnectionManager connManager;
+  private static final String[] TEST_GROUP = new String[]{"TEST_GROUP"};
+  private static final UserGroupInformation TEST_USER1 =
+      UserGroupInformation.createUserForTesting("user1", TEST_GROUP);
+  private static final UserGroupInformation TEST_USER2 =
+      UserGroupInformation.createUserForTesting("user2", TEST_GROUP);
+  private static final UserGroupInformation TEST_USER3 =
+      UserGroupInformation.createUserForTesting("user3", TEST_GROUP);
+  private static final String TEST_NN_ADDRESS = "nn1:8080";
+
+  @Before
+  public void setup() throws Exception {
+    conf = new Configuration();
+    connManager = new ConnectionManager(conf);
+    NetUtils.addStaticResolution("nn1", "localhost");
+    NetUtils.createSocketAddrForHost("nn1", 8080);
+    connManager.start();
+  }
+
+  @After
+  public void shutdown() {
+    if (connManager != null) {
+      connManager.close();
+    }
+  }
+
+  @Test
+  public void testCleanup() throws Exception {
+    Map<ConnectionPoolId, ConnectionPool> poolMap = connManager.getPools();
+
+    ConnectionPool pool1 = new ConnectionPool(
+        conf, TEST_NN_ADDRESS, TEST_USER1, 0, 10);
+    addConnectionsToPool(pool1, 9, 4);
+    poolMap.put(new ConnectionPoolId(TEST_USER1, TEST_NN_ADDRESS), pool1);
+
+    ConnectionPool pool2 = new ConnectionPool(
+        conf, TEST_NN_ADDRESS, TEST_USER2, 0, 10);
+    addConnectionsToPool(pool2, 10, 10);
+    poolMap.put(new ConnectionPoolId(TEST_USER2, TEST_NN_ADDRESS), pool2);
+
+    checkPoolConnections(TEST_USER1, 9, 4);
+    checkPoolConnections(TEST_USER2, 10, 10);
+
+    // Clean up first pool, one connection should be removed, and second pool
+    // should remain the same.
+    connManager.cleanup(pool1);
+    checkPoolConnections(TEST_USER1, 8, 4);
+    checkPoolConnections(TEST_USER2, 10, 10);
+
+    // Clean up the first pool again, it should have no effect since it reached
+    // the MIN_ACTIVE_RATIO.
+    connManager.cleanup(pool1);
+    checkPoolConnections(TEST_USER1, 8, 4);
+    checkPoolConnections(TEST_USER2, 10, 10);
+
+    // Make sure the number of connections doesn't go below minSize
+    ConnectionPool pool3 = new ConnectionPool(
+        conf, TEST_NN_ADDRESS, TEST_USER3, 2, 10);
+    addConnectionsToPool(pool3, 10, 0);
+    poolMap.put(new ConnectionPoolId(TEST_USER2, TEST_NN_ADDRESS), pool3);
+    connManager.cleanup(pool3);
+    checkPoolConnections(TEST_USER3, 2, 0);
+    // With active connections added to pool, make sure it honors the
+    // MIN_ACTIVE_RATIO again
+    addConnectionsToPool(pool3, 10, 2);
+    connManager.cleanup(pool3);
+    checkPoolConnections(TEST_USER3, 4, 2);
+  }
+
+  @Test
+  public void testGetConnection() throws Exception {
+    Map<ConnectionPoolId, ConnectionPool> poolMap = connManager.getPools();
+    final int totalConns = 10;
+    int activeConns = 5;
+
+    ConnectionPool pool = new ConnectionPool(
+        conf, TEST_NN_ADDRESS, TEST_USER1, 0, 10);
+    addConnectionsToPool(pool, totalConns, activeConns);
+    poolMap.put(new ConnectionPoolId(TEST_USER1, TEST_NN_ADDRESS), pool);
+
+    // All remaining connections should be usable
+    final int remainingSlots = totalConns - activeConns;
+    for (int i = 0; i < remainingSlots; i++) {
+      ConnectionContext cc = pool.getConnection();
+      assertTrue(cc.isUsable());
+      cc.getClient();
+      activeConns++;
+    }
+
+    checkPoolConnections(TEST_USER1, totalConns, activeConns);
+
+    // Ask for more and this returns an active connection
+    ConnectionContext cc = pool.getConnection();
+    assertTrue(cc.isActive());
+  }
+
+  private void addConnectionsToPool(ConnectionPool pool, int numTotalConn,
+      int numActiveConn) throws IOException {
+    for (int i = 0; i < numTotalConn; i++) {
+      ConnectionContext cc = pool.newConnection();
+      pool.addConnection(cc);
+      if (i < numActiveConn) {
+        cc.getClient();
+      }
+    }
+  }
+
+  private void checkPoolConnections(UserGroupInformation ugi,
+      int numOfConns, int numOfActiveConns) {
+    for (Map.Entry<ConnectionPoolId, ConnectionPool> e :
+        connManager.getPools().entrySet()) {
+      if (e.getKey().getUgi() == ugi) {
+        assertEquals(numOfConns, e.getValue().getNumConnections());
+        assertEquals(numOfActiveConns, e.getValue().getNumActiveConnections());
+      }
+    }
+  }
+
+}

+ 34 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java

@@ -44,7 +44,6 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -110,7 +109,7 @@ public class TestRouterAdminCLI {
     String src = "/test-addmounttable";
     String dest = "/addmounttable";
     String[] argv = new String[] {"-add", src, nsId, dest};
-    Assert.assertEquals(0, ToolRunner.run(admin, argv));
+    assertEquals(0, ToolRunner.run(admin, argv));
 
     stateStore.loadCache(MountTableStoreImpl.class, true);
     GetMountTableEntriesRequest getRequest = GetMountTableEntriesRequest
@@ -130,7 +129,7 @@ public class TestRouterAdminCLI {
     // test mount table update behavior
     dest = dest + "-new";
     argv = new String[] {"-add", src, nsId, dest, "-readonly"};
-    Assert.assertEquals(0, ToolRunner.run(admin, argv));
+    assertEquals(0, ToolRunner.run(admin, argv));
     stateStore.loadCache(MountTableStoreImpl.class, true);
 
     getResponse = client.getMountTableManager()
@@ -212,7 +211,7 @@ public class TestRouterAdminCLI {
   @Test
   public void testMountTableDefaultACL() throws Exception {
     String[] argv = new String[] {"-add", "/testpath0", "ns0", "/testdir0"};
-    Assert.assertEquals(0, ToolRunner.run(admin, argv));
+    assertEquals(0, ToolRunner.run(admin, argv));
 
     stateStore.loadCache(MountTableStoreImpl.class, true);
     GetMountTableEntriesRequest getRequest = GetMountTableEntriesRequest
@@ -397,6 +396,37 @@ public class TestRouterAdminCLI {
     assertTrue(out.toString().contains("false"));
   }
 
+  @Test
+  public void testCreateInvalidEntry() throws Exception {
+    String[] argv = new String[] {
+        "-add", "test-createInvalidEntry", "ns0", "/createInvalidEntry"};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+
+    argv = new String[] {
+        "-add", "/test-createInvalidEntry", "ns0", "createInvalidEntry"};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+
+    argv = new String[] {
+        "-add", null, "ns0", "/createInvalidEntry"};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+
+    argv = new String[] {
+        "-add", "/test-createInvalidEntry", "ns0", null};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+
+    argv = new String[] {
+        "-add", "", "ns0", "/createInvalidEntry"};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+
+    argv = new String[] {
+        "-add", "/test-createInvalidEntry", null, "/createInvalidEntry"};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+
+    argv = new String[] {
+        "-add", "/test-createInvalidEntry", "", "/createInvalidEntry"};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+  }
+
   /**
    * Wait for the Router transforming to expected state.
    * @param expectedState Expected Router state.

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

@@ -414,7 +414,7 @@ public class TestRouterQuota {
 
     // mkdir and write a new file
     final FileSystem routerFs = routerContext.getFileSystem();
-    routerFs.mkdirs(new Path(path + UUID.randomUUID()));
+    routerFs.mkdirs(new Path(path + "/" + UUID.randomUUID()));
     DFSClient routerClient = routerContext.getClient();
     routerClient.create(path + "/file", true).close();
     appendData(path + "/file", routerClient, BLOCK_SIZE);

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuotaManager.java

@@ -57,6 +57,18 @@ public class TestRouterQuotaManager {
     assertTrue(childrenPaths.contains("/path1/subdir")
         && childrenPaths.contains("/path1/subdir/subdir")
         && childrenPaths.contains("/path1"));
+
+    // test for corner case
+    manager.put("/path3", quotaUsage);
+    manager.put("/path3/subdir", quotaUsage);
+    manager.put("/path3-subdir", quotaUsage);
+
+    childrenPaths = manager.getPaths("/path3");
+    assertEquals(2, childrenPaths.size());
+    // path /path3-subdir should not be returned
+    assertTrue(childrenPaths.contains("/path3")
+        && childrenPaths.contains("/path3/subdir")
+        && !childrenPaths.contains("/path3-subdir"));
   }
 
   @Test

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterSafemode.java

@@ -32,6 +32,7 @@ import java.net.URISyntaxException;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
 import org.apache.hadoop.service.Service.STATE;
 import org.apache.hadoop.util.Time;
@@ -65,6 +66,14 @@ public class TestRouterSafemode {
     // 2 sec post cache update before entering safemode (2 intervals)
     conf.setTimeDuration(DFS_ROUTER_SAFEMODE_EXPIRATION,
         TimeUnit.SECONDS.toMillis(2), TimeUnit.MILLISECONDS);
+
+    conf.set(DFSConfigKeys.DFS_ROUTER_RPC_BIND_HOST_KEY, "0.0.0.0");
+    conf.set(DFSConfigKeys.DFS_ROUTER_RPC_ADDRESS_KEY, "127.0.0.1:0");
+    conf.set(DFSConfigKeys.DFS_ROUTER_ADMIN_ADDRESS_KEY, "127.0.0.1:0");
+    conf.set(DFSConfigKeys.DFS_ROUTER_ADMIN_BIND_HOST_KEY, "0.0.0.0");
+    conf.set(DFSConfigKeys.DFS_ROUTER_HTTP_ADDRESS_KEY, "127.0.0.1:0");
+    conf.set(DFSConfigKeys.DFS_ROUTER_HTTPS_ADDRESS_KEY, "127.0.0.1:0");
+
     // RPC + State Store + Safe Mode only
     conf = new RouterConfigBuilder(conf)
         .rpc()

+ 43 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/records/TestMountTable.java

@@ -19,9 +19,12 @@ package org.apache.hadoop.hdfs.server.federation.store.records;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -32,6 +35,7 @@ import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
 import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
 import org.apache.hadoop.hdfs.server.federation.router.RouterQuotaUsage;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 
 /**
@@ -213,4 +217,43 @@ public class TestMountTable {
     assertEquals(SS_COUNT, quotaGet.getSpaceConsumed());
     assertEquals(SS_QUOTA, quotaGet.getSpaceQuota());
   }
+
+  @Test
+  public void testValidation() throws IOException {
+    Map<String, String> destinations = new HashMap<>();
+    destinations.put("ns0", "/testValidate-dest");
+    try {
+      MountTable.newInstance("testValidate", destinations);
+      fail("Mount table entry should be created failed.");
+    } catch (Exception e) {
+      GenericTestUtils.assertExceptionContains(
+          MountTable.ERROR_MSG_MUST_START_WITH_BACK_SLASH, e);
+    }
+
+    destinations.clear();
+    destinations.put("ns0", "testValidate-dest");
+    try {
+      MountTable.newInstance("/testValidate", destinations);
+      fail("Mount table entry should be created failed.");
+    } catch (Exception e) {
+      GenericTestUtils.assertExceptionContains(
+          MountTable.ERROR_MSG_ALL_DEST_MUST_START_WITH_BACK_SLASH, e);
+    }
+
+    destinations.clear();
+    destinations.put("", "/testValidate-dest");
+    try {
+      MountTable.newInstance("/testValidate", destinations);
+      fail("Mount table entry should be created failed.");
+    } catch (Exception e) {
+      GenericTestUtils.assertExceptionContains(
+          MountTable.ERROR_MSG_INVAILD_DEST_NS, e);
+    }
+
+    destinations.clear();
+    destinations.put("ns0", "/testValidate-dest");
+    MountTable record = MountTable.newInstance("/testValidate", destinations);
+    assertNotNull(record);
+
+  }
 }

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.util.NativeCodeLoader;
 import org.junit.Assert;
 
 import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -74,6 +75,7 @@ import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Time;
+import org.junit.Assume;
 import org.junit.Test;
 
 import static org.junit.Assert.assertArrayEquals;
@@ -99,6 +101,13 @@ public class TestFSImage {
     setCompressCodec(conf, "org.apache.hadoop.io.compress.DefaultCodec");
     setCompressCodec(conf, "org.apache.hadoop.io.compress.GzipCodec");
     setCompressCodec(conf, "org.apache.hadoop.io.compress.BZip2Codec");
+  }
+
+  @Test
+  public void testNativeCompression() throws IOException {
+    Assume.assumeTrue(NativeCodeLoader.isNativeCodeLoaded());
+    Configuration conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, true);
     setCompressCodec(conf, "org.apache.hadoop.io.compress.Lz4Codec");
   }
 

+ 43 - 36
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java

@@ -17,23 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -46,47 +29,71 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
-import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
-import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner;
-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.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.datanode.*;
+import org.apache.hadoop.hdfs.server.datanode.checker.DatasetVolumeChecker;
+import org.apache.hadoop.hdfs.server.datanode.checker.ThrottledAsyncChecker;
+import org.apache.hadoop.hdfs.server.namenode.*;
+import org.apache.hadoop.hdfs.server.namenode.top.metrics.TopMetrics;
+import org.apache.hadoop.http.HttpRequestLog;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.ipc.ProtobufRpcEngine.Server;
 import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.GSet;
 import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+import java.util.*;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Helper for writing snapshot related tests
  */
 public class SnapshotTestHelper {
-  public static final Log LOG = LogFactory.getLog(SnapshotTestHelper.class);
+  static final Logger LOG = LoggerFactory.getLogger(SnapshotTestHelper.class);
 
   /** Disable the logs that are not very useful for snapshot related tests. */
   public static void disableLogs() {
     final String[] lognames = {
+        "org.apache.hadoop.hdfs.server.common.Util",
+        "org.apache.hadoop.hdfs.server.blockmanagement.BlockReportLeaseManager",
+        "org.apache.hadoop.hdfs.server.namenode.FileJournalManager",
+        "org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager",
+        "org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf",
+        "org.apache.hadoop.hdfs.server.namenode.FSEditLog",
         "org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceScanner",
+        "org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.BlockPoolSlice",
         "org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImpl",
         "org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetAsyncDiskService",
+        "org.apache.hadoop.hdfs.server.datanode.fsdataset.impl" +
+            ".RamDiskAsyncLazyPersistService",
     };
     for(String n : lognames) {
-      GenericTestUtils.disableLog(LogFactory.getLog(n));
+      GenericTestUtils.disableLog(LoggerFactory.getLogger(n));
     }
     
-    GenericTestUtils.disableLog(LogFactory.getLog(UserGroupInformation.class));
-    GenericTestUtils.disableLog(LogFactory.getLog(BlockManager.class));
-    GenericTestUtils.disableLog(LogFactory.getLog(FSNamesystem.class));
-    GenericTestUtils.disableLog(LogFactory.getLog(DirectoryScanner.class));
-    GenericTestUtils.disableLog(LogFactory.getLog(MetricsSystemImpl.class));
-    
+    GenericTestUtils.disableLog(LoggerFactory.getLogger(
+        UserGroupInformation.class));
+    GenericTestUtils.disableLog(LoggerFactory.getLogger(BlockManager.class));
+    GenericTestUtils.disableLog(LoggerFactory.getLogger(FSNamesystem.class));
+    GenericTestUtils.disableLog(LoggerFactory.getLogger(
+        DirectoryScanner.class));
+    GenericTestUtils.disableLog(LoggerFactory.getLogger(
+        MetricsSystemImpl.class));
+
+    GenericTestUtils.disableLog(DatasetVolumeChecker.LOG);
+    GenericTestUtils.disableLog(DatanodeDescriptor.LOG);
+    GenericTestUtils.disableLog(GSet.LOG);
+    GenericTestUtils.disableLog(TopMetrics.LOG);
+    GenericTestUtils.disableLog(HttpRequestLog.LOG);
+    GenericTestUtils.disableLog(ThrottledAsyncChecker.LOG);
+    GenericTestUtils.disableLog(VolumeScanner.LOG);
     GenericTestUtils.disableLog(BlockScanner.LOG);
     GenericTestUtils.disableLog(HttpServer2.LOG);
     GenericTestUtils.disableLog(DataNode.LOG);

+ 50 - 79
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

@@ -17,22 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyObject;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.spy;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Random;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -41,12 +25,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSOutputStream;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.*;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
@@ -55,18 +34,10 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
-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.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.INodeReference;
+import org.apache.hadoop.hdfs.server.namenode.*;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
-import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
-import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 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.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
@@ -76,6 +47,18 @@ import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.internal.util.reflection.Whitebox;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.spy;
+
 /** Testing rename with snapshots. */
 public class TestRenameWithSnapshots {
   static {
@@ -104,7 +87,11 @@ public class TestRenameWithSnapshots {
   static private final String snap1 = "snap1";
   static private final String snap2 = "snap2";
 
-  
+  static void assertSizes(int createdSize, int deletedSize, ChildrenDiff diff) {
+    assertEquals(createdSize, diff.getCreatedUnmodifiable().size());
+    assertEquals(deletedSize, diff.getDeletedUnmodifiable().size());
+  }
+
   @Before
   public void setUp() throws Exception {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
@@ -1301,9 +1288,8 @@ public class TestRenameWithSnapshots {
     // after the undo of rename, both the created and deleted list of sdir1
     // should be empty
     ChildrenDiff childrenDiff = dir1Diffs.get(0).getChildrenDiff();
-    assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
-    assertEquals(0, childrenDiff.getList(ListType.CREATED).size());
-    
+    assertSizes(0, 0, childrenDiff);
+
     INode fooNode = fsdir.getINode4Write(foo.toString());
     assertTrue(fooNode.isDirectory() && fooNode.asDirectory().isWithSnapshot());
     DiffList<DirectoryDiff> fooDiffs =
@@ -1372,13 +1358,12 @@ public class TestRenameWithSnapshots {
     // after the undo of rename, the created list of sdir1 should contain 
     // 1 element
     ChildrenDiff childrenDiff = dir1Diffs.get(0).getChildrenDiff();
-    assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
-    assertEquals(1, childrenDiff.getList(ListType.CREATED).size());
-    
+    assertSizes(1, 0, childrenDiff);
+
     INode fooNode = fsdir.getINode4Write(foo.toString());
     assertTrue(fooNode instanceof INodeDirectory);
-    assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
-    
+    assertTrue(childrenDiff.getCreatedUnmodifiable().get(0) == fooNode);
+
     final Path foo_s1 = SnapshotTestHelper.getSnapshotPath(sdir1, "s1", "foo");
     assertFalse(hdfs.exists(foo_s1));
     
@@ -1438,13 +1423,12 @@ public class TestRenameWithSnapshots {
     assertEquals(1, dir2Diffs.size());
     assertEquals(s2.getId(), dir2Diffs.get(0).getSnapshotId());
     ChildrenDiff childrenDiff = dir2Diffs.get(0).getChildrenDiff();
-    assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
-    assertEquals(1, childrenDiff.getList(ListType.CREATED).size());
+    assertSizes(1, 0, childrenDiff);
     final Path foo_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2", "foo2");
     assertFalse(hdfs.exists(foo_s2));
     
     INode fooNode = fsdir.getINode4Write(foo_dir2.toString());
-    assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
+    assertTrue(childrenDiff.getCreatedUnmodifiable().get(0) == fooNode);
     assertTrue(fooNode instanceof INodeReference.DstReference);
     DiffList<DirectoryDiff> fooDiffs =
         fooNode.asDirectory().getDiffs().asList();
@@ -1468,14 +1452,12 @@ public class TestRenameWithSnapshots {
     assertEquals(s3.getId(), dir2Diffs.get(1).getSnapshotId());
     
     childrenDiff = dir2Diffs.get(0).getChildrenDiff();
-    assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
-    assertEquals(1, childrenDiff.getList(ListType.CREATED).size());
-    assertTrue(childrenDiff.getList(ListType.CREATED).get(0) == fooNode);
+    assertSizes(1, 0, childrenDiff);
+    assertTrue(childrenDiff.getCreatedUnmodifiable().get(0) == fooNode);
     
     childrenDiff = dir2Diffs.get(1).getChildrenDiff();
-    assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
-    assertEquals(0, childrenDiff.getList(ListType.CREATED).size());
-    
+    assertSizes(0, 0, childrenDiff);
+
     final Path foo_s3 = SnapshotTestHelper.getSnapshotPath(sdir2, "s3", "foo2");
     assertFalse(hdfs.exists(foo_s2));
     assertTrue(hdfs.exists(foo_s3));
@@ -1600,9 +1582,8 @@ public class TestRenameWithSnapshots {
         .getDiffs().asList();
     assertEquals(1, diffList.size());
     DirectoryDiff diff = diffList.get(0);
-    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
-    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
-    
+    assertSizes(0, 0, diff.getChildrenDiff());
+
     // check dir2
     INodeDirectory dir2Node = fsdir2.getINode4Write(dir2.toString()).asDirectory();
     assertTrue(dir2Node.isSnapshottable());
@@ -1618,8 +1599,7 @@ public class TestRenameWithSnapshots {
     diffList = dir2Node.getDiffs().asList();
     assertEquals(1, diffList.size());
     diff = diffList.get(0);
-    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
-    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
+    assertSizes(0, 0, diff.getChildrenDiff());
   }
   
   /**
@@ -1674,9 +1654,8 @@ public class TestRenameWithSnapshots {
         .getDiffs().asList();
     assertEquals(1, diffList.size());
     DirectoryDiff diff = diffList.get(0);
-    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
-    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
-    
+    assertSizes(0, 0, diff.getChildrenDiff());
+
     // check dir2
     INodeDirectory dir2Node = fsdir2.getINode4Write(dir2.toString()).asDirectory();
     assertTrue(dir2Node.isSnapshottable());
@@ -1696,8 +1675,7 @@ public class TestRenameWithSnapshots {
     diffList = (  dir2Node).getDiffs().asList();
     assertEquals(1, diffList.size());
     diff = diffList.get(0);
-    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
-    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
+    assertSizes(0, 0, diff.getChildrenDiff());
   }
   
   /**
@@ -1737,9 +1715,8 @@ public class TestRenameWithSnapshots {
     Snapshot s1 = rootNode.getSnapshot(DFSUtil.string2Bytes(snap1));
     assertEquals(s1.getId(), diff.getSnapshotId());
     // after undo, the diff should be empty
-    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
-    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
-    
+    assertSizes(0, 0, diff.getChildrenDiff());
+
     // bar was converted to filewithsnapshot while renaming
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
     assertSame(barNode, children.get(0));
@@ -1989,9 +1966,8 @@ public class TestRenameWithSnapshots {
     Snapshot s1 = dir1Node.getSnapshot(DFSUtil.string2Bytes("s1"));
     assertEquals(s1.getId(), diffList.get(0).getSnapshotId());
     ChildrenDiff diff = diffList.get(0).getChildrenDiff();
-    assertEquals(0, diff.getList(ListType.CREATED).size());
-    assertEquals(0, diff.getList(ListType.DELETED).size());
-    
+    assertSizes(0, 0, diff);
+
     restartClusterAndCheckImage(true);
   }
   
@@ -2062,9 +2038,8 @@ public class TestRenameWithSnapshots {
     assertEquals(s1.getId(), diffList.get(0).getSnapshotId());
     ChildrenDiff diff = diffList.get(0).getChildrenDiff();
     // bar2 and bar3 in the created list
-    assertEquals(2, diff.getList(ListType.CREATED).size());
-    assertEquals(0, diff.getList(ListType.DELETED).size());
-    
+    assertSizes(2, 0, diff);
+
     final INode fooRef2 = fsdir.getINode4Write(foo.toString());
     assertTrue(fooRef2 instanceof INodeReference.DstReference);
     INodeReference.WithCount wc2 = 
@@ -2235,13 +2210,9 @@ public class TestRenameWithSnapshots {
         .asDirectory();
     DiffList<DirectoryDiff> dir1DiffList = dir1Node.getDiffs().asList();
     assertEquals(1, dir1DiffList.size());
-    List<INode> dList = dir1DiffList.get(0).getChildrenDiff()
-        .getList(ListType.DELETED);
-    assertTrue(dList.isEmpty());
-    List<INode> cList = dir1DiffList.get(0).getChildrenDiff()
-        .getList(ListType.CREATED);
-    assertEquals(1, cList.size());
-    INode cNode = cList.get(0);
+    final ChildrenDiff childrenDiff = dir1DiffList.get(0).getChildrenDiff();
+    assertSizes(1, 0, childrenDiff);
+    INode cNode = childrenDiff.getCreatedUnmodifiable().get(0);
     INode fooNode = fsdir.getINode4Write(newfoo.toString());
     assertSame(cNode, fooNode);
     
@@ -2259,7 +2230,7 @@ public class TestRenameWithSnapshots {
     Snapshot s0 = testNode.getSnapshot(DFSUtil.string2Bytes("s0"));
     assertEquals(s0.getId(), diff.getSnapshotId());
     // and file should be stored in the deleted list of this snapshot diff
-    assertEquals("file", diff.getChildrenDiff().getList(ListType.DELETED)
+    assertEquals("file", diff.getChildrenDiff().getDeletedUnmodifiable()
         .get(0).getLocalName());
     
     // check dir2: a WithName instance for foo should be in the deleted list
@@ -2269,7 +2240,8 @@ public class TestRenameWithSnapshots {
     DiffList<DirectoryDiff> dir2DiffList = dir2Node.getDiffs().asList();
     // dir2Node should contain 1 snapshot diffs for s2
     assertEquals(1, dir2DiffList.size());
-    dList = dir2DiffList.get(0).getChildrenDiff().getList(ListType.DELETED);
+    final List<INode> dList = dir2DiffList.get(0).getChildrenDiff()
+        .getDeletedUnmodifiable();
     assertEquals(1, dList.size());
     final Path foo_s2 = SnapshotTestHelper.getSnapshotPath(dir2, "s2", 
         foo.getName());
@@ -2323,8 +2295,7 @@ public class TestRenameWithSnapshots {
     DiffList<DirectoryDiff> diffList = barNode.getDiffs().asList();
     assertEquals(1, diffList.size());
     DirectoryDiff diff = diffList.get(0);
-    assertEquals(0, diff.getChildrenDiff().getList(ListType.DELETED).size());
-    assertEquals(0, diff.getChildrenDiff().getList(ListType.CREATED).size());
+    assertSizes(0, 0, diff.getChildrenDiff());
   }
 
   /**

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSetQuotaWithSnapshot.java

@@ -38,7 +38,6 @@ 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.DirectoryDiff;
-import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -153,8 +152,9 @@ public class TestSetQuotaWithSnapshot {
         subNode.asDirectory().getDiffs().asList();
     assertEquals(1, diffList.size());
     Snapshot s2 = dirNode.getSnapshot(DFSUtil.string2Bytes("s2"));
-    assertEquals(s2.getId(), diffList.get(0).getSnapshotId());
-    List<INode> createdList = diffList.get(0).getChildrenDiff().getList(ListType.CREATED);
+    final DirectoryDiff diff = diffList.get(0);
+    assertEquals(s2.getId(), diff.getSnapshotId());
+    List<INode> createdList = diff.getChildrenDiff().getCreatedUnmodifiable();
     assertEquals(1, createdList.size());
     assertSame(fsdir.getINode4Write(file.toString()), createdList.get(0));
   }

+ 38 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java

@@ -885,4 +885,42 @@ public class TestDFSAdmin {
     assertEquals(-1, ToolRunner.run(dfsAdmin,
         new String[]{"-setBalancerBandwidth", "-10m"}));
   }
+
+  @Test(timeout = 300000L)
+  public void testCheckNumOfBlocksInReportCommand() throws Exception {
+    Configuration config = new Configuration();
+    config.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 512);
+    config.set(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, "3s");
+
+    int numOfDatanodes = 1;
+    MiniDFSCluster miniDFSCluster = new MiniDFSCluster.Builder(config)
+        .numDataNodes(numOfDatanodes).build();
+    try {
+      miniDFSCluster.waitActive();
+      DistributedFileSystem dfs = miniDFSCluster.getFileSystem();
+      Path path= new Path("/tmp.txt");
+
+      DatanodeInfo[] dn = dfs.getDataNodeStats();
+      assertEquals(dn.length, numOfDatanodes);
+      //Block count should be 0, as no files are created
+      assertEquals(dn[0].getNumBlocks(), 0);
+
+
+      //Create a file with 2 blocks
+      DFSTestUtil.createFile(dfs, path, 1024, (short) 1, 0);
+      int expectedBlockCount = 2;
+
+      //Wait for One Heartbeat
+      Thread.sleep(3 * 1000);
+
+      dn = dfs.getDataNodeStats();
+      assertEquals(dn.length, numOfDatanodes);
+
+      //Block count should be 2, as file is created with block count 2
+      assertEquals(dn[0].getNumBlocks(), expectedBlockCount);
+
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }

+ 183 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java

@@ -44,6 +44,7 @@ import java.net.SocketTimeoutException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
+import java.nio.charset.StandardCharsets;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.Random;
@@ -81,6 +82,7 @@ import org.apache.hadoop.hdfs.TestDFSClientRetries;
 import org.apache.hadoop.hdfs.TestFileCreation;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -109,6 +111,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.log4j.Level;
+import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Assert;
@@ -625,7 +628,7 @@ public class TestWebHDFS {
   }
 
   /**
-   * Test snapshot deletion through WebHdfs
+   * Test snapshot deletion through WebHdfs.
    */
   @Test
   public void testWebHdfsDeleteSnapshot() throws Exception {
@@ -670,7 +673,7 @@ public class TestWebHDFS {
   }
 
   /**
-   * Test snapshot diff through WebHdfs
+   * Test snapshot diff through WebHdfs.
    */
   @Test
   public void testWebHdfsSnapshotDiff() throws Exception {
@@ -735,6 +738,75 @@ public class TestWebHDFS {
     }
   }
 
+  /**
+   * Test snapshottable directory list through WebHdfs.
+   */
+  @Test
+  public void testWebHdfsSnapshottableDirectoryList() throws Exception {
+    MiniDFSCluster cluster = null;
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final WebHdfsFileSystem webHdfs = WebHdfsTestUtil
+          .getWebHdfsFileSystem(conf, WebHdfsConstants.WEBHDFS_SCHEME);
+      final Path foo = new Path("/foo");
+      final Path bar = new Path("/bar");
+      dfs.mkdirs(foo);
+      dfs.mkdirs(bar);
+      dfs.allowSnapshot(foo);
+      dfs.allowSnapshot(bar);
+      Path file0 = new Path(foo, "file0");
+      DFSTestUtil.createFile(dfs, file0, 100, (short) 1, 0);
+      Path file1 = new Path(bar, "file1");
+      DFSTestUtil.createFile(dfs, file1, 100, (short) 1, 0);
+      SnapshottableDirectoryStatus[] statuses =
+          webHdfs.getSnapshottableDirectoryList();
+      SnapshottableDirectoryStatus[] dfsStatuses =
+          dfs.getSnapshottableDirListing();
+
+      for (int i = 0; i < dfsStatuses.length; i++) {
+        Assert.assertEquals(statuses[i].getSnapshotNumber(),
+            dfsStatuses[i].getSnapshotNumber());
+        Assert.assertEquals(statuses[i].getSnapshotQuota(),
+            dfsStatuses[i].getSnapshotQuota());
+        Assert.assertTrue(Arrays.equals(statuses[i].getParentFullPath(),
+            dfsStatuses[i].getParentFullPath()));
+        Assert.assertEquals(dfsStatuses[i].getDirStatus().getChildrenNum(),
+            statuses[i].getDirStatus().getChildrenNum());
+        Assert.assertEquals(dfsStatuses[i].getDirStatus().getModificationTime(),
+            statuses[i].getDirStatus().getModificationTime());
+        Assert.assertEquals(dfsStatuses[i].getDirStatus().isDir(),
+            statuses[i].getDirStatus().isDir());
+        Assert.assertEquals(dfsStatuses[i].getDirStatus().getAccessTime(),
+            statuses[i].getDirStatus().getAccessTime());
+        Assert.assertEquals(dfsStatuses[i].getDirStatus().getPermission(),
+            statuses[i].getDirStatus().getPermission());
+        Assert.assertEquals(dfsStatuses[i].getDirStatus().getOwner(),
+            statuses[i].getDirStatus().getOwner());
+        Assert.assertEquals(dfsStatuses[i].getDirStatus().getGroup(),
+            statuses[i].getDirStatus().getGroup());
+        Assert.assertEquals(dfsStatuses[i].getDirStatus().getPath(),
+            statuses[i].getDirStatus().getPath());
+        Assert.assertEquals(dfsStatuses[i].getDirStatus().getFileId(),
+            statuses[i].getDirStatus().getFileId());
+        Assert.assertEquals(dfsStatuses[i].getDirStatus().hasAcl(),
+            statuses[i].getDirStatus().hasAcl());
+        Assert.assertEquals(dfsStatuses[i].getDirStatus().isEncrypted(),
+            statuses[i].getDirStatus().isEncrypted());
+        Assert.assertEquals(dfsStatuses[i].getDirStatus().isErasureCoded(),
+            statuses[i].getDirStatus().isErasureCoded());
+        Assert.assertEquals(dfsStatuses[i].getDirStatus().isSnapshotEnabled(),
+            statuses[i].getDirStatus().isSnapshotEnabled());
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   @Test
   public void testWebHdfsCreateNonRecursive() throws IOException, URISyntaxException {
     MiniDFSCluster cluster = null;
@@ -760,7 +832,7 @@ public class TestWebHDFS {
     }
   }
   /**
-   * Test snapshot rename through WebHdfs
+   * Test snapshot rename through WebHdfs.
    */
   @Test
   public void testWebHdfsRenameSnapshot() throws Exception {
@@ -1578,6 +1650,114 @@ public class TestWebHDFS {
     }
   }
 
+  /**
+   * Tests that the LISTSTATUS ang GETFILESTATUS WebHDFS calls return the
+   * ecPolicy for EC files.
+   */
+  @Test(timeout=300000)
+  public void testECPolicyInFileStatus() throws Exception {
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    final ErasureCodingPolicy ecPolicy = SystemErasureCodingPolicies
+        .getByID(SystemErasureCodingPolicies.RS_3_2_POLICY_ID);
+    final String ecPolicyName = ecPolicy.getName();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(5)
+        .build();
+    cluster.waitActive();
+    final DistributedFileSystem fs = cluster.getFileSystem();
+
+    // Create an EC dir and write a test file in it
+    final Path ecDir = new Path("/ec");
+    Path ecFile = new Path(ecDir, "ec_file.txt");
+    Path nonEcFile = new Path(ecDir, "non_ec_file.txt");
+    fs.mkdirs(ecDir);
+
+    // Create a non-EC file before enabling ec policy
+    DFSTestUtil.createFile(fs, nonEcFile, 1024, (short) 1, 0);
+
+    fs.enableErasureCodingPolicy(ecPolicyName);
+    fs.setErasureCodingPolicy(ecDir, ecPolicyName);
+
+    // Create a EC file
+    DFSTestUtil.createFile(fs, ecFile, 1024, (short) 1, 0);
+
+    // Query webhdfs REST API to list statuses of files/directories in ecDir
+    InetSocketAddress addr = cluster.getNameNode().getHttpAddress();
+    URL listStatusUrl = new URL("http", addr.getHostString(), addr.getPort(),
+        WebHdfsFileSystem.PATH_PREFIX + ecDir.toString() + "?op=LISTSTATUS");
+
+    HttpURLConnection conn = (HttpURLConnection) listStatusUrl.openConnection();
+    conn.setRequestMethod("GET");
+    conn.setInstanceFollowRedirects(false);
+    String listStatusResponse = IOUtils.toString(conn.getInputStream(),
+        StandardCharsets.UTF_8);
+    Assert.assertEquals("Response wasn't " + HttpURLConnection.HTTP_OK,
+        HttpURLConnection.HTTP_OK, conn.getResponseCode());
+
+    // Verify that ecPolicy is set in the ListStatus response for ec file
+    String ecpolicyForECfile = getECPolicyFromFileStatusJson(
+        getFileStatusJson(listStatusResponse, ecFile.getName()));
+    assertEquals("EC policy for ecFile should match the set EC policy",
+        ecpolicyForECfile, ecPolicyName);
+
+    // Verify that ecPolicy is not set in the ListStatus response for non-ec
+    // file
+    String ecPolicyForNonECfile = getECPolicyFromFileStatusJson(
+        getFileStatusJson(listStatusResponse, nonEcFile.getName()));
+    assertEquals("EC policy for nonEcFile should be null (not set)",
+        ecPolicyForNonECfile, null);
+
+    // Query webhdfs REST API to get fileStatus for ecFile
+    URL getFileStatusUrl = new URL("http", addr.getHostString(), addr.getPort(),
+        WebHdfsFileSystem.PATH_PREFIX + ecFile.toString() +
+            "?op=GETFILESTATUS");
+
+    conn = (HttpURLConnection) getFileStatusUrl.openConnection();
+    conn.setRequestMethod("GET");
+    conn.setInstanceFollowRedirects(false);
+    String getFileStatusResponse = IOUtils.toString(conn.getInputStream(),
+        StandardCharsets.UTF_8);
+    Assert.assertEquals("Response wasn't " + HttpURLConnection.HTTP_OK,
+        HttpURLConnection.HTTP_OK, conn.getResponseCode());
+
+    // Verify that ecPolicy is set in getFileStatus response for ecFile
+    JSONObject fileStatusObject = new JSONObject(getFileStatusResponse)
+        .getJSONObject("FileStatus");
+    ecpolicyForECfile = getECPolicyFromFileStatusJson(fileStatusObject);
+    assertEquals("EC policy for ecFile should match the set EC policy",
+        ecpolicyForECfile, ecPolicyName);
+  }
+
+  /**
+   * Get FileStatus JSONObject from ListStatus response.
+   */
+  private JSONObject getFileStatusJson(String response, String fileName)
+      throws JSONException {
+    JSONObject listStatusResponseJson = new JSONObject(response);
+    JSONArray fileStatusArray = listStatusResponseJson
+        .getJSONObject("FileStatuses")
+        .getJSONArray("FileStatus");
+    for (int i = 0; i < fileStatusArray.length(); i++) {
+      JSONObject fileStatusJsonObject = fileStatusArray.getJSONObject(i);
+      if (fileName.equals(fileStatusJsonObject.get("pathSuffix"))) {
+        return fileStatusJsonObject;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Get ECPolicy name from FileStatus JSONObject.
+   */
+  private String getECPolicyFromFileStatusJson(JSONObject fileStatusJsonObject)
+      throws JSONException {
+    if (fileStatusJsonObject.has("ecPolicy")) {
+      return fileStatusJsonObject.getString("ecPolicy");
+    } else {
+      return null;
+    }
+  }
+
   final static class DummyThrowable extends Throwable {
   }
 }

+ 25 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml

@@ -214,7 +214,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Set erasure coding policy RS-6-3-1024k on /ecdir</expected-output>
+          <expected-output>Set RS-6-3-1024k erasure coding policy on /ecdir</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -232,7 +232,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Set erasure coding policy RS-6-3-1024k on /ecdir</expected-output>
+          <expected-output>Set RS-6-3-1024k erasure coding policy on /ecdir</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -311,7 +311,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Warning: setting erasure coding policy on a non-empty directory will not automatically convert existing files to RS-6-3-1024</expected-output>
+          <expected-output>Warning: setting erasure coding policy on a non-empty directory will not automatically convert existing files to RS-6-3-1024k erasure coding policy</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -695,6 +695,28 @@
       </comparators>
     </test>
 
+    <test>
+      <description>setPolicy : set erasure coding policy without given a specific policy name on a non empty directory</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <command>-fs NAMENODE -mkdir /ecdir/ecsubdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -path /ecdir</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Set default erasure coding policy on /ecdir</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Warning: setting erasure coding policy on a non-empty directory will not automatically convert existing files to default erasure coding policy</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
     <test>
       <description>getPolicy: get the default policy after setPolicy without given a specific policy name</description>
       <test-commands>

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml

@@ -16550,6 +16550,10 @@
           <type>RegexpComparator</type>
           <expected-output>Last contact: [a-zA-Z]+ [a-zA-Z]+ [0-9]+ [0-9:]+ [A-Z\-\+\:0-9]+ [0-9]+</expected-output>
         </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>Num of Blocks: [0-9]+</expected-output>
+        </comparator>
         <comparator>
           <type>TokenComparator</type>
           <expected-output>Live datanodes</expected-output>

+ 3 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java

@@ -393,11 +393,10 @@ public class AppController extends Controller implements AMParams {
    */
   boolean checkAccess(Job job) {
     String remoteUser = request().getRemoteUser();
-    if (remoteUser == null) {
-      return false;
+    UserGroupInformation callerUGI = null;
+    if (remoteUser != null) {
+      callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
     }
-    UserGroupInformation callerUGI =
-        UserGroupInformation.createRemoteUser(remoteUser);
     if (callerUGI != null && !job.checkAccess(callerUGI, JobACL.VIEW_JOB)) {
       return false;
     }

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

@@ -421,22 +421,6 @@
   </description>
 </property>
 
-<property>
-  <name>mapreduce.map.cpu.vcores</name>
-  <value>1</value>
-  <description>
-      The number of virtual cores required for each map task.
-  </description>
-</property>
-
-<property>
-  <name>mapreduce.reduce.cpu.vcores</name>
-  <value>1</value>
-  <description>
-      The number of virtual cores required for each reduce task.
-  </description>
-</property>
-
 <property>
   <name>mapreduce.reduce.merge.inmem.threshold</name>
   <value>1000</value>

+ 100 - 0
hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/paralleltests/CreateDirsMojo.java

@@ -0,0 +1,100 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.maven.plugin.paralleltests;
+
+import java.io.File;
+
+import org.apache.maven.plugin.AbstractMojo;
+import org.apache.maven.plugin.MojoExecutionException;
+import org.apache.maven.plugins.annotations.LifecyclePhase;
+import org.apache.maven.plugins.annotations.Mojo;
+import org.apache.maven.plugins.annotations.Parameter;
+
+
+/**
+ * Goal which creates the parallel-test directories.
+ */
+@Mojo(name="parallel-tests-createdir",
+      defaultPhase = LifecyclePhase.GENERATE_TEST_RESOURCES)
+public class CreateDirsMojo extends AbstractMojo {
+
+  /**
+   * Location of the test.build.dir.
+   */
+  @Parameter(defaultValue="${project.build.directory}/test-dir")
+  private File testBuildDir;
+
+  /**
+   * Location of the test.build.data.
+   */
+  @Parameter(defaultValue="${project.build.directory}/test-dir")
+  private File testBuildData;
+
+  /**
+   * Location of the test.build.data.
+   */
+  @Parameter(defaultValue="${project.build.directory}/tmp")
+  private File hadoopTmpDir;
+
+  /**
+   * Thread count.
+   */
+  @Parameter(defaultValue="${testsThreadCount}")
+  private String testsThreadCount;
+
+  public void execute() throws MojoExecutionException {
+    int numDirs=getTestsThreadCount();
+
+    mkParallelDirs(testBuildDir, numDirs);
+    mkParallelDirs(testBuildData, numDirs);
+    mkParallelDirs(hadoopTmpDir, numDirs);
+  }
+
+  /**
+   * Get the real number of parallel threads.
+   * @return int number of threads
+   */
+
+  public int getTestsThreadCount() {
+    int threadCount = 1;
+    if (testsThreadCount != null) {
+      String trimProp = testsThreadCount.trim();
+      if (trimProp.endsWith("C")) {
+        double multiplier = Double.parseDouble(
+            trimProp.substring(0, trimProp.length()-1));
+        double calculated = multiplier * ((double) Runtime
+            .getRuntime()
+            .availableProcessors());
+        threadCount = calculated > 0d ? Math.max((int) calculated, 1) : 0;
+      } else {
+        threadCount = Integer.parseInt(testsThreadCount);
+      }
+    }
+    return threadCount;
+  }
+
+  private void mkParallelDirs(File testDir, int numDirs)
+      throws MojoExecutionException {
+    for (int i=1; i<=numDirs; i++) {
+      File newDir = new File(testDir, String.valueOf(i));
+      if (!newDir.exists()) {
+        getLog().info("Creating " + newDir.toString());
+        if (!newDir.mkdirs()) {
+          throw new MojoExecutionException("Unable to create "
+              + newDir.toString());
+        }
+      }
+    }
+  }
+}

+ 1 - 1
hadoop-project/src/site/markdown/index.md.vm

@@ -223,7 +223,7 @@ functionality, except the mount table is managed on the server-side by the
 routing layer rather than on the client. This simplifies access to a federated
 cluster for existing HDFS clients.
 
-See [HDFS-10467](https://issues.apache.org/jira/browse/HADOOP-10467) and the
+See [HDFS-10467](https://issues.apache.org/jira/browse/HDFS-10467) and the
 HDFS Router-based Federation
 [documentation](./hadoop-project-dist/hadoop-hdfs/HDFSRouterFederation.html) for
 more details.

+ 5 - 21
hadoop-tools/hadoop-aws/pom.xml

@@ -85,30 +85,13 @@
       <build>
         <plugins>
           <plugin>
-            <artifactId>maven-antrun-plugin</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-maven-plugins</artifactId>
             <executions>
               <execution>
-                <id>create-parallel-tests-dirs</id>
-                <phase>test-compile</phase>
-                <configuration>
-                  <target>
-                    <script language="javascript"><![CDATA[
-                      var baseDirs = [
-                          "${test.build.data}",
-                          "${test.build.dir}",
-                          "${hadoop.tmp.dir}" ];
-                      for (var i in baseDirs) {
-                        for (var j = 1; j <= ${testsThreadCount}; ++j) {
-                          var mkdir = project.createTask("mkdir");
-                          mkdir.setDir(new java.io.File(baseDirs[i], j));
-                          mkdir.perform();
-                        }
-                      }
-                    ]]></script>
-                  </target>
-                </configuration>
+                <id>parallel-tests-createdir</id>
                 <goals>
-                  <goal>run</goal>
+                  <goal>parallel-tests-createdir</goal>
                 </goals>
               </execution>
             </executions>
@@ -121,6 +104,7 @@
               <reuseForks>false</reuseForks>
               <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
               <systemPropertyVariables>
+                <testsThreadCount>${testsThreadCount}</testsThreadCount>
                 <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
                 <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
                 <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>

+ 11 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java

@@ -542,4 +542,15 @@ public final class Constants {
    */
   public static final String RETRY_THROTTLE_INTERVAL_DEFAULT = "500ms";
 
+  /**
+   * Should etags be exposed as checksums?
+   */
+  public static final String ETAG_CHECKSUM_ENABLED =
+      "fs.s3a.etag.checksum.enabled";
+
+  /**
+   * Default value: false.
+   */
+  public static final boolean ETAG_CHECKSUM_ENABLED_DEFAULT = false;
+
 }

+ 25 - 13
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -2993,17 +2993,21 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   }
 
   /**
-   * Get the etag of a object at the path via HEAD request and return it
-   * as a checksum object. This has the whatever guarantees about equivalence
-   * the S3 implementation offers.
+   * When enabled, get the etag of a object at the path via HEAD request and
+   * return it as a checksum object.
    * <ol>
    *   <li>If a tag has not changed, consider the object unchanged.</li>
    *   <li>Two tags being different does not imply the data is different.</li>
    * </ol>
    * Different S3 implementations may offer different guarantees.
+   *
+   * This check is (currently) only made if
+   * {@link Constants#ETAG_CHECKSUM_ENABLED} is set; turning it on
+   * has caused problems with Distcp (HADOOP-15273).
+   *
    * @param f The file path
    * @param length The length of the file range for checksum calculation
-   * @return The EtagChecksum or null if checksums are not supported.
+   * @return The EtagChecksum or null if checksums are not enabled or supported.
    * @throws IOException IO failure
    * @see <a href="http://docs.aws.amazon.com/AmazonS3/latest/API/RESTCommonResponseHeaders.html">Common Response Headers</a>
    */
@@ -3012,15 +3016,23 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   public EtagChecksum getFileChecksum(Path f, final long length)
       throws IOException {
     Preconditions.checkArgument(length >= 0);
-    Path path = qualify(f);
-    LOG.debug("getFileChecksum({})", path);
-    return once("getFileChecksum", path.toString(),
-        () -> {
-          // this always does a full HEAD to the object
-          ObjectMetadata headers = getObjectMetadata(path);
-          String eTag = headers.getETag();
-          return eTag != null ? new EtagChecksum(eTag) : null;
-        });
+    entryPoint(INVOCATION_GET_FILE_CHECKSUM);
+
+    if (getConf().getBoolean(ETAG_CHECKSUM_ENABLED,
+        ETAG_CHECKSUM_ENABLED_DEFAULT)) {
+      Path path = qualify(f);
+      LOG.debug("getFileChecksum({})", path);
+      return once("getFileChecksum", path.toString(),
+          () -> {
+            // this always does a full HEAD to the object
+            ObjectMetadata headers = getObjectMetadata(path);
+            String eTag = headers.getETag();
+            return eTag != null ? new EtagChecksum(eTag) : null;
+          });
+    } else {
+      // disabled
+      return null;
+    }
   }
 
   /**

+ 1 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java

@@ -139,6 +139,7 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
       INVOCATION_CREATE_NON_RECURSIVE,
       INVOCATION_DELETE,
       INVOCATION_EXISTS,
+      INVOCATION_GET_FILE_CHECKSUM,
       INVOCATION_GET_FILE_STATUS,
       INVOCATION_GLOB_STATUS,
       INVOCATION_IS_DIRECTORY,

+ 2 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java

@@ -57,6 +57,8 @@ public enum Statistic {
       "Calls of delete()"),
   INVOCATION_EXISTS(CommonStatisticNames.OP_EXISTS,
       "Calls of exists()"),
+  INVOCATION_GET_FILE_CHECKSUM(CommonStatisticNames.OP_GET_FILE_CHECKSUM,
+      "Calls of getFileChecksum()"),
   INVOCATION_GET_FILE_STATUS(CommonStatisticNames.OP_GET_FILE_STATUS,
       "Calls of getFileStatus()"),
   INVOCATION_GLOB_STATUS(CommonStatisticNames.OP_GLOB_STATUS,

+ 39 - 2
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md

@@ -1656,11 +1656,48 @@ in these metrics.
 
 ##<a name="further_reading"></a> Other Topics
 
-### Copying Data with distcp
+### <a name="distcp"></a> Copying Data with distcp
 
-Hadoop's `distcp` application can be used to copy data between a Hadoop
+Hadoop's `distcp` tool is often used to copy data between a Hadoop
 cluster and Amazon S3.
 See [Copying Data Between a Cluster and Amazon S3](https://hortonworks.github.io/hdp-aws/s3-copy-data/index.html)
 for details on S3 copying specifically.
 
+The `distcp update` command tries to do incremental updates of data.
+It is straightforward to verify when files do not match when they are of
+different length, but not when they are the same size.
+
+Distcp addresses this by comparing file checksums on the source and destination
+filesystems, which it tries to do *even if the filesystems have incompatible
+checksum algorithms*.
+
+The S3A connector can provide the HTTP etag header to the caller as the
+checksum of the uploaded file. Doing so will break distcp operations
+between hdfs and s3a.
+
+For this reason, the etag-as-checksum feature is disabled by default.
+
+```xml
+<property>
+  <name>fs.s3a.etag.checksum.enabled</name>
+  <value>false</value>
+  <description>
+    Should calls to getFileChecksum() return the etag value of the remote
+    object.
+    WARNING: if enabled, distcp operations between HDFS and S3 will fail unless
+    -skipcrccheck is set.
+  </description>
+</property>
+```
+
+If enabled, `distcp` between two S3 buckets can use the checksum to compare
+objects. Their checksums should be identical if they were either each uploaded
+as a single file PUT, or, if in a multipart PUT, in blocks of the same size,
+as configured by the value `fs.s3a.multipart.size`.
+
+To disable checksum verification in `distcp`, use the `-skipcrccheck` option:
+
+```bash
+hadoop distcp -update -skipcrccheck /user/alice/datasets s3a://alice-backup/datasets
+```
 

+ 47 - 6
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java

@@ -39,12 +39,30 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
 
 /**
  * Tests of the S3A FileSystem which don't have a specific home and can share
- * a filesystem instance with others..
+ * a filesystem instance with others.
+ * Checksums are turned on unless explicitly disabled for a test case.
  */
 public class ITestS3AMiscOperations extends AbstractS3ATestBase {
 
   private static final byte[] HELLO = "hello".getBytes(StandardCharsets.UTF_8);
 
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    // checksums are forced on.
+    enableChecksums(true);
+  }
+
+  /**
+   * Turn checksums on.
+   * Relies on the FS not caching the configuration option
+   * @param enabled enabled flag.
+   */
+  protected void enableChecksums(final boolean enabled) {
+    getFileSystem().getConf().setBoolean(Constants.ETAG_CHECKSUM_ENABLED,
+        enabled);
+  }
+
   @Test
   public void testCreateNonRecursiveSuccess() throws IOException {
     Path shouldWork = path("nonrecursivenode");
@@ -124,12 +142,26 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
     Path file1 = touchFile("file1");
     EtagChecksum checksum1 = fs.getFileChecksum(file1, 0);
     LOG.info("Checksum for {}: {}", file1, checksum1);
-    assertNotNull("file 1 checksum", checksum1);
+    assertNotNull("Null file 1 checksum", checksum1);
     assertNotEquals("file 1 checksum", 0, checksum1.getLength());
     assertEquals("checksums", checksum1,
         fs.getFileChecksum(touchFile("file2"), 0));
   }
 
+  /**
+   * Make sure that when checksums are disabled, the caller
+   * gets null back.
+   */
+  @Test
+  public void testChecksumDisabled() throws Throwable {
+    // checksums are forced off.
+    enableChecksums(false);
+    final S3AFileSystem fs = getFileSystem();
+    Path file1 = touchFile("file1");
+    EtagChecksum checksum1 = fs.getFileChecksum(file1, 0);
+    assertNull("Checksums are being generated", checksum1);
+  }
+
   /**
    * Verify that different file contents have different
    * checksums, and that that they aren't the same as the empty file.
@@ -138,6 +170,7 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
   @Test
   public void testNonEmptyFileChecksums() throws Throwable {
     final S3AFileSystem fs = getFileSystem();
+
     final Path file3 = mkFile("file3", HELLO);
     final EtagChecksum checksum1 = fs.getFileChecksum(file3, 0);
     assertNotNull("file 3 checksum", checksum1);
@@ -178,12 +211,20 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
   }
 
   @Test
-  public void testLengthPastEOF() throws Throwable {
+  public void testNegativeLengthDisabledChecksum() throws Throwable {
+    enableChecksums(false);
+    LambdaTestUtils.intercept(IllegalArgumentException.class,
+        () -> getFileSystem().getFileChecksum(mkFile("negative", HELLO), -1));
+  }
+
+  @Test
+  public void testChecksumLengthPastEOF() throws Throwable {
+    enableChecksums(true);
     final S3AFileSystem fs = getFileSystem();
     Path f = mkFile("file5", HELLO);
-    assertEquals(
-        fs.getFileChecksum(f, HELLO.length),
-        fs.getFileChecksum(f, HELLO.length * 2));
+    EtagChecksum l = fs.getFileChecksum(f, HELLO.length);
+    assertNotNull("Null checksum", l);
+    assertEquals(l, fs.getFileChecksum(f, HELLO.length * 2));
   }
 
   @Test

+ 0 - 5
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java

@@ -534,11 +534,6 @@ public final class DistCpOptions {
             + "mutually exclusive");
       }
 
-      if (!syncFolder && skipCRC) {
-        throw new IllegalArgumentException(
-            "Skip CRC is valid only with update options");
-      }
-
       if (!syncFolder && append) {
         throw new IllegalArgumentException(
             "Append is valid only with update options");

+ 38 - 15
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java

@@ -210,15 +210,30 @@ public class RetriableFileCopyCommand extends RetriableCommand {
       throws IOException {
     if (!DistCpUtils.checksumsAreEqual(sourceFS, source, sourceChecksum,
         targetFS, target)) {
-      StringBuilder errorMessage = new StringBuilder("Check-sum mismatch between ")
-          .append(source).append(" and ").append(target).append(".");
-      if (sourceFS.getFileStatus(source).getBlockSize() !=
+      StringBuilder errorMessage =
+          new StringBuilder("Checksum mismatch between ")
+              .append(source).append(" and ").append(target).append(".");
+      boolean addSkipHint = false;
+      String srcScheme = sourceFS.getScheme();
+      String targetScheme = targetFS.getScheme();
+      if (!srcScheme.equals(targetScheme)
+          && !(srcScheme.contains("hdfs") && targetScheme.contains("hdfs"))) {
+        // the filesystems are different and they aren't both hdfs connectors
+        errorMessage.append("Source and destination filesystems are of"
+            + " different types\n")
+            .append("Their checksum algorithms may be incompatible");
+        addSkipHint = true;
+      } else if (sourceFS.getFileStatus(source).getBlockSize() !=
           targetFS.getFileStatus(target).getBlockSize()) {
-        errorMessage.append(" Source and target differ in block-size.")
-            .append(" Use -pb to preserve block-sizes during copy.")
-            .append(" Alternatively, skip checksum-checks altogether, using -skipCrc.")
+        errorMessage.append(" Source and target differ in block-size.\n")
+            .append(" Use -pb to preserve block-sizes during copy.");
+        addSkipHint = true;
+      }
+      if (addSkipHint) {
+        errorMessage.append(" You can skip checksum-checks altogether "
+            + " with -skipcrccheck.\n")
             .append(" (NOTE: By skipping checksums, one runs the risk of " +
-                "masking data-corruption during file-transfer.)");
+                "masking data-corruption during file-transfer.)\n");
       }
       throw new IOException(errorMessage.toString());
     }
@@ -260,7 +275,8 @@ public class RetriableFileCopyCommand extends RetriableCommand {
     boolean finished = false;
     try {
       inStream = getInputStream(source, context.getConfiguration());
-      int bytesRead = readBytes(inStream, buf, sourceOffset);
+      seekIfRequired(inStream, sourceOffset);
+      int bytesRead = readBytes(inStream, buf);
       while (bytesRead >= 0) {
         if (chunkLength > 0 &&
             (totalBytesRead + bytesRead) >= chunkLength) {
@@ -276,7 +292,7 @@ public class RetriableFileCopyCommand extends RetriableCommand {
         if (finished) {
           break;
         }
-        bytesRead = readBytes(inStream, buf, sourceOffset);
+        bytesRead = readBytes(inStream, buf);
       }
       outStream.close();
       outStream = null;
@@ -299,13 +315,20 @@ public class RetriableFileCopyCommand extends RetriableCommand {
     context.setStatus(message.toString());
   }
 
-  private static int readBytes(ThrottledInputStream inStream, byte buf[],
-      long position) throws IOException {
+  private static int readBytes(ThrottledInputStream inStream, byte buf[])
+      throws IOException {
+    try {
+      return inStream.read(buf);
+    } catch (IOException e) {
+      throw new CopyReadException(e);
+    }
+  }
+
+  private static void seekIfRequired(ThrottledInputStream inStream,
+      long sourceOffset) throws IOException {
     try {
-      if (position == 0) {
-        return inStream.read(buf);
-      } else {
-        return inStream.read(position, buf, 0, buf.length);
+      if (sourceOffset != inStream.getPos()) {
+        inStream.seek(sourceOffset);
       }
     } catch (IOException e) {
       throw new CopyReadException(e);

+ 1 - 1
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java

@@ -527,7 +527,7 @@ public class DistCpUtils {
   /**
    * Utility to compare checksums for the paths specified.
    *
-   * If checksums's can't be retrieved, it doesn't fail the test
+   * If checksums can't be retrieved, it doesn't fail the test
    * Only time the comparison would fail is when checksums are
    * available and they don't match
    *

+ 27 - 21
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/ThrottledInputStream.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.tools.util;
 
-import org.apache.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -33,7 +33,7 @@ import java.io.InputStream;
  * (Thus, while the read-rate might exceed the maximum for a given short interval,
  * the average tends towards the specified maximum, overall.)
  */
-public class ThrottledInputStream extends InputStream {
+public class ThrottledInputStream extends InputStream implements Seekable {
 
   private final InputStream rawStream;
   private final float maxBytesPerSec;
@@ -95,25 +95,6 @@ public class ThrottledInputStream extends InputStream {
     return readLen;
   }
 
-  /**
-   * Read bytes starting from the specified position. This requires rawStream is
-   * an instance of {@link PositionedReadable}.
-   */
-  public int read(long position, byte[] buffer, int offset, int length)
-      throws IOException {
-    if (!(rawStream instanceof PositionedReadable)) {
-      throw new UnsupportedOperationException(
-          "positioned read is not supported by the internal stream");
-    }
-    throttle();
-    int readLen = ((PositionedReadable) rawStream).read(position, buffer,
-        offset, length);
-    if (readLen != -1) {
-      bytesRead += readLen;
-    }
-    return readLen;
-  }
-
   private void throttle() throws IOException {
     while (getBytesPerSec() > maxBytesPerSec) {
       try {
@@ -165,4 +146,29 @@ public class ThrottledInputStream extends InputStream {
         ", totalSleepTime=" + totalSleepTime +
         '}';
   }
+
+  private void checkSeekable() throws IOException {
+    if (!(rawStream instanceof Seekable)) {
+      throw new UnsupportedOperationException(
+          "seek operations are unsupported by the internal stream");
+    }
+  }
+
+  @Override
+  public void seek(long pos) throws IOException {
+    checkSeekable();
+    ((Seekable) rawStream).seek(pos);
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    checkSeekable();
+    return ((Seekable) rawStream).getPos();
+  }
+
+  @Override
+  public boolean seekToNewSource(long targetPos) throws IOException {
+    checkSeekable();
+    return ((Seekable) rawStream).seekToNewSource(targetPos);
+  }
 }

+ 30 - 8
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java

@@ -42,8 +42,10 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.tools.CopyListingFileStatus;
 import org.apache.hadoop.tools.DistCpConstants;
 import org.apache.hadoop.tools.DistCpOptionSwitch;
@@ -55,6 +57,10 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
+import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+
 public class TestCopyMapper {
   private static final Log LOG = LogFactory.getLog(TestCopyMapper.class);
   private static List<Path> pathList = new ArrayList<Path>();
@@ -248,7 +254,11 @@ public class TestCopyMapper {
 
     // do the distcp again with -update and -append option
     CopyMapper copyMapper = new CopyMapper();
-    StubContext stubContext = new StubContext(getConfiguration(), null, 0);
+    Configuration conf = getConfiguration();
+    // set the buffer size to 1/10th the size of the file.
+    conf.setInt(DistCpOptionSwitch.COPY_BUFFER_SIZE.getConfigLabel(),
+        DEFAULT_FILE_SIZE/10);
+    StubContext stubContext = new StubContext(conf, null, 0);
     Mapper<Text, CopyListingFileStatus, Text, Text>.Context context =
         stubContext.getContext();
     // Enable append 
@@ -257,6 +267,10 @@ public class TestCopyMapper {
     copyMapper.setup(context);
 
     int numFiles = 0;
+    MetricsRecordBuilder rb =
+        getMetrics(cluster.getDataNodes().get(0).getMetrics().name());
+    String readCounter = "ReadsFromLocalClient";
+    long readsFromClient = getLongCounter(readCounter, rb);
     for (Path path: pathList) {
       if (fs.getFileStatus(path).isFile()) {
         numFiles++;
@@ -274,6 +288,15 @@ public class TestCopyMapper {
         .getValue());
     Assert.assertEquals(numFiles, stubContext.getReporter().
         getCounter(CopyMapper.Counter.COPY).getValue());
+    rb = getMetrics(cluster.getDataNodes().get(0).getMetrics().name());
+    /*
+     * added as part of HADOOP-15292 to ensure that multiple readBlock()
+     * operations are not performed to read a block from a single Datanode.
+     * assert assumes that there is only one block per file, and that the number
+     * of files appended to in appendSourceData() above is captured by the
+     * variable numFiles.
+     */
+    assertCounter(readCounter, readsFromClient + numFiles, rb);
   }
 
   private void testCopy(boolean preserveChecksum) throws Exception {
@@ -915,7 +938,7 @@ public class TestCopyMapper {
   }
 
   @Test(timeout=40000)
-  public void testCopyFailOnBlockSizeDifference() {
+  public void testCopyFailOnBlockSizeDifference() throws Exception {
     try {
       deleteState();
       createSourceDataWithDifferentBlockSize();
@@ -942,12 +965,11 @@ public class TestCopyMapper {
 
       Assert.fail("Copy should have failed because of block-size difference.");
     }
-    catch (Exception exception) {
-      // Check that the exception suggests the use of -pb/-skipCrc.
-      Assert.assertTrue("Failure exception should have suggested the use of -pb.",
-          exception.getCause().getCause().getMessage().contains("pb"));
-      Assert.assertTrue("Failure exception should have suggested the use of -skipCrc.",
-          exception.getCause().getCause().getMessage().contains("skipCrc"));
+    catch (IOException exception) {
+      // Check that the exception suggests the use of -pb/-skipcrccheck.
+      Throwable cause = exception.getCause().getCause();
+      GenericTestUtils.assertExceptionContains("-pb", cause);
+      GenericTestUtils.assertExceptionContains("-skipcrccheck", cause);
     }
   }
 

+ 27 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -3585,6 +3585,22 @@ public class YarnConfiguration extends Configuration {
       DEFAULT_TIMELINE_SERVICE_COLLECTOR_WEBAPP_HTTPS_ADDRESS =
       DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS;
 
+  /**
+   * Settings for NUMA awareness.
+   */
+  public static final String NM_NUMA_AWARENESS_ENABLED = NM_PREFIX
+      + "numa-awareness.enabled";
+  public static final boolean DEFAULT_NM_NUMA_AWARENESS_ENABLED = false;
+  public static final String NM_NUMA_AWARENESS_READ_TOPOLOGY = NM_PREFIX
+      + "numa-awareness.read-topology";
+  public static final boolean DEFAULT_NM_NUMA_AWARENESS_READ_TOPOLOGY = false;
+  public static final String NM_NUMA_AWARENESS_NODE_IDS = NM_PREFIX
+      + "numa-awareness.node-ids";
+  public static final String NM_NUMA_AWARENESS_NUMACTL_CMD = NM_PREFIX
+      + "numa-awareness.numactl.cmd";
+  public static final String DEFAULT_NM_NUMA_AWARENESS_NUMACTL_CMD =
+      "/usr/bin/numactl";
+
   public YarnConfiguration() {
     super();
   }
@@ -3791,6 +3807,17 @@ public class YarnConfiguration extends Configuration {
         YarnConfiguration.DEFAULT_SYSTEM_METRICS_PUBLISHER_ENABLED);
   }
 
+  /**
+   * Returns whether the NUMA awareness is enabled.
+   *
+   * @param conf the configuration
+   * @return whether the NUMA awareness is enabled.
+   */
+  public static boolean numaAwarenessEnabled(Configuration conf) {
+    return conf.getBoolean(NM_NUMA_AWARENESS_ENABLED,
+        DEFAULT_NM_NUMA_AWARENESS_ENABLED);
+  }
+
   /* For debugging. mp configurations to system output as XML format. */
   public static void main(String[] args) throws Exception {
     new YarnConfiguration(new Configuration()).writeXml(System.out);

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

@@ -123,8 +123,8 @@ public class ApiServer {
             return null;
           }
         });
-        serviceStatus.setDiagnostics("Service "+service.getName() +
-            " saved.");
+        serviceStatus.setDiagnostics("Service " + service.getName() +
+            " version " + service.getVersion() + " saved.");
       } else {
         ApplicationId applicationId = ugi
             .doAs(new PrivilegedExceptionAction<ApplicationId>() {

+ 8 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md

@@ -21,6 +21,8 @@ POST URL - http://localhost:9191/ws/v1/services
 ```json
 {
   "name": "hello-world",
+  "version": "1.0.0",
+  "description": "hello world example",
   "components" :
     [
       {
@@ -48,6 +50,8 @@ Note, lifetime value of -1 means unlimited lifetime.
 ```json
 {
     "name": "hello-world",
+    "version": "1.0.0",
+    "description": "hello world example",
     "id": "application_1503963985568_0002",
     "lifetime": -1,
     "components": [
@@ -72,7 +76,7 @@ Note, lifetime value of -1 means unlimited lifetime.
                     "state": "READY",
                     "launch_time": 1504051512412,
                     "bare_host": "10.22.8.143",
-                    "component_name": "hello-0"
+                    "component_instance_name": "hello-0"
                 },
                 {
                     "id": "container_e03_1503963985568_0002_01_000002",
@@ -81,7 +85,7 @@ Note, lifetime value of -1 means unlimited lifetime.
                     "state": "READY",
                     "launch_time": 1504051536450,
                     "bare_host": "10.22.8.143",
-                    "component_name": "hello-1"
+                    "component_instance_name": "hello-1"
                 }
             ],
             "launch_command": "./start_nginx.sh",
@@ -154,6 +158,8 @@ POST URL - http://localhost:9191:/ws/v1/services/hbase-app-1
 ```json
 {
   "name": "hbase-app-1",
+  "version": "1.0.0",
+  "description": "hbase service",
   "lifetime": "3600",
   "components": [
     {

+ 7 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml

@@ -197,10 +197,17 @@ definitions:
     description: a service resource has the following attributes.
     required:
       - name
+      - version
     properties:
       name:
         type: string
         description: A unique service name. If Registry DNS is enabled, the max length is 63 characters.
+      version:
+        type: string
+        description: Version of the service.
+      description:
+        type: string
+        description: Description of the service.
       id:
         type: string
         description: A unique service id.

Some files were not shown because too many files changed in this diff