Browse Source

Merge remote-tracking branch 'apache-commit/trunk' into HDFS-12996

Arpit Agarwal 7 years ago
parent
commit
8d4aa014d7
100 changed files with 2362 additions and 1218 deletions
  1. 2 2
      BUILDING.txt
  2. 14 0
      NOTICE.txt
  3. 145 0
      dev-support/bin/verify-license-files
  4. 1 1
      dev-support/docker/Dockerfile
  5. 1 1
      hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
  6. 2 3
      hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml
  7. 6 1
      hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml
  8. 12 0
      hadoop-common-project/hadoop-auth/pom.xml
  9. 5 21
      hadoop-common-project/hadoop-common/pom.xml
  10. 3 0
      hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
  11. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
  12. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  13. 11 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
  14. 4 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicy.java
  15. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/Log4Json.java
  16. 83 76
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
  17. 3 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationFilterInitializer.java
  18. 0 124
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationWithProxyUserFilter.java
  19. 91 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/RuleBasedLdapGroupsMapping.java
  20. 12 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
  21. 24 0
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  22. 3 1
      hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
  23. 8 0
      hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm
  24. 3 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
  25. 22 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
  26. 0 494
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpengo.java
  27. 2 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java
  28. 40 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
  29. 10 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java
  30. 0 79
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationWithProxyUserFilter.java
  31. 99 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRuleBasedLdapGroupsMapping.java
  32. 46 22
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
  33. 1 1
      hadoop-common-project/hadoop-kms/pom.xml
  34. 1 1
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
  35. 5 1
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
  36. 0 39
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSWithZK.java
  37. 2 1
      hadoop-common-project/hadoop-minikdc/src/test/java/org/apache/hadoop/minikdc/TestMiniKdc.java
  38. 15 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  39. 2 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java
  40. 55 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  41. 86 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  42. 2 44
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  43. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
  44. 1 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  45. 6 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
  46. 4 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
  47. 8 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
  48. 6 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RequestHedgingProxyProvider.java
  49. 83 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
  50. 56 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  51. 3 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
  52. 45 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRequestHedgingProxyProvider.java
  53. 1 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
  54. 18 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
  55. 27 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
  56. 13 5
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
  57. 174 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
  58. 8 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerWebServer.java
  59. 5 21
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  60. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  61. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
  62. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
  63. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
  64. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
  65. 36 35
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
  66. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
  67. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
  68. 4 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  69. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockRecoveryWorker.java
  70. 26 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  71. 20 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
  72. 33 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java
  73. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java
  74. 0 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java
  75. 16 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
  76. 14 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
  77. 33 26
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
  78. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java
  79. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPoolId.java
  80. 20 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
  81. 13 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
  82. 13 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaManager.java
  83. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
  84. 12 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
  85. 18 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipState.java
  86. 26 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java
  87. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/RouterState.java
  88. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
  89. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
  90. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  91. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
  92. 4 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  93. 0 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  94. 30 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  95. 16 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
  96. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffList.java
  97. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffListByArrayList.java
  98. 585 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffListBySkipList.java
  99. 70 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryDiffListFactory.java
  100. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java

+ 2 - 2
BUILDING.txt

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

+ 14 - 0
NOTICE.txt

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

+ 145 - 0
dev-support/bin/verify-license-files

@@ -0,0 +1,145 @@
+#!/usr/bin/env bash
+#
+# 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.
+
+
+## @description  check a file
+## @audience     private
+## @stability    evolving
+## @replaceable  no
+## @param        filename
+## @param        jarfile
+## @return       0 = destroy verify dir
+## @return       1 = keep verify dir
+function process_file
+{
+  declare check=$1
+  declare fqfn=$2
+  declare fn
+  declare keepdir
+  declare tf
+  declare count
+  declare valid
+
+  fn=$(basename "${fqfn}")
+  keepdir=false
+  valid=0
+  count=0
+
+  unzip -o -d "${WORK_DIR}/${fn}" "${fqfn}" '*'"${check}"'*' >/dev/null 2>&1
+
+  while read -r tf; do
+    ((count = count + 1))
+    if diff -q "${DIST_DIR}/${check}.txt" "${tf}" >/dev/null 2>&1; then
+      ((valid = valid + 1))
+    fi
+  done < <(find "${WORK_DIR}/${fn}" -name "${check}"'*')
+
+  if [[ "${count}" -eq 0 ]]; then
+    hadoop_error "ERROR: ${fn}: Missing a ${check} file"
+  elif [[ "${count}" -gt 1 ]]; then
+    hadoop_error "WARNING: ${fn}: Found ${count} ${check} files (${valid} were valid)"
+    keepdir=true
+  fi
+
+  if [[ "${valid}" -eq 0 ]] && [[ "${count}" -gt 0 ]]; then
+      hadoop_error "ERROR: ${fn}: No valid ${check} found"
+      keepdir=true
+  fi
+
+  if [[ "${keepdir}" = "false" ]]; then
+    return 0
+  else
+    return 1
+  fi
+}
+
+
+## @description  check a jar
+## @audience     private
+## @stability    evolving
+## @replaceable  no
+## @param        jarfile
+## @return       0 - success
+## @return       1 - errors
+function process_jar
+{
+  declare fqfn=$1
+  declare fn
+  declare keepwork
+
+  fn=$(basename "${fqfn}")
+  keepwork=false
+
+  if [[ ! ${fn} =~ hadoop-.*-${PROJ_VERSION} ]]; then
+    return
+  fi
+
+  mkdir -p "${WORK_DIR}/${fn}"
+
+  if ! process_file LICENSE "${fqfn}"; then
+    keepwork=true
+  fi
+
+  if ! process_file NOTICE "${fqfn}"; then
+    keepwork=true
+  fi
+
+  if [[ "${keepwork}" = "false" ]]; then
+    rm -rf "${WORK_DIR:?}/${fn}"
+    return 0
+  else
+    hadoop_error ""
+    return 1
+  fi
+}
+
+
+MYNAME="${BASH_SOURCE-$0}"
+#shellcheck disable=SC2034
+HADOOP_SHELL_EXECNAME="${MYNAME##*/}"
+BINDIR=$(cd -P -- "$(dirname -- "${MYNAME}")" >/dev/null && pwd -P)
+#shellcheck disable=SC2034
+HADOOP_LIBEXEC_DIR="${BINDIR}/../../hadoop-common-project/hadoop-common/src/main/bin"
+
+#shellcheck disable=SC1090
+. "${HADOOP_LIBEXEC_DIR}/hadoop-functions.sh"
+
+HADOOP_LIBEXEC_DIR=$(hadoop_abs "${HADOOP_LIBEXEC_DIR}")
+BINDIR=$(hadoop_abs "${BINDIR}")
+BASEDIR=$(hadoop_abs "${BINDIR}/../..")
+
+pushd "${BASEDIR}" >/dev/null
+#shellcheck disable=SC2016
+PROJ_VERSION=$(mvn -q -Dexec.executable="echo" -Dexec.args='${project.version}' --non-recursive exec:exec)
+popd >/dev/null
+
+DIST_DIR="${BASEDIR}/hadoop-dist/target/hadoop-${PROJ_VERSION}"
+WORK_DIR="${BASEDIR}/patchprocess/verify"
+
+rm -rf "${WORK_DIR:?}"
+mkdir -p "${WORK_DIR}"
+
+while read -r filename; do
+  process_jar "${filename}"
+  ((ret = ret + $? ))
+done < <(find "${DIST_DIR}" \
+  -name "hadoop-*${PROJ_VERSION}*.jar" -or \
+  -name "hadoop-*${PROJ_VERSION}*.war")
+if [[ ${ret} -gt 0 ]]; then
+  exit 1
+fi
+exit 0

+ 1 - 1
dev-support/docker/Dockerfile

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

+ 1 - 1
hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml

@@ -236,7 +236,7 @@
     </moduleSet>
     <moduleSet>
       <includes>
-        <include>org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase-server</include>
+        <include>org.apache.hadoop:${hbase-server-artifactid}</include>
       </includes>
       <binaries>
         <outputDirectory>share/hadoop/${hadoop.component}/timelineservice</outputDirectory>

+ 2 - 3
hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml

@@ -50,8 +50,6 @@
 <module name="Checker">
 
     <module name="SuppressWarningsFilter"/>
-    <module name="SuppressionCommentFilter"/>
-    <module name="SuppressWithNearbyCommentFilter"/>
 
     <!-- Checks that a package.html file exists for each package.     -->
     <!-- See http://checkstyle.sf.net/config_javadoc.html#PackageHtml -->
@@ -72,7 +70,8 @@
     <module name="TreeWalker">
 
         <module name="SuppressWarningsHolder"/>
-        <module name="FileContentsHolder"/>
+        <module name="SuppressionCommentFilter"/>
+        <module name="SuppressWithNearbyCommentFilter"/>
 
 
         <!-- Checks for Javadoc comments.                     -->

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

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

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

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

+ 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

@@ -306,3 +306,6 @@ log4j.appender.EWMA.maxUniqueMessages=${yarn.ewma.maxUniqueMessages}
 #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

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java

@@ -502,7 +502,7 @@ public class FileStatus implements Writable, Comparable<Object>,
     setPath(other.getPath());
     attr = attributes(other.hasAcl(), other.isEncrypted(),
         other.isErasureCoded(), other.isSnapshotEnabled());
-    assert (isDirectory() && getSymlink() == null) || !isDirectory();
+    assert !(isDirectory() && isSymlink()) : "A directory cannot be a symlink";
   }
 
   /**

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

@@ -1649,7 +1649,7 @@ public abstract class FileSystem extends Configured implements Closeable {
   /**
    * Mark a path to be deleted when its FileSystem is closed.
    * When the JVM shuts down cleanly, all cached FileSystem objects will be
-   * closed automatically —these the marked paths will be deleted as a result.
+   * closed automatically. These the marked paths will be deleted as a result.
    *
    * If a FileSystem instance is not cached, i.e. has been created with
    * {@link #createFileSystem(URI, Configuration)}, then the paths will

+ 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);

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

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

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/Log4Json.java

@@ -103,7 +103,7 @@ public class Log4Json extends Layout {
   /**
    * Jackson factories are thread safe when constructing parsers and generators.
    * They are not thread safe in configure methods; if there is to be any
-   * configuration it must be done in a static intializer block.
+   * configuration it must be done in a static initializer block.
    */
   private static final JsonFactory factory = new MappingJsonFactory();
   private static final ObjectReader READER = new ObjectMapper(factory).reader();

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

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

+ 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;
-  }
-}

+ 91 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/RuleBasedLdapGroupsMapping.java

@@ -0,0 +1,91 @@
+/**
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * This class uses {@link LdapGroupsMapping} for group lookup and applies the
+ * rule configured on the group names.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+@InterfaceStability.Evolving
+public class RuleBasedLdapGroupsMapping extends LdapGroupsMapping {
+
+  public static final String CONVERSION_RULE_KEY = LDAP_CONFIG_PREFIX +
+      ".conversion.rule";
+
+  private static final String CONVERSION_RULE_DEFAULT = "none";
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RuleBasedLdapGroupsMapping.class);
+
+  private Rule rule;
+
+  /**
+   * Supported rules applicable for group name modification.
+   */
+  private enum Rule {
+    TO_UPPER, TO_LOWER, NONE
+  }
+
+  @Override
+  public synchronized void setConf(Configuration conf) {
+    super.setConf(conf);
+    String value = conf.get(CONVERSION_RULE_KEY, CONVERSION_RULE_DEFAULT);
+    try {
+      rule = Rule.valueOf(value.toUpperCase());
+    } catch (IllegalArgumentException iae) {
+      LOG.warn("Invalid {} configured: '{}'. Using default value: '{}'",
+          CONVERSION_RULE_KEY, value, CONVERSION_RULE_DEFAULT);
+    }
+  }
+
+    /**
+     * Returns list of groups for a user.
+     * This calls {@link LdapGroupsMapping}'s getGroups and applies the
+     * configured rules on group names before returning.
+     *
+     * @param user get groups for this user
+     * @return list of groups for a given user
+     */
+  @Override
+  public synchronized List<String> getGroups(String user) {
+    List<String> groups = super.getGroups(user);
+    switch (rule) {
+    case TO_UPPER:
+      return groups.stream().map(StringUtils::toUpperCase).collect(
+          Collectors.toList());
+    case TO_LOWER:
+      return groups.stream().map(StringUtils::toLowerCase).collect(
+          Collectors.toList());
+    case NONE:
+    default:
+      return groups;
+    }
+  }
+
+}

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

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

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

@@ -275,6 +275,19 @@
   </description>
 </property>
 
+<property>
+  <name>hadoop.security.group.mapping.ldap.conversion.rule</name>
+  <value>none</value>
+  <description>
+    The rule is applied on the group names received from LDAP when
+    RuleBasedLdapGroupsMapping is configured.
+    Supported rules are "to_upper", "to_lower" and "none".
+    to_upper: This will convert all the group names to uppercase.
+    to_lower: This will convert all the group names to lowercase.
+    none: This will retain the source formatting, this is default value.
+  </description>
+</property>
+
 <property>
   <name>hadoop.security.credential.clear-text-fallback</name>
   <value>true</value>
@@ -1534,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>

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

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

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

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

+ 3 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.security.CompositeGroupsMapping;
 import org.apache.hadoop.security.HttpCrossOriginFilterInitializer;
 import org.apache.hadoop.security.LdapGroupsMapping;
+import org.apache.hadoop.security.RuleBasedLdapGroupsMapping;
 import org.apache.hadoop.security.http.CrossOriginFilter;
 import org.apache.hadoop.security.ssl.SSLFactory;
 
@@ -74,7 +75,8 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
         ZKFailoverController.class,
         SSLFactory.class,
         CompositeGroupsMapping.class,
-        CodecUtil.class
+        CodecUtil.class,
+        RuleBasedLdapGroupsMapping.class
         };
 
     // Initialize used variables

+ 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());
     }
   }

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

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

+ 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);
-  }
-
-}

+ 99 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRuleBasedLdapGroupsMapping.java

@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import javax.naming.NamingException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.security.RuleBasedLdapGroupsMapping
+    .CONVERSION_RULE_KEY;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.eq;
+
+/**
+ * Test cases to verify the rules supported by RuleBasedLdapGroupsMapping.
+ */
+public class TestRuleBasedLdapGroupsMapping  {
+
+  @Test
+  public void testGetGroupsToUpper() throws NamingException {
+    RuleBasedLdapGroupsMapping groupsMapping = Mockito.spy(
+        new RuleBasedLdapGroupsMapping());
+    List<String> groups = new ArrayList<>();
+    groups.add("group1");
+    groups.add("group2");
+    Mockito.doReturn(groups).when((LdapGroupsMapping) groupsMapping)
+        .doGetGroups(eq("admin"), anyInt());
+
+    Configuration conf = new Configuration();
+    conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
+    conf.set(CONVERSION_RULE_KEY, "to_upper");
+    groupsMapping.setConf(conf);
+
+    List<String> groupsUpper = new ArrayList<>();
+    groupsUpper.add("GROUP1");
+    groupsUpper.add("GROUP2");
+    Assert.assertEquals(groupsUpper, groupsMapping.getGroups("admin"));
+  }
+
+  @Test
+  public void testGetGroupsToLower() throws NamingException {
+    RuleBasedLdapGroupsMapping groupsMapping = Mockito.spy(
+        new RuleBasedLdapGroupsMapping());
+    List<String> groups = new ArrayList<>();
+    groups.add("GROUP1");
+    groups.add("GROUP2");
+    Mockito.doReturn(groups).when((LdapGroupsMapping) groupsMapping)
+        .doGetGroups(eq("admin"), anyInt());
+
+    Configuration conf = new Configuration();
+    conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
+    conf.set(CONVERSION_RULE_KEY, "to_lower");
+    groupsMapping.setConf(conf);
+
+    List<String> groupsLower = new ArrayList<>();
+    groupsLower.add("group1");
+    groupsLower.add("group2");
+    Assert.assertEquals(groupsLower, groupsMapping.getGroups("admin"));
+  }
+
+  @Test
+  public void testGetGroupsInvalidRule() throws NamingException {
+    RuleBasedLdapGroupsMapping groupsMapping = Mockito.spy(
+        new RuleBasedLdapGroupsMapping());
+    List<String> groups = new ArrayList<>();
+    groups.add("group1");
+    groups.add("GROUP2");
+    Mockito.doReturn(groups).when((LdapGroupsMapping) groupsMapping)
+        .doGetGroups(eq("admin"), anyInt());
+
+    Configuration conf = new Configuration();
+    conf.set(LdapGroupsMapping.LDAP_URL_KEY, "ldap://test");
+    conf.set(CONVERSION_RULE_KEY, "none");
+    groupsMapping.setConf(conf);
+
+    Assert.assertEquals(groups, groupsMapping.getGroups("admin"));
+  }
+
+}

+ 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;
+  }
+
 }

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

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

+ 1 - 1
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java

@@ -152,7 +152,7 @@ public class KMSWebApp implements ServletContextListener {
 
       kmsAudit = new KMSAudit(kmsConf);
 
-      // intializing the KeyProvider
+      // initializing the KeyProvider
       String providerString = kmsConf.get(KMSConfiguration.KEY_PROVIDER_URI);
       if (providerString == null) {
         throw new IllegalStateException("No KeyProvider has been defined");

+ 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;
               }

+ 0 - 39
hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSWithZK.java

@@ -19,58 +19,19 @@ package org.apache.hadoop.crypto.key.kms.server;
 
 import org.apache.curator.test.TestingServer;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.KeyProvider;
-import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
-import org.apache.hadoop.crypto.key.KeyProvider.Options;
-import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
-import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
-import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
-import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
 import org.apache.hadoop.crypto.key.kms.KMSRESTConstants;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.minikdc.MiniKdc;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.util.ZKSignerSecretProvider;
-import org.apache.hadoop.security.authorize.AuthorizationException;
-import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
-import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-import javax.security.auth.Subject;
-import javax.security.auth.kerberos.KerberosPrincipal;
-import javax.security.auth.login.AppConfigurationEntry;
-import javax.security.auth.login.LoginContext;
 import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.Writer;
 import java.net.HttpURLConnection;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.net.SocketTimeoutException;
-import java.net.URI;
 import java.net.URL;
-import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.Callable;
 
 public class TestKMSWithZK {
 

+ 2 - 1
hadoop-common-project/hadoop-minikdc/src/test/java/org/apache/hadoop/minikdc/TestMiniKdc.java

@@ -165,7 +165,8 @@ public class TestMiniKdc extends KerberosSecurityTestcase {
       loginContext.logout();
 
     } finally {
-      if (loginContext != null) {
+      if (loginContext != null && loginContext.getSubject() != null
+          && !loginContext.getSubject().getPrincipals().isEmpty()) {
         loginContext.logout();
       }
     }

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

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

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

@@ -87,6 +87,8 @@ public class DFSOpsCountStatistics extends StorageStatistics {
     SET_STORAGE_POLICY("op_set_storagePolicy"),
     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");
 

+ 55 - 8
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -852,7 +852,19 @@ public class DFSOutputStream extends FSOutputSummer
 
   protected synchronized void closeImpl() throws IOException {
     if (isClosed()) {
-      getStreamer().getLastException().check(true);
+      LOG.debug("Closing an already closed stream. [Stream:{}, streamer:{}]",
+          closed, getStreamer().streamerClosed());
+      try {
+        getStreamer().getLastException().check(true);
+      } catch (IOException ioe) {
+        cleanupAndRethrowIOException(ioe);
+      } finally {
+        if (!closed) {
+          // If stream is not closed but streamer closed, clean up the stream.
+          // Most importantly, end the file lease.
+          closeThreads(true);
+        }
+      }
       return;
     }
 
@@ -867,14 +879,12 @@ public class DFSOutputStream extends FSOutputSummer
         setCurrentPacketToEmpty();
       }
 
-      flushInternal();             // flush all data to Datanodes
-      // get last block before destroying the streamer
-      ExtendedBlock lastBlock = getStreamer().getBlock();
-
-      try (TraceScope ignored =
-               dfsClient.getTracer().newScope("completeFile")) {
-        completeFile(lastBlock);
+      try {
+        flushInternal();             // flush all data to Datanodes
+      } catch (IOException ioe) {
+        cleanupAndRethrowIOException(ioe);
       }
+      completeFile();
     } catch (ClosedChannelException ignored) {
     } finally {
       // Failures may happen when flushing data.
@@ -886,6 +896,43 @@ public class DFSOutputStream extends FSOutputSummer
     }
   }
 
+  private void completeFile() throws IOException {
+    // get last block before destroying the streamer
+    ExtendedBlock lastBlock = getStreamer().getBlock();
+    try (TraceScope ignored =
+        dfsClient.getTracer().newScope("completeFile")) {
+      completeFile(lastBlock);
+    }
+  }
+
+  /**
+   * Determines whether an IOException thrown needs extra cleanup on the stream.
+   * Space quota exceptions will be thrown when getting new blocks, so the
+   * open HDFS file need to be closed.
+   *
+   * @param ioe the IOException
+   * @return whether the stream needs cleanup for the given IOException
+   */
+  private boolean exceptionNeedsCleanup(IOException ioe) {
+    return ioe instanceof DSQuotaExceededException
+        || ioe instanceof QuotaByStorageTypeExceededException;
+  }
+
+  private void cleanupAndRethrowIOException(IOException ioe)
+      throws IOException {
+    if (exceptionNeedsCleanup(ioe)) {
+      final MultipleIOException.Builder b = new MultipleIOException.Builder();
+      b.add(ioe);
+      try {
+        completeFile();
+      } catch (IOException e) {
+        b.add(e);
+        throw b.build();
+      }
+    }
+    throw ioe;
+  }
+
   // should be called holding (this) lock since setTestFilename() may
   // be called during unit tests
   protected void completeFile(ExtendedBlock last) throws IOException {

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

@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStreamBuilder;
 import org.apache.hadoop.fs.FSLinkResolver;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
@@ -2020,8 +2021,13 @@ public class DistributedFileSystem extends FileSystem
       @Override
       public RemoteIterator<SnapshotDiffReportListing> doCall(final Path p)
           throws IOException {
-        return new SnapshotDiffReportListingIterator(
-            getPathName(p), fromSnapshot, toSnapshot);
+        if (!isValidSnapshotName(fromSnapshot) || !isValidSnapshotName(
+            toSnapshot)) {
+          throw new UnsupportedOperationException("Remote Iterator is"
+              + "supported for snapshotDiffReport between two snapshots");
+        }
+        return new SnapshotDiffReportListingIterator(getPathName(p),
+            fromSnapshot, toSnapshot);
       }
 
       @Override
@@ -2081,9 +2087,23 @@ public class DistributedFileSystem extends FileSystem
     }
   }
 
+  private boolean isValidSnapshotName(String snapshotName) {
+    // If any of the snapshots specified in the getSnapshotDiffReport call
+    // is null or empty, it points to the current tree.
+    return (snapshotName != null && !snapshotName.isEmpty());
+  }
+
   private SnapshotDiffReport getSnapshotDiffReportInternal(
       final String snapshotDir, final String fromSnapshot,
       final String toSnapshot) throws IOException {
+    // In case the diff needs to be computed between a snapshot and the current
+    // tree, we should not do iterative diffReport computation as the iterative
+    // approach might fail if in between the rpc calls the current tree
+    // changes in absence of the global fsn lock.
+    if (!isValidSnapshotName(fromSnapshot) || !isValidSnapshotName(
+        toSnapshot)) {
+      return dfs.getSnapshotDiffReport(snapshotDir, fromSnapshot, toSnapshot);
+    }
     byte[] startPath = DFSUtilClient.EMPTY_BYTES;
     int index = -1;
     SnapshotDiffReportGenerator snapshotDiffReport;
@@ -2581,6 +2601,70 @@ public class DistributedFileSystem extends FileSystem
     }.resolve(this, absF);
   }
 
+  /* HDFS only */
+  public void provisionEZTrash(final Path path,
+      final FsPermission trashPermission) throws IOException {
+    Path absF = fixRelativePart(path);
+    new FileSystemLinkResolver<Void>() {
+      @Override
+      public Void doCall(Path p) throws IOException {
+        provisionEZTrash(getPathName(p), trashPermission);
+        return null;
+      }
+
+      @Override
+      public Void next(FileSystem fs, Path p) throws IOException {
+        if (fs instanceof DistributedFileSystem) {
+          DistributedFileSystem myDfs = (DistributedFileSystem)fs;
+          myDfs.provisionEZTrash(p, trashPermission);
+          return null;
+        }
+        throw new UnsupportedOperationException("Cannot provisionEZTrash " +
+            "through a symlink to a non-DistributedFileSystem: " + fs + " -> "
+            + p);
+      }
+    }.resolve(this, absF);
+  }
+
+  private void provisionEZTrash(String path, FsPermission trashPermission)
+      throws IOException {
+    // make sure the path is an EZ
+    EncryptionZone ez = dfs.getEZForPath(path);
+    if (ez == null) {
+      throw new IllegalArgumentException(path + " is not an encryption zone.");
+    }
+
+    String ezPath = ez.getPath();
+    if (!path.toString().equals(ezPath)) {
+      throw new IllegalArgumentException(path + " is not the root of an " +
+          "encryption zone. Do you mean " + ez.getPath() + "?");
+    }
+
+    // check if the trash directory exists
+    Path trashPath = new Path(ez.getPath(), FileSystem.TRASH_PREFIX);
+    try {
+      FileStatus trashFileStatus = getFileStatus(trashPath);
+      String errMessage = "Will not provision new trash directory for " +
+          "encryption zone " + ez.getPath() + ". Path already exists.";
+      if (!trashFileStatus.isDirectory()) {
+        errMessage += "\r\n" +
+            "Warning: " + trashPath.toString() + " is not a directory";
+      }
+      if (!trashFileStatus.getPermission().equals(trashPermission)) {
+        errMessage += "\r\n" +
+            "Warning: the permission of " +
+            trashPath.toString() + " is not " + trashPermission;
+      }
+      throw new FileAlreadyExistsException(errMessage);
+    } catch (FileNotFoundException ignored) {
+      // no trash path
+    }
+
+    // Update the permission bits
+    mkdir(trashPath, trashPermission);
+    setPermission(trashPath, trashPermission);
+  }
+
   @Override
   public void setXAttr(Path path, final String name, final byte[] value,
       final EnumSet<XAttrSetFlag> flag) throws IOException {

+ 2 - 44
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java

@@ -24,9 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.CacheFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
@@ -330,7 +328,7 @@ public class HdfsAdmin {
         throw new HadoopIllegalArgumentException(
             "can not have both PROVISION_TRASH and NO_TRASH flags");
       }
-      this.provisionEZTrash(path);
+      dfs.provisionEZTrash(path, TRASH_PERMISSION);
     }
   }
 
@@ -341,7 +339,7 @@ public class HdfsAdmin {
    * @throws IOException if the trash directory can not be created.
    */
   public void provisionEncryptionZoneTrash(Path path) throws IOException {
-    this.provisionEZTrash(path);
+    dfs.provisionEZTrash(path, TRASH_PERMISSION);
   }
 
   /**
@@ -603,46 +601,6 @@ public class HdfsAdmin {
     dfs.disableErasureCodingPolicy(ecPolicyName);
   }
 
-  private void provisionEZTrash(Path path) throws IOException {
-    // make sure the path is an EZ
-    EncryptionZone ez = dfs.getEZForPath(path);
-    if (ez == null) {
-      throw new IllegalArgumentException(path + " is not an encryption zone.");
-    }
-
-    String ezPath = ez.getPath();
-    if (!path.toString().equals(ezPath)) {
-      throw new IllegalArgumentException(path + " is not the root of an " +
-          "encryption zone. Do you mean " + ez.getPath() + "?");
-    }
-
-    // check if the trash directory exists
-
-    Path trashPath = new Path(ez.getPath(), FileSystem.TRASH_PREFIX);
-
-    try {
-      FileStatus trashFileStatus = dfs.getFileStatus(trashPath);
-      String errMessage = "Will not provision new trash directory for " +
-          "encryption zone " + ez.getPath() + ". Path already exists.";
-      if (!trashFileStatus.isDirectory()) {
-        errMessage += "\r\n" +
-            "Warning: " + trashPath.toString() + " is not a directory";
-      }
-      if (!trashFileStatus.getPermission().equals(TRASH_PERMISSION)) {
-        errMessage += "\r\n" +
-            "Warning: the permission of " +
-            trashPath.toString() + " is not " + TRASH_PERMISSION;
-      }
-      throw new FileAlreadyExistsException(errMessage);
-    } catch (FileNotFoundException ignored) {
-      // no trash path
-    }
-
-    // Update the permission bits
-    dfs.mkdir(trashPath, TRASH_PERMISSION);
-    dfs.setPermission(trashPath, TRASH_PERMISSION);
-  }
-
   /**
    * Returns a RemoteIterator which can be used to list all open files
    * currently managed by the NameNode. For large numbers of open files,

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java

@@ -74,7 +74,7 @@ import org.slf4j.LoggerFactory;
  */
 @InterfaceAudience.Private
 public class LeaseRenewer {
-  static final Logger LOG = LoggerFactory.getLogger(LeaseRenewer.class);
+  public static final Logger LOG = LoggerFactory.getLogger(LeaseRenewer.class);
 
   private static long leaseRenewerGraceDefault = 60*1000L;
   static final long LEASE_RENEWER_SLEEP_DEFAULT = 1000L;

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

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

+ 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 + "}";
   }
 }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java

@@ -69,6 +69,10 @@ public class SnapshotDiffReport {
       }
       return null;
     }
+
+    public static DiffType parseDiffType(String s){
+      return DiffType.valueOf(s.toUpperCase());
+    }
   }
 
   /**

+ 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
    */

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RequestHedgingProxyProvider.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
+import java.io.IOException;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
@@ -87,6 +88,11 @@ public class RequestHedgingProxyProvider<T> extends
         // Optimization : if only 2 proxies are configured and one had failed
         // over, then we dont need to create a threadpool etc.
         targetProxies.remove(toIgnore);
+        if (targetProxies.size() == 0) {
+          LOG.trace("No valid proxies left");
+          throw new RemoteException(IOException.class.getName(),
+              "No valid proxies left. All NameNode proxies have failed over.");
+        }
         if (targetProxies.size() == 1) {
           ProxyInfo<T> proxyInfo = targetProxies.values().iterator().next();
           try {

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

@@ -37,6 +37,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
 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;
@@ -49,6 +51,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
 
@@ -693,4 +696,84 @@ class JsonUtilClient {
         encryptDataTransfer, trashInterval, type, keyProviderUri,
         storagepolicyId);
   }
+
+  public static SnapshotDiffReport toSnapshotDiffReport(final Map<?, ?> json) {
+    if (json == null) {
+      return null;
+    }
+    Map<?, ?> m =
+        (Map<?, ?>) json.get(SnapshotDiffReport.class.getSimpleName());
+    String snapshotRoot = (String) m.get("snapshotRoot");
+    String fromSnapshot = (String) m.get("fromSnapshot");
+    String toSnapshot = (String) m.get("toSnapshot");
+    List<SnapshotDiffReport.DiffReportEntry> diffList =
+        toDiffList(getList(m, "diffList"));
+    return new SnapshotDiffReport(snapshotRoot, fromSnapshot, toSnapshot,
+        diffList);
+  }
+
+  private static List<SnapshotDiffReport.DiffReportEntry> toDiffList(
+      List<?> objs) {
+    if (objs == null) {
+      return null;
+    }
+    List<SnapshotDiffReport.DiffReportEntry> diffList =
+        new ChunkedArrayList<>();
+    for (int i = 0; i < objs.size(); i++) {
+      diffList.add(toDiffReportEntry((Map<?, ?>) objs.get(i)));
+    }
+    return diffList;
+  }
+
+  private static SnapshotDiffReport.DiffReportEntry toDiffReportEntry(
+      Map<?, ?> json) {
+    if (json == null) {
+      return null;
+    }
+    SnapshotDiffReport.DiffType type =
+        SnapshotDiffReport.DiffType.parseDiffType((String) json.get("type"));
+    byte[] sourcePath = toByteArray((String) json.get("sourcePath"));
+    byte[] targetPath = toByteArray((String) json.get("targetPath"));
+    return new SnapshotDiffReport.DiffReportEntry(type, sourcePath, targetPath);
+  }
+
+  private static byte[] toByteArray(String str) {
+    if (str == null) {
+      return null;
+    }
+    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;
+  }
 }

+ 56 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -37,6 +37,8 @@ import java.net.InetSocketAddress;
 import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URL;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
 import java.nio.charset.StandardCharsets;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
@@ -96,6 +98,8 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 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;
@@ -597,8 +601,32 @@ public class WebHdfsFileSystem extends FileSystem
   URL toUrl(final HttpOpParam.Op op, final Path fspath,
       final Param<?,?>... parameters) throws IOException {
     //initialize URI path and query
+
+    Path encodedFSPath = fspath;
+    if (fspath != null) {
+      URI fspathUri = fspath.toUri();
+      String fspathUriDecoded = fspathUri.getPath();
+      try {
+        fspathUriDecoded = URLDecoder.decode(fspathUri.getPath(), "UTF-8");
+      } catch (IllegalArgumentException ex) {
+        LOG.trace("Cannot decode URL encoded file", ex);
+      }
+      String[] fspathItems = fspathUriDecoded.split("/");
+
+      if (fspathItems.length > 0) {
+        StringBuilder fsPathEncodedItems = new StringBuilder();
+        for (String fsPathItem : fspathItems) {
+          fsPathEncodedItems.append("/");
+          fsPathEncodedItems.append(URLEncoder.encode(fsPathItem, "UTF-8"));
+        }
+        encodedFSPath = new Path(fspathUri.getScheme(),
+                fspathUri.getAuthority(), fsPathEncodedItems.substring(1));
+      }
+    }
+
     final String path = PATH_PREFIX
-        + (fspath == null? "/": makeQualified(fspath).toUri().getRawPath());
+        + (encodedFSPath == null ? "/" :
+            makeQualified(encodedFSPath).toUri().getRawPath());
     final String query = op.toQueryString()
         + Param.toSortedString("&", getAuthParameters(op))
         + Param.toSortedString("&", parameters);
@@ -1317,6 +1345,33 @@ public class WebHdfsFileSystem extends FileSystem
         new SnapshotNameParam(snapshotNewName)).run();
   }
 
+  public SnapshotDiffReport getSnapshotDiffReport(final Path snapshotDir,
+      final String fromSnapshot, final String toSnapshot) throws IOException {
+    storageStatistics.incrementOpCounter(OpType.GET_SNAPSHOT_DIFF);
+    final HttpOpParam.Op op = GetOpParam.Op.GETSNAPSHOTDIFF;
+    return new FsPathResponseRunner<SnapshotDiffReport>(op, snapshotDir,
+        new OldSnapshotNameParam(fromSnapshot),
+        new SnapshotNameParam(toSnapshot)) {
+      @Override
+      SnapshotDiffReport decodeResponse(Map<?, ?> json) {
+        return JsonUtilClient.toSnapshotDiffReport(json);
+      }
+    }.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 {

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

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

+ 45 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRequestHedgingProxyProvider.java

@@ -28,6 +28,7 @@ import java.util.Iterator;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
@@ -289,6 +290,50 @@ public class TestRequestHedgingProxyProvider {
     }
   }
 
+  @Test
+  public void testSingleProxyFailover() throws Exception {
+    String singleNS = "mycluster-" + Time.monotonicNow();
+    URI singleNNUri = new URI("hdfs://" + singleNS);
+    Configuration singleConf = new Configuration();
+    singleConf.set(HdfsClientConfigKeys.DFS_NAMESERVICES, singleNS);
+    singleConf.set(HdfsClientConfigKeys.
+        DFS_HA_NAMENODES_KEY_PREFIX + "." + singleNS, "nn1");
+
+    singleConf.set(HdfsClientConfigKeys.
+            DFS_NAMENODE_RPC_ADDRESS_KEY + "." + singleNS + ".nn1",
+        RandomStringUtils.randomAlphabetic(8) + ".foo.bar:9820");
+    ClientProtocol active = Mockito.mock(ClientProtocol.class);
+    Mockito
+        .when(active.getBlockLocations(Matchers.anyString(),
+            Matchers.anyLong(), Matchers.anyLong()))
+        .thenThrow(new RemoteException("java.io.FileNotFoundException",
+            "File does not exist!"));
+
+    RequestHedgingProxyProvider<ClientProtocol> provider =
+        new RequestHedgingProxyProvider<>(singleConf, singleNNUri,
+            ClientProtocol.class, createFactory(active));
+    try {
+      provider.getProxy().proxy.getBlockLocations("/tmp/test.file", 0L, 20L);
+      Assert.fail("Should fail since the active namenode throws"
+          + " FileNotFoundException!");
+    } catch (RemoteException ex) {
+      Exception rEx = ex.unwrapRemoteException();
+      Assert.assertTrue(rEx instanceof FileNotFoundException);
+    }
+    //Perform failover now, there will be no active proxies now
+    provider.performFailover(active);
+    try {
+      provider.getProxy().proxy.getBlockLocations("/tmp/test.file", 0L, 20L);
+      Assert.fail("Should fail since the active namenode throws"
+          + " FileNotFoundException!");
+    } catch (RemoteException ex) {
+      Exception rEx = ex.unwrapRemoteException();
+      Assert.assertTrue(rEx instanceof IOException);
+      Assert.assertTrue(rEx.getMessage().equals("No valid proxies left."
+          + " All NameNode proxies have failed over."));
+    }
+  }
+
   @Test
   public void testPerformFailoverWith3Proxies() throws Exception {
     conf.set(HdfsClientConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + "." + ns,

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java

@@ -104,6 +104,7 @@ public class HttpFSFileSystem extends FileSystem
   public static final String REPLICATION_PARAM = "replication";
   public static final String BLOCKSIZE_PARAM = "blocksize";
   public static final String PERMISSION_PARAM = "permission";
+  public static final String UNMASKED_PERMISSION_PARAM = "unmaskedpermission";
   public static final String ACLSPEC_PARAM = "aclspec";
   public static final String DESTINATION_PARAM = "destination";
   public static final String RECURSIVE_PARAM = "recursive";

+ 18 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.util.StringUtils;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
+import org.apache.hadoop.fs.permission.FsCreateModes;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -473,6 +474,7 @@ public class FSOperations {
     private InputStream is;
     private Path path;
     private short permission;
+    private short unmaskedPermission;
     private boolean override;
     private short replication;
     private long blockSize;
@@ -486,12 +488,14 @@ public class FSOperations {
      * @param override if the file should be overriden if it already exist.
      * @param repl the replication factor for the file.
      * @param blockSize the block size for the file.
+     * @param unmaskedPerm unmasked permissions for the file
      */
     public FSCreate(InputStream is, String path, short perm, boolean override,
-                    short repl, long blockSize) {
+                    short repl, long blockSize, short unmaskedPerm) {
       this.is = is;
       this.path = new Path(path);
       this.permission = perm;
+      this.unmaskedPermission = unmaskedPerm;
       this.override = override;
       this.replication = repl;
       this.blockSize = blockSize;
@@ -515,6 +519,10 @@ public class FSOperations {
         blockSize = fs.getDefaultBlockSize(path);
       }
       FsPermission fsPermission = new FsPermission(permission);
+      if (unmaskedPermission != -1) {
+        fsPermission = FsCreateModes.create(fsPermission,
+            new FsPermission(unmaskedPermission));
+      }
       int bufferSize = fs.getConf().getInt(HTTPFS_BUFFER_SIZE_KEY,
           HTTP_BUFFER_SIZE_DEFAULT);
       OutputStream os = fs.create(path, fsPermission, override, bufferSize, replication, blockSize, null);
@@ -748,16 +756,20 @@ public class FSOperations {
 
     private Path path;
     private short permission;
+    private short unmaskedPermission;
 
     /**
      * Creates a mkdirs executor.
      *
      * @param path directory path to create.
      * @param permission permission to use.
+     * @param unmaskedPermission unmasked permissions for the directory
      */
-    public FSMkdirs(String path, short permission) {
+    public FSMkdirs(String path, short permission,
+        short unmaskedPermission) {
       this.path = new Path(path);
       this.permission = permission;
+      this.unmaskedPermission = unmaskedPermission;
     }
 
     /**
@@ -773,6 +785,10 @@ public class FSOperations {
     @Override
     public JSONObject execute(FileSystem fs) throws IOException {
       FsPermission fsPermission = new FsPermission(permission);
+      if (unmaskedPermission != -1) {
+        fsPermission = FsCreateModes.create(fsPermission,
+            new FsPermission(unmaskedPermission));
+      }
       boolean mkdirs = fs.mkdirs(path, fsPermission);
       return toJSON(HttpFSFileSystem.MKDIRS_JSON, mkdirs);
     }

+ 27 - 3
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java

@@ -66,9 +66,11 @@ public class HttpFSParametersProvider extends ParametersProvider {
     PARAMS_DEF.put(Operation.CONCAT, new Class[]{SourcesParam.class});
     PARAMS_DEF.put(Operation.TRUNCATE, new Class[]{NewLengthParam.class});
     PARAMS_DEF.put(Operation.CREATE,
-      new Class[]{PermissionParam.class, OverwriteParam.class,
-                  ReplicationParam.class, BlockSizeParam.class, DataParam.class});
-    PARAMS_DEF.put(Operation.MKDIRS, new Class[]{PermissionParam.class});
+        new Class[]{PermissionParam.class, OverwriteParam.class,
+            ReplicationParam.class, BlockSizeParam.class, DataParam.class,
+            UnmaskedPermissionParam.class});
+    PARAMS_DEF.put(Operation.MKDIRS, new Class[]{PermissionParam.class,
+        UnmaskedPermissionParam.class});
     PARAMS_DEF.put(Operation.RENAME, new Class[]{DestinationParam.class});
     PARAMS_DEF.put(Operation.SETOWNER,
         new Class[]{OwnerParam.class, GroupParam.class});
@@ -384,6 +386,28 @@ public class HttpFSParametersProvider extends ParametersProvider {
 
   }
 
+  /**
+   * Class for unmaskedpermission parameter.
+   */
+  @InterfaceAudience.Private
+  public static class UnmaskedPermissionParam extends ShortParam {
+
+    /**
+     * Parameter name.
+     */
+    public static final String NAME =
+        HttpFSFileSystem.UNMASKED_PERMISSION_PARAM;
+
+
+    /**
+     * Constructor.
+     */
+    public UnmaskedPermissionParam() {
+      super(NAME, (short) -1, 8);
+    }
+
+  }
+
   /**
    * Class for AclPermission parameter.
    */

+ 13 - 5
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.PolicyNameParam
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.RecursiveParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.ReplicationParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.SourcesParam;
+import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.UnmaskedPermissionParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.SnapshotNameParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrEncodingParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrNameParam;
@@ -578,6 +579,8 @@ public class HttpFSServer {
         } else {
           Short permission = params.get(PermissionParam.NAME,
                                          PermissionParam.class);
+          Short unmaskedPermission = params.get(UnmaskedPermissionParam.NAME,
+              UnmaskedPermissionParam.class);
           Boolean override = params.get(OverwriteParam.NAME,
                                         OverwriteParam.class);
           Short replication = params.get(ReplicationParam.NAME,
@@ -586,11 +589,13 @@ public class HttpFSServer {
                                       BlockSizeParam.class);
           FSOperations.FSCreate command =
             new FSOperations.FSCreate(is, path, permission, override,
-                                      replication, blockSize);
+                replication, blockSize, unmaskedPermission);
           fsExecute(user, command);
           AUDIT_LOG.info(
-            "[{}] permission [{}] override [{}] replication [{}] blockSize [{}]",
-            new Object[]{path, permission, override, replication, blockSize});
+              "[{}] permission [{}] override [{}] "+
+              "replication [{}] blockSize [{}] unmaskedpermission [{}]",
+              new Object[]{path, permission,  override, replication, blockSize,
+                  unmaskedPermission});
           response = Response.status(Response.Status.CREATED).build();
         }
         break;
@@ -646,10 +651,13 @@ public class HttpFSServer {
       case MKDIRS: {
         Short permission = params.get(PermissionParam.NAME,
                                        PermissionParam.class);
+        Short unmaskedPermission = params.get(UnmaskedPermissionParam.NAME,
+            UnmaskedPermissionParam.class);
         FSOperations.FSMkdirs command =
-          new FSOperations.FSMkdirs(path, permission);
+            new FSOperations.FSMkdirs(path, permission, unmaskedPermission);
         JSONObject json = fsExecute(user, command);
-        AUDIT_LOG.info("[{}] permission [{}]", path, permission);
+        AUDIT_LOG.info("[{}] permission [{}] unmaskedpermission [{}]",
+            path, permission, unmaskedPermission);
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
       }

+ 174 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java

@@ -41,6 +41,7 @@ import java.net.URL;
 import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -48,6 +49,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.XAttrCodec;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclEntryScope;
+import org.apache.hadoop.fs.permission.AclEntryType;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.lib.server.Service;
 import org.apache.hadoop.lib.server.ServiceException;
@@ -406,6 +412,19 @@ public class TestHttpFSServer extends HFSTestCase {
    * @throws Exception
    */
   private void createWithHttp(String filename, String perms) throws Exception {
+    createWithHttp(filename, perms, null);
+  }
+
+  /**
+   * Talks to the http interface to create a file.
+   *
+   * @param filename The file to create
+   * @param perms The permission field, if any (may be null)
+   * @param unmaskedPerms The unmaskedPermission field, if any (may be null)
+   * @throws Exception
+   */
+  private void createWithHttp(String filename, String perms,
+      String unmaskedPerms) throws Exception {
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     // Remove leading / from filename
     if (filename.charAt(0) == '/') {
@@ -421,6 +440,9 @@ public class TestHttpFSServer extends HFSTestCase {
               "/webhdfs/v1/{0}?user.name={1}&permission={2}&op=CREATE",
               filename, user, perms);
     }
+    if (unmaskedPerms != null) {
+      pathOps = pathOps+"&unmaskedpermission="+unmaskedPerms;
+    }
     URL url = new URL(TestJettyHelper.getJettyURL(), pathOps);
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
     conn.addRequestProperty("Content-Type", "application/octet-stream");
@@ -429,6 +451,41 @@ public class TestHttpFSServer extends HFSTestCase {
     Assert.assertEquals(HttpURLConnection.HTTP_CREATED, conn.getResponseCode());
   }
 
+  /**
+   * Talks to the http interface to create a directory.
+   *
+   * @param dirname The directory to create
+   * @param perms The permission field, if any (may be null)
+   * @param unmaskedPerms The unmaskedPermission field, if any (may be null)
+   * @throws Exception
+   */
+  private void createDirWithHttp(String dirname, String perms,
+      String unmaskedPerms) throws Exception {
+    String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
+    // Remove leading / from filename
+    if (dirname.charAt(0) == '/') {
+      dirname = dirname.substring(1);
+    }
+    String pathOps;
+    if (perms == null) {
+      pathOps = MessageFormat.format(
+              "/webhdfs/v1/{0}?user.name={1}&op=MKDIRS",
+              dirname, user);
+    } else {
+      pathOps = MessageFormat.format(
+              "/webhdfs/v1/{0}?user.name={1}&permission={2}&op=MKDIRS",
+              dirname, user, perms);
+    }
+    if (unmaskedPerms != null) {
+      pathOps = pathOps+"&unmaskedpermission="+unmaskedPerms;
+    }
+    URL url = new URL(TestJettyHelper.getJettyURL(), pathOps);
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod("PUT");
+    conn.connect();
+    Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+  }
+
   /**
    * Talks to the http interface to get the json output of a *STATUS command
    * on the given file.
@@ -577,6 +634,27 @@ public class TestHttpFSServer extends HFSTestCase {
     }
   }
 
+  /**
+   *
+   * @param stat AclStatus object from a call to getAclStatus
+   * @param name The name of the ACL being searched for
+   * @return The AclEntry if found, or null otherwise
+   * @throws IOException
+   */
+  private AclEntry findAclWithName(AclStatus stat, String name)
+      throws IOException{
+    AclEntry relevantAcl = null;
+    Iterator<AclEntry> it = stat.getEntries().iterator();
+    while (it.hasNext()) {
+      AclEntry e = it.next();
+      if (e.getName().equals(name)) {
+        relevantAcl = e;
+        break;
+      }
+    }
+    return relevantAcl;
+  }
+
   /**
    * Validate that files are created with 755 permissions when no
    * 'permissions' attribute is specified, and when 'permissions'
@@ -837,6 +915,102 @@ public class TestHttpFSServer extends HFSTestCase {
     Assert.assertEquals(-1, is.read());
   }
 
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testCreateFileWithUnmaskedPermissions() throws Exception {
+    createHttpFSServer(false, false);
+
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    // Create a folder with a default acl default:user2:rw-
+    fs.mkdirs(new Path("/tmp"));
+    AclEntry acl = new org.apache.hadoop.fs.permission.AclEntry.Builder()
+        .setType(AclEntryType.USER)
+        .setScope(AclEntryScope.DEFAULT)
+        .setName("user2")
+        .setPermission(FsAction.READ_WRITE)
+        .build();
+    fs.setAcl(new Path("/tmp"), new ArrayList<AclEntry>(Arrays.asList(acl)));
+
+    String notUnmaskedFile = "/tmp/notUnmasked";
+    String unmaskedFile = "/tmp/unmasked";
+
+    // Create a file inside the folder. It should inherit the default acl
+    // but the mask should affect the ACL permissions. The mask is controlled
+    // by the group permissions, which are 0, and hence the mask will make
+    // the effective permission of the inherited ACL be NONE.
+    createWithHttp(notUnmaskedFile, "700");
+
+    // Pull the relevant ACL from the FS object and check the mask has affected
+    // its permissions.
+    AclStatus aclStatus = fs.getAclStatus(new Path(notUnmaskedFile));
+    AclEntry theAcl = findAclWithName(aclStatus, "user2");
+
+    Assert.assertNotNull(theAcl);
+    Assert.assertEquals(FsAction.NONE,
+        aclStatus.getEffectivePermission(theAcl));
+
+    // Create another file, this time pass a mask of 777. Now the inherited
+    // permissions should be as expected
+    createWithHttp(unmaskedFile, "700", "777");
+
+    aclStatus = fs.getAclStatus(new Path(unmaskedFile));
+    theAcl = findAclWithName(aclStatus, "user2");
+
+    Assert.assertNotNull(theAcl);
+    Assert.assertEquals(FsAction.READ_WRITE,
+        aclStatus.getEffectivePermission(theAcl));
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testMkdirWithUnmaskedPermissions() throws Exception {
+    createHttpFSServer(false, false);
+
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    // Create a folder with a default acl default:user2:rw-
+    fs.mkdirs(new Path("/tmp"));
+    AclEntry acl = new org.apache.hadoop.fs.permission.AclEntry.Builder()
+        .setType(AclEntryType.USER)
+        .setScope(AclEntryScope.DEFAULT)
+        .setName("user2")
+        .setPermission(FsAction.READ_WRITE)
+        .build();
+    fs.setAcl(new Path("/tmp"), new ArrayList<AclEntry>(Arrays.asList(acl)));
+
+    String notUnmaskedDir = "/tmp/notUnmaskedDir";
+    String unmaskedDir = "/tmp/unmaskedDir";
+
+    // Create a file inside the folder. It should inherit the default acl
+    // but the mask should affect the ACL permissions. The mask is controlled
+    // by the group permissions, which are 0, and hence the mask will make
+    // the effective permission of the inherited ACL be NONE.
+    createDirWithHttp(notUnmaskedDir, "700", null);
+
+    // Pull the relevant ACL from the FS object and check the mask has affected
+    // its permissions.
+    AclStatus aclStatus = fs.getAclStatus(new Path(notUnmaskedDir));
+    AclEntry theAcl = findAclWithName(aclStatus, "user2");
+
+    Assert.assertNotNull(theAcl);
+    Assert.assertEquals(FsAction.NONE,
+        aclStatus.getEffectivePermission(theAcl));
+
+    // Create another file, this time pass a mask of 777. Now the inherited
+    // permissions should be as expected
+    createDirWithHttp(unmaskedDir, "700", "777");
+
+    aclStatus = fs.getAclStatus(new Path(unmaskedDir));
+    theAcl = findAclWithName(aclStatus, "user2");
+
+    Assert.assertNotNull(theAcl);
+    Assert.assertEquals(FsAction.READ_WRITE,
+        aclStatus.getEffectivePermission(theAcl));
+  }
+
   @Test
   @TestDir
   @TestJetty

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerWebServer.java

@@ -22,9 +22,12 @@ import java.io.File;
 import java.io.InputStreamReader;
 import java.net.HttpURLConnection;
 import java.net.URL;
+import java.nio.charset.Charset;
 import java.text.MessageFormat;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.HadoopUsersConfTestHelper;
 import org.junit.Assert;
@@ -57,7 +60,8 @@ public class TestHttpFSServerWebServer {
     System.setProperty("httpfs.home.dir", homeDir.getAbsolutePath());
     System.setProperty("httpfs.log.dir", logsDir.getAbsolutePath());
     System.setProperty("httpfs.config.dir", confDir.getAbsolutePath());
-    new File(confDir, "httpfs-signature.secret").createNewFile();
+    FileUtils.writeStringToFile(new File(confDir, "httpfs-signature.secret"),
+        "foo", Charset.forName("UTF-8"));
   }
 
   @Before
@@ -65,6 +69,8 @@ public class TestHttpFSServerWebServer {
     Configuration conf = new Configuration();
     conf.set(HttpFSServerWebServer.HTTP_HOSTNAME_KEY, "localhost");
     conf.setInt(HttpFSServerWebServer.HTTP_PORT_KEY, 0);
+    conf.set(AuthenticationFilter.SIGNATURE_SECRET_FILE,
+        "httpfs-signature.secret");
     Configuration sslConf = new Configuration();
     webServer = new HttpFSServerWebServer(conf, sslConf);
   }
@@ -76,7 +82,7 @@ public class TestHttpFSServerWebServer {
     URL url = new URL(webServer.getUrl(), MessageFormat.format(
         "/webhdfs/v1/?user.name={0}&op=liststatus", user));
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+    Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
     BufferedReader reader = new BufferedReader(
         new InputStreamReader(conn.getInputStream()));
     reader.readLine();

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

@@ -515,30 +515,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>
@@ -551,6 +534,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 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -419,6 +419,15 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.namenode.snapshot.max.limit";
 
   public static final int DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT = 65536;
+  public static final String DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL =
+      "dfs.namenode.snapshot.skiplist.interval";
+  public static final int DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL_DEFAULT =
+      10;
+  public static final String DFS_NAMENODE_SNAPSHOT_SKIPLIST_MAX_LEVELS =
+      "dfs.namenode.snapshot.skiplist.max.levels";
+  public static final int
+      DFS_NAMENODE_SNAPSHOT_SKIPLIST_MAX_SKIP_LEVELS_DEFAULT = 0;
+
   // Whether to enable datanode's stale state detection and usage for reads
   public static final String DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY = "dfs.namenode.avoid.read.stale.datanode";
   public static final boolean DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_DEFAULT = false;
@@ -1162,6 +1171,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.disk.balancer.max.disk.errors";
   public static final int DFS_DISK_BALANCER_MAX_DISK_ERRORS_DEFAULT = 5;
 
+  public static final String DFS_DISK_BALANCER_PLAN_VALID_INTERVAL =
+      "dfs.disk.balancer.plan.valid.interval";
+  public static final String DFS_DISK_BALANCER_PLAN_VALID_INTERVAL_DEFAULT =
+      "1d";
+
 
   public static final String DFS_DISK_BALANCER_BLOCK_TOLERANCE =
       "dfs.disk.balancer.block.tolerance.percent";

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

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

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java

@@ -86,7 +86,8 @@ public class NamenodeProtocolServerSideTranslatorPB implements
         .build();
     BlocksWithLocations blocks;
     try {
-      blocks = impl.getBlocks(dnInfo, request.getSize());
+      blocks = impl.getBlocks(dnInfo, request.getSize(),
+          request.getMinBlockSize());
     } catch (IOException e) {
       throw new ServiceException(e);
     }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java

@@ -99,11 +99,12 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
   }
 
   @Override
-  public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
+  public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size, long
+      minBlockSize)
       throws IOException {
     GetBlocksRequestProto req = GetBlocksRequestProto.newBuilder()
         .setDatanode(PBHelperClient.convert((DatanodeID)datanode)).setSize(size)
-        .build();
+        .setMinBlockSize(minBlockSize).build();
     try {
       return PBHelper.convert(rpcProxy.getBlocks(NULL_CONTROLLER, req)
           .getBlocks());

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

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

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

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

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java

@@ -785,7 +785,7 @@ public class Dispatcher {
     private long getBlockList() throws IOException {
       final long size = Math.min(getBlocksSize, blocksToReceive);
       final BlocksWithLocations newBlksLocs =
-          nnc.getBlocks(getDatanodeInfo(), size);
+          nnc.getBlocks(getDatanodeInfo(), size, getBlocksMinBlockSize);
 
       if (LOG.isTraceEnabled()) {
         LOG.trace("getBlocks(" + getDatanodeInfo() + ", "

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java

@@ -162,9 +162,10 @@ public class NameNodeConnector implements Closeable {
   }
 
   /** @return blocks with locations. */
-  public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
+  public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size, long
+      minBlockSize)
       throws IOException {
-    return namenode.getBlocks(datanode, size);
+    return namenode.getBlocks(datanode, size, minBlockSize);
   }
 
   /**

+ 4 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -408,13 +408,6 @@ public class BlockManager implements BlockStatsMXBean {
    */
   private int numBlocksPerIteration;
 
-  /**
-   * Minimum size that a block can be sent to Balancer through getBlocks.
-   * And after HDFS-8824, the small blocks are unused anyway, so there's no
-   * point to send them to balancer.
-   */
-  private long getBlocksMinBlockSize = -1;
-
   /**
    * Progress of the Reconstruction queues initialisation.
    */
@@ -539,9 +532,6 @@ public class BlockManager implements BlockStatsMXBean {
     this.numBlocksPerIteration = conf.getInt(
         DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT,
         DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT_DEFAULT);
-    this.getBlocksMinBlockSize = conf.getLongBytes(
-        DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY,
-        DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_DEFAULT);
 
     final int minMaintenanceR = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY,
@@ -1469,7 +1459,8 @@ public class BlockManager implements BlockStatsMXBean {
 
   /** Get all blocks with location information from a datanode. */
   public BlocksWithLocations getBlocksWithLocations(final DatanodeID datanode,
-      final long size) throws UnregisteredNodeException {
+      final long size, final long minBlockSize) throws
+      UnregisteredNodeException {
     final DatanodeDescriptor node = getDatanodeManager().getDatanode(datanode);
     if (node == null) {
       blockLog.warn("BLOCK* getBlocks: Asking for blocks from an" +
@@ -1491,7 +1482,7 @@ public class BlockManager implements BlockStatsMXBean {
     while(totalSize<size && iter.hasNext()) {
       curBlock = iter.next();
       if(!curBlock.isComplete())  continue;
-      if (curBlock.getNumBytes() < getBlocksMinBlockSize) {
+      if (curBlock.getNumBytes() < minBlockSize) {
         continue;
       }
       totalSize += addBlock(curBlock, results);
@@ -1501,7 +1492,7 @@ public class BlockManager implements BlockStatsMXBean {
       for(int i=0; i<startBlock&&totalSize<size; i++) {
         curBlock = iter.next();
         if(!curBlock.isComplete())  continue;
-        if (curBlock.getNumBytes() < getBlocksMinBlockSize) {
+        if (curBlock.getNumBytes() < minBlockSize) {
           continue;
         }
         totalSize += addBlock(curBlock, results);

+ 2 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockRecoveryWorker.java

@@ -307,10 +307,8 @@ public class BlockRecoveryWorker {
         }
       }
 
-      // If any of the data-nodes failed, the recovery fails, because
-      // we never know the actual state of the replica on failed data-nodes.
-      // The recovery should be started over.
-      if (!failedList.isEmpty()) {
+      // Abort if all failed.
+      if (successList.isEmpty()) {
         throw new IOException("Cannot recover " + block
             + ", the following datanodes failed: " + failedList);
       }

+ 26 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

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

+ 20 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java

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

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

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

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java

@@ -58,6 +58,7 @@ import java.io.OutputStream;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.net.URLDecoder;
 import java.nio.charset.StandardCharsets;
 import java.security.PrivilegedExceptionAction;
 import java.util.EnumSet;
@@ -127,7 +128,7 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
     params = new ParameterParser(queryString, conf);
     DataNodeUGIProvider ugiProvider = new DataNodeUGIProvider(params);
     ugi = ugiProvider.ugi();
-    path = params.path();
+    path = URLDecoder.decode(params.path(), "UTF-8");
 
     injectToken();
     ugi.doAs(new PrivilegedExceptionAction<Void>() {

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

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

+ 16 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java

@@ -48,6 +48,8 @@ public class ExecuteCommand extends Command {
     super(conf);
     addValidCommandParameters(DiskBalancerCLI.EXECUTE,
         "Executes a given plan.");
+    addValidCommandParameters(DiskBalancerCLI.SKIPDATECHECK,
+        "skips the date check and force execute the plan");
   }
 
   /**
@@ -69,7 +71,16 @@ public class ExecuteCommand extends Command {
     try (FSDataInputStream plan = open(planFile)) {
       planData = IOUtils.toString(plan);
     }
-    submitPlan(planFile, planData);
+
+    boolean skipDateCheck = false;
+    if(cmd.hasOption(DiskBalancerCLI.SKIPDATECHECK)) {
+      skipDateCheck = true;
+      LOG.warn("Skipping date check on this plan. This could mean we are " +
+          "executing an old plan and may not be the right plan for this " +
+          "data node.");
+    }
+
+    submitPlan(planFile, planData, skipDateCheck);
   }
 
   /**
@@ -77,9 +88,11 @@ public class ExecuteCommand extends Command {
    *
    * @param planFile - Plan file name
    * @param planData - Plan data in json format
+   * @param skipDateCheck - skips date check
    * @throws IOException
    */
-  private void submitPlan(final String planFile, final String planData)
+  private void submitPlan(final String planFile, final String planData,
+                          boolean skipDateCheck)
           throws IOException {
     Preconditions.checkNotNull(planData);
     NodePlan plan = NodePlan.parseJson(planData);
@@ -88,9 +101,8 @@ public class ExecuteCommand extends Command {
     ClientDatanodeProtocol dataNode = getDataNodeProxy(dataNodeAddress);
     String planHash = DigestUtils.shaHex(planData);
     try {
-      // TODO : Support skipping date check.
       dataNode.submitDiskBalancerPlan(planHash, DiskBalancerCLI.PLAN_VERSION,
-                                      planFile, planData, false);
+                                      planFile, planData, skipDateCheck);
     } catch (DiskBalancerException ex) {
       LOG.error("Submitting plan on  {} failed. Result: {}, Message: {}",
           plan.getNodeName(), ex.getResult().toString(), ex.getMessage());

+ 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 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java

@@ -101,7 +101,7 @@ public class Router extends CompositeService {
   /** Interface to identify the active NN for a nameservice or blockpool ID. */
   private ActiveNamenodeResolver namenodeResolver;
   /** Updates the namenode status in the namenode resolver. */
-  private Collection<NamenodeHeartbeatService> namenodeHearbeatServices;
+  private Collection<NamenodeHeartbeatService> namenodeHeartbeatServices;
 
   /** Router metrics. */
   private RouterMetricsService metrics;
@@ -196,13 +196,13 @@ public class Router extends CompositeService {
         DFSConfigKeys.DFS_ROUTER_HEARTBEAT_ENABLE_DEFAULT)) {
 
       // Create status updater for each monitored Namenode
-      this.namenodeHearbeatServices = createNamenodeHearbeatServices();
+      this.namenodeHeartbeatServices = createNamenodeHeartbeatServices();
       for (NamenodeHeartbeatService hearbeatService :
-          this.namenodeHearbeatServices) {
+          this.namenodeHeartbeatServices) {
         addService(hearbeatService);
       }
 
-      if (this.namenodeHearbeatServices.isEmpty()) {
+      if (this.namenodeHeartbeatServices.isEmpty()) {
         LOG.error("Heartbeat is enabled but there are no namenodes to monitor");
       }
 
@@ -411,7 +411,7 @@ public class Router extends CompositeService {
    * @return List of heartbeat services.
    */
   protected Collection<NamenodeHeartbeatService>
-      createNamenodeHearbeatServices() {
+      createNamenodeHeartbeatServices() {
 
     Map<String, NamenodeHeartbeatService> ret = new HashMap<>();
 
@@ -645,4 +645,12 @@ public class Router extends CompositeService {
   RouterQuotaUpdateService getQuotaCacheUpdateService() {
     return this.quotaUpdateService;
   }
+
+  /**
+   * Get the list of namenode heartbeat service.
+   */
+  @VisibleForTesting
+  Collection<NamenodeHeartbeatService> getNamenodeHearbeatServices() {
+    return this.namenodeHeartbeatServices;
+  }
 }

+ 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();
     }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java

@@ -73,7 +73,7 @@ public abstract class StateStoreDriver implements StateStoreRecordOperations {
 
     boolean success = initDriver();
     if (!success) {
-      LOG.error("Cannot intialize driver for {}", getDriverName());
+      LOG.error("Cannot initialize driver for {}", getDriverName());
       return false;
     }
 

+ 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

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

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

+ 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;

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

@@ -1718,13 +1718,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param datanode on which blocks are located
    * @param size total size of blocks
    */
-  public BlocksWithLocations getBlocks(DatanodeID datanode, long size)
-      throws IOException {
+  public BlocksWithLocations getBlocks(DatanodeID datanode, long size, long
+      minimumBlockSize) throws IOException {
     checkOperation(OperationCategory.READ);
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return getBlockManager().getBlocksWithLocations(datanode, size);
+      return getBlockManager().getBlocksWithLocations(datanode, size,
+          minimumBlockSize);
     } finally {
       readUnlock("getBlocks");
     }

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

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

+ 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;

+ 30 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -34,6 +34,7 @@ import static org.apache.hadoop.util.Time.now;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
@@ -189,6 +190,7 @@ import org.apache.hadoop.ipc.RefreshResponse;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Groups;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.ProxyUsers;
@@ -616,15 +618,20 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   // NamenodeProtocol
   /////////////////////////////////////////////////////
   @Override // NamenodeProtocol
-  public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
-  throws IOException {
+  public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size, long
+      minBlockSize)
+      throws IOException {
     if(size <= 0) {
       throw new IllegalArgumentException(
-        "Unexpected not positive size: "+size);
+          "Unexpected not positive size: "+size);
+    }
+    if(minBlockSize < 0) {
+      throw new IllegalArgumentException(
+          "Unexpected not positive size: "+size);
     }
     checkNNStartup();
     namesystem.checkSuperuserPrivilege();
-    return namesystem.getBlocks(datanode, size);
+    return namesystem.getBlocks(datanode, size, minBlockSize);
   }
 
   @Override // NamenodeProtocol
@@ -2253,6 +2260,24 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     // guaranteed to have been written by this NameNode.)
     boolean readInProgress = syncTxid > 0;
 
+    // doas the NN login user for the actual operations to get edits.
+    // Notably this is necessary when polling from the remote edits via https.
+    // We have validated the client is a superuser from the NN RPC, so this
+    // running as the login user here is safe.
+    EventBatchList ret = SecurityUtil.doAsLoginUser(
+        new PrivilegedExceptionAction<EventBatchList>() {
+          @Override
+          public EventBatchList run() throws IOException {
+            return getEventBatchList(syncTxid, txid, log, readInProgress,
+                maxEventsPerRPC);
+          }
+        });
+    return ret;
+  }
+
+  private EventBatchList getEventBatchList(long syncTxid, long txid,
+      FSEditLog log, boolean readInProgress, int maxEventsPerRPC)
+      throws IOException {
     List<EventBatch> batches = Lists.newArrayList();
     int totalEvents = 0;
     long maxSeenTxid = -1;
@@ -2271,7 +2296,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       // and are using QJM -- the edit log will be closed and this exception
       // will result
       LOG.info("NN is transitioning from active to standby and FSEditLog " +
-      "is closed -- could not read edits");
+          "is closed -- could not read edits");
       return new EventBatchList(batches, firstSeenTxid, maxSeenTxid, syncTxid);
     }
 

+ 16 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java

@@ -138,10 +138,14 @@ abstract class AbstractINodeDiffList<N extends INode,
     return n == 0 ? null : diffs.get(n - 1);
   }
 
+  DiffList<D> newDiffs() {
+    return new DiffListByArrayList<>(
+        INodeDirectory.DEFAULT_FILES_PER_DIRECTORY);
+  }
+
   private void createDiffsIfNeeded() {
     if (diffs == null) {
-      diffs =
-          new DiffListByArrayList<>(INodeDirectory.DEFAULT_FILES_PER_DIRECTORY);
+      diffs = newDiffs();
     }
   }
 
@@ -231,6 +235,12 @@ abstract class AbstractINodeDiffList<N extends INode,
     return diff == null ? Snapshot.CURRENT_STATE_ID : diff.getSnapshotId();
   }
 
+  public final int getDiffIndexById(final int snapshotId) {
+    int diffIndex = diffs.binarySearch(snapshotId);
+    diffIndex = diffIndex < 0 ? (-diffIndex - 1) : diffIndex;
+    return diffIndex;
+  }
+
   final int[] changedBetweenSnapshots(Snapshot from, Snapshot to) {
     if (diffs == null) {
       return null;
@@ -243,10 +253,10 @@ abstract class AbstractINodeDiffList<N extends INode,
     }
 
     final int size = diffs.size();
-    int earlierDiffIndex = diffs.binarySearch(earlier.getId());
+    int earlierDiffIndex = getDiffIndexById(earlier.getId());
     int laterDiffIndex = later == null ? size
-        : diffs.binarySearch(later.getId());
-    if (-earlierDiffIndex - 1 == size) {
+        : getDiffIndexById(later.getId());
+    if (earlierDiffIndex == size) {
       // if the earlierSnapshot is after the latest SnapshotDiff stored in
       // diffs, no modification happened after the earlierSnapshot
       return null;
@@ -256,10 +266,6 @@ abstract class AbstractINodeDiffList<N extends INode,
       // before it, no modification happened before the laterSnapshot
       return null;
     }
-    earlierDiffIndex = earlierDiffIndex < 0 ? (-earlierDiffIndex - 1)
-        : earlierDiffIndex;
-    laterDiffIndex = laterDiffIndex < 0 ? (-laterDiffIndex - 1)
-        : laterDiffIndex;
     return new int[]{earlierDiffIndex, laterDiffIndex};
   }
 
@@ -300,7 +306,7 @@ abstract class AbstractINodeDiffList<N extends INode,
     }
     return diff;
   }
-  
+
   @Override
   public Iterator<D> iterator() {
     return diffs != null ? diffs.iterator() : Collections.emptyIterator();

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

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

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

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

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

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

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

@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.snapshot;
+
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
+import org.slf4j.Logger;
+
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.IntFunction;
+
+/** For creating {@link DiffList} for {@link DirectoryDiff}. */
+public abstract class DirectoryDiffListFactory {
+  public static DiffList<DirectoryDiff> createDiffList(int capacity) {
+    return constructor.apply(capacity);
+  }
+
+  public static void init(int interval, int maxSkipLevels, Logger log) {
+    DirectoryDiffListFactory.skipInterval = interval;
+    DirectoryDiffListFactory.maxLevels = maxSkipLevels;
+
+    if (maxLevels > 0) {
+      constructor = c -> new DiffListBySkipList(c);
+      log.info("SkipList is enabled with skipInterval=" + skipInterval
+          + ", maxLevels=" + maxLevels);
+    } else {
+      constructor = c -> new DiffListByArrayList<>(c);
+      log.info("SkipList is disabled");
+    }
+  }
+
+  private static volatile IntFunction<DiffList<DirectoryDiff>> constructor
+      = c -> new DiffListByArrayList<>(c);
+
+  private static volatile int skipInterval;
+  private static volatile int maxLevels;
+
+  /**
+   * Returns the level of a skip list node.
+   * @return A value in the range 0 to maxLevels.
+   */
+  public static int randomLevel() {
+    final Random r = ThreadLocalRandom.current();
+    for (int level = 0; level < maxLevels; level++) {
+      // skip to the next level with probability 1/skipInterval
+      if (r.nextInt(skipInterval) > 0) {
+        return level;
+      }
+    }
+    return maxLevels;
+  }
+
+
+  private DirectoryDiffListFactory() {}
+}

+ 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));

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