Browse Source

Merge branch 'trunk' into HDFS-7240

Xiaoyu Yao 7 years ago
parent
commit
1e634d49da
100 changed files with 3486 additions and 1483 deletions
  1. 145 0
      dev-support/bin/verify-license-files
  2. 20 2
      hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
  3. 1 0
      hadoop-client-modules/hadoop-client-minicluster/pom.xml
  4. 53 27
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
  5. 14 8
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/Signer.java
  6. 29 0
      hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java
  7. 3 8
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ConfServlet.java
  8. 84 62
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  9. 0 37
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/CorePropertyTag.java
  10. 0 41
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/HDFSPropertyTag.java
  11. 0 49
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/OzonePropertyTag.java
  12. 0 30
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/PropertyTag.java
  13. 0 39
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/YarnPropertyTag.java
  14. 2 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
  15. 0 11
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
  16. 2 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
  17. 31 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java
  18. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/Log4Json.java
  19. 91 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/RuleBasedLdapGroupsMapping.java
  20. 429 462
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
  21. 17 9
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  22. 278 0
      hadoop-common-project/hadoop-common/src/site/markdown/AdminCompatibilityGuide.md
  23. 21 12
      hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
  24. 3 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
  25. 0 58
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java
  26. 59 62
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  27. 107 126
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemCaching.java
  28. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpengo.java
  29. 14 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java
  30. 99 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestRuleBasedLdapGroupsMapping.java
  31. 402 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java
  32. 12 42
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java
  33. 95 18
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
  34. 2 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java
  35. 1 1
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
  36. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java
  37. 55 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  38. 87 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  39. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
  40. 4 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
  41. 7 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RequestHedgingProxyProvider.java
  42. 49 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
  43. 15 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  44. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
  45. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestHdfsFileStatusMethods.java
  46. 45 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRequestHedgingProxyProvider.java
  47. 1 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/DummyGroupMapping.java
  48. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh
  49. 9 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  50. 14 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java
  51. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockRecoveryWorker.java
  52. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
  53. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
  54. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java
  55. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
  56. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMBean.java
  57. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java
  58. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java
  59. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
  60. 60 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
  61. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java
  62. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
  63. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
  64. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
  65. 11 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java
  66. 22 32
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
  67. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
  68. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
  69. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
  70. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java
  71. 4 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
  72. 21 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
  73. 16 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
  74. 21 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
  75. 124 66
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  76. 27 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  77. 10 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/XmlImageVisitor.java
  78. 33 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  79. 19 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  80. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
  81. 107 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
  82. 13 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
  83. 151 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCClientRetries.java
  84. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
  85. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java
  86. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
  87. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
  88. 130 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
  89. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java
  90. 71 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
  91. 10 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
  92. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  93. 47 20
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
  94. 50 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
  95. 38 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java
  96. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
  97. 39 45
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java
  98. 8 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java
  99. BIN
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/resources/testdata/concat.bz2
  100. BIN
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/resources/testdata/concat.gz

+ 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

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

@@ -213,7 +213,11 @@
     </fileSet>
     <!-- Copy dependecies from hadoop-yarn-server-timelineservice as well -->
     <fileSet>
-      <directory>hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/target/lib</directory>
+      <directory>hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/target/lib</directory>
+      <outputDirectory>share/hadoop/${hadoop.component}/timelineservice/lib</outputDirectory>
+    </fileSet>
+    <fileSet>
+      <directory>hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/target/lib</directory>
       <outputDirectory>share/hadoop/${hadoop.component}/timelineservice/lib</outputDirectory>
     </fileSet>
   </fileSets>
@@ -221,12 +225,26 @@
     <moduleSet>
       <includes>
         <include>org.apache.hadoop:hadoop-yarn-server-timelineservice</include>
-        <include>org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase</include>
+        <include>org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase-client</include>
+        <include>org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase-common</include>
+      </includes>
+      <binaries>
+        <outputDirectory>share/hadoop/${hadoop.component}/timelineservice</outputDirectory>
+        <includeDependencies>false</includeDependencies>
+        <unpack>false</unpack>
+      </binaries>
+    </moduleSet>
+    <moduleSet>
+      <includes>
+        <include>org.apache.hadoop:hadoop-yarn-server-timelineservice-hbase-server</include>
       </includes>
       <binaries>
         <outputDirectory>share/hadoop/${hadoop.component}/timelineservice</outputDirectory>
         <includeDependencies>false</includeDependencies>
+        <!-- This is the id of the timelineservice-hbase-coprocessor assembly descriptor -->
+        <attachmentClassifier>coprocessor</attachmentClassifier>
         <unpack>false</unpack>
+        <outputFileNameMapping>hadoop-yarn-server-timelineservice-hbase-coprocessor-${module.version}.${module.extension}</outputFileNameMapping>
       </binaries>
     </moduleSet>
     <moduleSet>

+ 1 - 0
hadoop-client-modules/hadoop-client-minicluster/pom.xml

@@ -615,6 +615,7 @@
                       <excludes>
                         <exclude>testjar/*</exclude>
                         <exclude>testshell/*</exclude>
+                        <exclude>testdata/*</exclude>
                       </excludes>
                     </filter>
                     <!-- Mockito tries to include its own unrelocated copy of hamcrest. :( -->

+ 53 - 27
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java

@@ -13,6 +13,8 @@
  */
 package org.apache.hadoop.security.authentication.client;
 
+import com.google.common.annotations.VisibleForTesting;
+import java.lang.reflect.Constructor;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.security.authentication.server.HttpConstants;
 import org.apache.hadoop.security.authentication.util.AuthToken;
@@ -177,41 +179,65 @@ public class KerberosAuthenticator implements Authenticator {
    */
   @Override
   public void authenticate(URL url, AuthenticatedURL.Token token)
-    throws IOException, AuthenticationException {
+      throws IOException, AuthenticationException {
     if (!token.isSet()) {
       this.url = url;
       base64 = new Base64(0);
-      HttpURLConnection conn = token.openConnection(url, connConfigurator);
-      conn.setRequestMethod(AUTH_HTTP_METHOD);
-      conn.connect();
-      
-      boolean needFallback = false;
-      if (conn.getResponseCode() == HttpURLConnection.HTTP_OK) {
-        LOG.debug("JDK performed authentication on our behalf.");
-        // If the JDK already did the SPNEGO back-and-forth for
-        // us, just pull out the token.
-        AuthenticatedURL.extractToken(conn, token);
-        if (isTokenKerberos(token)) {
-          return;
+      try {
+        HttpURLConnection conn = token.openConnection(url, connConfigurator);
+        conn.setRequestMethod(AUTH_HTTP_METHOD);
+        conn.connect();
+
+        boolean needFallback = false;
+        if (conn.getResponseCode() == HttpURLConnection.HTTP_OK) {
+          LOG.debug("JDK performed authentication on our behalf.");
+          // If the JDK already did the SPNEGO back-and-forth for
+          // us, just pull out the token.
+          AuthenticatedURL.extractToken(conn, token);
+          if (isTokenKerberos(token)) {
+            return;
+          }
+          needFallback = true;
         }
-        needFallback = true;
-      }
-      if (!needFallback && isNegotiate(conn)) {
-        LOG.debug("Performing our own SPNEGO sequence.");
-        doSpnegoSequence(token);
-      } else {
-        LOG.debug("Using fallback authenticator sequence.");
-        Authenticator auth = getFallBackAuthenticator();
-        // Make sure that the fall back authenticator have the same
-        // ConnectionConfigurator, since the method might be overridden.
-        // Otherwise the fall back authenticator might not have the information
-        // to make the connection (e.g., SSL certificates)
-        auth.setConnectionConfigurator(connConfigurator);
-        auth.authenticate(url, token);
+        if (!needFallback && isNegotiate(conn)) {
+          LOG.debug("Performing our own SPNEGO sequence.");
+          doSpnegoSequence(token);
+        } else {
+          LOG.debug("Using fallback authenticator sequence.");
+          Authenticator auth = getFallBackAuthenticator();
+          // Make sure that the fall back authenticator have the same
+          // ConnectionConfigurator, since the method might be overridden.
+          // Otherwise the fall back authenticator might not have the
+          // information to make the connection (e.g., SSL certificates)
+          auth.setConnectionConfigurator(connConfigurator);
+          auth.authenticate(url, token);
+        }
+      } catch (IOException ex){
+        throw wrapExceptionWithMessage(ex,
+            "Error while authenticating with endpoint: " + url);
+      } catch (AuthenticationException ex){
+        throw wrapExceptionWithMessage(ex,
+            "Error while authenticating with endpoint: " + url);
       }
     }
   }
 
+  @VisibleForTesting
+   static <T extends Exception> T wrapExceptionWithMessage(
+      T exception, String msg) {
+    Class<? extends Throwable> exceptionClass = exception.getClass();
+    try {
+      Constructor<? extends Throwable> ctor = exceptionClass
+          .getConstructor(String.class);
+      Throwable t = ctor.newInstance(msg);
+      return (T) (t.initCause(exception));
+    } catch (Throwable e) {
+      LOG.debug("Unable to wrap exception of type {}, it has "
+          + "no (String) constructor.", exceptionClass, e);
+      return exception;
+    }
+  }
+
   /**
    * If the specified URL does not support SPNEGO authentication, a fallback {@link Authenticator} will be used.
    * <p>

+ 14 - 8
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/Signer.java

@@ -14,8 +14,11 @@
 package org.apache.hadoop.security.authentication.util;
 
 import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.codec.binary.StringUtils;
 
-import java.nio.charset.Charset;
+import javax.crypto.Mac;
+import javax.crypto.spec.SecretKeySpec;
+import java.security.InvalidKeyException;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 
@@ -24,6 +27,7 @@ import java.security.NoSuchAlgorithmException;
  */
 public class Signer {
   private static final String SIGNATURE = "&s=";
+  private static final String SIGNING_ALGORITHM = "HmacSHA256";
 
   private SignerSecretProvider secretProvider;
 
@@ -86,25 +90,27 @@ public class Signer {
    */
   protected String computeSignature(byte[] secret, String str) {
     try {
-      MessageDigest md = MessageDigest.getInstance("SHA");
-      md.update(str.getBytes(Charset.forName("UTF-8")));
-      md.update(secret);
-      byte[] digest = md.digest();
-      return new Base64(0).encodeToString(digest);
-    } catch (NoSuchAlgorithmException ex) {
+      SecretKeySpec key = new SecretKeySpec((secret), SIGNING_ALGORITHM);
+      Mac mac = Mac.getInstance(SIGNING_ALGORITHM);
+      mac.init(key);
+      byte[] sig = mac.doFinal(StringUtils.getBytesUtf8(str));
+      return new Base64(0).encodeToString(sig);
+    } catch (NoSuchAlgorithmException | InvalidKeyException ex) {
       throw new RuntimeException("It should not happen, " + ex.getMessage(), ex);
     }
   }
 
   protected void checkSignatures(String rawValue, String originalSignature)
       throws SignerException {
+    byte[] orginalSignatureBytes = StringUtils.getBytesUtf8(originalSignature);
     boolean isValid = false;
     byte[][] secrets = secretProvider.getAllSecrets();
     for (int i = 0; i < secrets.length; i++) {
       byte[] secret = secrets[i];
       if (secret != null) {
         String currentSignature = computeSignature(secret, rawValue);
-        if (originalSignature.equals(currentSignature)) {
+        if (MessageDigest.isEqual(orginalSignatureBytes,
+            StringUtils.getBytesUtf8(currentSignature))) {
           isValid = true;
           break;
         }

+ 29 - 0
hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java

@@ -20,6 +20,9 @@ import static org.apache.hadoop.security.authentication.server.KerberosAuthentic
 import static org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler.KEYTAB;
 import static org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler.NAME_RULES;
 
+import java.io.IOException;
+import java.nio.charset.CharacterCodingException;
+import javax.security.sasl.AuthenticationException;
 import org.apache.hadoop.minikdc.KerberosSecurityTestcase;
 import org.apache.hadoop.security.authentication.KerberosTestUtils;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
@@ -218,4 +221,30 @@ public class TestKerberosAuthenticator extends KerberosSecurityTestcase {
     });
   }
 
+  @Test(timeout = 60000)
+  public void testWrapExceptionWithMessage() {
+    IOException ex;
+    ex = new IOException("Induced exception");
+    ex = KerberosAuthenticator.wrapExceptionWithMessage(ex, "Error while "
+        + "authenticating with endpoint: localhost");
+    Assert.assertEquals("Induced exception", ex.getCause().getMessage());
+    Assert.assertEquals("Error while authenticating with endpoint: localhost",
+        ex.getMessage());
+
+    ex = new AuthenticationException("Auth exception");
+    ex = KerberosAuthenticator.wrapExceptionWithMessage(ex, "Error while "
+        + "authenticating with endpoint: localhost");
+    Assert.assertEquals("Auth exception", ex.getCause().getMessage());
+    Assert.assertEquals("Error while authenticating with endpoint: localhost",
+        ex.getMessage());
+
+    // Test for Exception with  no (String) constructor
+    // redirect the LOG to and check log message
+    ex = new CharacterCodingException();
+    Exception ex2 = KerberosAuthenticator.wrapExceptionWithMessage(ex,
+        "Error while authenticating with endpoint: localhost");
+    Assert.assertTrue(ex instanceof CharacterCodingException);
+    Assert.assertTrue(ex.equals(ex2));
+  }
+
 }

+ 3 - 8
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ConfServlet.java

@@ -116,20 +116,15 @@ public class ConfServlet extends HttpServlet {
 
     config.get("ozone.enabled");
     switch (cmd) {
-    case "getOzoneTags":
-      LOG.debug(
-          "Sending json for tags:" + gson.toJson(OzonePropertyTag.values()));
-      out.write(gson.toJson(OzonePropertyTag.values()));
-      break;
     case "getPropertyByTag":
       String tags = request.getParameter("tags");
       String tagGroup = request.getParameter("group");
       LOG.debug("Getting all properties for tags:" + tags + " group:" +
           tagGroup);
-      List<PropertyTag> tagList = new ArrayList<>();
+      List<String> tagList = new ArrayList<>();
       for (String tag : tags.split(",")) {
-        if (config.getPropertyTag(tag, tagGroup) != null) {
-          tagList.add(config.getPropertyTag(tag, tagGroup));
+        if (config.isPropertyTag(tag)) {
+          tagList.add(tag);
         }
       }
 

+ 84 - 62
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -83,7 +83,6 @@ import javax.xml.transform.stream.StreamResult;
 
 import com.google.common.base.Charsets;
 import org.apache.commons.collections.map.UnmodifiableMap;
-import org.apache.commons.io.FilenameUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -195,6 +194,30 @@ import static org.apache.commons.lang3.StringUtils.isNotBlank;
  * parameters and these are suppressible by configuring
  * <tt>log4j.logger.org.apache.hadoop.conf.Configuration.deprecation</tt> in
  * log4j.properties file.
+ *
+ * <h4 id="Tags">Tags</h4>
+ *
+ * <p>Optionally we can tag related properties together by using tag
+ * attributes. System tags are defined by hadoop.system.tags property. Users
+ * can define there own custom tags in  hadoop.custom.tags property.
+ *
+ * <p>For example, we can tag existing property as:
+ * <tt><pre>
+ *  &lt;property&gt;
+ *    &lt;name&gt;dfs.replication&lt;/name&gt;
+ *    &lt;value&gt;3&lt;/value&gt;
+ *    &lt;tag&gt;HDFS,REQUIRED&lt;/tag&gt;
+ *  &lt;/property&gt;
+ *
+ *  &lt;property&gt;
+ *    &lt;name&gt;dfs.data.transfer.protection&lt;/name&gt;
+ *    &lt;value&gt;3&lt;/value&gt;
+ *    &lt;tag&gt;HDFS,SECURITY&lt;/tag&gt;
+ *  &lt;/property&gt;
+ * </pre></tt>
+ * <p> Properties marked with tags can be retrieved with <tt>conf
+ * .getAllPropertiesByTag("HDFS")</tt> or <tt>conf.getAllPropertiesByTags
+ * (Arrays.asList("YARN","SECURITY"))</tt>.</p>
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
@@ -206,6 +229,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   private static final Logger LOG_DEPRECATION =
       LoggerFactory.getLogger(
           "org.apache.hadoop.conf.Configuration.deprecation");
+  private static final Set<String> TAGS = new HashSet<>();
 
   private boolean quietmode = true;
 
@@ -296,15 +320,10 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   private static final WeakHashMap<Configuration,Object> REGISTRY = 
     new WeakHashMap<Configuration,Object>();
 
-  /**
-   * Map to register all classes holding property tag enums.
-   */
-  private static final Map<String, Class>
-      REGISTERED_TAG_CLASS = new HashMap<>();
   /**
    * Map to hold properties by there tag groupings.
    */
-  private final Map<PropertyTag, Properties> propertyTagsMap =
+  private final Map<String, Properties> propertyTagsMap =
       new ConcurrentHashMap<>();
 
   /**
@@ -785,12 +804,6 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   public Configuration(boolean loadDefaults) {
     this.loadDefaults = loadDefaults;
 
-    // Register all classes holding property tags with
-    REGISTERED_TAG_CLASS.put("core", CorePropertyTag.class);
-    REGISTERED_TAG_CLASS.put("hdfs", HDFSPropertyTag.class);
-    REGISTERED_TAG_CLASS.put("yarn", YarnPropertyTag.class);
-    REGISTERED_TAG_CLASS.put("ozone", OzonePropertyTag.class);
-
     synchronized(Configuration.class) {
       REGISTRY.put(this, null);
     }
@@ -821,7 +834,6 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       this.finalParameters = Collections.newSetFromMap(
           new ConcurrentHashMap<String, Boolean>());
       this.finalParameters.addAll(other.finalParameters);
-      this.REGISTERED_TAG_CLASS.putAll(other.REGISTERED_TAG_CLASS);
       this.propertyTagsMap.putAll(other.propertyTagsMap);
     }
 
@@ -2920,6 +2932,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
         resources.set(i, ret);
       }
     }
+    this.removeUndeclaredTags(properties);
   }
   
   private Resource loadResource(Properties properties,
@@ -3124,7 +3137,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
               break;
             }
             confSource.add(name);
-            //Read tags and put them in propertyTagsMap
+            // Read tags and put them in propertyTagsMap
             if (confTag != null) {
               readTagFromConfig(confTag, confName, confValue, confSource);
             }
@@ -3166,48 +3179,61 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
   }
 
+  /**
+   * Removes undeclared tags and related properties from propertyTagsMap.
+   * Its required because ordering of properties in xml config files is not
+   * guaranteed.
+   * @param prop
+   */
+  private void removeUndeclaredTags(Properties prop) {
+    // Get all system tags
+    if (prop.containsKey(CommonConfigurationKeys.HADOOP_SYSTEM_TAGS)){
+      String systemTags = prop.getProperty(CommonConfigurationKeys
+              .HADOOP_SYSTEM_TAGS);
+      Arrays.stream(systemTags.split(",")).forEach(tag -> TAGS.add(tag));
+    }
+    // Get all custom tags
+    if (prop.containsKey(CommonConfigurationKeys.HADOOP_CUSTOM_TAGS)) {
+      String customTags = prop.getProperty(CommonConfigurationKeys
+          .HADOOP_CUSTOM_TAGS);
+      Arrays.stream(customTags.split(",")).forEach(tag -> TAGS.add(tag));
+    }
+
+    Set undeclaredTags = propertyTagsMap.keySet();
+    if (undeclaredTags.retainAll(TAGS)) {
+      LOG.info("Removed undeclared tags:");
+    }
+  }
+
+  /**
+   * Read the values passed as tags and store them in a
+   * map for later retrieval.
+   * @param attributeValue
+   * @param confName
+   * @param confValue
+   * @param confSource
+   */
   private void readTagFromConfig(String attributeValue, String confName, String
       confValue, List<String> confSource) {
     for (String tagStr : attributeValue.split(",")) {
       tagStr = tagStr.trim();
       try {
-        if (confSource.size() > 0) {
-          for (String source : confSource) {
-            PropertyTag tag1 = this.getPropertyTag(tagStr,
-                FilenameUtils.getName(source).split("-")[0]);
-            if (tag1 != null) {
-              //Handle property with no/null value
-              if (confValue == null) {
-                confValue = "";
-              }
-              if (propertyTagsMap.containsKey(tag1)) {
-                propertyTagsMap.get(tag1).setProperty(confName, confValue);
-              } else {
-                Properties props = new Properties();
-                props.setProperty(confName, confValue);
-                propertyTagsMap.put(tag1, props);
-              }
-            }
-          }
+        // Handle property with no/null value
+        if (confValue == null) {
+          confValue = "";
+        }
+        if (propertyTagsMap.containsKey(tagStr)) {
+          propertyTagsMap.get(tagStr).setProperty(confName, confValue);
         } else {
-          // If no source is set try to find tag in CorePropertyTag
-          if (propertyTagsMap.containsKey(CorePropertyTag.valueOf(tagStr))) {
-            propertyTagsMap.get(CorePropertyTag.valueOf(tagStr))
-                .setProperty(confName, confValue);
-          } else {
-            Properties props = new Properties();
-            props.setProperty(confName, confValue);
-            propertyTagsMap.put(CorePropertyTag.valueOf(tagStr),
-                props);
-          }
+          Properties props = new Properties();
+          props.setProperty(confName, confValue);
+          propertyTagsMap.put(tagStr, props);
         }
       } catch (Exception ex) {
-        // Log the invalid tag and continue to parse rest of the properties.
-        LOG.info("Invalid tag '" + tagStr + "' found for "
-            + "property:" + confName + " Source:" + Arrays
-            .toString(confSource.toArray()), ex);
+        // Log the exception at trace level.
+        LOG.trace("Tag '{}' for property:{} Source:{}", tagStr, confName,
+            Arrays.toString(confSource.toArray()), ex);
       }
-
     }
   }
 
@@ -3691,9 +3717,10 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
 
   /**
    * Get all properties belonging to tag.
-   * @return Properties with matching properties
+   * @param tag tag
+   * @return Properties with matching tag
    */
-  public Properties getAllPropertiesByTag(final PropertyTag tag) {
+  public Properties getAllPropertiesByTag(final String tag) {
     Properties props = new Properties();
     if (propertyTagsMap.containsKey(tag)) {
       props.putAll(propertyTagsMap.get(tag));
@@ -3704,12 +3731,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   /**
    * Get all properties belonging to list of input tags. Calls
    * getAllPropertiesByTag internally.
-   *
-   * @return Properties with all matching properties
+   * @param tagList list of input tags
+   * @return Properties with matching tags
    */
-  public Properties getAllPropertiesByTags(final List<PropertyTag> tagList) {
+  public Properties getAllPropertiesByTags(final List<String> tagList) {
     Properties prop = new Properties();
-    for (PropertyTag tag : tagList) {
+    for (String tag : tagList) {
       prop.putAll(this.getAllPropertiesByTag(tag));
     }
     return prop;
@@ -3719,15 +3746,10 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * Get Property tag Enum corresponding to given source.
    *
    * @param tagStr String representation of Enum
-   * @param group Group to which enum belongs.Ex hdfs,yarn
-   * @return Properties with all matching properties
+   * @return true if tagStr is a valid tag
    */
-  public PropertyTag getPropertyTag(String tagStr, String group) {
-    PropertyTag tag = null;
-    if (REGISTERED_TAG_CLASS.containsKey(group)) {
-      tag = (PropertyTag) Enum.valueOf(REGISTERED_TAG_CLASS.get(group), tagStr);
-    }
-    return tag;
+  public boolean isPropertyTag(String tagStr) {
+    return this.TAGS.contains(tagStr);
   }
 
   private void putIntoUpdatingResource(String key, String[] value) {

+ 0 - 37
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/CorePropertyTag.java

@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.hadoop.conf;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/***************************************************************************
- * Enum for tagging hadoop core properties according to there usage.
- * CorePropertyTag implements the
- * {@link org.apache.hadoop.conf.PropertyTag} interface,
- ***************************************************************************/
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public enum CorePropertyTag implements PropertyTag {
-  CORE,
-  REQUIRED,
-  PERFORMANCE,
-  CLIENT,
-  SERVER,
-  SECURITY,
-  DEBUG
-}

+ 0 - 41
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/HDFSPropertyTag.java

@@ -1,41 +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.conf;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/***************************************************************************
- * Enum for tagging hdfs properties according to there usage or application.
- * HDFSPropertyTag implements the
- * {@link org.apache.hadoop.conf.PropertyTag} interface,
- ***************************************************************************/
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public enum HDFSPropertyTag implements PropertyTag {
-  HDFS,
-  NAMENODE,
-  DATANODE,
-  REQUIRED,
-  SECURITY,
-  KERBEROS,
-  PERFORMANCE,
-  CLIENT,
-  SERVER,
-  DEBUG,
-  DEPRICATED
-}

+ 0 - 49
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/OzonePropertyTag.java

@@ -1,49 +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.conf;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/***************************************************************************
- * Enum for tagging ozone properties according to there usage.
- * OzonePropertyTag implements the
- * {@link org.apache.hadoop.conf.PropertyTag} interface,
- ***************************************************************************/
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public enum OzonePropertyTag implements PropertyTag {
-  OZONE,
-  CBLOCK,
-  MANAGEMENT,
-  SECURITY,
-  PERFORMANCE,
-  DEBUG,
-  CLIENT,
-  SERVER,
-  KSM,
-  SCM,
-  CRITICAL,
-  RATIS,
-  CONTAINER,
-  REQUIRED,
-  REST,
-  STORAGE,
-  PIPELINE,
-  STANDALONE,
-  KUBERNETES
-}

+ 0 - 30
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/PropertyTag.java

@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.hadoop.conf;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/***********************************************************
- * PropertyTag is used for creating extendable property tag Enums.
- * Property tags will group related properties together.
- ***********************************************************/
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public interface PropertyTag {
-
-}

+ 0 - 39
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/YarnPropertyTag.java

@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.hadoop.conf;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/***************************************************************************
- * Enum for tagging yarn properties according to there usage or application.
- * YarnPropertyTag implements the
- * {@link org.apache.hadoop.conf.PropertyTag} interface,
- ***************************************************************************/
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public enum YarnPropertyTag implements PropertyTag {
-  YARN,
-  RESOURCEMANAGER,
-  SECURITY,
-  KERBEROS,
-  PERFORMANCE,
-  CLIENT,
-  REQUIRED,
-  SERVER,
-  DEBUG
-}

+ 2 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java

@@ -199,9 +199,8 @@ public abstract class KeyProvider {
       return cipher;
     }
 
-    @SuppressWarnings("unchecked")
     public Map<String, String> getAttributes() {
-      return (attributes == null) ? Collections.EMPTY_MAP : attributes;
+      return (attributes == null) ? Collections.emptyMap() : attributes;
     }
 
     /**
@@ -370,9 +369,8 @@ public abstract class KeyProvider {
       return description;
     }
 
-    @SuppressWarnings("unchecked")
     public Map<String, String> getAttributes() {
-      return (attributes == null) ? Collections.EMPTY_MAP : attributes;
+      return (attributes == null) ? Collections.emptyMap() : attributes;
     }
 
     @Override

+ 0 - 11
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java

@@ -355,17 +355,6 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   public static final String HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS =
     "hadoop.user.group.metrics.percentiles.intervals";
 
-  /* When creating UGI with UserGroupInformation(Subject), treat the passed
-   * subject external if set to true, and assume the owner of the subject
-   * should do the credential renewal.
-   *
-   * This is a temporary config to solve the compatibility issue with
-   * HADOOP-13558 and HADOOP-13805 fix, see the jiras for discussions.
-   */
-  public static final String HADOOP_TREAT_SUBJECT_EXTERNAL_KEY =
-      "hadoop.treat.subject.external";
-  public static final boolean HADOOP_TREAT_SUBJECT_EXTERNAL_DEFAULT = false;
-
   public static final String RPC_METRICS_QUANTILE_ENABLE =
       "rpc.metrics.quantile.enable";
   public static final boolean RPC_METRICS_QUANTILE_ENABLE_DEFAULT = false;

+ 2 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java

@@ -860,5 +860,7 @@ public class CommonConfigurationKeysPublic {
           "credential$",
           "oauth.*token$",
           HADOOP_SECURITY_SENSITIVE_CONFIG_KEYS);
+  public static final String HADOOP_SYSTEM_TAGS = "hadoop.system.tags";
+  public static final String HADOOP_CUSTOM_TAGS = "hadoop.custom.tags";
 }
 

+ 31 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java

@@ -180,7 +180,8 @@ public class BZip2Codec implements Configurable, SplittableCompressionCodec {
       new DecompressorStream(in, decompressor,
                              conf.getInt(IO_FILE_BUFFER_SIZE_KEY,
                                  IO_FILE_BUFFER_SIZE_DEFAULT)) :
-      new BZip2CompressionInputStream(in);
+      new BZip2CompressionInputStream(
+              in, 0L, Long.MAX_VALUE, READ_MODE.BYBLOCK);
   }
 
   /**
@@ -362,9 +363,29 @@ public class BZip2Codec implements Configurable, SplittableCompressionCodec {
       bufferedIn = new BufferedInputStream(super.in);
       this.startingPos = super.getPos();
       this.readMode = readMode;
+      long numSkipped = 0;
       if (this.startingPos == 0) {
         // We only strip header if it is start of file
         bufferedIn = readStreamHeader();
+      } else if (this.readMode == READ_MODE.BYBLOCK  &&
+          this.startingPos <= HEADER_LEN + SUB_HEADER_LEN) {
+        // When we're in BYBLOCK mode and the start position is >=0
+        // and < HEADER_LEN + SUB_HEADER_LEN, we should skip to after
+        // start of the first bz2 block to avoid duplicated records
+        numSkipped = HEADER_LEN + SUB_HEADER_LEN + 1 - this.startingPos;
+        long skipBytes = numSkipped;
+        while (skipBytes > 0) {
+          long s = bufferedIn.skip(skipBytes);
+          if (s > 0) {
+            skipBytes -= s;
+          } else {
+            if (bufferedIn.read() == -1) {
+              break; // end of the split
+            } else {
+              skipBytes--;
+            }
+          }
+        }
       }
       input = new CBZip2InputStream(bufferedIn, readMode);
       if (this.isHeaderStripped) {
@@ -375,7 +396,15 @@ public class BZip2Codec implements Configurable, SplittableCompressionCodec {
         input.updateReportedByteCount(SUB_HEADER_LEN);
       }
 
-      this.updatePos(false);
+      if (numSkipped > 0) {
+        input.updateReportedByteCount((int) numSkipped);
+      }
+
+      // To avoid dropped records, not advertising a new byte position
+      // when we are in BYBLOCK mode and the start position is 0
+      if (!(this.readMode == READ_MODE.BYBLOCK && this.startingPos == 0)) {
+        this.updatePos(false);
+      }
     }
 
     private BufferedInputStream readStreamHeader() throws IOException {

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

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

File diff suppressed because it is too large
+ 429 - 462
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java


+ 17 - 9
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>
@@ -2960,16 +2973,11 @@
     </description>
   </property>
   <property>
-    <name>hadoop.treat.subject.external</name>
-    <value>false</value>
+    <name>hadoop.system.tags</name>
+    <value>YARN,HDFS,NAMENODE,DATANODE,REQUIRED,SECURITY,KERBEROS,PERFORMANCE,CLIENT
+      ,SERVER,DEBUG,DEPRICATED,COMMON,OPTIONAL</value>
     <description>
-      When creating UGI with UserGroupInformation(Subject), treat the passed
-      subject external if set to true, and assume the owner of the subject
-      should do the credential renewal.
-
-      When true this property will introduce an incompatible change which
-      may require changes in client code. For more details, see the jiras:
-      HADOOP-13805,HADOOP-13558.
+      System tags to group related properties together.
     </description>
   </property>
 </configuration>

+ 278 - 0
hadoop-common-project/hadoop-common/src/site/markdown/AdminCompatibilityGuide.md

@@ -0,0 +1,278 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+Apache Hadoop Compatibility
+===========================
+
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
+
+Purpose
+-------
+
+This purpose of this document is to distill down the
+[Hadoop Compatibility Guidelines](./Compatibility.html) into the information
+relevant for a system administrator.
+
+### Target Audience
+
+The target audience is administrators who are responsible for maintaining
+Apache Hadoop clusters and who must plan for and execute cluster upgrades.
+
+Hadoop Releases
+---------------
+
+The Hadoop development community periodically produces new Hadoop releases to
+introduce new functionality and fix existing issues. Realeses fall into three
+categories:
+
+* Major: a major release will typically include significant new functionality and generally represents the largest upgrade compatibility risk. A major release increments the first number of the release version, e.g. going from 2.8.2 to 3.0.0.
+* Minor: a minor release will typically include some new functionality as well as fixes for some notable issues. A minor release should not pose much upgrade risk in most cases. A minor release increments the middle number of release version, e.g. going from 2.8.2 to 2.9.0.
+* Maintenance: a maintenance release should not include any new functionality. The purpose of a maintenance release is to resolve a set of issues that are deemed by the developer community to be significant enough to be worth pushing a new release to address them. Maintenance releases should pose very little upgrade risk. A maintenance release increments the final number in the release version, e.g. going from 2.8.2 to 2.8.3.
+
+Platform Dependencies
+---------------------
+
+The set of native components on which Hadoop depends is considered part of the
+Hadoop ABI. The Hadoop development community endeavors to maintain ABI
+compatibility to the fullest extent possible. Between minor releases the
+minimum supported version numbers for Hadoop's native dependencies will not
+be increased unless necessary, such as for security or licensing issues. When
+such changes occur, the Hadoop developer community to try to keep the same
+major version and only update the minor version.
+
+Hadoop depends on the Java virtual machine. The minimum supported
+version of the JVM will not change between major releases of Hadoop. In the
+event that the current minimum supported JVM version becomes unsupported
+between major releases, the minimum supported JVM version may be changed in a
+minor release.
+
+Network
+-------
+
+Hadoop has dependencies on some transport level technologies, such as SSL. The
+minimum supported version of these dependencies will not
+be increased unless necessary, such as for security or licensing issues. When
+such changes occur, the Hadoop developer community to try to keep the same
+major version and only update the minor version.
+
+Service port numbers for Hadoop will remain the same within a major version,
+though may be changed in a major release.
+
+Hadoop's internal wire protocols will be maintained as backward and forward
+compatible across minor releases within the same major version, both between
+clients and servers and between servers, with the intent of enabling rolling
+upgrades. Forward and backward compatibility of wire protocols across major
+releases may be possible and may allow for rolling upgrades under certain
+conditions, but no guarantees are made.
+
+Scripting and Automation
+------------------------
+
+### REST APIs
+
+The Hadoop REST APIs provide an easy mechanism for collecting information about
+the state of the Hadoop system. To support REST clients, the Hadoop
+REST APIs are versioned and will not change incompatibly within a version.
+Both the endpoint itself along with the list of supported parameters and the
+output from the endpoint are prohibited from changing incompatibly within a
+REST endpoint version. Note, however, that introducing new fields and other
+additive changes are considered compatible changes, so any consumer of the
+REST API should be flexible enough to ignore unknown fields.
+
+The REST API version is a single number and has no relationship with the Hadoop
+version number. The version number is encoded in the endpoint URL prefixed
+with a 'v', for example 'v1'. A new REST endpoint version may only be
+introduced with a minor or major release. A REST endpoint version may only be
+removed after being labeled as deprecated for a full major release.
+
+### Parsing Hadoop Output
+
+Hadoop produces a variety of outputs that could conceivably parsed by automated
+tools. When consuming output from Hadoop, please consider the following:
+
+* Hadoop log output is not expected to change with a maintenance release unless it resolves a correctness issue. While log output can be consumed by software directly, it is intended primarily for a human reader.
+* Hadoop produces audit logs for a variety of operations. The audit logs are intended to be machine readable, though the addition of new records and fields are considered to be compatible changes. Any consumer of the audit logs should allow for unexpected records and fields. The audit log format may not change incompatibly between major releases.
+* Metrics data produced by Hadoop is mostly intended for automated consumption. The metrics format may not change in an incompatible way between major releases, but new records and fields can be compatibly added at any time. Consumers of the metrics data should allow for unknown records and fields.
+
+### CLIs
+
+Hadoop's set of CLIs provide the ability to manage various aspects of the
+system as well as discover information about the system's state.
+Between major releases, no CLI tool options will be removed or
+change semantically. The exception to that rule is CLI tools and tool options
+that are explicitly labeled as experimental and subject to change.
+The output from CLI tools will likewise remain the same
+within a major version number unless otherwise documented.
+
+Note that any change to CLI tool output is
+considered an incompatible change, so between major versions, the CLI output
+will not change. Note that the CLI tool output is distinct from
+the log output produced by the CLI tools. Log output is not intended for
+automated consumption and may change at any time.
+
+### Web UI
+
+The web UIs that are exposed by Hadoop are for human consumption only. Scraping
+the UIs for data is not a supported use. No effort is made to ensure any
+kind of compatibility between the data displayed in any of the web UIs
+across releases.
+
+Hadoop State Data
+-----------------
+
+Hadoop's internal system state is private and should not be modified directly.
+The following policies govern the upgrade characteristics of the various
+internal state stores:
+
+* The internal MapReduce state data will remain compatible across minor releases within the same major version to facilitate rolling upgrades while MapReduce workloads execute.
+* HDFS maintains metadata about the data stored in HDFS in a private, internal format that is versioned. In the event of an incompatible change, the store's version number will be incremented. When upgrading an existing cluster, the metadata store will automatically be upgraded if possible. After the metadata store has been upgraded, it is always possible to reverse the upgrade process.
+* The AWS S3A guard keeps a private, internal metadata store that is versioned. Incompatible changes will cause the version number to be incremented. If an upgrade requires reformatting the store, it will be indicated in the release notes.
+* The YARN resource manager keeps a private, internal state store of application and scheduler information that is versioned. Incompatible changes will cause the version number to be incremented. If an upgrade requires reformatting the store, it will be indicated in the release notes.
+* The YARN node manager keeps a private, internal state store of application information that is versioned. Incompatible changes will cause the version number to be incremented. If an upgrade requires reformatting the store, it will be indicated in the release notes.
+* The YARN federation service keeps a private, internal state store of application and cluster information that is versioned. Incompatible changes will cause the version number to be incremented. If an upgrade requires reformatting the store, it will be indicated in the release notes.
+
+Hadoop Configurations
+---------------------
+
+Hadoop uses two primary forms of configuration files: XML configuration files
+and logging configuration files.
+
+### XML Configuration Files
+
+The XML configuration files contain a set of properties as name-value pairs.
+The names and meanings of the properties are defined by Hadoop and are
+guaranteed to be stable across minor releases. A property can only be removed
+in a major release and only if it has been marked as deprecated for at least a
+full major release. Most properties have a default value that will be used if
+the property is not explicitly set in the XML configuration files. The default
+property values will not be changed during a maintenance releas.  For details
+about the properties supported by the various Hadoop components,
+see the component documentation.
+
+Downstream projects and users can add their own properties into the XML
+configuration files for use by their tools and applications. While Hadoop
+makes no formal restrictions about defining new properties, a new property
+that conflicts with a property defined by Hadoop can lead to unexpected and
+undesirable results. Users are encouraged to avoid using custom configuration
+property names that conflict with the namespace of Hadoop-defined properties
+and thus should avoid using any prefixes used by Hadoop,
+e.g. hadoop, io, ipc, fs, net, file, ftp, kfs, ha, file, dfs, mapred,
+mapreduce, and yarn.
+
+### Logging Configuration Files
+
+The log output produced by Hadoop daemons and CLIs is governed by a set of
+configuration files. These files control the minimum level of log message that
+will be output by the various components of Hadoop, as well as where and how
+those messages are stored. Between minor releases no changes will be made to
+the log configuration that reduce, eliminate, or redirect the log messages.
+
+### Other Configuration Files
+
+Hadoop makes use of a number of other types of configuration files in a variety
+of formats, such as the JSON resource profiles configuration or the XML fair
+scheduler configuration. No incompatible changes will be introduced to the
+configuration file formats within a minor release. Even between minor releases
+incompatible configuration file format changes will be avoided if possible.
+
+Hadoop Distribution
+-------------------
+
+### Configuration Files
+
+The location and general structure of the Hadoop configuration files, job
+history information (as consumed by the job history server), and logs files
+generated by Hadoop will be maintained across maintenance releases.
+
+### JARs, etc.
+
+The contents of the Hadoop distribution, e.g. JAR files, are
+subject to change at any time and should not be treated as reliable, except
+for the client artifacts. Client artifacts and their contents will remain
+compatible within a major release. It is the goal of the Hadoop development
+community to allow application code to continue to function unchanged across
+minor releases and, whenever possible, across major releases. The current list
+of client artifacts is as follows:
+
+* hadoop-client
+* hadoop-client-api
+* hadoop-client-minicluster
+* hadoop-client-runtime
+* hadoop-hdfs-client
+* hadoop-hdfs-native-client
+* hadoop-mapreduce-client-app
+* hadoop-mapreduce-client-common
+* hadoop-mapreduce-client-core
+* hadoop-mapreduce-client-jobclient
+* hadoop-mapreduce-client-nativetask
+* hadoop-yarn-client
+
+### Environment Variables
+
+Some Hadoop components receive information through environment variables. For
+example, the ```HADOOP_OPTS``` environment variable is interpreted by most
+Hadoop processes as a string of additional JVM arguments to be used when
+starting a new JVM. Between minor releases the way Hadoop interprets environment
+variables will not change in an incompatible way. In other words, the same value
+placed into the same variable should produce the same result for all Hadoop
+releases within the same major version.
+
+### Library Dependencies
+
+Hadoop relies on a large number of third-party libraries for its operation. As
+much as possible the Hadoop developer community works to hide these dependencies
+from downstream developers. Nonetheless Hadoop does expose some
+of its dependencies, especially prior to Hadoop 3. No new dependency
+will be exposed by Hadoop via the client artifacts between major releases.
+
+A common downstream anti-pattern is to use the output of ```hadoop classpath```
+to set the downstream application's classpath or add all third-party JARs
+included with Hadoop to the downstream application's classpath. This practice
+creates a tight coupling between the downstream application and Hadoop's
+third-party dependencies, which leads to a fragile application that is hard to
+maintain as Hadoop's dependencies change. This practice is strongly discouraged.
+
+Hadoop also includes several native components, including compression, the
+container executor binary, and various native integrations. These native
+components introduce a set of native dependencies for Hadoop. The set of
+native dependencies can change in a minor release, but the Hadoop developer
+community will try to limit any dependency version changes to minor version
+changes as much as possible.
+
+### Hardware and OS Dependencies
+
+Hadoop is currently supported by the Hadoop developer community on Linux and
+Windows running on x86 and AMD processors. These OSes and processors are likely
+to remain supported for the foreseeable future. In the event that support plans
+change, the OS or processor to be dropped will be documented as deprecated
+for at least a full minor release, but ideally a full major release, before
+actually being dropped. Hadoop may function on other OSes and processor
+architectures, but the community may not be able to provide assistance in the
+event of issues.
+
+There are no guarantees on how the minimum resources required by Hadoop daemons
+will change between releases, even maintenance releases. Nonetheless, the
+Hadoop developer community will try to avoid increasing the requirements within
+a minor release.
+
+Any file systems supported Hadoop, such as through the FileSystem API, will
+in most cases continue to be supported throughout a major release. The only
+case where support for a file system can be dropped within a major version is
+if a clean migration path to an alternate client implementation is provided.
+
+Questions
+---------
+
+For question about developing applications and projects against Apache Hadoop,
+please contact the [user mailing list](mailto:user@hadoop.apache.org).

+ 21 - 12
hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md

@@ -220,10 +220,10 @@ dependencies is part of the Hadoop ABI.
 
 The minimum required versions of the native components on which Hadoop depends
 at compile time and/or runtime SHALL be considered
-[Evolving](./InterfaceClassification.html#Evolving). Changes to the minimum
+[Evolving](./InterfaceClassification.html#Evolving). The minimum
 required versions SHOULD NOT increase between minor releases within a major
 version, though updates because of security issues, license issues, or other
-reasons may occur. When the native components on which Hadoop depends must
+reasons MAY occur. When the native components on which Hadoop depends must
 be updated between minor releases within a major release, where possible the
 changes SHOULD only change the minor versions of the components without
 changing the major versions.
@@ -256,7 +256,11 @@ cross-version communications requires that the transports supported also be
 stable. The most likely source of transport changes stems from secure
 transports, such as SSL. Upgrading a service from SSLv2 to SSLv3 may break
 existing SSLv2 clients. The minimum supported major version of any transports
-MUST not increase across minor releases within a major version.
+SHOULD NOT increase between minor releases within a major version, though
+updates because of security issues, license issues, or other reasons MAY occur.
+When a transport must be updated between minor releases within a major release,
+where possible the changes SHOULD only change the minor versions of the
+components without changing the major versions.
 
 Service ports are considered as part of the transport mechanism. Default
 service port numbers must be kept consistent to prevent breaking clients.
@@ -310,15 +314,18 @@ according to the following:
 * Client-Server compatibility MUST be maintained so as to allow upgrading individual components without upgrading others. For example, upgrade HDFS from version 2.1.0 to 2.2.0 without upgrading MapReduce.
 * Server-Server compatibility MUST be maintained so as to allow mixed versions within an active cluster so the cluster may be upgraded without downtime in a rolling fashion.
 
-Existing transport mechanisms MUST continue to be supported across
-minor versions within a major version. Default service port numbers MUST remain
-consistent across minor version numbers within a major version.
+New transport mechanisms MUST only be introduced with minor or major version
+changes. Existing transport mechanisms MUST continue to be supported across
+minor versions within a major version. Default service port numbers SHALL be
+considered [Stable](./InterfaceClassification.html#Stable).
 
 ### REST APIs
 
-REST API compatibility applies to the REST endpoints (URLs) and response data
-format. Hadoop REST APIs are specifically meant for stable use by clients across
-releases, even major ones. The following is a non-exhaustive list of the
+REST API compatibility applies to the exposed REST endpoints (URLs) and response
+data format. Hadoop REST APIs are specifically meant for stable use by clients
+across releases, even major ones. For purposes of this document, an exposed
+PEST API is one that is documented in the public documentation.
+The following is a non-exhaustive list of the
 exposed REST APIs:
 
 * [WebHDFS](../hadoop-hdfs/WebHDFS.html)
@@ -334,10 +341,10 @@ increment the API version number.
 
 #### Policy
 
-The Hadoop REST APIs SHALL be considered
+The exposed Hadoop REST APIs SHALL be considered
 [Public](./InterfaceClassification.html#Public) and
 [Evolving](./InterfaceClassification.html#Evolving). With respect to API version
-numbers, the Hadoop REST APIs SHALL be considered
+numbers, the exposed Hadoop REST APIs SHALL be considered
 [Public](./InterfaceClassification.html#Public) and
 [Stable](./InterfaceClassification.html#Stable), i.e. no incompatible changes
 are allowed to within an API version number. A REST API version must be labeled
@@ -549,7 +556,9 @@ command return codes and output break compatibility and adversely affect users.
 
 All Hadoop CLI paths, usage, and output SHALL be considered
 [Public](./InterfaceClassification.html#Public) and
-[Stable](./InterfaceClassification.html#Stable).
+[Stable](./InterfaceClassification.html#Stable) unless documented as
+experimental and subject to change.
+
 Note that the CLI output SHALL be considered distinct from the log output
 generated by the Hadoop CLIs. The latter SHALL be governed by the policy on log
 output. Note also that for CLI output, all changes SHALL be considered

+ 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

+ 0 - 58
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java

@@ -247,62 +247,4 @@ public class TestConfServlet {
     }
     assertEquals("", sw.toString());
   }
-
-  @Test
-  public void testOzoneConfigTagCommands() throws Exception {
-    StringWriter sw = null;
-    PrintWriter pw = null;
-    ConfServlet service = null;
-    try {
-      service = new ConfServlet();
-      ServletContext context = mock(ServletContext.class);
-      ServletConfig servletConf = mock(ServletConfig.class);
-      service.init(servletConf);
-      Configuration config = mock(OzoneConfiguration.class);
-      when(context.getAttribute(HttpServer2.CONF_CONTEXT_ATTRIBUTE))
-          .thenReturn(new Configuration());
-      when(service.getServletContext())
-          .thenReturn(context);
-
-      HttpServletRequest request = mock(HttpServletRequest.class);
-      when(request.getParameter("cmd"))
-          .thenReturn("getOzoneTags");
-      HttpServletResponse response = mock(HttpServletResponse.class);
-      sw = new StringWriter();
-      pw = new PrintWriter(sw);
-      when(response.getWriter()).thenReturn(pw);
-
-      // response request
-      service.doGet(request, response);
-      String result = sw.toString().trim();
-
-      for (OzonePropertyTag ozoneTag : OzonePropertyTag.values()) {
-        assertTrue(result.contains(ozoneTag.toString()));
-      }
-
-      when(request.getParameter("cmd"))
-          .thenReturn("badCommand");
-      when(request.getParameter("tags"))
-          .thenReturn("OZONE,CLIENT");
-      when(request.getParameter("group"))
-          .thenReturn("ozone");
-
-      service.doGet(request, response);
-      Mockito.verify(response).sendError(
-          Mockito.eq(HttpServletResponse.SC_NOT_FOUND),
-          Mockito.eq("badCommand is not a valid command."));
-
-    } finally {
-      if (sw != null) {
-        sw.close();
-      }
-      if (pw != null) {
-        pw.close();
-      }
-      if (service != null) {
-        service.destroy();
-      }
-    }
-
-  }
 }

+ 59 - 62
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -26,11 +26,14 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStreamWriter;
+import java.io.PrintStream;
 import java.io.StringWriter;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -2339,22 +2342,29 @@ public class TestConfiguration {
   @Test
   public void testGetAllPropertiesByTags() throws Exception {
 
-    out = new BufferedWriter(new FileWriter(CONFIG_CORE));
-    startConfig();
-    appendPropertyByTag("dfs.cblock.trace.io", "false", "DEBUG");
-    appendPropertyByTag("dfs.replication", "1", "PERFORMANCE,REQUIRED");
-    appendPropertyByTag("dfs.namenode.logging.level", "INFO", "CLIENT,DEBUG");
-    endConfig();
+    try{
+      out = new BufferedWriter(new FileWriter(CONFIG_CORE));
+      startConfig();
+      appendProperty("hadoop.system.tags", "YARN,HDFS,NAMENODE");
+      appendProperty("hadoop.custom.tags", "MYCUSTOMTAG");
+      appendPropertyByTag("dfs.cblock.trace.io", "false", "YARN");
+      appendPropertyByTag("dfs.replication", "1", "HDFS");
+      appendPropertyByTag("dfs.namenode.logging.level", "INFO", "NAMENODE");
+      appendPropertyByTag("dfs.random.key", "XYZ", "MYCUSTOMTAG");
+      endConfig();
 
-    Path fileResource = new Path(CONFIG_CORE);
-    conf.addResource(fileResource);
-    conf.getProps();
+      Path fileResource = new Path(CONFIG_CORE);
+      conf.addResource(fileResource);
+      conf.getProps();
 
-    List<PropertyTag> tagList = new ArrayList<>();
-    tagList.add(CorePropertyTag.REQUIRED);
-    tagList.add(CorePropertyTag.PERFORMANCE);
-    tagList.add(CorePropertyTag.DEBUG);
-    tagList.add(CorePropertyTag.CLIENT);
+    } finally {
+      out.close();
+    }
+    System.out.println(Files.readAllLines(Paths.get(CONFIG_CORE)));
+    List<String> tagList = new ArrayList<>();
+    tagList.add("YARN");
+    tagList.add("HDFS");
+    tagList.add("NAMENODE");
 
     Properties properties = conf.getAllPropertiesByTags(tagList);
     String[] sources = conf.getPropertySources("dfs.replication");
@@ -2366,58 +2376,45 @@ public class TestConfiguration {
     assertEq(true, properties.containsKey("dfs.replication"));
     assertEq(true, properties.containsKey("dfs.cblock.trace.io"));
     assertEq(false, properties.containsKey("namenode.host"));
+
+    properties = conf.getAllPropertiesByTag("DEBUG");
+    assertEq(0, properties.size());
+    assertEq(false, properties.containsKey("dfs.namenode.logging.level"));
+    assertEq(true, conf.isPropertyTag("YARN"));
+    assertEq(true, conf.isPropertyTag("HDFS"));
+    assertEq(true, conf.isPropertyTag("NAMENODE"));
+    assertEq(true, conf.isPropertyTag("MYCUSTOMTAG"));
+    assertEq(false, conf.isPropertyTag("CMYCUSTOMTAG2"));
   }
 
   @Test
-  public void testGetAllPropertiesWithSourceByTags() throws Exception {
-
-    out = new BufferedWriter(new FileWriter(CONFIG));
-    startConfig();
-    appendPropertyByTag("dfs.cblock.trace.io", "false", "DEBUG",
-        "hdfs-default.xml", "core-site.xml");
-    appendPropertyByTag("dfs.replication", "1", "PERFORMANCE,HDFS",
-        "hdfs-default.xml");
-    appendPropertyByTag("yarn.resourcemanager.work-preserving-recovery"
-        + ".enabled", "INFO", "CLIENT,DEBUG", "yarn-default.xml", "yarn-site"
-        + ".xml");
-    endConfig();
-
-    Path fileResource = new Path(CONFIG);
-    conf.addResource(fileResource);
-    conf.getProps();
-
-    List<PropertyTag> tagList = new ArrayList<>();
-    tagList.add(CorePropertyTag.REQUIRED);
-
-    Properties properties;
-    properties = conf.getAllPropertiesByTags(tagList);
-    assertNotEquals(3, properties.size());
-
-    tagList.add(HDFSPropertyTag.DEBUG);
-    tagList.add(YarnPropertyTag.CLIENT);
-    tagList.add(HDFSPropertyTag.PERFORMANCE);
-    tagList.add(HDFSPropertyTag.HDFS);
-    properties = conf.getAllPropertiesByTags(tagList);
-    assertEq(3, properties.size());
+  public void testInvalidTags() throws Exception {
+    PrintStream output = System.out;
+    try {
+      ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+      System.setOut(new PrintStream(bytes));
 
-    assertEq(true, properties.containsKey("dfs.cblock.trace.io"));
-    assertEq(true, properties.containsKey("dfs.replication"));
-    assertEq(true, properties
-        .containsKey("yarn.resourcemanager.work-preserving-recovery.enabled"));
-    assertEq(false, properties.containsKey("namenode.host"));
+      out = new BufferedWriter(new FileWriter(CONFIG));
+      startConfig();
+      appendPropertyByTag("dfs.cblock.trace.io", "false", "MYOWNTAG,TAG2");
+      endConfig();
 
-    tagList.clear();
-    tagList.add(HDFSPropertyTag.DEBUG);
-    properties = conf.getAllPropertiesByTags(tagList);
-    assertEq(true, properties.containsKey("dfs.cblock.trace.io"));
-    assertEq(false, properties.containsKey("yarn.resourcemanager"
-        + ".work-preserving-recovery"));
-
-    tagList.clear();
-    tagList.add(YarnPropertyTag.DEBUG);
-    properties = conf.getAllPropertiesByTags(tagList);
-    assertEq(false, properties.containsKey("dfs.cblock.trace.io"));
-    assertEq(true, properties.containsKey("yarn.resourcemanager"
-        + ".work-preserving-recovery.enabled"));
+      Path fileResource = new Path(CONFIG);
+      conf.addResource(fileResource);
+      conf.getProps();
+
+      List<String> tagList = new ArrayList<>();
+      tagList.add("REQUIRED");
+      tagList.add("MYOWNTAG");
+      tagList.add("TAG2");
+
+      Properties properties = conf.getAllPropertiesByTags(tagList);
+      assertEq(0, properties.size());
+      assertFalse(properties.containsKey("dfs.cblock.trace.io"));
+      assertFalse(bytes.toString().contains("Invalid tag "));
+      assertFalse(bytes.toString().contains("Tag"));
+    } finally {
+      System.setOut(output);
+    }
   }
 }

+ 107 - 126
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileSystemCaching.java

@@ -18,22 +18,20 @@
 
 package org.apache.hadoop.fs;
 
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertNotSame;
-
 import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+
 import org.junit.Test;
 import java.security.PrivilegedExceptionAction;
 import java.util.concurrent.Semaphore;
 
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
@@ -42,14 +40,13 @@ public class TestFileSystemCaching {
 
   @Test
   public void testCacheEnabled() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set("fs.cachedfile.impl", FileSystem.getFileSystemClass("file", null).getName());
+    Configuration conf = newConf();
     FileSystem fs1 = FileSystem.get(new URI("cachedfile://a"), conf);
     FileSystem fs2 = FileSystem.get(new URI("cachedfile://a"), conf);
     assertSame(fs1, fs2);
   }
 
-  static class DefaultFs extends LocalFileSystem {
+  private static class DefaultFs extends LocalFileSystem {
     URI uri;
     @Override
     public void initialize(URI uri, Configuration conf) {
@@ -67,43 +64,30 @@ public class TestFileSystemCaching {
     conf.set("fs.defaultfs.impl", DefaultFs.class.getName());
     final URI defaultUri = URI.create("defaultfs://host");
     FileSystem.setDefaultUri(conf, defaultUri);
-    FileSystem fs = null;
-    
+
     // sanity check default fs
     final FileSystem defaultFs = FileSystem.get(conf);
     assertEquals(defaultUri, defaultFs.getUri());
     
     // has scheme, no auth
-    fs = FileSystem.get(URI.create("defaultfs:/"), conf);
-    assertSame(defaultFs, fs);
-    fs = FileSystem.get(URI.create("defaultfs:///"), conf);
-    assertSame(defaultFs, fs);
+    assertSame(defaultFs, FileSystem.get(URI.create("defaultfs:/"), conf));
+    assertSame(defaultFs, FileSystem.get(URI.create("defaultfs:///"), conf));
     
     // has scheme, same auth
-    fs = FileSystem.get(URI.create("defaultfs://host"), conf);
-    assertSame(defaultFs, fs);
+    assertSame(defaultFs, FileSystem.get(URI.create("defaultfs://host"), conf));
     
     // has scheme, different auth
-    fs = FileSystem.get(URI.create("defaultfs://host2"), conf);
-    assertNotSame(defaultFs, fs);
+    assertNotSame(defaultFs,
+        FileSystem.get(URI.create("defaultfs://host2"), conf));
     
     // no scheme, no auth
-    fs = FileSystem.get(URI.create("/"), conf);
-    assertSame(defaultFs, fs);
+    assertSame(defaultFs, FileSystem.get(URI.create("/"), conf));
     
     // no scheme, same auth
-    try {
-      fs = FileSystem.get(URI.create("//host"), conf);
-      fail("got fs with auth but no scheme");
-    } catch (UnsupportedFileSystemException e) {
-    }
-
-    // no scheme, different auth
-    try {
-      fs = FileSystem.get(URI.create("//host2"), conf);
-      fail("got fs with auth but no scheme");
-    } catch (UnsupportedFileSystemException e) {
-    }
+    intercept(UnsupportedFileSystemException.class,
+        () -> FileSystem.get(URI.create("//host"), conf));
+    intercept(UnsupportedFileSystemException.class,
+        () -> FileSystem.get(URI.create("//host2"), conf));
   }
   
   public static class InitializeForeverFileSystem extends LocalFileSystem {
@@ -132,9 +116,7 @@ public class TestFileSystemCaching {
          "TestFileSystemCaching$InitializeForeverFileSystem");
         try {
           FileSystem.get(new URI("localfs1://a"), conf);
-        } catch (IOException e) {
-          e.printStackTrace();
-        } catch (URISyntaxException e) {
+        } catch (IOException | URISyntaxException e) {
           e.printStackTrace();
         }
       }
@@ -162,31 +144,15 @@ public class TestFileSystemCaching {
   @SuppressWarnings("unchecked")
   @Test
   public <T extends TokenIdentifier> void testCacheForUgi() throws Exception {
-    final Configuration conf = new Configuration();
-    conf.set("fs.cachedfile.impl", FileSystem.getFileSystemClass("file", null).getName());
+    final Configuration conf = newConf();
     UserGroupInformation ugiA = UserGroupInformation.createRemoteUser("foo");
     UserGroupInformation ugiB = UserGroupInformation.createRemoteUser("bar");
-    FileSystem fsA = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
-      @Override
-      public FileSystem run() throws Exception {
-        return FileSystem.get(new URI("cachedfile://a"), conf);
-      }
-    });
-    FileSystem fsA1 = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
-      @Override
-      public FileSystem run() throws Exception {
-        return FileSystem.get(new URI("cachedfile://a"), conf);
-      }
-    });
+    FileSystem fsA = getCachedFS(ugiA, conf);
+    FileSystem fsA1 = getCachedFS(ugiA, conf);
     //Since the UGIs are the same, we should have the same filesystem for both
     assertSame(fsA, fsA1);
     
-    FileSystem fsB = ugiB.doAs(new PrivilegedExceptionAction<FileSystem>() {
-      @Override
-      public FileSystem run() throws Exception {
-        return FileSystem.get(new URI("cachedfile://a"), conf);
-      }
-    });
+    FileSystem fsB = getCachedFS(ugiB, conf);
     //Since the UGIs are different, we should end up with different filesystems
     //corresponding to the two UGIs
     assertNotSame(fsA, fsB);
@@ -194,47 +160,56 @@ public class TestFileSystemCaching {
     Token<T> t1 = mock(Token.class);
     UserGroupInformation ugiA2 = UserGroupInformation.createRemoteUser("foo");
     
-    fsA = ugiA2.doAs(new PrivilegedExceptionAction<FileSystem>() {
-      @Override
-      public FileSystem run() throws Exception {
-        return FileSystem.get(new URI("cachedfile://a"), conf);
-      }
-    });
+    fsA = getCachedFS(ugiA2, conf);
     // Although the users in the UGI are same, they have different subjects
     // and so are different.
     assertNotSame(fsA, fsA1);
     
     ugiA.addToken(t1);
     
-    fsA = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
-      @Override
-      public FileSystem run() throws Exception {
-        return FileSystem.get(new URI("cachedfile://a"), conf);
-      }
-    });
+    fsA = getCachedFS(ugiA, conf);
     // Make sure that different UGI's with the same subject lead to the same
     // file system.
     assertSame(fsA, fsA1);
   }
-  
+
+  /**
+   * Get the cached filesystem for "cachedfile://a" for the supplied user
+   * @param ugi user
+   * @param conf configuration
+   * @return the filesystem
+   * @throws IOException failure to get/init
+   * @throws InterruptedException part of the signature of UGI.doAs()
+   */
+  private FileSystem getCachedFS(UserGroupInformation ugi, Configuration conf)
+      throws IOException, InterruptedException {
+    return ugi.doAs((PrivilegedExceptionAction<FileSystem>)
+            () -> FileSystem.get(new URI("cachedfile://a"), conf));
+  }
+
   @Test
   public void testUserFS() throws Exception {
-    final Configuration conf = new Configuration();
-    conf.set("fs.cachedfile.impl", FileSystem.getFileSystemClass("file", null).getName());
+    final Configuration conf = newConf();
     FileSystem fsU1 = FileSystem.get(new URI("cachedfile://a"), conf, "bar");
     FileSystem fsU2 = FileSystem.get(new URI("cachedfile://a"), conf, "foo");
     
     assertNotSame(fsU1, fsU2);   
   }
-  
+
+  private Configuration newConf() throws IOException {
+    final Configuration conf = new Configuration();
+    conf.set("fs.cachedfile.impl",
+        FileSystem.getFileSystemClass("file", null).getName());
+    return conf;
+  }
+
   @Test
   public void testFsUniqueness() throws Exception {
-    final Configuration conf = new Configuration();
-    conf.set("fs.cachedfile.impl", FileSystem.getFileSystemClass("file", null).getName());
+    final Configuration conf = newConf();
     // multiple invocations of FileSystem.get return the same object.
     FileSystem fs1 = FileSystem.get(conf);
     FileSystem fs2 = FileSystem.get(conf);
-    assertTrue(fs1 == fs2);
+    assertSame(fs1, fs2);
 
     // multiple invocations of FileSystem.newInstance return different objects
     fs1 = FileSystem.newInstance(new URI("cachedfile://a"), conf, "bar");
@@ -246,33 +221,17 @@ public class TestFileSystemCaching {
   
   @Test
   public void testCloseAllForUGI() throws Exception {
-    final Configuration conf = new Configuration();
-    conf.set("fs.cachedfile.impl", FileSystem.getFileSystemClass("file", null).getName());
+    final Configuration conf = newConf();
     UserGroupInformation ugiA = UserGroupInformation.createRemoteUser("foo");
-    FileSystem fsA = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
-      @Override
-      public FileSystem run() throws Exception {
-        return FileSystem.get(new URI("cachedfile://a"), conf);
-      }
-    });
+    FileSystem fsA = getCachedFS(ugiA, conf);
     //Now we should get the cached filesystem
-    FileSystem fsA1 = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
-      @Override
-      public FileSystem run() throws Exception {
-        return FileSystem.get(new URI("cachedfile://a"), conf);
-      }
-    });
+    FileSystem fsA1 = getCachedFS(ugiA, conf);
     assertSame(fsA, fsA1);
     
     FileSystem.closeAllForUGI(ugiA);
     
     //Now we should get a different (newly created) filesystem
-    fsA1 = ugiA.doAs(new PrivilegedExceptionAction<FileSystem>() {
-      @Override
-      public FileSystem run() throws Exception {
-        return FileSystem.get(new URI("cachedfile://a"), conf);
-      }
-    });
+    fsA1 = getCachedFS(ugiA, conf);
     assertNotSame(fsA, fsA1);
   }
   
@@ -292,16 +251,17 @@ public class TestFileSystemCaching {
   @Test
   public void testDeleteOnExit() throws IOException {
     FileSystem mockFs = mock(FileSystem.class);
-    FileSystem fs = new FilterFileSystem(mockFs);
     Path path = new Path("/a");
+    try (FileSystem fs = new FilterFileSystem(mockFs)) {
 
-    // delete on close if path does exist
-    when(mockFs.getFileStatus(eq(path))).thenReturn(new FileStatus());
-    assertTrue(fs.deleteOnExit(path));
-    verify(mockFs).getFileStatus(eq(path));
-    reset(mockFs);
-    when(mockFs.getFileStatus(eq(path))).thenReturn(new FileStatus());
-    fs.close();
+      // delete on close if path does exist
+      when(mockFs.getFileStatus(eq(path))).thenReturn(new FileStatus());
+      assertTrue(fs.deleteOnExit(path));
+      verify(mockFs).getFileStatus(eq(path));
+      reset(mockFs);
+      when(mockFs.getFileStatus(eq(path))).thenReturn(new FileStatus());
+      fs.close();
+    }
     verify(mockFs).getFileStatus(eq(path));
     verify(mockFs).delete(eq(path), eq(true));
   }
@@ -309,14 +269,16 @@ public class TestFileSystemCaching {
   @Test
   public void testDeleteOnExitFNF() throws IOException {
     FileSystem mockFs = mock(FileSystem.class);
-    FileSystem fs = new FilterFileSystem(mockFs);
-    Path path = new Path("/a");
+    Path path;
+    try (FileSystem fs = new FilterFileSystem(mockFs)) {
+      path = new Path("/a");
 
-    // don't delete on close if path doesn't exist
-    assertFalse(fs.deleteOnExit(path));
-    verify(mockFs).getFileStatus(eq(path));
-    reset(mockFs);
-    fs.close();
+      // don't delete on close if path doesn't exist
+      assertFalse(fs.deleteOnExit(path));
+      verify(mockFs).getFileStatus(eq(path));
+      reset(mockFs);
+      fs.close();
+    }
     verify(mockFs, never()).getFileStatus(eq(path));
     verify(mockFs, never()).delete(any(Path.class), anyBoolean());
   }
@@ -325,15 +287,17 @@ public class TestFileSystemCaching {
   @Test
   public void testDeleteOnExitRemoved() throws IOException {
     FileSystem mockFs = mock(FileSystem.class);
-    FileSystem fs = new FilterFileSystem(mockFs);
-    Path path = new Path("/a");
+    Path path;
+    try (FileSystem fs = new FilterFileSystem(mockFs)) {
+      path = new Path("/a");
 
-    // don't delete on close if path existed, but later removed
-    when(mockFs.getFileStatus(eq(path))).thenReturn(new FileStatus());
-    assertTrue(fs.deleteOnExit(path));
-    verify(mockFs).getFileStatus(eq(path));
-    reset(mockFs);
-    fs.close();
+      // don't delete on close if path existed, but later removed
+      when(mockFs.getFileStatus(eq(path))).thenReturn(new FileStatus());
+      assertTrue(fs.deleteOnExit(path));
+      verify(mockFs).getFileStatus(eq(path));
+      reset(mockFs);
+      fs.close();
+    }
     verify(mockFs).getFileStatus(eq(path));
     verify(mockFs, never()).delete(any(Path.class), anyBoolean());
   }
@@ -341,18 +305,35 @@ public class TestFileSystemCaching {
   @Test
   public void testCancelDeleteOnExit() throws IOException {
     FileSystem mockFs = mock(FileSystem.class);
-    FileSystem fs = new FilterFileSystem(mockFs);
-    Path path = new Path("/a");
+    try (FileSystem fs = new FilterFileSystem(mockFs)) {
+      Path path = new Path("/a");
 
-    // don't delete on close if path existed, but later cancelled
-    when(mockFs.getFileStatus(eq(path))).thenReturn(new FileStatus());
-    assertTrue(fs.deleteOnExit(path));
-    verify(mockFs).getFileStatus(eq(path));
-    assertTrue(fs.cancelDeleteOnExit(path));
-    assertFalse(fs.cancelDeleteOnExit(path)); // false because not registered
-    reset(mockFs);
-    fs.close();
+      // don't delete on close if path existed, but later cancelled
+      when(mockFs.getFileStatus(eq(path))).thenReturn(new FileStatus());
+      assertTrue(fs.deleteOnExit(path));
+      verify(mockFs).getFileStatus(eq(path));
+      assertTrue(fs.cancelDeleteOnExit(path));
+      assertFalse(fs.cancelDeleteOnExit(path)); // false because not registered
+      reset(mockFs);
+      fs.close();
+    }
     verify(mockFs, never()).getFileStatus(any(Path.class));
     verify(mockFs, never()).delete(any(Path.class), anyBoolean());
   }
+
+  @Test
+  public void testCacheIncludesURIUserInfo() throws Throwable {
+    URI containerA = new URI("wasb://a@account.blob.core.windows.net");
+    URI containerB = new URI("wasb://b@account.blob.core.windows.net");
+    Configuration conf = new Configuration(false);
+    FileSystem.Cache.Key keyA = new FileSystem.Cache.Key(containerA, conf);
+    FileSystem.Cache.Key keyB = new FileSystem.Cache.Key(containerB, conf);
+    assertNotEquals(keyA, keyB);
+    assertNotEquals(keyA, new FileSystem.Cache.Key(
+        new URI("wasb://account.blob.core.windows.net"), conf));
+    assertEquals(keyA, new FileSystem.Cache.Key(
+        new URI("wasb://A@account.blob.core.windows.net"), conf));
+    assertNotEquals(keyA, new FileSystem.Cache.Key(
+        new URI("wasb://a:password@account.blob.core.windows.net"), conf));
+  }
 }

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpengo.java

@@ -383,8 +383,9 @@ public class TestHttpServerWithSpengo {
           Assert.fail("should fail with no credentials");
         } catch (AuthenticationException ae) {
           Assert.assertNotNull(ae.getCause());
-          Assert.assertEquals(GSSException.class, ae.getCause().getClass());
-          GSSException gsse = (GSSException)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);

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

@@ -356,7 +356,10 @@ public class TestLogLevel extends KerberosSecurityTestcase {
       fail("A HTTPS Client should not have succeeded in connecting to a " +
           "HTTP server");
     } catch (SSLException e) {
-      GenericTestUtils.assertExceptionContains("Unrecognized SSL message", e);
+      GenericTestUtils.assertExceptionContains("Error while authenticating "
+          + "with endpoint", e);
+      GenericTestUtils.assertExceptionContains("Unrecognized SSL message", e
+          .getCause());
     }
   }
 
@@ -374,7 +377,10 @@ public class TestLogLevel extends KerberosSecurityTestcase {
       fail("A HTTPS Client should not have succeeded in connecting to a " +
           "HTTP server");
     } catch (SSLException e) {
-      GenericTestUtils.assertExceptionContains("Unrecognized SSL message", e);
+      GenericTestUtils.assertExceptionContains("Error while authenticating "
+          + "with endpoint", e);
+      GenericTestUtils.assertExceptionContains("Unrecognized SSL message", e
+          .getCause());
     }
   }
 
@@ -393,8 +399,10 @@ public class TestLogLevel extends KerberosSecurityTestcase {
       fail("A HTTP Client should not have succeeded in connecting to a " +
           "HTTPS server");
     } catch (SocketException e) {
+      GenericTestUtils.assertExceptionContains("Error while authenticating "
+          + "with endpoint", e);
       GenericTestUtils.assertExceptionContains(
-          "Unexpected end of file from server", e);
+          "Unexpected end of file from server", e.getCause());
     }
   }
 
@@ -413,8 +421,10 @@ public class TestLogLevel extends KerberosSecurityTestcase {
       fail("A HTTP Client should not have succeeded in connecting to a " +
           "HTTPS server");
     }  catch (SocketException e) {
+      GenericTestUtils.assertExceptionContains("Error while authenticating "
+          + "with endpoint", e);
       GenericTestUtils.assertExceptionContains(
-          "Unexpected end of file from server", e);
+          "Unexpected end of file from server", e.getCause());
     }
   }
 }

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

+ 402 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGILoginFromKeytab.java

@@ -21,14 +21,41 @@ package org.apache.hadoop.security;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
+import java.io.IOException;
+import java.security.Principal;
+import java.security.PrivilegedExceptionAction;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.login.LoginContext;
 
 /**
  * Verify UGI login from keytab. Check that the UGI is
@@ -39,6 +66,7 @@ public class TestUGILoginFromKeytab {
 
   private MiniKdc kdc;
   private File workDir;
+  private ExecutorService executor;
 
   @Rule
   public final TemporaryFolder folder = new TemporaryFolder();
@@ -51,9 +79,12 @@ public class TestUGILoginFromKeytab {
     conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
         "kerberos");
     UserGroupInformation.setConfiguration(conf);
+    UserGroupInformation.setShouldRenewImmediatelyForTests(true);
     workDir = folder.getRoot();
     kdc = new MiniKdc(MiniKdc.createConf(), workDir);
     kdc.start();
+    executor = Executors.newCachedThreadPool();
+
   }
 
   @After
@@ -61,6 +92,9 @@ public class TestUGILoginFromKeytab {
     if (kdc != null) {
       kdc.stop();
     }
+    if (executor != null) {
+      executor.shutdownNow();
+    }
   }
 
   /**
@@ -69,7 +103,6 @@ public class TestUGILoginFromKeytab {
    */
   @Test
   public void testUGILoginFromKeytab() throws Exception {
-    UserGroupInformation.setShouldRenewImmediatelyForTests(true);
     String principal = "foo";
     File keytab = new File(workDir, "foo.keytab");
     kdc.createPrincipal(keytab, principal);
@@ -80,12 +113,379 @@ public class TestUGILoginFromKeytab {
         ugi.isFromKeytab());
 
     // Verify relogin from keytab.
-    User user = ugi.getSubject().getPrincipals(User.class).iterator().next();
+    User user = getUser(ugi.getSubject());
     final long firstLogin = user.getLastLogin();
+    final LoginContext login1 = user.getLogin();
+    Assert.assertNotNull(login1);
+
     ugi.reloginFromKeytab();
     final long secondLogin = user.getLastLogin();
+    final LoginContext login2 = user.getLogin();
     Assert.assertTrue("User should have been able to relogin from keytab",
         secondLogin > firstLogin);
+    Assert.assertNotNull(login2);
+    Assert.assertNotSame(login1, login2);
+  }
+
+  @Test
+  public void testGetUGIFromKnownSubject() throws Exception {
+    KerberosPrincipal principal = new KerberosPrincipal("user");
+    File keytab = new File(workDir, "user.keytab");
+    kdc.createPrincipal(keytab, principal.getName());
+
+    UserGroupInformation ugi1 =
+      UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        principal.getName(), keytab.getPath());
+    Subject subject = ugi1.getSubject();
+    User user = getUser(subject);
+    Assert.assertNotNull(user);
+    LoginContext login = user.getLogin();
+    Assert.assertNotNull(login);
+
+    // User instance and/or login context should not change.
+    UserGroupInformation ugi2 = UserGroupInformation.getUGIFromSubject(subject);
+    Assert.assertSame(user, getUser(ugi2.getSubject()));
+    Assert.assertSame(login, user.getLogin());
+  }
+
+  @Test
+  public void testGetUGIFromExternalSubject() throws Exception {
+    KerberosPrincipal principal = new KerberosPrincipal("user");
+    File keytab = new File(workDir, "user.keytab");
+    kdc.createPrincipal(keytab, principal.getName());
+
+    UserGroupInformation ugi =
+      UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        principal.getName(), keytab.getPath());
+    Subject subject = ugi.getSubject();
+    removeUser(subject);
+
+    // first call to get the ugi should add the User instance w/o a login
+    // context.
+    UserGroupInformation ugi1 = UserGroupInformation.getUGIFromSubject(subject);
+    Assert.assertSame(subject, ugi1.getSubject());
+    User user = getUser(subject);
+    Assert.assertNotNull(user);
+    Assert.assertEquals(principal.getName(), user.getName());
+    Assert.assertNull(user.getLogin());
+
+    // subsequent call should not change the existing User instance.
+    UserGroupInformation ugi2 = UserGroupInformation.getUGIFromSubject(subject);
+    Assert.assertSame(subject, ugi2.getSubject());
+    Assert.assertSame(user, getUser(ugi2.getSubject()));
+    Assert.assertNull(user.getLogin());
+  }
+
+  @Test
+  public void testGetUGIFromExternalSubjectWithLogin() throws Exception {
+    KerberosPrincipal principal = new KerberosPrincipal("user");
+    File keytab = new File(workDir, "user.keytab");
+    kdc.createPrincipal(keytab, principal.getName());
+
+    // alter the User's login context to appear to be a foreign and
+    // unmanagable context.
+    UserGroupInformation ugi =
+      UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        principal.getName(), keytab.getPath());
+    Subject subject = ugi.getSubject();
+    User user = getUser(subject);
+    final LoginContext dummyLogin = Mockito.mock(LoginContext.class);
+    user.setLogin(dummyLogin);
+
+    // nothing should change.
+    UserGroupInformation ugi2 = UserGroupInformation.getUGIFromSubject(subject);
+    Assert.assertSame(subject, ugi2.getSubject());
+    Assert.assertSame(user, getUser(ugi2.getSubject()));
+    Assert.assertSame(dummyLogin, user.getLogin());
+  }
+
+
+  private static KerberosTicket getTicket(UserGroupInformation ugi) {
+    Set<KerberosTicket> tickets =
+        ugi.getSubject().getPrivateCredentials(KerberosTicket.class);
+    return tickets.isEmpty() ? null : tickets.iterator().next();
   }
 
+  // verify ugi has expected principal, a keytab, and has a ticket for
+  // the expected principal.
+  private static KerberosTicket checkTicketAndKeytab(UserGroupInformation ugi,
+      KerberosPrincipal principal, boolean expectIsKeytab) {
+    Assert.assertEquals("wrong principal",
+      principal.getName(), ugi.getUserName());
+    Assert.assertEquals("is not keytab",
+      expectIsKeytab, ugi.isFromKeytab());
+    KerberosTicket ticket = getTicket(ugi);
+    Assert.assertNotNull("no ticket", ticket);
+    Assert.assertEquals("wrong principal", principal, ticket.getClient());
+    return ticket;
+  }
+
+  @Test
+  public void testReloginForUGIFromSubject() throws Exception {
+    KerberosPrincipal principal1 = new KerberosPrincipal("user1");
+    File keytab1 = new File(workDir, "user1.keytab");
+    kdc.createPrincipal(keytab1, principal1.getName());
+
+    KerberosPrincipal principal2 = new KerberosPrincipal("user2");
+    File keytab2 = new File(workDir, "user2.keytab");
+    kdc.createPrincipal(keytab2, principal2.getName());
+
+    // Login a user and remove the User instance so it looks like an
+    // "external" subject.
+    final Subject extSubject =
+      UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        principal2.getName(), keytab2.getPath()).getSubject();
+    removeUser(extSubject);
+
+    // Login another user.
+    UserGroupInformation.loginUserFromKeytab(
+        principal1.getName(), keytab1.getPath());
+    final UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
+
+    loginUser.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws IOException {
+        KerberosTicket loginTicket =
+            checkTicketAndKeytab(loginUser, principal1, true);
+
+        // get the ugi for the previously logged in subject.
+        UserGroupInformation extSubjectUser =
+            UserGroupInformation.getUGIFromSubject(extSubject);
+        KerberosTicket ticket =
+          checkTicketAndKeytab(extSubjectUser, principal2, false);
+
+        // verify login user got a new ticket.
+        loginUser.reloginFromKeytab();
+        KerberosTicket newLoginTicket =
+            checkTicketAndKeytab(loginUser, principal1, true);
+        Assert.assertNotEquals(loginTicket.getAuthTime(),
+            newLoginTicket.getAuthTime());
+
+        // verify an "external" subject ticket does not change.
+        extSubjectUser.reloginFromKeytab();
+        Assert.assertSame(ticket,
+            checkTicketAndKeytab(extSubjectUser, principal2, false));
+
+        // verify subject ugi relogin did not affect the login user.
+        Assert.assertSame(newLoginTicket,
+            checkTicketAndKeytab(loginUser, principal1, true));
+
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void testReloginForLoginFromSubject() throws Exception {
+    KerberosPrincipal principal1 = new KerberosPrincipal("user1");
+    File keytab1 = new File(workDir, "user1.keytab");
+    kdc.createPrincipal(keytab1, principal1.getName());
+
+    KerberosPrincipal principal2 = new KerberosPrincipal("user2");
+    File keytab2 = new File(workDir, "user2.keytab");
+    kdc.createPrincipal(keytab2, principal2.getName());
+
+    // login principal1 with a keytab.
+    UserGroupInformation.loginUserFromKeytab(
+        principal1.getName(), keytab1.getPath());
+    final UserGroupInformation originalLoginUser =
+        UserGroupInformation.getLoginUser();
+    Assert.assertNotNull(getUser(originalLoginUser.getSubject()).getLogin());
+
+    originalLoginUser.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws IOException {
+        KerberosTicket originalLoginUserTicket =
+            checkTicketAndKeytab(originalLoginUser, principal1, true);
+
+        // login principal2 from a subject with keytab.  it's external so
+        // no login context should be attached to the user.
+        final Subject subject =
+          UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+            principal2.getName(), keytab2.getPath()).getSubject();
+        removeUser(subject);
+
+        // verify the new login user is external.
+        UserGroupInformation.loginUserFromSubject(subject);
+        Assert.assertNull(getUser(subject).getLogin());
+        UserGroupInformation extLoginUser =
+          UserGroupInformation.getLoginUser();
+        KerberosTicket extLoginUserTicket =
+            checkTicketAndKeytab(extLoginUser, principal2, false);
+
+        // verify subject-based login user does not get a new ticket, and
+        // original login user not affected.
+        extLoginUser.reloginFromKeytab();
+        Assert.assertSame(extLoginUserTicket,
+          checkTicketAndKeytab(extLoginUser, principal2, false));
+        Assert.assertSame(originalLoginUserTicket,
+          checkTicketAndKeytab(originalLoginUser, principal1, true));
+
+        // verify original login user gets a new ticket, new login user
+        // not affected.
+        originalLoginUser.reloginFromKeytab();
+        Assert.assertNotSame(originalLoginUserTicket,
+            checkTicketAndKeytab(originalLoginUser, principal1, true));
+        Assert.assertSame(extLoginUserTicket,
+            checkTicketAndKeytab(extLoginUser, principal2, false));
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void testReloginAfterFailedRelogin() throws Exception {
+    KerberosPrincipal principal = new KerberosPrincipal("user1");
+    File keytab = new File(workDir, "user1.keytab");
+    File keytabBackup = new File(keytab + ".backup");
+    kdc.createPrincipal(keytab, principal.getName());
+
+    UserGroupInformation.loginUserFromKeytab(
+        principal.getName(), keytab.getPath());
+    final UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
+    checkTicketAndKeytab(loginUser, principal, true);
+
+    // move the keytab to induce a relogin failure.
+    Assert.assertTrue(keytab.renameTo(keytabBackup));
+    try {
+      loginUser.reloginFromKeytab();
+      Assert.fail("relogin should fail");
+    } catch (KerberosAuthException kae) {
+      // expected.
+    }
+
+    // even though no KeyTab object, ugi should know it's keytab based.
+    Assert.assertTrue(loginUser.isFromKeytab());
+    Assert.assertNull(getTicket(loginUser));
+
+    // move keytab back to enable relogin to succeed.
+    Assert.assertTrue(keytabBackup.renameTo(keytab));
+    loginUser.reloginFromKeytab();
+    checkTicketAndKeytab(loginUser, principal, true);
+  }
+
+  // verify getting concurrent relogins blocks to avoid indeterminate
+  // credentials corruption, but getting a ugi for the subject does not block.
+  @Test(timeout=180000)
+  public void testConcurrentRelogin() throws Exception {
+    final CyclicBarrier barrier = new CyclicBarrier(2);
+    final CountDownLatch latch = new CountDownLatch(1);
+    assertTrue(UserGroupInformation.isSecurityEnabled());
+
+    KerberosPrincipal principal = new KerberosPrincipal("testUser");
+    File keytab = new File(workDir, "user1.keytab");
+    kdc.createPrincipal(keytab, principal.getName());
+
+    // create a keytab ugi.
+    final UserGroupInformation loginUgi =
+        UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+          principal.getName(), keytab.getPath());
+    assertEquals(AuthenticationMethod.KERBEROS,
+        loginUgi.getAuthenticationMethod());
+    assertTrue(loginUgi.isFromKeytab());
+
+    // create a new ugi instance based on subject from the logged in user.
+    final UserGroupInformation clonedUgi =
+        UserGroupInformation.getUGIFromSubject(loginUgi.getSubject());
+    assertEquals(AuthenticationMethod.KERBEROS,
+        clonedUgi.getAuthenticationMethod());
+    assertTrue(clonedUgi.isFromKeytab());
+
+    // cause first relogin to block on a barrier in logout to verify relogins
+    // are atomic.
+    User user = getUser(loginUgi.getSubject());
+    final LoginContext spyLogin = Mockito.spy(user.getLogin());
+    user.setLogin(spyLogin);
+    Mockito.doAnswer(new Answer<Void>(){
+      @Override
+      public Void answer(InvocationOnMock invocation)
+          throws Throwable {
+        invocation.callRealMethod();
+        latch.countDown();
+        barrier.await();
+        return null;
+      }
+    }).when(spyLogin).logout();
+
+    Future<Void> relogin = executor.submit(
+        new Callable<Void>(){
+          @Override
+          public Void call() throws Exception {
+            Thread.currentThread().setName("relogin");
+            loginUgi.reloginFromKeytab();
+            return null;
+          }
+        });
+    // wait for the thread to block on the barrier in the logout of the
+    // relogin.
+    assertTrue("first relogin didn't block",
+      latch.await(2, TimeUnit.SECONDS));
+
+    // although the logout removed the keytab instance, verify the ugi
+    // knows from its login params that it is supposed to be from a keytab.
+    assertTrue(clonedUgi.isFromKeytab());
+
+    // another concurrent re-login should block.
+    Mockito.doNothing().when(spyLogin).logout();
+    Mockito.doNothing().when(spyLogin).login();
+    Future<UserGroupInformation> clonedRelogin = executor.submit(
+        new Callable<UserGroupInformation>(){
+          @Override
+          public UserGroupInformation call() throws Exception {
+            Thread.currentThread().setName("clonedRelogin");
+            clonedUgi.reloginFromKeytab();
+            return clonedUgi;
+          }
+        });
+
+    try {
+      clonedRelogin.get(2, TimeUnit.SECONDS);
+      fail("second relogin didn't block!");
+    } catch (TimeoutException te) {
+      // expected
+    }
+
+    // concurrent UGI instantiation should not block and again should
+    // know it's supposed to be from a keytab.
+    loginUgi.doAs(new PrivilegedExceptionAction<Void>(){
+      @Override
+      public Void run() throws Exception {
+        UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+        assertEquals(principal.getName(), ugi.getUserName());
+        assertTrue(ugi.isFromKeytab());
+        return null;
+      }
+    });
+    clonedUgi.doAs(new PrivilegedExceptionAction<Void>(){
+      @Override
+      public Void run() throws Exception {
+        UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+        assertEquals(principal.getName(), ugi.getUserName());
+        assertTrue(ugi.isFromKeytab());
+        return null;
+      }
+    });
+
+    // second relogin should still be blocked until the original relogin
+    // is blocked.
+    assertFalse(clonedRelogin.isDone());
+    barrier.await();
+    relogin.get();
+    clonedRelogin.get();
+  }
+
+  private User getUser(Subject subject) {
+    Iterator<User> iter = subject.getPrincipals(User.class).iterator();
+    return iter.hasNext() ? iter.next() : null;
+  }
+
+  private void removeUser(Subject subject) {
+    // remove User instance so it appears to not be logged in.
+    for (Iterator<Principal> iter = subject.getPrincipals().iterator();
+         iter.hasNext(); ) {
+      if (iter.next() instanceof User) {
+        iter.remove();
+      }
+    }
+  }
 }

+ 12 - 42
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java

@@ -58,9 +58,11 @@ public class TestUGIWithMiniKdc {
 
   private void setupKdc() throws Exception {
     Properties kdcConf = MiniKdc.createConf();
-    // tgt expire time = 30 seconds
-    kdcConf.setProperty(MiniKdc.MAX_TICKET_LIFETIME, "30");
-    kdcConf.setProperty(MiniKdc.MIN_TICKET_LIFETIME, "30");
+    // tgt expire time = 2 seconds.  just testing that renewal thread retries
+    // for expiring tickets, so no need to waste time waiting for expiry to
+    // arrive.
+    kdcConf.setProperty(MiniKdc.MAX_TICKET_LIFETIME, "2");
+    kdcConf.setProperty(MiniKdc.MIN_TICKET_LIFETIME, "2");
     File kdcDir = new File(System.getProperty("test.dir", "target"));
     kdc = new MiniKdc(kdcConf, kdcDir);
     kdc.start();
@@ -70,12 +72,14 @@ public class TestUGIWithMiniKdc {
   public void testAutoRenewalThreadRetryWithKdc() throws Exception {
     GenericTestUtils.setLogLevel(UserGroupInformation.LOG, Level.DEBUG);
     final Configuration conf = new Configuration();
+    // can't rely on standard kinit, else test fails when user running
+    // the test is kinit'ed because the test renews _their TGT_.
+    conf.set("hadoop.kerberos.kinit.command", "bogus-kinit-cmd");
     // Relogin every 1 second
     conf.setLong(HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN, 1);
     SecurityUtil.setAuthenticationMethod(
         UserGroupInformation.AuthenticationMethod.KERBEROS, conf);
     UserGroupInformation.setConfiguration(conf);
-    UserGroupInformation.setEnableRenewThreadCreationForTest(true);
 
     LoginContext loginContext = null;
     try {
@@ -87,44 +91,10 @@ public class TestUGIWithMiniKdc {
       setupKdc();
       kdc.createPrincipal(keytab, principal);
 
-      // client login
-      final Subject subject =
-          new Subject(false, principals, new HashSet<>(), new HashSet<>());
-
-      loginContext = new LoginContext("", subject, null,
-          new javax.security.auth.login.Configuration() {
-            @Override
-            public AppConfigurationEntry[] getAppConfigurationEntry(
-                String name) {
-              Map<String, String> options = new HashMap<>();
-              options.put("principal", principal);
-              options.put("refreshKrb5Config", "true");
-              if (PlatformName.IBM_JAVA) {
-                options.put("useKeytab", keytab.getPath());
-                options.put("credsType", "both");
-              } else {
-                options.put("keyTab", keytab.getPath());
-                options.put("useKeyTab", "true");
-                options.put("storeKey", "true");
-                options.put("doNotPrompt", "true");
-                options.put("useTicketCache", "true");
-                options.put("renewTGT", "true");
-                options.put("isInitiator", Boolean.toString(true));
-              }
-              String ticketCache = System.getenv("KRB5CCNAME");
-              if (ticketCache != null) {
-                options.put("ticketCache", ticketCache);
-              }
-              options.put("debug", "true");
-              return new AppConfigurationEntry[] {new AppConfigurationEntry(
-                  KerberosUtil.getKrb5LoginModuleName(),
-                  AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
-                  options)};
-            }
-          });
-      loginContext.login();
-      final Subject loginSubject = loginContext.getSubject();
-      UserGroupInformation.loginUserFromSubject(loginSubject);
+      UserGroupInformation.loginUserFromKeytab(principal, keytab.getPath());
+      UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+      // no ticket cache, so force the thread to test for failures.
+      ugi.spawnAutoRenewalThreadForUserCreds(true);
 
       // Verify retry happens. Do not verify retry count to reduce flakiness.
       // Detailed back-off logic is tested separately in

+ 95 - 18
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java

@@ -38,6 +38,9 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.event.Level;
@@ -53,15 +56,19 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.lang.reflect.Method;
+import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.ConcurrentModificationException;
 import java.util.Date;
 import java.util.LinkedHashSet;
 import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_TREAT_SUBJECT_EXTERNAL_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL;
@@ -112,7 +119,14 @@ public class TestUserGroupInformation {
       throw new RuntimeException("UGI is not using its own security conf!");
     } 
   }
-  
+
+  // must be set immediately to avoid inconsistent testing issues.
+  static {
+    // fake the realm is kerberos is enabled
+    System.setProperty("java.security.krb5.kdc", "");
+    System.setProperty("java.security.krb5.realm", "DEFAULT.REALM");
+  }
+
   /** configure ugi */
   @BeforeClass
   public static void setup() {
@@ -123,9 +137,6 @@ public class TestUserGroupInformation {
     // that finds winutils.exe
     String home = System.getenv("HADOOP_HOME");
     System.setProperty("hadoop.home.dir", (home != null ? home : "."));
-    // fake the realm is kerberos is enabled
-    System.setProperty("java.security.krb5.kdc", "");
-    System.setProperty("java.security.krb5.realm", "DEFAULT.REALM");
   }
   
   @Before
@@ -1021,7 +1032,8 @@ public class TestUserGroupInformation {
     assertTrue(credsugiTokens.contains(token2));
   }
 
-  private void testCheckTGTAfterLoginFromSubjectHelper() throws Exception {
+  @Test
+  public void testCheckTGTAfterLoginFromSubject() throws Exception {
     // security on, default is remove default realm
     SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
     UserGroupInformation.setConfiguration(conf);
@@ -1043,17 +1055,6 @@ public class TestUserGroupInformation {
     });
   }
 
-  @Test(expected = KerberosAuthException.class)
-  public void testCheckTGTAfterLoginFromSubject() throws Exception {
-    testCheckTGTAfterLoginFromSubjectHelper();
-  }
-
-  @Test
-  public void testCheckTGTAfterLoginFromSubjectFix() throws Exception {
-    conf.setBoolean(HADOOP_TREAT_SUBJECT_EXTERNAL_KEY, true);
-    testCheckTGTAfterLoginFromSubjectHelper();
-  }
-
   @Test
   public void testGetNextRetryTime() throws Exception {
     GenericTestUtils.setLogLevel(UserGroupInformation.LOG, Level.DEBUG);
@@ -1134,4 +1135,80 @@ public class TestUserGroupInformation {
     LOG.info(str);
     assertTrue(str, lower <= lastRetry && lastRetry < upper);
   }
+
+  // verify that getCurrentUser on the same and different subjects can be
+  // concurrent.  Ie. no synchronization.
+  @Test(timeout=8000)
+  public void testConcurrentGetCurrentUser() throws Exception {
+    final CyclicBarrier barrier = new CyclicBarrier(2);
+    final CountDownLatch latch = new CountDownLatch(1);
+
+    final UserGroupInformation testUgi1 =
+        UserGroupInformation.createRemoteUser("testUgi1");
+
+    final UserGroupInformation testUgi2 =
+        UserGroupInformation.createRemoteUser("testUgi2");
+
+    // swap the User with a spy to allow getCurrentUser to block when the
+    // spy is called for the user name.
+    Set<Principal> principals = testUgi1.getSubject().getPrincipals();
+    User user =
+        testUgi1.getSubject().getPrincipals(User.class).iterator().next();
+    final User spyUser = Mockito.spy(user);
+    principals.remove(user);
+    principals.add(spyUser);
+    when(spyUser.getName()).thenAnswer(new Answer<String>(){
+      @Override
+      public String answer(InvocationOnMock invocation) throws Throwable {
+        latch.countDown();
+        barrier.await();
+        return (String)invocation.callRealMethod();
+      }
+    });
+    // wait for the thread to block on the barrier in getCurrentUser.
+    Future<UserGroupInformation> blockingLookup =
+        Executors.newSingleThreadExecutor().submit(
+            new Callable<UserGroupInformation>(){
+              @Override
+              public UserGroupInformation call() throws Exception {
+                return testUgi1.doAs(
+                    new PrivilegedExceptionAction<UserGroupInformation>() {
+                      @Override
+                      public UserGroupInformation run() throws Exception {
+                        return UserGroupInformation.getCurrentUser();
+                      }
+                    });
+              }
+            });
+    latch.await();
+
+    // old versions of mockito synchronize on returning mocked answers so
+    // the blocked getCurrentUser will block all other calls to getName.
+    // workaround this by swapping out the spy with the original User.
+    principals.remove(spyUser);
+    principals.add(user);
+    // concurrent getCurrentUser on ugi1 should not be blocked.
+    UserGroupInformation ugi;
+    ugi = testUgi1.doAs(
+        new PrivilegedExceptionAction<UserGroupInformation>() {
+          @Override
+          public UserGroupInformation run() throws Exception {
+            return UserGroupInformation.getCurrentUser();
+          }
+        });
+    assertSame(testUgi1.getSubject(), ugi.getSubject());
+    // concurrent getCurrentUser on ugi2 should not be blocked.
+    ugi = testUgi2.doAs(
+        new PrivilegedExceptionAction<UserGroupInformation>() {
+          @Override
+          public UserGroupInformation run() throws Exception {
+            return UserGroupInformation.getCurrentUser();
+          }
+        });
+    assertSame(testUgi2.getSubject(), ugi.getSubject());
+
+    // unblock the original call.
+    barrier.await();
+    assertSame(testUgi1.getSubject(), blockingLookup.get().getSubject());
+  }
 }

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java

@@ -364,7 +364,7 @@ public class TestWebDelegationToken {
         aUrl.getDelegationToken(nonAuthURL, token, FOO_USER);
         Assert.fail();
       } catch (Exception ex) {
-        Assert.assertTrue(ex.getMessage().contains("401"));
+        Assert.assertTrue(ex.getCause().getMessage().contains("401"));
       }
 
       aUrl.getDelegationToken(authURL, token, FOO_USER);
@@ -776,7 +776,7 @@ public class TestWebDelegationToken {
         aUrl.getDelegationToken(url, token, FOO_USER, doAsUser);
         Assert.fail();
       } catch (AuthenticationException ex) {
-        Assert.assertTrue(ex.getMessage().contains("GSSException"));
+        Assert.assertTrue(ex.getCause().getMessage().contains("GSSException"));
       }
 
       doAsKerberosUser("client", keytabFile.getAbsolutePath(),

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

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

@@ -87,6 +87,7 @@ 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"),
     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 {

+ 87 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -1994,6 +1994,93 @@ public class DistributedFileSystem extends FileSystem
     }.resolve(this, absF);
   }
 
+  /**
+   * Returns a remote iterator so that followup calls are made on demand
+   * while consuming the SnapshotDiffReportListing entries.
+   * This reduces memory consumption overhead in case the snapshotDiffReport
+   * is huge.
+   *
+   * @param snapshotDir
+   *          full path of the directory where snapshots are taken
+   * @param fromSnapshot
+   *          snapshot name of the from point. Null indicates the current
+   *          tree
+   * @param toSnapshot
+   *          snapshot name of the to point. Null indicates the current
+   *          tree.
+   * @return Remote iterator
+   */
+  public RemoteIterator
+      <SnapshotDiffReportListing> snapshotDiffReportListingRemoteIterator(
+      final Path snapshotDir, final String fromSnapshot,
+      final String toSnapshot) throws IOException {
+    Path absF = fixRelativePart(snapshotDir);
+    return new FileSystemLinkResolver
+        <RemoteIterator<SnapshotDiffReportListing>>() {
+      @Override
+      public RemoteIterator<SnapshotDiffReportListing> doCall(final Path p)
+          throws IOException {
+        return new SnapshotDiffReportListingIterator(
+            getPathName(p), fromSnapshot, toSnapshot);
+      }
+
+      @Override
+      public RemoteIterator<SnapshotDiffReportListing> next(final FileSystem fs,
+          final Path p) throws IOException {
+        return ((DistributedFileSystem) fs)
+            .snapshotDiffReportListingRemoteIterator(p, fromSnapshot,
+                toSnapshot);
+      }
+    }.resolve(this, absF);
+
+  }
+
+  /**
+   * This class defines an iterator that returns
+   * the SnapshotDiffReportListing for a snapshottable directory
+   * between two given snapshots.
+   */
+  private final class SnapshotDiffReportListingIterator implements
+      RemoteIterator<SnapshotDiffReportListing> {
+    private final String snapshotDir;
+    private final String fromSnapshot;
+    private final String toSnapshot;
+
+    private byte[] startPath;
+    private int index;
+    private boolean hasNext = true;
+
+    private SnapshotDiffReportListingIterator(String snapshotDir,
+        String fromSnapshot, String toSnapshot) {
+      this.snapshotDir = snapshotDir;
+      this.fromSnapshot = fromSnapshot;
+      this.toSnapshot = toSnapshot;
+      this.startPath = DFSUtilClient.EMPTY_BYTES;
+      this.index = -1;
+    }
+
+    @Override
+    public boolean hasNext() {
+      return hasNext;
+    }
+
+    @Override
+    public SnapshotDiffReportListing next() throws IOException {
+      if (!hasNext) {
+        throw new java.util.NoSuchElementException(
+            "No more entry in SnapshotDiffReport for " + snapshotDir);
+      }
+      final SnapshotDiffReportListing part =
+          dfs.getSnapshotDiffReportListing(snapshotDir, fromSnapshot,
+              toSnapshot, startPath, index);
+      startPath = part.getLastPath();
+      index = part.getLastIndex();
+      hasNext =
+          !(Arrays.equals(startPath, DFSUtilClient.EMPTY_BYTES) && index == -1);
+      return part;
+    }
+  }
+
   private SnapshotDiffReport getSnapshotDiffReportInternal(
       final String snapshotDir, final String fromSnapshot,
       final String toSnapshot) throws IOException {

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

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

+ 7 - 1
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;
@@ -45,7 +46,7 @@ import org.slf4j.LoggerFactory;
  * per-se. It constructs a wrapper proxy that sends the request to ALL
  * underlying proxies simultaneously. It assumes the in an HA setup, there will
  * be only one Active, and the active should respond faster than any configured
- * standbys. Once it receive a response from any one of the configred proxies,
+ * standbys. Once it receive a response from any one of the configured proxies,
  * outstanding requests to other proxies are immediately cancelled.
  */
 public class RequestHedgingProxyProvider<T> extends
@@ -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 {

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

@@ -37,6 +37,7 @@ 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.DatanodeInfo.DatanodeInfoBuilder;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -49,6 +50,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 +695,51 @@ 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);
+  }
 }

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

@@ -96,6 +96,7 @@ 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.proto.HdfsProtos.FileEncryptionInfoProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -1317,6 +1318,20 @@ 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();
+  }
+
   @Override
   public boolean setReplication(final Path p, final short replication
   ) throws IOException {

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

@@ -47,7 +47,8 @@ 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);
 
     final boolean redirect;
     final int expectedHttpResponseCode;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestHdfsFileStatusMethods.java

@@ -54,7 +54,7 @@ public class TestHdfsFileStatusMethods {
     assertEquals(fsM.stream()
             .map(MethodSignature::toString)
             .collect(joining("\n")),
-        Collections.EMPTY_SET, fsM);
+        Collections.emptySet(), fsM);
   }
 
   /** Map non-static, declared methods for this class to signatures. */

+ 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 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/DummyGroupMapping.java

@@ -28,7 +28,6 @@ import org.apache.hadoop.test.HadoopUsersConfTestHelper;
 public class DummyGroupMapping implements GroupMappingServiceProvider {
 
   @Override
-  @SuppressWarnings("unchecked")
   public List<String> getGroups(String user) throws IOException {
     if (user.equals("root")) {
       return Arrays.asList("admin");
@@ -37,7 +36,7 @@ public class DummyGroupMapping implements GroupMappingServiceProvider {
       return Arrays.asList("nobody");
     } else {
       String[] groups = HadoopUsersConfTestHelper.getHadoopUserGroups(user);
-      return (groups != null) ? Arrays.asList(groups) : Collections.EMPTY_LIST;
+      return (groups != null) ? Arrays.asList(groups) : Collections.emptyList();
     }
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh

@@ -28,7 +28,7 @@
 #    if hdfs_subcmd_user is not defined, error
 #
 # For secure daemons, this means both the secure and insecure env vars need to be
-# defined.  e.g., HDFS_DATANODE_USER=root HADOOP_SECURE_DN_USER=hdfs
+# defined.  e.g., HDFS_DATANODE_USER=root HDFS_DATANODE_SECURE_USER=hdfs
 #
 
 ## @description  usage info

+ 9 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -1146,7 +1146,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   // Disk Balancer Keys
   public static final String DFS_DISK_BALANCER_ENABLED =
       "dfs.disk.balancer.enabled";
-  public static final boolean DFS_DISK_BALANCER_ENABLED_DEFAULT = false;
+  public static final boolean DFS_DISK_BALANCER_ENABLED_DEFAULT = true;
 
   public static final String DFS_DISK_BALANCER_MAX_DISK_THROUGHPUT =
       "dfs.disk.balancer.max.disk.throughputInMBperSec";
@@ -1246,6 +1246,14 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long DFS_ROUTER_NAMENODE_CONNECTION_CLEAN_MS_DEFAULT =
       TimeUnit.SECONDS.toMillis(10);
 
+  // HDFS Router RPC client
+  public static final String DFS_ROUTER_CLIENT_THREADS_SIZE =
+      FEDERATION_ROUTER_PREFIX + "client.thread-size";
+  public static final int DFS_ROUTER_CLIENT_THREADS_SIZE_DEFAULT = 32;
+  public static final String DFS_ROUTER_CLIENT_MAX_ATTEMPTS =
+      FEDERATION_ROUTER_PREFIX + "client.retry.max.attempts";
+  public static final int DFS_ROUTER_CLIENT_MAX_ATTEMPTS_DEFAULT = 3;
+
   // HDFS Router State Store connection
   public static final String FEDERATION_FILE_RESOLVER_CLIENT_CLASS =
       FEDERATION_ROUTER_PREFIX + "file.resolver.client.class";

+ 14 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java

@@ -21,8 +21,9 @@ import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.util.AbstractList;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
 import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Queue;
@@ -139,7 +140,7 @@ public class DatanodeAdminManager {
         new ThreadFactoryBuilder().setNameFormat("DatanodeAdminMonitor-%d")
             .setDaemon(true).build());
     outOfServiceNodeBlocks = new TreeMap<>();
-    pendingNodes = new LinkedList<>();
+    pendingNodes = new ArrayDeque<>();
   }
 
   /**
@@ -219,7 +220,7 @@ public class DatanodeAdminManager {
         pendingNodes.add(node);
       }
     } else {
-      LOG.trace("startDecommission: Node {} in {}, nothing to do." +
+      LOG.trace("startDecommission: Node {} in {}, nothing to do.",
           node, node.getAdminState());
     }
   }
@@ -242,7 +243,7 @@ public class DatanodeAdminManager {
       pendingNodes.remove(node);
       outOfServiceNodeBlocks.remove(node);
     } else {
-      LOG.trace("stopDecommission: Node {} in {}, nothing to do." +
+      LOG.trace("stopDecommission: Node {} in {}, nothing to do.",
           node, node.getAdminState());
     }
   }
@@ -272,7 +273,7 @@ public class DatanodeAdminManager {
       // IN_MAINTENANCE to support maintenance expiration.
       pendingNodes.add(node);
     } else {
-      LOG.trace("startMaintenance: Node {} in {}, nothing to do." +
+      LOG.trace("startMaintenance: Node {} in {}, nothing to do.",
           node, node.getAdminState());
     }
   }
@@ -321,7 +322,7 @@ public class DatanodeAdminManager {
       pendingNodes.remove(node);
       outOfServiceNodeBlocks.remove(node);
     } else {
-      LOG.trace("stopMaintenance: Node {} in {}, nothing to do." +
+      LOG.trace("stopMaintenance: Node {} in {}, nothing to do.",
           node, node.getAdminState());
     }
   }
@@ -395,7 +396,7 @@ public class DatanodeAdminManager {
     for (DatanodeStorageInfo storage : storages) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       nodeList.append(node);
-      nodeList.append(" ");
+      nodeList.append(' ');
     }
     NameNode.blockStateChangeLog.info(
         "Block: " + block + ", Expected Replicas: "
@@ -517,7 +518,7 @@ public class DatanodeAdminManager {
       final Iterator<Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>>
           it = new CyclicIteration<>(outOfServiceNodeBlocks,
               iterkey).iterator();
-      final LinkedList<DatanodeDescriptor> toRemove = new LinkedList<>();
+      final List<DatanodeDescriptor> toRemove = new ArrayList<>();
 
       while (it.hasNext() && !exceededNumBlocksPerCheck() && namesystem
           .isRunning()) {
@@ -583,12 +584,12 @@ public class DatanodeAdminManager {
                   "A node is in an invalid state!");
             }
             LOG.debug("Node {} is sufficiently replicated and healthy, "
-                + "marked as {}.", dn.getAdminState());
+                + "marked as {}.", dn, dn.getAdminState());
           } else {
             LOG.debug("Node {} {} healthy."
                 + " It needs to replicate {} more blocks."
                 + " {} is still in progress.", dn,
-                isHealthy? "is": "isn't", blocks.size(), dn.getAdminState());
+                isHealthy ? "is": "isn't", blocks.size(), dn.getAdminState());
           }
         } else {
           LOG.debug("Node {} still has {} blocks to replicate "
@@ -744,10 +745,10 @@ public class DatanodeAdminManager {
         lowRedundancyBlocks++;
         if (bc.isUnderConstruction()) {
           INode ucFile = namesystem.getFSDirectory().getInode(bc.getId());
-          if(!(ucFile instanceof  INodeFile) ||
+          if (!(ucFile instanceof  INodeFile) ||
               !ucFile.asFile().isUnderConstruction()) {
-            LOG.warn("File " + ucFile.getLocalName() + " is not under " +
-                "construction. Skipping add to low redundancy open files!");
+            LOG.warn("File {} is not under construction. Skipping add to " +
+                "low redundancy open files!", ucFile.getLocalName());
           } else {
             lowRedundancyBlocksInOpenFiles++;
             lowRedundancyOpenFiles.add(ucFile.getId());

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

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java

@@ -53,7 +53,6 @@ abstract public class ReplicaInfo extends Block
    * Constructor.
    * @param block a block
    * @param vol volume where replica is located
-   * @param dir directory path where block and meta files are located
    */
   ReplicaInfo(Block block, FsVolumeSpi vol) {
     this(vol, block.getBlockId(), block.getNumBytes(),

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java

@@ -124,6 +124,14 @@ public class PlanCommand extends Command {
       throw new IllegalArgumentException("Unable to find the specified node. " +
           cmd.getOptionValue(DiskBalancerCLI.PLAN));
     }
+
+    try (FSDataOutputStream beforeStream = create(String.format(
+        DiskBalancerCLI.BEFORE_TEMPLATE,
+        cmd.getOptionValue(DiskBalancerCLI.PLAN)))) {
+      beforeStream.write(getCluster().toJson()
+          .getBytes(StandardCharsets.UTF_8));
+    }
+
     this.thresholdPercentage = getThresholdPercentage(cmd);
 
     LOG.debug("threshold Percentage is {}", this.thresholdPercentage);
@@ -138,14 +146,6 @@ public class PlanCommand extends Command {
       plan = plans.get(0);
     }
 
-
-    try (FSDataOutputStream beforeStream = create(String.format(
-        DiskBalancerCLI.BEFORE_TEMPLATE,
-        cmd.getOptionValue(DiskBalancerCLI.PLAN)))) {
-      beforeStream.write(getCluster().toJson()
-          .getBytes(StandardCharsets.UTF_8));
-    }
-
     try {
       if (plan != null && plan.getVolumeSetPlans().size() > 0) {
         outputLine = String.format("Writing plan to:");

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java

@@ -144,8 +144,6 @@ class DBNameNodeConnector implements ClusterConnector {
       // Does it make sense ? Balancer does do that. Right now
       // we only deal with volumes and not blockPools
 
-      volume.setUsed(report.getDfsUsed());
-
       volume.setUuid(storage.getStorageID());
 
       // we will skip this volume for disk balancer if

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java

@@ -269,7 +269,9 @@ public class DiskBalancerVolume {
    * @param dfsUsedSpace - dfsUsedSpace for this volume.
    */
   public void setUsed(long dfsUsedSpace) {
-    Preconditions.checkArgument(dfsUsedSpace < this.getCapacity());
+    Preconditions.checkArgument(dfsUsedSpace < this.getCapacity(),
+        "DiskBalancerVolume.setUsed: dfsUsedSpace(%s) < capacity(%s)",
+        dfsUsedSpace, getCapacity());
     this.used = dfsUsedSpace;
   }
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMBean.java

@@ -42,6 +42,8 @@ public interface FederationRPCMBean {
 
   long getProxyOpNotImplemented();
 
+  long getProxyOpRetries();
+
   long getRouterFailureStateStoreOps();
 
   long getRouterFailureReadOnlyOps();

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java

@@ -56,6 +56,8 @@ public class FederationRPCMetrics implements FederationRPCMBean {
   private MutableCounterLong proxyOpFailureCommunicate;
   @Metric("Number of operations not implemented")
   private MutableCounterLong proxyOpNotImplemented;
+  @Metric("Number of operation retries")
+  private MutableCounterLong proxyOpRetries;
 
   @Metric("Failed requests due to State Store unavailable")
   private MutableCounterLong routerFailureStateStore;
@@ -126,6 +128,15 @@ public class FederationRPCMetrics implements FederationRPCMBean {
     return proxyOpNotImplemented.value();
   }
 
+  public void incrProxyOpRetries() {
+    proxyOpRetries.incr();
+  }
+
+  @Override
+  public long getProxyOpRetries() {
+    return proxyOpRetries.value();
+  }
+
   public void incrRouterFailureStateStore() {
     routerFailureStateStore.incr();
   }

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java

@@ -158,6 +158,11 @@ public class FederationRPCPerformanceMonitor implements RouterRpcMonitor {
     metrics.incrProxyOpNotImplemented();
   }
 
+  @Override
+  public void proxyOpRetries() {
+    metrics.incrProxyOpRetries();
+  }
+
   @Override
   public void routerFailureStateStore() {
     metrics.incrRouterFailureStateStore();
@@ -208,4 +213,9 @@ public class FederationRPCPerformanceMonitor implements RouterRpcMonitor {
     }
     return -1;
   }
+
+  @Override
+  public FederationRPCMetrics getRPCMetrics() {
+    return this.metrics;
+  }
 }

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java

@@ -390,6 +390,14 @@ public class NamenodeStatusReport {
     return this.numOfBlocksPendingDeletion;
   }
 
+  /**
+   * Set the validity of registration.
+   * @param isValid The desired value to be set.
+   */
+  public void setRegistrationValid(boolean isValid) {
+    this.registrationValid = isValid;
+  }
+
   @Override
   public String toString() {
     return String.format("%s-%s:%s",

+ 60 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java

@@ -46,12 +46,14 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
 import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
@@ -123,10 +125,14 @@ public class RouterRpcClient {
     this.connectionManager = new ConnectionManager(conf);
     this.connectionManager.start();
 
+    int numThreads = conf.getInt(
+        DFSConfigKeys.DFS_ROUTER_CLIENT_THREADS_SIZE,
+        DFSConfigKeys.DFS_ROUTER_CLIENT_THREADS_SIZE_DEFAULT);
     ThreadFactory threadFactory = new ThreadFactoryBuilder()
         .setNameFormat("RPC Router Client-%d")
         .build();
-    this.executorService = Executors.newCachedThreadPool(threadFactory);
+    this.executorService = Executors.newFixedThreadPool(
+        numThreads, threadFactory);
 
     this.rpcMonitor = monitor;
 
@@ -134,8 +140,8 @@ public class RouterRpcClient {
         HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY,
         HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_DEFAULT);
     int maxRetryAttempts = conf.getInt(
-        HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY,
-        HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_DEFAULT);
+        DFSConfigKeys.DFS_ROUTER_CLIENT_MAX_ATTEMPTS,
+        DFSConfigKeys.DFS_ROUTER_CLIENT_MAX_ATTEMPTS_DEFAULT);
     int failoverSleepBaseMillis = conf.getInt(
         HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_KEY,
         HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT);
@@ -274,11 +280,24 @@ public class RouterRpcClient {
    *
    * @param ioe IOException reported.
    * @param retryCount Number of retries.
+   * @param nsId Nameservice ID.
    * @return Retry decision.
-   * @throws IOException Original exception if the retry policy generates one.
+   * @throws IOException Original exception if the retry policy generates one
+   *                     or IOException for no available namenodes.
    */
-  private RetryDecision shouldRetry(final IOException ioe, final int retryCount)
-      throws IOException {
+  private RetryDecision shouldRetry(final IOException ioe, final int retryCount,
+      final String nsId) throws IOException {
+    // check for the case of cluster unavailable state
+    if (isClusterUnAvailable(nsId)) {
+      // we allow to retry once if cluster is unavailable
+      if (retryCount == 0) {
+        return RetryDecision.RETRY;
+      } else {
+        throw new IOException("No namenode available under nameservice " + nsId,
+            ioe);
+      }
+    }
+
     try {
       final RetryPolicy.RetryAction a =
           this.retryPolicy.shouldRetry(ioe, retryCount, 0, true);
@@ -329,7 +348,7 @@ public class RouterRpcClient {
         connection = this.getConnection(ugi, nsId, rpcAddress);
         ProxyAndInfo<ClientProtocol> client = connection.getClient();
         ClientProtocol proxy = client.getProxy();
-        ret = invoke(0, method, proxy, params);
+        ret = invoke(nsId, 0, method, proxy, params);
         if (failover) {
           // Success on alternate server, update
           InetSocketAddress address = client.getAddress();
@@ -400,6 +419,8 @@ public class RouterRpcClient {
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
+   * @param nsId Identifier for the namespace
+   * @param retryCount Current retry times
    * @param method Method to invoke
    * @param obj Target object for the method
    * @param params Variable parameters
@@ -407,8 +428,8 @@ public class RouterRpcClient {
    * @throws IOException
    * @throws InterruptedException
    */
-  private Object invoke(int retryCount, final Method method, final Object obj,
-      final Object... params) throws IOException {
+  private Object invoke(String nsId, int retryCount, final Method method,
+      final Object obj, final Object... params) throws IOException {
     try {
       return method.invoke(obj, params);
     } catch (IllegalAccessException e) {
@@ -421,11 +442,16 @@ public class RouterRpcClient {
       Throwable cause = e.getCause();
       if (cause instanceof IOException) {
         IOException ioe = (IOException) cause;
+
         // Check if we should retry.
-        RetryDecision decision = shouldRetry(ioe, retryCount);
+        RetryDecision decision = shouldRetry(ioe, retryCount, nsId);
         if (decision == RetryDecision.RETRY) {
+          if (this.rpcMonitor != null) {
+            this.rpcMonitor.proxyOpRetries();
+          }
+
           // retry
-          return invoke(++retryCount, method, obj, params);
+          return invoke(nsId, ++retryCount, method, obj, params);
         } else if (decision == RetryDecision.FAILOVER_AND_RETRY) {
           // failover, invoker looks for standby exceptions for failover.
           if (ioe instanceof StandbyException) {
@@ -447,6 +473,29 @@ public class RouterRpcClient {
     }
   }
 
+  /**
+   * Check if the cluster of given nameservice id is available.
+   * @param nsId nameservice ID.
+   * @return
+   * @throws IOException
+   */
+  private boolean isClusterUnAvailable(String nsId) throws IOException {
+    List<? extends FederationNamenodeContext> nnState = this.namenodeResolver
+        .getNamenodesForNameserviceId(nsId);
+
+    if (nnState != null) {
+      for (FederationNamenodeContext nnContext : nnState) {
+        // Once we find one NN is in active state, we assume this
+        // cluster is available.
+        if (nnContext.getState() == FederationNamenodeServiceState.ACTIVE) {
+          return false;
+        }
+      }
+    }
+
+    return true;
+  }
+
   /**
    * Get a clean copy of the exception. Sometimes the exceptions returned by the
    * server contain the full stack trace in the message.

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

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.federation.router;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.metrics.FederationRPCMetrics;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 
 /**
@@ -35,6 +36,12 @@ public interface RouterRpcMonitor {
   void init(
       Configuration conf, RouterRpcServer server, StateStoreService store);
 
+  /**
+   * Get Router RPC metrics info.
+   * @return The instance of FederationRPCMetrics.
+   */
+  FederationRPCMetrics getRPCMetrics();
+
   /**
    * Close the monitor.
    */
@@ -73,6 +80,12 @@ public interface RouterRpcMonitor {
    */
   void proxyOpNotImplemented();
 
+  /**
+   * Retry to proxy an operation to a Namenode because of an unexpected
+   * exception.
+   */
+  void proxyOpRetries();
+
   /**
    * If the Router cannot contact the State Store in an operation.
    */

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

@@ -105,6 +105,7 @@ import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.federation.metrics.FederationRPCMetrics;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
@@ -2177,4 +2178,12 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
   public Quota getQuotaModule() {
     return this.quotaCall;
   }
+
+  /**
+   * Get RPC metrics info.
+   * @return The instance of FederationRPCMetrics.
+   */
+  public FederationRPCMetrics getRPCMetrics() {
+    return this.rpcMonitor.getRPCMetrics();
+  }
 }

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

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

@@ -154,9 +154,10 @@ public class EncryptionZoneManager {
   public void pauseForTestingAfterNthCheckpoint(final String zone,
       final int count) throws IOException {
     INodesInPath iip;
+    final FSPermissionChecker pc = dir.getPermissionChecker();
     dir.readLock();
     try {
-      iip = dir.resolvePath(dir.getPermissionChecker(), zone, DirOp.READ);
+      iip = dir.resolvePath(pc, zone, DirOp.READ);
     } finally {
       dir.readUnlock();
     }

+ 11 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java

@@ -36,11 +36,10 @@ import java.util.List;
 
 class FSDirAclOp {
   static FileStatus modifyAclEntries(
-      FSDirectory fsd, final String srcArg, List<AclEntry> aclSpec)
-      throws IOException {
+      FSDirectory fsd, FSPermissionChecker pc, final String srcArg,
+      List<AclEntry> aclSpec) throws IOException {
     String src = srcArg;
     checkAclsConfigFlag(fsd);
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     INodesInPath iip;
     fsd.writeLock();
     try {
@@ -61,11 +60,10 @@ class FSDirAclOp {
   }
 
   static FileStatus removeAclEntries(
-      FSDirectory fsd, final String srcArg, List<AclEntry> aclSpec)
-      throws IOException {
+      FSDirectory fsd, FSPermissionChecker pc, final String srcArg,
+      List<AclEntry> aclSpec) throws IOException {
     String src = srcArg;
     checkAclsConfigFlag(fsd);
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     INodesInPath iip;
     fsd.writeLock();
     try {
@@ -85,11 +83,10 @@ class FSDirAclOp {
     return fsd.getAuditFileInfo(iip);
   }
 
-  static FileStatus removeDefaultAcl(FSDirectory fsd, final String srcArg)
-      throws IOException {
+  static FileStatus removeDefaultAcl(FSDirectory fsd, FSPermissionChecker pc,
+      final String srcArg) throws IOException {
     String src = srcArg;
     checkAclsConfigFlag(fsd);
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     INodesInPath iip;
     fsd.writeLock();
     try {
@@ -109,11 +106,10 @@ class FSDirAclOp {
     return fsd.getAuditFileInfo(iip);
   }
 
-  static FileStatus removeAcl(FSDirectory fsd, final String srcArg)
-      throws IOException {
+  static FileStatus removeAcl(FSDirectory fsd, FSPermissionChecker pc,
+      final String srcArg) throws IOException {
     String src = srcArg;
     checkAclsConfigFlag(fsd);
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     INodesInPath iip;
     fsd.writeLock();
     try {
@@ -129,11 +125,10 @@ class FSDirAclOp {
   }
 
   static FileStatus setAcl(
-      FSDirectory fsd, final String srcArg, List<AclEntry> aclSpec)
-      throws IOException {
+      FSDirectory fsd, FSPermissionChecker pc, final String srcArg,
+      List<AclEntry> aclSpec) throws IOException {
     String src = srcArg;
     checkAclsConfigFlag(fsd);
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     INodesInPath iip;
     fsd.writeLock();
     try {
@@ -148,9 +143,8 @@ class FSDirAclOp {
   }
 
   static AclStatus getAclStatus(
-      FSDirectory fsd, String src) throws IOException {
+      FSDirectory fsd, FSPermissionChecker pc, String src) throws IOException {
     checkAclsConfigFlag(fsd);
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     fsd.readLock();
     try {
       INodesInPath iip = fsd.resolvePath(pc, src, DirOp.READ);

+ 22 - 32
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java

@@ -51,12 +51,11 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KE
 
 public class FSDirAttrOp {
   static FileStatus setPermission(
-      FSDirectory fsd, final String src, FsPermission permission)
-      throws IOException {
+      FSDirectory fsd, FSPermissionChecker pc, final String src,
+      FsPermission permission) throws IOException {
     if (FSDirectory.isExactReservedName(src)) {
       throw new InvalidPathException(src);
     }
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     INodesInPath iip;
     fsd.writeLock();
     try {
@@ -71,12 +70,11 @@ public class FSDirAttrOp {
   }
 
   static FileStatus setOwner(
-      FSDirectory fsd, String src, String username, String group)
-      throws IOException {
+      FSDirectory fsd, FSPermissionChecker pc, String src, String username,
+      String group) throws IOException {
     if (FSDirectory.isExactReservedName(src)) {
       throw new InvalidPathException(src);
     }
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     INodesInPath iip;
     fsd.writeLock();
     try {
@@ -101,10 +99,8 @@ public class FSDirAttrOp {
   }
 
   static FileStatus setTimes(
-      FSDirectory fsd, String src, long mtime, long atime)
-      throws IOException {
-    FSPermissionChecker pc = fsd.getPermissionChecker();
-
+      FSDirectory fsd, FSPermissionChecker pc, String src, long mtime,
+      long atime) throws IOException {
     INodesInPath iip;
     fsd.writeLock();
     try {
@@ -129,11 +125,10 @@ public class FSDirAttrOp {
   }
 
   static boolean setReplication(
-      FSDirectory fsd, BlockManager bm, String src, final short replication)
-      throws IOException {
+      FSDirectory fsd, FSPermissionChecker pc, BlockManager bm, String src,
+      final short replication) throws IOException {
     bm.verifyReplication(src, replication, null);
     final boolean isFile;
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     fsd.writeLock();
     try {
       final INodesInPath iip = fsd.resolvePath(pc, src, DirOp.WRITE);
@@ -153,33 +148,31 @@ public class FSDirAttrOp {
     return isFile;
   }
 
-  static FileStatus unsetStoragePolicy(FSDirectory fsd, BlockManager bm,
-      String src) throws IOException {
-    return setStoragePolicy(fsd, bm, src,
+  static FileStatus unsetStoragePolicy(FSDirectory fsd, FSPermissionChecker pc,
+      BlockManager bm, String src) throws IOException {
+    return setStoragePolicy(fsd, pc, bm, src,
         HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, "unset");
   }
 
-  static FileStatus setStoragePolicy(FSDirectory fsd, BlockManager bm,
-      String src, final String policyName) throws IOException {
+  static FileStatus setStoragePolicy(FSDirectory fsd, FSPermissionChecker pc,
+      BlockManager bm, String src, final String policyName) throws IOException {
     // get the corresponding policy and make sure the policy name is valid
     BlockStoragePolicy policy = bm.getStoragePolicy(policyName);
     if (policy == null) {
       throw new HadoopIllegalArgumentException(
           "Cannot find a block policy with the name " + policyName);
     }
-
-    return setStoragePolicy(fsd, bm, src, policy.getId(), "set");
+    return setStoragePolicy(fsd, pc, bm, src, policy.getId(), "set");
   }
 
-  static FileStatus setStoragePolicy(FSDirectory fsd, BlockManager bm,
-      String src, final byte policyId, final String operation)
+  static FileStatus setStoragePolicy(FSDirectory fsd, FSPermissionChecker pc,
+      BlockManager bm, String src, final byte policyId, final String operation)
       throws IOException {
     if (!fsd.isStoragePolicyEnabled()) {
       throw new IOException(String.format(
           "Failed to %s storage policy since %s is set to false.", operation,
           DFS_STORAGE_POLICY_ENABLED_KEY));
     }
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     INodesInPath iip;
     fsd.writeLock();
     try {
@@ -202,9 +195,8 @@ public class FSDirAttrOp {
     return bm.getStoragePolicies();
   }
 
-  static BlockStoragePolicy getStoragePolicy(FSDirectory fsd, BlockManager bm,
-      String path) throws IOException {
-    FSPermissionChecker pc = fsd.getPermissionChecker();
+  static BlockStoragePolicy getStoragePolicy(FSDirectory fsd,
+      FSPermissionChecker pc, BlockManager bm, String path) throws IOException {
     fsd.readLock();
     try {
       final INodesInPath iip = fsd.resolvePath(pc, path, DirOp.READ_LINK);
@@ -222,9 +214,8 @@ public class FSDirAttrOp {
     }
   }
 
-  static long getPreferredBlockSize(FSDirectory fsd, String src)
-      throws IOException {
-    FSPermissionChecker pc = fsd.getPermissionChecker();
+  static long getPreferredBlockSize(FSDirectory fsd, FSPermissionChecker pc,
+      String src) throws IOException {
     fsd.readLock();
     try {
       final INodesInPath iip = fsd.resolvePath(pc, src, DirOp.READ_LINK);
@@ -240,9 +231,8 @@ public class FSDirAttrOp {
    *
    * Note: This does not support ".inodes" relative path.
    */
-  static void setQuota(FSDirectory fsd, String src, long nsQuota, long ssQuota,
-      StorageType type) throws IOException {
-    FSPermissionChecker pc = fsd.getPermissionChecker();
+  static void setQuota(FSDirectory fsd, FSPermissionChecker pc, String src,
+      long nsQuota, long ssQuota, StorageType type) throws IOException {
     if (fsd.isPermissionEnabled()) {
       pc.checkSuperuserPrivilege();
     }

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

@@ -48,14 +48,13 @@ import static org.apache.hadoop.util.Time.now;
  */
 class FSDirConcatOp {
 
-  static FileStatus concat(FSDirectory fsd, String target, String[] srcs,
-    boolean logRetryCache) throws IOException {
+  static FileStatus concat(FSDirectory fsd, FSPermissionChecker pc,
+      String target, String[] srcs, boolean logRetryCache) throws IOException {
     validatePath(target, srcs);
     assert srcs != null;
     if (FSDirectory.LOG.isDebugEnabled()) {
       FSDirectory.LOG.debug("concat {} to {}", Arrays.toString(srcs), target);
     }
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     final INodesInPath targetIIP = fsd.resolvePath(pc, target, DirOp.WRITE);
     // write permission for the target
     if (fsd.isPermissionEnabled()) {

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

@@ -88,6 +88,7 @@ class FSDirDeleteOp {
    * For small directory or file the deletion is done in one shot.
    *
    * @param fsn namespace
+   * @param pc FS permission checker
    * @param src path name to be deleted
    * @param recursive boolean true to apply to all sub-directories recursively
    * @param logRetryCache whether to record RPC ids in editlog for retry cache
@@ -96,10 +97,9 @@ class FSDirDeleteOp {
    * @throws IOException
    */
   static BlocksMapUpdateInfo delete(
-      FSNamesystem fsn, String src, boolean recursive, boolean logRetryCache)
-      throws IOException {
+      FSNamesystem fsn, FSPermissionChecker pc, String src, boolean recursive,
+      boolean logRetryCache) throws IOException {
     FSDirectory fsd = fsn.getFSDirectory();
-    FSPermissionChecker pc = fsd.getPermissionChecker();
 
     if (FSDirectory.isExactReservedName(src)) {
       throw new InvalidPathException(src);
@@ -130,7 +130,7 @@ class FSDirDeleteOp {
    * <br>
    *
    * @param fsd the FSDirectory instance
-   * @param src a string representation of a path to an inode
+   * @param iip inodes of a path to be deleted
    * @param mtime the time the inode is removed
    */
   static void deleteForEditLog(FSDirectory fsd, INodesInPath iip, long mtime)

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

@@ -693,11 +693,12 @@ final class FSDirEncryptionZoneOp {
    * a different ACL. HDFS should not try to operate on additional ACLs, but
    * rather use the generate ACL it already has.
    */
-  static String getCurrentKeyVersion(final FSDirectory dir, final String zone)
-      throws IOException {
+  static String getCurrentKeyVersion(final FSDirectory dir,
+      final FSPermissionChecker pc, final String zone) throws IOException {
     assert dir.getProvider() != null;
     assert !dir.hasReadLock();
-    final String keyName = FSDirEncryptionZoneOp.getKeyNameForZone(dir, zone);
+    final String keyName = FSDirEncryptionZoneOp.getKeyNameForZone(dir,
+        pc, zone);
     if (keyName == null) {
       throw new IOException(zone + " is not an encryption zone.");
     }
@@ -719,11 +720,10 @@ final class FSDirEncryptionZoneOp {
    * Resolve the zone to an inode, find the encryption zone info associated with
    * that inode, and return the key name. Does not contact the KMS.
    */
-  static String getKeyNameForZone(final FSDirectory dir, final String zone)
-      throws IOException {
+  static String getKeyNameForZone(final FSDirectory dir,
+      final FSPermissionChecker pc, final String zone) throws IOException {
     assert dir.getProvider() != null;
     final INodesInPath iip;
-    final FSPermissionChecker pc = dir.getPermissionChecker();
     dir.readLock();
     try {
       iip = dir.resolvePath(pc, zone, DirOp.READ);

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

@@ -39,13 +39,12 @@ import static org.apache.hadoop.util.Time.now;
 
 class FSDirMkdirOp {
 
-  static FileStatus mkdirs(FSNamesystem fsn, String src,
+  static FileStatus mkdirs(FSNamesystem fsn, FSPermissionChecker pc, String src,
       PermissionStatus permissions, boolean createParent) throws IOException {
     FSDirectory fsd = fsn.getFSDirectory();
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
     }
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     fsd.writeLock();
     try {
       INodesInPath iip = fsd.resolvePath(pc, src, DirOp.CREATE);

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

@@ -47,14 +47,12 @@ import static org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooL
 class FSDirRenameOp {
   @Deprecated
   static RenameResult renameToInt(
-      FSDirectory fsd, final String src, final String dst,
-      boolean logRetryCache)
-      throws IOException {
+      FSDirectory fsd, FSPermissionChecker pc, final String src,
+      final String dst, boolean logRetryCache) throws IOException {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: " + src +
           " to " + dst);
     }
-    FSPermissionChecker pc = fsd.getPermissionChecker();
 
     // Rename does not operate on link targets
     // Do not resolveLink when checking permissions of src and dst
@@ -230,8 +228,8 @@ class FSDirRenameOp {
    * The new rename which has the POSIX semantic.
    */
   static RenameResult renameToInt(
-      FSDirectory fsd, final String srcArg, final String dstArg,
-      boolean logRetryCache, Options.Rename... options)
+      FSDirectory fsd, FSPermissionChecker pc, final String srcArg,
+      final String dstArg, boolean logRetryCache, Options.Rename... options)
       throws IOException {
     String src = srcArg;
     String dst = dstArg;
@@ -239,7 +237,6 @@ class FSDirRenameOp {
       NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: with options -" +
           " " + src + " to " + dst);
     }
-    final FSPermissionChecker pc = fsd.getPermissionChecker();
 
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
     // returns resolved path

+ 21 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java

@@ -80,14 +80,17 @@ class FSDirSnapshotOp {
 
   /**
    * Create a snapshot
+   * @param fsd FS directory
+   * @param pc FS permission checker
    * @param snapshotRoot The directory path where the snapshot is taken
    * @param snapshotName The name of the snapshot
+   * @param logRetryCache whether to record RPC ids in editlog for retry cache
+   *                      rebuilding.
    */
   static String createSnapshot(
-      FSDirectory fsd, SnapshotManager snapshotManager, String snapshotRoot,
-      String snapshotName, boolean logRetryCache)
+      FSDirectory fsd, FSPermissionChecker pc, SnapshotManager snapshotManager,
+      String snapshotRoot, String snapshotName, boolean logRetryCache)
       throws IOException {
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     final INodesInPath iip = fsd.resolvePath(pc, snapshotRoot, DirOp.WRITE);
     if (fsd.isPermissionEnabled()) {
       fsd.checkOwner(pc, iip);
@@ -115,10 +118,9 @@ class FSDirSnapshotOp {
     return snapshotPath;
   }
 
-  static void renameSnapshot(FSDirectory fsd, SnapshotManager snapshotManager,
-      String path, String snapshotOldName, String snapshotNewName,
-      boolean logRetryCache) throws IOException {
-    FSPermissionChecker pc = fsd.getPermissionChecker();
+  static void renameSnapshot(FSDirectory fsd, FSPermissionChecker pc,
+      SnapshotManager snapshotManager, String path, String snapshotOldName,
+      String snapshotNewName, boolean logRetryCache) throws IOException {
     final INodesInPath iip = fsd.resolvePath(pc, path, DirOp.WRITE);
     if (fsd.isPermissionEnabled()) {
       fsd.checkOwner(pc, iip);
@@ -136,8 +138,8 @@ class FSDirSnapshotOp {
   }
 
   static SnapshottableDirectoryStatus[] getSnapshottableDirListing(
-      FSDirectory fsd, SnapshotManager snapshotManager) throws IOException {
-    FSPermissionChecker pc = fsd.getPermissionChecker();
+      FSDirectory fsd, FSPermissionChecker pc, SnapshotManager snapshotManager)
+      throws IOException {
     fsd.readLock();
     try {
       final String user = pc.isSuperUser()? null : pc.getUser();
@@ -148,10 +150,9 @@ class FSDirSnapshotOp {
   }
 
   static SnapshotDiffReport getSnapshotDiffReport(FSDirectory fsd,
-      SnapshotManager snapshotManager, String path,
+      FSPermissionChecker pc, SnapshotManager snapshotManager, String path,
       String fromSnapshot, String toSnapshot) throws IOException {
     SnapshotDiffReport diffs;
-    final FSPermissionChecker pc = fsd.getPermissionChecker();
     fsd.readLock();
     try {
       INodesInPath iip = fsd.resolvePath(pc, path, DirOp.READ);
@@ -167,11 +168,10 @@ class FSDirSnapshotOp {
   }
 
   static SnapshotDiffReportListing getSnapshotDiffReportListing(FSDirectory fsd,
-      SnapshotManager snapshotManager, String path, String fromSnapshot,
-      String toSnapshot, byte[] startPath, int index,
+      FSPermissionChecker pc, SnapshotManager snapshotManager, String path,
+      String fromSnapshot, String toSnapshot, byte[] startPath, int index,
       int snapshotDiffReportLimit) throws IOException {
     SnapshotDiffReportListing diffs;
-    final FSPermissionChecker pc = fsd.getPermissionChecker();
     fsd.readLock();
     try {
       INodesInPath iip = fsd.resolvePath(pc, path, DirOp.READ);
@@ -226,15 +226,19 @@ class FSDirSnapshotOp {
 
   /**
    * Delete a snapshot of a snapshottable directory
+   * @param fsd The FS directory
+   * @param pc The permission checker
+   * @param snapshotManager The snapshot manager
    * @param snapshotRoot The snapshottable directory
    * @param snapshotName The name of the to-be-deleted snapshot
+   * @param logRetryCache whether to record RPC ids in editlog for retry cache
+   *                      rebuilding.
    * @throws IOException
    */
   static INode.BlocksMapUpdateInfo deleteSnapshot(
-      FSDirectory fsd, SnapshotManager snapshotManager, String snapshotRoot,
-      String snapshotName, boolean logRetryCache)
+      FSDirectory fsd, FSPermissionChecker pc, SnapshotManager snapshotManager,
+      String snapshotRoot, String snapshotName, boolean logRetryCache)
       throws IOException {
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     final INodesInPath iip = fsd.resolvePath(pc, snapshotRoot, DirOp.WRITE);
     if (fsd.isPermissionEnabled()) {
       fsd.checkOwner(pc, iip);

+ 16 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java

@@ -51,9 +51,9 @@ import java.util.EnumSet;
 import static org.apache.hadoop.util.Time.now;
 
 class FSDirStatAndListingOp {
-  static DirectoryListing getListingInt(FSDirectory fsd, final String srcArg,
-      byte[] startAfter, boolean needLocation) throws IOException {
-    final FSPermissionChecker pc = fsd.getPermissionChecker();
+  static DirectoryListing getListingInt(FSDirectory fsd, FSPermissionChecker pc,
+      final String srcArg, byte[] startAfter, boolean needLocation)
+      throws IOException {
     final INodesInPath iip = fsd.resolvePath(pc, srcArg, DirOp.READ);
 
     // Get file name when startAfter is an INodePath.  This is not the
@@ -85,7 +85,8 @@ class FSDirStatAndListingOp {
 
   /**
    * Get the file info for a specific file.
-   *
+   * @param fsd The FS directory
+   * @param pc The permission checker
    * @param srcArg The string representation of the path to the file
    * @param resolveLink whether to throw UnresolvedLinkException
    *        if src refers to a symlink
@@ -95,11 +96,10 @@ class FSDirStatAndListingOp {
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  static HdfsFileStatus getFileInfo(FSDirectory fsd, String srcArg,
-      boolean resolveLink, boolean needLocation, boolean needBlockToken)
-      throws IOException {
+  static HdfsFileStatus getFileInfo(FSDirectory fsd, FSPermissionChecker pc,
+      String srcArg, boolean resolveLink, boolean needLocation,
+      boolean needBlockToken) throws IOException {
     DirOp dirOp = resolveLink ? DirOp.READ : DirOp.READ_LINK;
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     final INodesInPath iip;
     if (pc.isSuperUser()) {
       // superuser can only get an ACE if an existing ancestor is a file.
@@ -119,19 +119,18 @@ class FSDirStatAndListingOp {
   /**
    * Returns true if the file is closed
    */
-  static boolean isFileClosed(FSDirectory fsd, String src) throws IOException {
-    FSPermissionChecker pc = fsd.getPermissionChecker();
+  static boolean isFileClosed(FSDirectory fsd, FSPermissionChecker pc,
+      String src) throws IOException {
     final INodesInPath iip = fsd.resolvePath(pc, src, DirOp.READ);
     return !INodeFile.valueOf(iip.getLastINode(), src).isUnderConstruction();
   }
 
   static ContentSummary getContentSummary(
-      FSDirectory fsd, String src) throws IOException {
-    FSPermissionChecker pc = fsd.getPermissionChecker();
+      FSDirectory fsd, FSPermissionChecker pc, String src) throws IOException {
     final INodesInPath iip = fsd.resolvePath(pc, src, DirOp.READ_LINK);
     // getContentSummaryInt() call will check access (if enabled) when
     // traversing all sub directories.
-    return getContentSummaryInt(fsd, iip);
+    return getContentSummaryInt(fsd, pc, iip);
   }
 
   /**
@@ -516,7 +515,7 @@ class FSDirStatAndListingOp {
   }
 
   private static ContentSummary getContentSummaryInt(FSDirectory fsd,
-      INodesInPath iip) throws IOException {
+      FSPermissionChecker pc, INodesInPath iip) throws IOException {
     fsd.readLock();
     try {
       INode targetNode = iip.getLastINode();
@@ -528,8 +527,7 @@ class FSDirStatAndListingOp {
         // processed. 0 means disabled. I.e. blocking for the entire duration.
         ContentSummaryComputationContext cscc =
             new ContentSummaryComputationContext(fsd, fsd.getFSNamesystem(),
-                fsd.getContentCountLimit(), fsd.getContentSleepMicroSec(),
-                fsd.getPermissionChecker());
+                fsd.getContentCountLimit(), fsd.getContentSleepMicroSec(), pc);
         ContentSummary cs = targetNode.computeAndConvertContentSummary(
             iip.getPathSnapshotId(), cscc);
         fsd.addYieldCount(cscc.getYieldCount());
@@ -541,8 +539,7 @@ class FSDirStatAndListingOp {
   }
 
   static QuotaUsage getQuotaUsage(
-      FSDirectory fsd, String src) throws IOException {
-    FSPermissionChecker pc = fsd.getPermissionChecker();
+      FSDirectory fsd, FSPermissionChecker pc, String src) throws IOException {
     final INodesInPath iip;
     fsd.readLock();
     try {
@@ -559,7 +556,7 @@ class FSDirStatAndListingOp {
       return usage;
     } else {
       //If quota isn't set, fall back to getContentSummary.
-      return getContentSummaryInt(fsd, iip);
+      return getContentSummaryInt(fsd, pc, iip);
     }
   }
 

+ 21 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java

@@ -51,22 +51,27 @@ class FSDirXAttrOp {
 
   /**
    * Set xattr for a file or directory.
-   *
+   * @param fsd
+   *          - FS directory
+   * @param pc
+   *          - FS permission checker
    * @param src
    *          - path on which it sets the xattr
    * @param xAttr
    *          - xAttr details to set
    * @param flag
    *          - xAttrs flags
+   * @param logRetryCache
+   *          - whether to record RPC ids in editlog for retry cache
+   *          rebuilding.
    * @throws IOException
    */
   static FileStatus setXAttr(
-      FSDirectory fsd, String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
-      boolean logRetryCache)
+      FSDirectory fsd, FSPermissionChecker pc, String src, XAttr xAttr,
+      EnumSet<XAttrSetFlag> flag, boolean logRetryCache)
       throws IOException {
     checkXAttrsConfigFlag(fsd);
     checkXAttrSize(fsd, xAttr);
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     XAttrPermissionFilter.checkPermissionForApi(
         pc, xAttr, FSDirectory.isReservedRawName(src));
     List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
@@ -85,12 +90,10 @@ class FSDirXAttrOp {
     return fsd.getAuditFileInfo(iip);
   }
 
-  static List<XAttr> getXAttrs(FSDirectory fsd, final String srcArg,
-                               List<XAttr> xAttrs)
-      throws IOException {
+  static List<XAttr> getXAttrs(FSDirectory fsd, FSPermissionChecker pc,
+      final String srcArg, List<XAttr> xAttrs) throws IOException {
     String src = srcArg;
     checkXAttrsConfigFlag(fsd);
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     final boolean isRawPath = FSDirectory.isReservedRawName(src);
     boolean getAll = xAttrs == null || xAttrs.isEmpty();
     if (!getAll) {
@@ -131,9 +134,8 @@ class FSDirXAttrOp {
   }
 
   static List<XAttr> listXAttrs(
-      FSDirectory fsd, String src) throws IOException {
+      FSDirectory fsd, FSPermissionChecker pc, String src) throws IOException {
     FSDirXAttrOp.checkXAttrsConfigFlag(fsd);
-    final FSPermissionChecker pc = fsd.getPermissionChecker();
     final boolean isRawPath = FSDirectory.isReservedRawName(src);
     final INodesInPath iip = fsd.resolvePath(pc, src, DirOp.READ);
     if (fsd.isPermissionEnabled()) {
@@ -146,18 +148,23 @@ class FSDirXAttrOp {
 
   /**
    * Remove an xattr for a file or directory.
-   *
+   * @param fsd
+   *          - FS direcotry
+   * @param pc
+   *          - FS permission checker
    * @param src
    *          - path to remove the xattr from
    * @param xAttr
    *          - xAttr to remove
+   * @param logRetryCache
+   *          - whether to record RPC ids in editlog for retry cache
+   *          rebuilding.
    * @throws IOException
    */
   static FileStatus removeXAttr(
-      FSDirectory fsd, String src, XAttr xAttr, boolean logRetryCache)
-      throws IOException {
+      FSDirectory fsd, FSPermissionChecker pc, String src, XAttr xAttr,
+      boolean logRetryCache) throws IOException {
     FSDirXAttrOp.checkXAttrsConfigFlag(fsd);
-    FSPermissionChecker pc = fsd.getPermissionChecker();
     XAttrPermissionFilter.checkPermissionForApi(
         pc, xAttr, FSDirectory.isReservedRawName(src));
 

+ 124 - 66
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1869,11 +1869,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "setPermission";
     FileStatus auditStat;
     checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set permission for " + src);
-      auditStat = FSDirAttrOp.setPermission(dir, src, permission);
+      auditStat = FSDirAttrOp.setPermission(dir, pc, src, permission);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -1893,11 +1894,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "setOwner";
     FileStatus auditStat;
     checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set owner for " + src);
-      auditStat = FSDirAttrOp.setOwner(dir, src, username, group);
+      auditStat = FSDirAttrOp.setOwner(dir, pc, src, username, group);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -1917,7 +1919,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "open";
     checkOperation(OperationCategory.READ);
     GetBlockLocationsResult res = null;
-    FSPermissionChecker pc = getPermissionChecker();
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
@@ -2030,11 +2032,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "concat";
     FileStatus stat = null;
     boolean success = false;
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot concat " + target);
-      stat = FSDirConcatOp.concat(dir, target, srcs, logRetryCache);
+      stat = FSDirConcatOp.concat(dir, pc, target, srcs, logRetryCache);
       success = true;
     } catch (AccessControlException ace) {
       logAuditEvent(success, operationName, Arrays.toString(srcs),
@@ -2058,11 +2061,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "setTimes";
     FileStatus auditStat;
     checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set times " + src);
-      auditStat = FSDirAttrOp.setTimes(dir, src, mtime, atime);
+      auditStat = FSDirAttrOp.setTimes(dir, pc, src, mtime, atime);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -2096,8 +2100,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         throw new HadoopIllegalArgumentException(
             "Cannot truncate to a negative file size: " + newLength + ".");
       }
-      final FSPermissionChecker pc = getPermissionChecker();
       checkOperation(OperationCategory.WRITE);
+      final FSPermissionChecker pc = getPermissionChecker();
       writeLock();
       BlocksMapUpdateInfo toRemoveBlocks = new BlocksMapUpdateInfo();
       try {
@@ -2166,11 +2170,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "setReplication";
     boolean success = false;
     checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set replication for " + src);
-      success = FSDirAttrOp.setReplication(dir, blockManager, src, replication);
+      success = FSDirAttrOp.setReplication(dir, pc, blockManager, src,
+          replication);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -2194,11 +2200,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "setStoragePolicy";
     FileStatus auditStat;
     checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set storage policy for " + src);
-      auditStat = FSDirAttrOp.setStoragePolicy(dir, blockManager, src,
+      auditStat = FSDirAttrOp.setStoragePolicy(dir, pc, blockManager, src,
                                                policyName);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
@@ -2219,11 +2226,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "unsetStoragePolicy";
     FileStatus auditStat;
     checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot unset storage policy for " + src);
-      auditStat = FSDirAttrOp.unsetStoragePolicy(dir, blockManager, src);
+      auditStat = FSDirAttrOp.unsetStoragePolicy(dir, pc, blockManager, src);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -2242,10 +2250,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   BlockStoragePolicy getStoragePolicy(String src) throws IOException {
     checkOperation(OperationCategory.READ);
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return FSDirAttrOp.getStoragePolicy(dir, blockManager, src);
+      return FSDirAttrOp.getStoragePolicy(dir, pc, blockManager, src);
     } finally {
       readUnlock("getStoragePolicy");
     }
@@ -2267,10 +2276,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   long getPreferredBlockSize(String src) throws IOException {
     checkOperation(OperationCategory.READ);
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return FSDirAttrOp.getPreferredBlockSize(dir, src);
+      return FSDirAttrOp.getPreferredBlockSize(dir, pc, src);
     } finally {
       readUnlock("getPreferredBlockSize");
     }
@@ -2374,13 +2384,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           "ecPolicyName are exclusive parameters. Set both is not allowed!");
     }
 
-    FSPermissionChecker pc = getPermissionChecker();
     INodesInPath iip = null;
     boolean skipSync = true; // until we do something that might create edits
     HdfsFileStatus stat = null;
     BlocksMapUpdateInfo toRemoveBlocks = null;
 
     checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -2461,8 +2471,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   boolean recoverLease(String src, String holder, String clientMachine)
       throws IOException {
     boolean skipSync = false;
-    FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -2601,8 +2611,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       boolean skipSync = false;
       LastBlockWithStatus lbs = null;
-      final FSPermissionChecker pc = getPermissionChecker();
       checkOperation(OperationCategory.WRITE);
+      final FSPermissionChecker pc = getPermissionChecker();
       writeLock();
       try {
         checkOperation(OperationCategory.WRITE);
@@ -2657,8 +2667,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     LocatedBlock[] onRetryBlock = new LocatedBlock[1];
     FSDirWriteFileOp.ValidateAddBlockResult r;
-    FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.READ);
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
@@ -2708,7 +2718,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final List<DatanodeStorageInfo> chosen;
     final BlockType blockType;
     checkOperation(OperationCategory.READ);
-    FSPermissionChecker pc = getPermissionChecker();
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
@@ -2756,7 +2766,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     NameNode.stateChangeLog.debug(
         "BLOCK* NameSystem.abandonBlock: {} of file {}", b, src);
     checkOperation(OperationCategory.WRITE);
-    FSPermissionChecker pc = getPermissionChecker();
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -2821,7 +2831,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     throws IOException {
     boolean success = false;
     checkOperation(OperationCategory.WRITE);
-    FSPermissionChecker pc = getPermissionChecker();
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -2899,11 +2909,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     final String operationName = "rename";
     FSDirRenameOp.RenameResult ret = null;
+    checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot rename " + src);
-      ret = FSDirRenameOp.renameToInt(dir, src, dst, logRetryCache);
+      ret = FSDirRenameOp.renameToInt(dir, pc, src, dst, logRetryCache);
     } catch (AccessControlException e)  {
       logAuditEvent(false, operationName, src, dst, null);
       throw e;
@@ -2923,11 +2935,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     final String operationName = "rename";
     FSDirRenameOp.RenameResult res = null;
+    checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot rename " + src);
-      res = FSDirRenameOp.renameToInt(dir, src, dst, logRetryCache, options);
+      res = FSDirRenameOp.renameToInt(dir, pc, src, dst, logRetryCache,
+          options);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName + " (options=" +
           Arrays.toString(options) + ")", src, dst, null);
@@ -2958,13 +2973,15 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     final String operationName = "delete";
     BlocksMapUpdateInfo toRemovedBlocks = null;
+    checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     boolean ret = false;
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot delete " + src);
       toRemovedBlocks = FSDirDeleteOp.delete(
-          this, src, recursive, logRetryCache);
+          this, pc, src, recursive, logRetryCache);
       ret = toRemovedBlocks != null;
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
@@ -3063,11 +3080,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = needBlockToken ? "open" : "getfileinfo";
     checkOperation(OperationCategory.READ);
     HdfsFileStatus stat = null;
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
       stat = FSDirStatAndListingOp.getFileInfo(
-          dir, src, resolveLink, needLocation, needBlockToken);
+          dir, pc, src, resolveLink, needLocation, needBlockToken);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -3084,10 +3102,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   boolean isFileClosed(final String src) throws IOException {
     final String operationName = "isFileClosed";
     checkOperation(OperationCategory.READ);
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      return FSDirStatAndListingOp.isFileClosed(dir, src);
+      return FSDirStatAndListingOp.isFileClosed(dir, pc, src);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -3104,11 +3123,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "mkdirs";
     FileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot create directory " + src);
-      auditStat = FSDirMkdirOp.mkdirs(this, src, permissions, createParent);
+      auditStat = FSDirMkdirOp.mkdirs(this, pc, src, permissions,
+          createParent);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -3137,12 +3158,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   ContentSummary getContentSummary(final String src) throws IOException {
     checkOperation(OperationCategory.READ);
     final String operationName = "contentSummary";
-    readLock();
     boolean success = true;
     ContentSummary cs;
+    final FSPermissionChecker pc = getPermissionChecker();
+    readLock();
     try {
       checkOperation(OperationCategory.READ);
-      cs = FSDirStatAndListingOp.getContentSummary(dir, src);
+      cs = FSDirStatAndListingOp.getContentSummary(dir, pc, src);
     } catch (AccessControlException ace) {
       success = false;
       logAuditEvent(success, operationName, src);
@@ -3172,11 +3194,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.READ);
     final String operationName = "quotaUsage";
     QuotaUsage quotaUsage;
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     boolean success = true;
     try {
       checkOperation(OperationCategory.READ);
-      quotaUsage = FSDirStatAndListingOp.getQuotaUsage(dir, src);
+      quotaUsage = FSDirStatAndListingOp.getQuotaUsage(dir, pc, src);
     } catch (AccessControlException ace) {
       success = false;
       logAuditEvent(success, operationName, src);
@@ -3202,12 +3225,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
     checkOperation(OperationCategory.WRITE);
     final String operationName = getQuotaCommand(nsQuota, ssQuota);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     boolean success = false;
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set quota on " + src);
-      FSDirAttrOp.setQuota(dir, src, nsQuota, ssQuota, type);
+      FSDirAttrOp.setQuota(dir, pc, src, nsQuota, ssQuota, type);
       success = true;
     } catch (AccessControlException ace) {
       logAuditEvent(success, operationName, src);
@@ -3234,8 +3258,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     NameNode.stateChangeLog.info("BLOCK* fsync: " + src + " for " + clientName);
     checkOperation(OperationCategory.WRITE);
-
-    FSPermissionChecker pc = getPermissionChecker();
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -3739,10 +3762,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.READ);
     final String operationName = "listStatus";
     DirectoryListing dl = null;
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(NameNode.OperationCategory.READ);
-      dl = getListingInt(dir, src, startAfter, needLocation);
+      dl = getListingInt(dir, pc, src, startAfter, needLocation);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -4678,6 +4702,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
+  void checkSuperuserPrivilege(FSPermissionChecker pc)
+      throws AccessControlException {
+    if (isPermissionEnabled) {
+      pc.checkSuperuserPrivilege();
+    }
+  }
+
   /**
    * Check to see if we have exceeded the limit on the number
    * of inodes.
@@ -6365,14 +6396,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   String createSnapshot(String snapshotRoot, String snapshotName,
                         boolean logRetryCache) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     final String operationName = "createSnapshot";
     String snapshotPath = null;
     boolean success = false;
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot create snapshot for " + snapshotRoot);
-      snapshotPath = FSDirSnapshotOp.createSnapshot(dir,
+      snapshotPath = FSDirSnapshotOp.createSnapshot(dir, pc,
           snapshotManager, snapshotRoot, snapshotName, logRetryCache);
       success = true;
     } catch (AccessControlException ace) {
@@ -6399,15 +6432,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void renameSnapshot(
       String path, String snapshotOldName, String snapshotNewName,
       boolean logRetryCache) throws IOException {
+    checkOperation(OperationCategory.WRITE);
     final String operationName = "renameSnapshot";
     boolean success = false;
     String oldSnapshotRoot = Snapshot.getSnapshotPath(path, snapshotOldName);
     String newSnapshotRoot = Snapshot.getSnapshotPath(path, snapshotNewName);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot rename snapshot for " + path);
-      FSDirSnapshotOp.renameSnapshot(dir, snapshotManager, path,
+      FSDirSnapshotOp.renameSnapshot(dir, pc, snapshotManager, path,
           snapshotOldName, snapshotNewName, logRetryCache);
       success = true;
     } catch (AccessControlException ace) {
@@ -6435,10 +6470,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     SnapshottableDirectoryStatus[] status = null;
     checkOperation(OperationCategory.READ);
     boolean success = false;
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      status = FSDirSnapshotOp.getSnapshottableDirListing(dir, snapshotManager);
+      status = FSDirSnapshotOp.getSnapshottableDirListing(dir, pc,
+          snapshotManager);
       success = true;
     } catch (AccessControlException ace) {
       logAuditEvent(success, operationName, null, null, null);
@@ -6475,10 +6512,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         path : Snapshot.getSnapshotPath(path, fromSnapshot);
     String toSnapshotRoot = (toSnapshot == null || toSnapshot.isEmpty()) ?
         path : Snapshot.getSnapshotPath(path, toSnapshot);
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      diffs = FSDirSnapshotOp.getSnapshotDiffReport(dir, snapshotManager,
+      diffs = FSDirSnapshotOp.getSnapshotDiffReport(dir, pc, snapshotManager,
           path, fromSnapshot, toSnapshot);
       success = true;
     } catch (AccessControlException ace) {
@@ -6530,11 +6568,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     String toSnapshotRoot =
         (toSnapshot == null || toSnapshot.isEmpty()) ? path :
             Snapshot.getSnapshotPath(path, toSnapshot);
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
       diffs = FSDirSnapshotOp
-          .getSnapshotDiffReportListing(dir, snapshotManager, path,
+          .getSnapshotDiffReportListing(dir, pc, snapshotManager, path,
               fromSnapshot, toSnapshot, startPath, index,
               snapshotDiffReportLimit);
       success = true;
@@ -6562,14 +6601,15 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "deleteSnapshot";
     boolean success = false;
     String rootPath = null;
-    writeLock();
     BlocksMapUpdateInfo blocksToBeDeleted = null;
+    final FSPermissionChecker pc = getPermissionChecker();
+    writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot delete snapshot for " + snapshotRoot);
       rootPath = Snapshot.getSnapshotPath(snapshotRoot, snapshotName);
-      blocksToBeDeleted = FSDirSnapshotOp.deleteSnapshot(dir, snapshotManager,
-          snapshotRoot, snapshotName, logRetryCache);
+      blocksToBeDeleted = FSDirSnapshotOp.deleteSnapshot(dir, pc,
+          snapshotManager, snapshotRoot, snapshotName, logRetryCache);
       success = true;
     } catch (AccessControlException ace) {
       logAuditEvent(success, operationName, rootPath, null, null);
@@ -7051,11 +7091,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "modifyAclEntries";
     FileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot modify ACL entries on " + src);
-      auditStat = FSDirAclOp.modifyAclEntries(dir, src, aclSpec);
+      auditStat = FSDirAclOp.modifyAclEntries(dir, pc, src, aclSpec);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -7071,11 +7112,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "removeAclEntries";
     checkOperation(OperationCategory.WRITE);
     FileStatus auditStat = null;
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove ACL entries on " + src);
-      auditStat = FSDirAclOp.removeAclEntries(dir, src, aclSpec);
+      auditStat = FSDirAclOp.removeAclEntries(dir, pc, src, aclSpec);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -7090,11 +7132,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "removeDefaultAcl";
     FileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove default ACL entries on " + src);
-      auditStat = FSDirAclOp.removeDefaultAcl(dir, src);
+      auditStat = FSDirAclOp.removeDefaultAcl(dir, pc, src);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -7109,11 +7152,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "removeAcl";
     FileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove ACL on " + src);
-      auditStat = FSDirAclOp.removeAcl(dir, src);
+      auditStat = FSDirAclOp.removeAcl(dir, pc, src);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -7128,11 +7172,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "setAcl";
     FileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set ACL on " + src);
-      auditStat = FSDirAclOp.setAcl(dir, src, aclSpec);
+      auditStat = FSDirAclOp.setAcl(dir, pc, src, aclSpec);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -7147,10 +7192,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "getAclStatus";
     checkOperation(OperationCategory.READ);
     final AclStatus ret;
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      ret = FSDirAclOp.getAclStatus(dir, src);
+      ret = FSDirAclOp.getAclStatus(dir, pc, src);
     } catch(AccessControlException ace) {
       logAuditEvent(false, operationName, src);
       throw ace;
@@ -7179,13 +7225,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       Metadata metadata = FSDirEncryptionZoneOp.ensureKeyIsInitialized(dir,
           keyName, src);
-      checkSuperuserPrivilege();
-      FSPermissionChecker pc = getPermissionChecker();
+      final FSPermissionChecker pc = getPermissionChecker();
+      checkSuperuserPrivilege(pc);
       checkOperation(OperationCategory.WRITE);
       final FileStatus resultingStat;
       writeLock();
       try {
-        checkSuperuserPrivilege();
+        checkSuperuserPrivilege(pc);
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot create encryption zone on " + src);
         resultingStat = FSDirEncryptionZoneOp.createEncryptionZone(dir, src,
@@ -7240,12 +7286,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     final String operationName = "listEncryptionZones";
     boolean success = false;
-    checkSuperuserPrivilege();
     checkOperation(OperationCategory.READ);
+    final FSPermissionChecker pc = getPermissionChecker();
+    checkSuperuserPrivilege(pc);
     readLock();
     try {
-      checkSuperuserPrivilege();
       checkOperation(OperationCategory.READ);
+      checkSuperuserPrivilege(pc);
       final BatchedListEntries<EncryptionZone> ret =
           FSDirEncryptionZoneOp.listEncryptionZones(dir, prevId);
       success = true;
@@ -7261,11 +7308,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean success = false;
     try {
       Preconditions.checkNotNull(zone, "zone is null.");
-      checkSuperuserPrivilege();
       checkOperation(OperationCategory.WRITE);
+      final FSPermissionChecker pc = dir.getPermissionChecker();
+      checkSuperuserPrivilege(pc);
       checkNameNodeSafeMode("NameNode in safemode, cannot " + action
           + " re-encryption on zone " + zone);
-      reencryptEncryptionZoneInt(zone, action, logRetryCache);
+      reencryptEncryptionZoneInt(pc, zone, action, logRetryCache);
       success = true;
     } finally {
       logAuditEvent(success, action + "reencryption", zone, null, null);
@@ -7276,12 +7324,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       final long prevId) throws IOException {
     final String operationName = "listReencryptionStatus";
     boolean success = false;
-    checkSuperuserPrivilege();
     checkOperation(OperationCategory.READ);
+    final FSPermissionChecker pc = getPermissionChecker();
+    checkSuperuserPrivilege(pc);
     readLock();
     try {
-      checkSuperuserPrivilege();
       checkOperation(OperationCategory.READ);
+      checkSuperuserPrivilege(pc);
       final BatchedListEntries<ZoneReencryptionStatus> ret =
           FSDirEncryptionZoneOp.listReencryptionStatus(dir, prevId);
       success = true;
@@ -7292,9 +7341,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
-  private void reencryptEncryptionZoneInt(final String zone,
-      final ReencryptAction action, final boolean logRetryCache)
-      throws IOException {
+  private void reencryptEncryptionZoneInt(final FSPermissionChecker pc,
+      final String zone, final ReencryptAction action,
+      final boolean logRetryCache) throws IOException {
     if (getProvider() == null) {
       throw new IOException("No key provider configured, re-encryption "
           + "operation is rejected");
@@ -7302,7 +7351,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     String keyVersionName = null;
     if (action == ReencryptAction.START) {
       // get zone's latest key version name out of the lock.
-      keyVersionName = FSDirEncryptionZoneOp.getCurrentKeyVersion(dir, zone);
+      keyVersionName =
+          FSDirEncryptionZoneOp.getCurrentKeyVersion(dir, pc, zone);
       if (keyVersionName == null) {
         throw new IOException("Failed to get key version name for " + zone);
       }
@@ -7311,11 +7361,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
     writeLock();
     try {
-      checkSuperuserPrivilege();
+      checkSuperuserPrivilege(pc);
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("NameNode in safemode, cannot " + action
           + " re-encryption on zone " + zone);
-      final FSPermissionChecker pc = dir.getPermissionChecker();
       List<XAttr> xattrs;
       dir.writeLock();
       try {
@@ -7550,7 +7599,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "getErasureCodingPolicy";
     boolean success = false;
     checkOperation(OperationCategory.READ);
-    FSPermissionChecker pc = getPermissionChecker();
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
@@ -7609,11 +7658,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     final String operationName = "setXAttr";
     FileStatus auditStat = null;
+    checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot set XAttr on " + src);
-      auditStat = FSDirXAttrOp.setXAttr(dir, src, xAttr, flag, logRetryCache);
+      auditStat = FSDirXAttrOp.setXAttr(dir, pc, src, xAttr, flag,
+          logRetryCache);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -7629,10 +7681,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "getXAttrs";
     checkOperation(OperationCategory.READ);
     List<XAttr> fsXattrs;
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      fsXattrs = FSDirXAttrOp.getXAttrs(dir, src, xAttrs);
+      fsXattrs = FSDirXAttrOp.getXAttrs(dir, pc, src, xAttrs);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -7647,10 +7700,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "listXAttrs";
     checkOperation(OperationCategory.READ);
     List<XAttr> fsXattrs;
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      fsXattrs = FSDirXAttrOp.listXAttrs(dir, src);
+      fsXattrs = FSDirXAttrOp.listXAttrs(dir, pc, src);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -7665,11 +7719,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     final String operationName = "removeXAttr";
     FileStatus auditStat = null;
+    checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove XAttr entry on " + src);
-      auditStat = FSDirXAttrOp.removeXAttr(dir, src, xAttr, logRetryCache);
+      auditStat = FSDirXAttrOp.removeXAttr(dir, pc, src, xAttr, logRetryCache);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -7683,7 +7739,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void checkAccess(String src, FsAction mode) throws IOException {
     final String operationName = "checkAccess";
     checkOperation(OperationCategory.READ);
-    FSPermissionChecker pc = getPermissionChecker();
+    final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
@@ -7934,6 +7990,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         .size();
   }
 
+  // This method logs operatoinName without super user privilege.
+  // It should be called without holding FSN lock.
   void checkSuperuserPrivilege(String operationName)
       throws IOException {
     try {

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

@@ -78,6 +78,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
@@ -239,7 +240,7 @@ public class NamenodeWebHdfsMethods {
       final String remoteAddr, final HdfsFileStatus status) throws IOException {
     FSNamesystem fsn = namenode.getNamesystem();
     if (fsn == null) {
-      throw new IOException("Namesystem has not been intialized yet.");
+      throw new IOException("Namesystem has not been initialized yet.");
     }
     final BlockManager bm = fsn.getBlockManager();
     
@@ -922,6 +923,10 @@ public class NamenodeWebHdfsMethods {
           final ExcludeDatanodesParam excludeDatanodes,
       @QueryParam(FsActionParam.NAME) @DefaultValue(FsActionParam.DEFAULT)
           final FsActionParam fsAction,
+      @QueryParam(SnapshotNameParam.NAME) @DefaultValue(SnapshotNameParam.DEFAULT)
+          final SnapshotNameParam snapshotName,
+      @QueryParam(OldSnapshotNameParam.NAME) @DefaultValue(OldSnapshotNameParam.DEFAULT)
+          final OldSnapshotNameParam oldSnapshotName,
       @QueryParam(TokenKindParam.NAME) @DefaultValue(TokenKindParam.DEFAULT)
           final TokenKindParam tokenKind,
       @QueryParam(TokenServiceParam.NAME) @DefaultValue(TokenServiceParam.DEFAULT)
@@ -932,8 +937,9 @@ public class NamenodeWebHdfsMethods {
           final StartAfterParam startAfter
       ) throws IOException, InterruptedException {
     return get(ugi, delegation, username, doAsUser, ROOT, op, offset, length,
-        renewer, bufferSize, xattrNames, xattrEncoding, excludeDatanodes, fsAction,
-        tokenKind, tokenService, noredirect, startAfter);
+        renewer, bufferSize, xattrNames, xattrEncoding, excludeDatanodes,
+        fsAction, snapshotName, oldSnapshotName, tokenKind, tokenService,
+        noredirect, startAfter);
   }
 
   /** Handle HTTP GET request. */
@@ -968,6 +974,10 @@ public class NamenodeWebHdfsMethods {
           final ExcludeDatanodesParam excludeDatanodes,
       @QueryParam(FsActionParam.NAME) @DefaultValue(FsActionParam.DEFAULT)
           final FsActionParam fsAction,
+      @QueryParam(SnapshotNameParam.NAME) @DefaultValue(SnapshotNameParam.DEFAULT)
+          final SnapshotNameParam snapshotName,
+      @QueryParam(OldSnapshotNameParam.NAME) @DefaultValue(OldSnapshotNameParam.DEFAULT)
+          final OldSnapshotNameParam oldSnapshotName,
       @QueryParam(TokenKindParam.NAME) @DefaultValue(TokenKindParam.DEFAULT)
           final TokenKindParam tokenKind,
       @QueryParam(TokenServiceParam.NAME) @DefaultValue(TokenServiceParam.DEFAULT)
@@ -980,15 +990,15 @@ public class NamenodeWebHdfsMethods {
 
     init(ugi, delegation, username, doAsUser, path, op, offset, length,
         renewer, bufferSize, xattrEncoding, excludeDatanodes, fsAction,
-        tokenKind, tokenService, startAfter);
+        snapshotName, oldSnapshotName, tokenKind, tokenService, startAfter);
 
     return doAs(ugi, new PrivilegedExceptionAction<Response>() {
       @Override
       public Response run() throws IOException, URISyntaxException {
-          return get(ugi, delegation, username, doAsUser,
-              path.getAbsolutePath(), op, offset, length, renewer, bufferSize,
-              xattrNames, xattrEncoding, excludeDatanodes, fsAction, tokenKind,
-              tokenService, noredirect, startAfter);
+        return get(ugi, delegation, username, doAsUser, path.getAbsolutePath(),
+            op, offset, length, renewer, bufferSize, xattrNames, xattrEncoding,
+            excludeDatanodes, fsAction, snapshotName, oldSnapshotName,
+            tokenKind, tokenService, noredirect, startAfter);
       }
     });
   }
@@ -1015,6 +1025,8 @@ public class NamenodeWebHdfsMethods {
       final XAttrEncodingParam xattrEncoding,
       final ExcludeDatanodesParam excludeDatanodes,
       final FsActionParam fsAction,
+      final SnapshotNameParam snapshotName,
+      final OldSnapshotNameParam oldSnapshotName,
       final TokenKindParam tokenKind,
       final TokenServiceParam tokenService,
       final NoRedirectParam noredirectParam,
@@ -1185,6 +1197,13 @@ public class NamenodeWebHdfsMethods {
       return Response.ok(serverDefaultsResponse)
           .type(MediaType.APPLICATION_JSON).build();
     }
+    case GETSNAPSHOTDIFF: {
+      SnapshotDiffReport diffReport =
+          cp.getSnapshotDiffReport(fullpath, oldSnapshotName.getValue(),
+              snapshotName.getValue());
+      final String js = JsonUtil.toJsonString(diffReport);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }

+ 10 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/XmlImageVisitor.java

@@ -18,16 +18,17 @@
 package org.apache.hadoop.hdfs.tools.offlineImageViewer;
 
 import java.io.IOException;
-import java.util.LinkedList;
+import java.util.ArrayDeque;
+import java.util.Deque;
 
 import org.apache.hadoop.hdfs.util.XMLUtils;
+
 /**
  * An XmlImageVisitor walks over an fsimage structure and writes out
  * an equivalent XML document that contains the fsimage's components.
  */
 public class XmlImageVisitor extends TextWriterImageVisitor {
-  final private LinkedList<ImageElement> tagQ =
-                                          new LinkedList<ImageElement>();
+  final private Deque<ImageElement> tagQ = new ArrayDeque<>();
 
   public XmlImageVisitor(String filename) throws IOException {
     super(filename, false);
@@ -51,9 +52,10 @@ public class XmlImageVisitor extends TextWriterImageVisitor {
 
   @Override
   void leaveEnclosingElement() throws IOException {
-    if(tagQ.size() == 0)
+    if (tagQ.isEmpty()) {
       throw new IOException("Tried to exit non-existent enclosing element " +
-                "in FSImage file");
+          "in FSImage file");
+    }
 
     ImageElement element = tagQ.pop();
     write("</" + element.toString() + ">\n");
@@ -71,7 +73,7 @@ public class XmlImageVisitor extends TextWriterImageVisitor {
 
   @Override
   void visitEnclosingElement(ImageElement element) throws IOException {
-    write("<" + element.toString() + ">\n");
+    write('<' + element.toString() + ">\n");
     tagQ.push(element);
   }
 
@@ -79,12 +81,12 @@ public class XmlImageVisitor extends TextWriterImageVisitor {
   void visitEnclosingElement(ImageElement element,
       ImageElement key, String value)
        throws IOException {
-    write("<" + element.toString() + " " + key + "=\"" + value +"\">\n");
+    write('<' + element.toString() + ' ' + key + "=\"" + value +"\">\n");
     tagQ.push(element);
   }
 
   private void writeTag(String tag, String value) throws IOException {
-    write("<" + tag + ">" +
+    write('<' + tag + '>' +
         XMLUtils.mangleXmlString(value, true) + "</" + tag + ">\n");
   }
 }

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

@@ -494,4 +494,37 @@ public class JsonUtil {
     m.put("defaultStoragePolicyId", serverDefaults.getDefaultStoragePolicyId());
     return m;
   }
+
+  public static String toJsonString(SnapshotDiffReport diffReport) {
+    return toJsonString(SnapshotDiffReport.class.getSimpleName(),
+        toJsonMap(diffReport));
+  }
+
+  private static Object toJsonMap(SnapshotDiffReport diffReport) {
+    final Map<String, Object> m = new TreeMap<String, Object>();
+    m.put("snapshotRoot", diffReport.getSnapshotRoot());
+    m.put("fromSnapshot", diffReport.getFromSnapshot());
+    m.put("toSnapshot", diffReport.getLaterSnapshotName());
+    Object[] diffList = new Object[diffReport.getDiffList().size()];
+    for (int i = 0; i < diffReport.getDiffList().size(); i++) {
+      diffList[i] = toJsonMap(diffReport.getDiffList().get(i));
+    }
+    m.put("diffList", diffList);
+    return m;
+  }
+
+  private static Object toJsonMap(
+      SnapshotDiffReport.DiffReportEntry diffReportEntry) {
+    final Map<String, Object> m = new TreeMap<String, Object>();
+    m.put("type", diffReportEntry.getType());
+    if (diffReportEntry.getSourcePath() != null) {
+      m.put("sourcePath",
+          DFSUtilClient.bytes2String(diffReportEntry.getSourcePath()));
+    }
+    if (diffReportEntry.getTargetPath() != null) {
+      m.put("targetPath",
+          DFSUtilClient.bytes2String(diffReportEntry.getTargetPath()));
+    }
+    return m;
+  }
 }

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

@@ -4633,10 +4633,10 @@
 
   <property>
     <name>dfs.disk.balancer.enabled</name>
-    <value>false</value>
+    <value>true</value>
     <description>
         This enables the diskbalancer feature on a cluster. By default, disk
-      balancer is disabled.
+      balancer is enabled.
     </description>
   </property>
 
@@ -5234,4 +5234,21 @@
       is assumed.
     </description>
   </property>
+
+  <property>
+    <name>dfs.federation.router.client.thread-size</name>
+    <value>32</value>
+    <description>
+      Max threads size for the RouterClient to execute concurrent
+      requests.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.client.retry.max.attempts</name>
+    <value>3</value>
+    <description>
+      Max retry attempts for the RouterClient talking to the Router.
+    </description>
+  </property>
 </configuration>

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java

@@ -227,6 +227,50 @@ public class TestLeaseRecovery {
     assertEquals(newFileLen, expectedNewFileLen);
   }
 
+  /**
+   * Block/lease recovery should be retried with failed nodes from the second
+   * stage removed to avoid perpetual recovery failures.
+   */
+  @Test
+  public void testBlockRecoveryRetryAfterFailedRecovery() throws Exception {
+    Configuration conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+    Path file = new Path("/testBlockRecoveryRetryAfterFailedRecovery");
+    DistributedFileSystem dfs = cluster.getFileSystem();
+
+    // Create a file.
+    FSDataOutputStream out = dfs.create(file);
+    final int FILE_SIZE = 128 * 1024;
+    int count = 0;
+    while (count < FILE_SIZE) {
+      out.writeBytes("DE K9SUL");
+      count += 8;
+    }
+    out.hsync();
+
+    // Abort the original stream.
+    ((DFSOutputStream) out.getWrappedStream()).abort();
+
+    LocatedBlocks locations = cluster.getNameNodeRpc().getBlockLocations(
+        file.toString(), 0, count);
+    ExtendedBlock block = locations.get(0).getBlock();
+
+    // Finalize one replica to simulate a partial close failure.
+    cluster.getDataNodes().get(0).getFSDataset().finalizeBlock(block, false);
+    // Delete the meta file to simulate a rename/move failure.
+    cluster.deleteMeta(0, block);
+
+    // Try to recover the lease.
+    DistributedFileSystem newDfs = (DistributedFileSystem) FileSystem
+        .newInstance(cluster.getConfiguration(0));
+    count = 0;
+    while (count++ < 15 && !newDfs.recoverLease(file)) {
+      Thread.sleep(1000);
+    }
+    // The lease should have been recovered.
+    assertTrue("File should be closed", newDfs.recoverLease(file));
+  }
+
   /**
    * Recover the lease on a file and append file from another client.
    */

+ 107 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java

@@ -35,6 +35,7 @@ import java.security.PrivilegedExceptionAction;
 import java.util.List;
 import java.util.Scanner;
 
+import com.google.common.base.Supplier;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -42,6 +43,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
@@ -58,13 +60,20 @@ import org.apache.hadoop.util.ToolRunner;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 
 import com.google.common.base.Charsets;
 import com.google.common.collect.Lists;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.event.Level;
+import org.slf4j.LoggerFactory;
 
 /** A class for testing quota-related commands */
 public class TestQuota {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestQuota.class);
   
   private static Configuration conf = null;
   private static final ByteArrayOutputStream OUT_STREAM = new ByteArrayOutputStream();
@@ -77,6 +86,9 @@ public class TestQuota {
   /* set a smaller block size so that we can test with smaller space quotas */
   private static final int DEFAULT_BLOCK_SIZE = 512;
 
+  @Rule
+  public final Timeout testTestout = new Timeout(120000);
+
   @BeforeClass
   public static void setUpClass() throws Exception {
     conf = new HdfsConfiguration();
@@ -1479,6 +1491,101 @@ public class TestQuota {
         "clrSpaceQuota");
   }
 
+  @Test
+  public void testSpaceQuotaExceptionOnClose() throws Exception {
+    GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.TRACE);
+    final DFSAdmin dfsAdmin = new DFSAdmin(conf);
+    final Path dir = new Path(PathUtils.getTestPath(getClass()),
+        GenericTestUtils.getMethodName());
+    assertTrue(dfs.mkdirs(dir));
+    final String[] args = new String[] {"-setSpaceQuota", "1", dir.toString()};
+    assertEquals(0, ToolRunner.run(dfsAdmin, args));
+
+    final Path testFile = new Path(dir, "file");
+    final FSDataOutputStream stream = dfs.create(testFile);
+    stream.write("whatever".getBytes());
+    try {
+      stream.close();
+      fail("close should fail");
+    } catch (DSQuotaExceededException expected) {
+    }
+
+    assertEquals(0, cluster.getNamesystem().getNumFilesUnderConstruction());
+  }
+
+  @Test
+  public void testSpaceQuotaExceptionOnFlush() throws Exception {
+    GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(DFSClient.LOG, Level.TRACE);
+    final DFSAdmin dfsAdmin = new DFSAdmin(conf);
+    final Path dir = new Path(PathUtils.getTestPath(getClass()),
+        GenericTestUtils.getMethodName());
+    assertTrue(dfs.mkdirs(dir));
+    final String[] args = new String[] {"-setSpaceQuota", "1", dir.toString()};
+    assertEquals(0, ToolRunner.run(dfsAdmin, args));
+
+    Path testFile = new Path(dir, "file");
+    FSDataOutputStream stream = dfs.create(testFile);
+    // get the lease renewer now so we can verify it later without calling
+    // getLeaseRenewer, which will automatically add the client into it.
+    final LeaseRenewer leaseRenewer = dfs.getClient().getLeaseRenewer();
+    stream.write("whatever".getBytes());
+    try {
+      stream.hflush();
+      fail("flush should fail");
+    } catch (DSQuotaExceededException expected) {
+    }
+    // even if we close the stream in finially, it won't help.
+    try {
+      stream.close();
+      fail("close should fail too");
+    } catch (DSQuotaExceededException expected) {
+    }
+
+    GenericTestUtils.setLogLevel(LeaseRenewer.LOG, Level.TRACE);
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        LOG.info("LeaseRenewer: {}", leaseRenewer);
+        return leaseRenewer.isEmpty();
+      }
+    }, 100, 10000);
+    assertEquals(0, cluster.getNamesystem().getNumFilesUnderConstruction());
+  }
+
+  @Test
+  public void testSpaceQuotaExceptionOnAppend() throws Exception {
+    GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.TRACE);
+    final DFSAdmin dfsAdmin = new DFSAdmin(conf);
+    final Path dir = new Path(PathUtils.getTestPath(getClass()),
+        GenericTestUtils.getMethodName());
+    dfs.delete(dir, true);
+    assertTrue(dfs.mkdirs(dir));
+    final String[] args =
+        new String[] {"-setSpaceQuota", "4000", dir.toString()};
+    ToolRunner.run(dfsAdmin, args);
+
+    final Path testFile = new Path(dir, "file");
+    OutputStream stream = dfs.create(testFile);
+    stream.write("whatever".getBytes());
+    stream.close();
+
+    assertEquals(0, cluster.getNamesystem().getNumFilesUnderConstruction());
+
+    stream = dfs.append(testFile);
+    byte[] buf = AppendTestUtil.initBuffer(4096);
+    stream.write(buf);
+    try {
+      stream.close();
+      fail("close after append should fail");
+    } catch (DSQuotaExceededException expected) {
+    }
+    assertEquals(0, cluster.getNamesystem().getNumFilesUnderConstruction());
+  }
+
   private void testSetAndClearSpaceQuotaNoAccessInternal(
       final String[] args,
       final int cmdRet,

+ 13 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java

@@ -83,6 +83,7 @@ public class TestJournalNode {
     File editsDir = new File(MiniDFSCluster.getBaseDirectory() +
         File.separator + "TestJournalNode");
     FileUtil.fullyDelete(editsDir);
+    journalId = "test-journalid-" + GenericTestUtils.uniqueSequenceId();
 
     if (testName.getMethodName().equals("testJournalDirPerNameSpace")) {
       setFederationConf();
@@ -113,33 +114,33 @@ public class TestJournalNode {
       conf.setBoolean(DFSConfigKeys.DFS_JOURNALNODE_ENABLE_SYNC_KEY,
           false);
       conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
-          "qjournal://jn0:9900;jn1:9901");
+          "qjournal://jn0:9900;jn1:9901/" + journalId);
     } else if (testName.getMethodName().equals(
         "testJournalNodeSyncerNotStartWhenSyncEnabledIncorrectURI")) {
       conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
-          "qjournal://journal0\\:9900;journal1:9901");
+          "qjournal://journal0\\:9900;journal1:9901/" + journalId);
     } else if (testName.getMethodName().equals(
         "testJournalNodeSyncerNotStartWhenSyncEnabled")) {
       conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
-          "qjournal://jn0:9900;jn1:9901");
+          "qjournal://jn0:9900;jn1:9901/" + journalId);
     } else if (testName.getMethodName().equals(
         "testJournalNodeSyncwithFederationTypeConfigWithNameServiceId")) {
       conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY +".ns1",
-          "qjournal://journalnode0:9900;journalnode0:9901");
+          "qjournal://journalnode0:9900;journalnode0:9901/" + journalId);
     } else if (testName.getMethodName().equals(
         "testJournalNodeSyncwithFederationTypeConfigWithNamenodeId")) {
       conf.set(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + ".ns1", "nn1,nn2");
       conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY +".ns1" +".nn1",
-          "qjournal://journalnode0:9900;journalnode1:9901");
+          "qjournal://journalnode0:9900;journalnode1:9901/" +journalId);
       conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY +".ns1" +".nn2",
-          "qjournal://journalnode0:9900;journalnode1:9901");
+          "qjournal://journalnode0:9900;journalnode1:9901/" +journalId);
     } else if (testName.getMethodName().equals(
         "testJournalNodeSyncwithFederationTypeIncorrectConfigWithNamenodeId")) {
       conf.set(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + ".ns1", "nn1,nn2");
       conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY +".ns1" +".nn1",
-          "qjournal://journalnode0:9900;journalnode1:9901");
+          "qjournal://journalnode0:9900;journalnode1:9901/" + journalId);
       conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY +".ns1" +".nn2",
-          "qjournal://journalnode0:9902;journalnode1:9903");
+          "qjournal://journalnode0:9902;journalnode1:9903/" + journalId);
     }
     jn = new JournalNode();
     jn.setConf(conf);
@@ -161,7 +162,6 @@ public class TestJournalNode {
         journal.format(fakeNameSpaceInfo);
       }
     } else {
-      journalId = "test-journalid-" + GenericTestUtils.uniqueSequenceId();
       journal = jn.getOrCreateJournal(journalId);
       journal.format(FAKE_NSINFO);
     }
@@ -176,16 +176,16 @@ public class TestJournalNode {
     //ns1
     conf.set(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + ".ns1", "nn1,nn2");
     conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY +".ns1" +".nn1",
-        "qjournal://journalnode0:9900;journalnode1:9901/ns1");
+        "qjournal://journalnode0:9900;journalnode1:9901/test-journalid-ns1");
     conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY +".ns1" +".nn2",
-        "qjournal://journalnode0:9900;journalnode1:9901/ns2");
+        "qjournal://journalnode0:9900;journalnode1:9901/test-journalid-ns1");
 
     //ns2
     conf.set(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + ".ns2", "nn3,nn4");
     conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY +".ns2" +".nn3",
-        "qjournal://journalnode0:9900;journalnode1:9901/ns2");
+        "qjournal://journalnode0:9900;journalnode1:9901/test-journalid-ns2");
     conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY +".ns2" +".nn4",
-        "qjournal://journalnode0:9900;journalnode1:9901/ns2");
+        "qjournal://journalnode0:9900;journalnode1:9901/test-journalid-ns2");
   }
   
   @After

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

@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.NamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.RouterContext;
+import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.metrics.FederationRPCMetrics;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.MembershipNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test retry behavior of the Router RPC Client.
+ */
+public class TestRouterRPCClientRetries {
+
+  private static StateStoreDFSCluster cluster;
+  private static NamenodeContext nnContext1;
+  private static RouterContext routerContext;
+  private static MembershipNamenodeResolver resolver;
+  private static ClientProtocol routerProtocol;
+
+  @Before
+  public void setUp() throws Exception {
+    // Build and start a federated cluster
+    cluster = new StateStoreDFSCluster(false, 2);
+    Configuration routerConf = new RouterConfigBuilder()
+        .stateStore()
+        .admin()
+        .rpc()
+        .build();
+
+    cluster.addRouterOverrides(routerConf);
+    cluster.startCluster();
+    cluster.startRouters();
+    cluster.waitClusterUp();
+
+    nnContext1 = cluster.getNamenode(cluster.getNameservices().get(0), null);
+    routerContext = cluster.getRandomRouter();
+    resolver = (MembershipNamenodeResolver) routerContext.getRouter()
+        .getNamenodeResolver();
+    routerProtocol = routerContext.getClient().getNamenode();
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.stopRouter(routerContext);
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  @Test
+  public void testRetryWhenAllNameServiceDown() throws Exception {
+    // shutdown the dfs cluster
+    MiniDFSCluster dfsCluster = cluster.getCluster();
+    dfsCluster.shutdown();
+
+    // register an invalid namenode report
+    registerInvalidNameReport();
+
+    // Create a directory via the router
+    String dirPath = "/testRetryWhenClusterisDown";
+    FsPermission permission = new FsPermission("705");
+    try {
+      routerProtocol.mkdirs(dirPath, permission, false);
+      fail("Should have thrown RemoteException error.");
+    } catch (RemoteException e) {
+      String ns0 = cluster.getNameservices().get(0);
+      GenericTestUtils.assertExceptionContains(
+          "No namenode available under nameservice " + ns0, e);
+    }
+
+    // Verify the retry times, it should only retry one time.
+    FederationRPCMetrics rpcMetrics = routerContext.getRouter()
+        .getRpcServer().getRPCMetrics();
+    assertEquals(1, rpcMetrics.getProxyOpRetries());
+  }
+
+  @Test
+  public void testRetryWhenOneNameServiceDown() throws Exception {
+    // shutdown the dfs cluster
+    MiniDFSCluster dfsCluster = cluster.getCluster();
+    dfsCluster.shutdownNameNode(0);
+
+    // register an invalid namenode report
+    registerInvalidNameReport();
+
+    DFSClient client = nnContext1.getClient();
+    // Renew lease for the DFS client, it will succeed.
+    routerProtocol.renewLease(client.getClientName());
+
+    // Verify the retry times, it will retry one time for ns0.
+    FederationRPCMetrics rpcMetrics = routerContext.getRouter()
+        .getRpcServer().getRPCMetrics();
+    assertEquals(1, rpcMetrics.getProxyOpRetries());
+  }
+
+  /**
+   * Register an invalid namenode report.
+   * @throws IOException
+   */
+  private void registerInvalidNameReport() throws IOException {
+    String ns0 = cluster.getNameservices().get(0);
+    List<? extends FederationNamenodeContext> origin = resolver
+        .getNamenodesForNameserviceId(ns0);
+    FederationNamenodeContext nnInfo = origin.get(0);
+    NamenodeStatusReport report = new NamenodeStatusReport(ns0,
+        nnInfo.getNamenodeId(), nnInfo.getRpcAddress(),
+        nnInfo.getServiceAddress(), nnInfo.getLifelineAddress(),
+        nnInfo.getWebAddress());
+    report.setRegistrationValid(false);
+    assertTrue(resolver.registerNamenode(report));
+    resolver.loadCache(true);
+  }
+}

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java

@@ -75,10 +75,13 @@ public class NameNodeAdapter {
       boolean resolveLink, boolean needLocation, boolean needBlockToken)
       throws AccessControlException, UnresolvedLinkException, StandbyException,
       IOException {
+    final FSPermissionChecker pc =
+        namenode.getNamesystem().getPermissionChecker();
     namenode.getNamesystem().readLock();
     try {
       return FSDirStatAndListingOp.getFileInfo(namenode.getNamesystem()
-          .getFSDirectory(), src, resolveLink, needLocation, needBlockToken);
+          .getFSDirectory(), pc, src, resolveLink, needLocation,
+          needBlockToken);
     } finally {
       namenode.getNamesystem().readUnlock();
     }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java

@@ -75,6 +75,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doThrow;
 
 /**
@@ -415,7 +416,7 @@ public class TestAuditLogger {
 
       final FSDirectory mockedDir = Mockito.spy(dir);
       AccessControlException ex = new AccessControlException();
-      doThrow(ex).when(mockedDir).getPermissionChecker();
+      doThrow(ex).when(mockedDir).checkTraverse(any(), any(), any());
       cluster.getNamesystem().setFSDirectory(mockedDir);
       assertTrue(DummyAuditLogger.initialized);
       DummyAuditLogger.resetLogCount();

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java

@@ -53,6 +53,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import org.mockito.Mockito;
+
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
@@ -618,7 +620,7 @@ public class TestAuditLoggerWithCommands {
     final FSDirectory dir = cluster.getNamesystem().getFSDirectory();
     final FSDirectory mockedDir = Mockito.spy(dir);
     AccessControlException ex = new AccessControlException();
-    doThrow(ex).when(mockedDir).getPermissionChecker();
+    doThrow(ex).when(mockedDir).checkTraverse(any(), any(), any());
     cluster.getNamesystem().setFSDirectory(mockedDir);
     String aceGetAclStatus =
         ".*allowed=false.*ugi=theDoctor.*cmd=getAclStatus.*";

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java

@@ -22,6 +22,8 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DiffList;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DiffListByArrayList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
@@ -156,11 +158,11 @@ public class TestTruncateQuotaUpdate {
     FileDiff diff = mock(FileDiff.class);
     when(diff.getBlocks()).thenReturn(blocks);
     FileDiffList diffList = new FileDiffList();
-    Whitebox.setInternalState(diffList, "diffs", new ArrayList<FileDiff>());
+    Whitebox.setInternalState(diffList, "diffs", new DiffListByArrayList<>(0));
     @SuppressWarnings("unchecked")
-    ArrayList<FileDiff> diffs = ((ArrayList<FileDiff>)Whitebox.getInternalState
-        (diffList, "diffs"));
-    diffs.add(diff);
+    DiffList<FileDiff> diffs = (DiffList<FileDiff>)Whitebox.getInternalState(
+        diffList, "diffs");
+    diffs.addFirst(diff);
     FileWithSnapshotFeature sf = new FileWithSnapshotFeature(diffList);
     file.addFeature(sf);
   }

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

@@ -28,11 +28,15 @@ import java.util.Date;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Random;
+import java.util.List;
+import java.util.ArrayList;
 
+import org.apache.commons.collections.list.TreeList;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -40,14 +44,17 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.apache.hadoop.hdfs.client.impl.SnapshotDiffReportGenerator;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.Time;
 import org.junit.After;
 import org.junit.Assert;
@@ -1409,4 +1416,127 @@ public class TestSnapshotDiffReport {
         new DiffReportEntry(DiffType.DELETE,
             DFSUtil.string2Bytes("dir3/file3")));
   }
+
+  private void verifyDiffReportForGivenReport(Path dirPath, String from,
+      String to, SnapshotDiffReport report, DiffReportEntry... entries)
+      throws IOException {
+    // reverse the order of from and to
+    SnapshotDiffReport inverseReport =
+        hdfs.getSnapshotDiffReport(dirPath, to, from);
+    LOG.info(report.toString());
+    LOG.info(inverseReport.toString() + "\n");
+
+    assertEquals(entries.length, report.getDiffList().size());
+    assertEquals(entries.length, inverseReport.getDiffList().size());
+
+    for (DiffReportEntry entry : entries) {
+      if (entry.getType() == DiffType.MODIFY) {
+        assertTrue(report.getDiffList().contains(entry));
+        assertTrue(inverseReport.getDiffList().contains(entry));
+      } else if (entry.getType() == DiffType.DELETE) {
+        assertTrue(report.getDiffList().contains(entry));
+        assertTrue(inverseReport.getDiffList().contains(
+            new DiffReportEntry(DiffType.CREATE, entry.getSourcePath())));
+      } else if (entry.getType() == DiffType.CREATE) {
+        assertTrue(report.getDiffList().contains(entry));
+        assertTrue(inverseReport.getDiffList().contains(
+            new DiffReportEntry(DiffType.DELETE, entry.getSourcePath())));
+      }
+    }
+  }
+
+  @Test
+  public void testSnapshotDiffReportRemoteIterator() throws Exception {
+    final Path root = new Path("/");
+    hdfs.mkdirs(root);
+    for (int i = 1; i <= 3; i++) {
+      final Path path = new Path(root, "dir" + i);
+      hdfs.mkdirs(path);
+    }
+    for (int i = 1; i <= 3; i++) {
+      final Path path = new Path(root, "dir" + i);
+      for (int j = 1; j < 4; j++) {
+        final Path file = new Path(path, "file" + j);
+        DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, SEED);
+      }
+    }
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
+    Path targetDir = new Path(root, "dir4");
+    //create directory dir4
+    hdfs.mkdirs(targetDir);
+    //moves files from dir1 to dir4
+    Path path = new Path(root, "dir1");
+    for (int j = 1; j < 4; j++) {
+      final Path srcPath = new Path(path, "file" + j);
+      final Path targetPath = new Path(targetDir, "file" + j);
+      hdfs.rename(srcPath, targetPath);
+    }
+    targetDir = new Path(root, "dir3");
+    //overwrite existing files in dir3 from files in dir1
+    path = new Path(root, "dir2");
+    for (int j = 1; j < 4; j++) {
+      final Path srcPath = new Path(path, "file" + j);
+      final Path targetPath = new Path(targetDir, "file" + j);
+      hdfs.rename(srcPath, targetPath, Rename.OVERWRITE);
+    }
+    final Path pathToRename = new Path(root, "dir2");
+    //move dir2 inside dir3
+    hdfs.rename(pathToRename, targetDir);
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s1");
+    RemoteIterator<SnapshotDiffReportListing> iterator =
+        hdfs.snapshotDiffReportListingRemoteIterator(root, "s0", "s1");
+    SnapshotDiffReportGenerator snapshotDiffReport;
+    List<SnapshotDiffReportListing.DiffReportListingEntry> modifiedList =
+        new TreeList();
+    List<SnapshotDiffReportListing.DiffReportListingEntry> createdList =
+        new ChunkedArrayList<>();
+    List<SnapshotDiffReportListing.DiffReportListingEntry> deletedList =
+        new ChunkedArrayList<>();
+    SnapshotDiffReportListing report = null;
+    List<SnapshotDiffReportListing> reportList = new ArrayList<>();
+    while (iterator.hasNext()) {
+      report = iterator.next();
+      reportList.add(report);
+      modifiedList.addAll(report.getModifyList());
+      createdList.addAll(report.getCreateList());
+      deletedList.addAll(report.getDeleteList());
+    }
+    try {
+      iterator.next();
+    } catch (Exception e) {
+      Assert.assertTrue(
+          e.getMessage().contains("No more entry in SnapshotDiffReport for /"));
+    }
+    Assert.assertNotEquals(0, reportList.size());
+    // generate the snapshotDiffReport and Verify
+    snapshotDiffReport = new SnapshotDiffReportGenerator("/", "s0", "s1",
+        report.getIsFromEarlier(), modifiedList, createdList, deletedList);
+    verifyDiffReportForGivenReport(root, "s0", "s1",
+        snapshotDiffReport.generateReport(),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("dir4")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir2"),
+            DFSUtil.string2Bytes("dir3/dir2")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir1")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir1/file1"),
+            DFSUtil.string2Bytes("dir4/file1")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir1/file2"),
+            DFSUtil.string2Bytes("dir4/file2")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir1/file3"),
+            DFSUtil.string2Bytes("dir4/file3")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir2")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir2/file1"),
+            DFSUtil.string2Bytes("dir3/file1")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir2/file2"),
+            DFSUtil.string2Bytes("dir3/file2")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir2/file3"),
+            DFSUtil.string2Bytes("dir3/file3")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir3")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("dir3/file1")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("dir3/file1")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("dir3/file3")));
+  }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/web/resources/TestWebHdfsDataLocality.java

@@ -243,7 +243,7 @@ public class TestWebHdfsDataLocality {
     NameNode nn = mock(NameNode.class);
     when(nn.getNamesystem()).thenReturn(null);
     exception.expect(IOException.class);
-    exception.expectMessage("Namesystem has not been intialized yet.");
+    exception.expectMessage("Namesystem has not been initialized yet.");
     NamenodeWebHdfsMethods.chooseDatanode(nn, "/path", PutOpParam.Op.CREATE, 0,
         DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT, null, LOCALHOST, null);
   }

+ 71 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java

@@ -73,6 +73,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -82,6 +83,9 @@ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import static org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+import static org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -579,7 +583,7 @@ public class TestWebHDFS {
   }
 
   /**
-   * Test snapshot creation through WebHdfs
+   * Test snapshot creation through WebHdfs.
    */
   @Test
   public void testWebHdfsCreateSnapshot() throws Exception {
@@ -665,6 +669,72 @@ public class TestWebHDFS {
     }
   }
 
+  /**
+   * Test snapshot diff through WebHdfs
+   */
+  @Test
+  public void testWebHdfsSnapshotDiff() throws Exception {
+    MiniDFSCluster cluster = null;
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final WebHdfsFileSystem webHdfs = WebHdfsTestUtil
+          .getWebHdfsFileSystem(conf, WebHdfsConstants.WEBHDFS_SCHEME);
+
+      final Path foo = new Path("/foo");
+      dfs.mkdirs(foo);
+      Path file0 = new Path(foo, "file0");
+      DFSTestUtil.createFile(dfs, file0, 100, (short) 1, 0);
+      Path file1 = new Path(foo, "file1");
+      DFSTestUtil.createFile(dfs, file1, 100, (short) 1, 0);
+      Path file2 = new Path(foo, "file2");
+      DFSTestUtil.createFile(dfs, file2, 100, (short) 1, 0);
+
+      dfs.allowSnapshot(foo);
+      webHdfs.createSnapshot(foo, "s1");
+      final Path s1path = SnapshotTestHelper.getSnapshotRoot(foo, "s1");
+      Assert.assertTrue(webHdfs.exists(s1path));
+
+      Path file3 = new Path(foo, "file3");
+      DFSTestUtil.createFile(dfs, file3, 100, (short) 1, 0);
+      DFSTestUtil.appendFile(dfs, file0, 100);
+      dfs.delete(file1, false);
+      Path file4 = new Path(foo, "file4");
+      dfs.rename(file2, file4);
+
+      webHdfs.createSnapshot(foo, "s2");
+      SnapshotDiffReport diffReport =
+          webHdfs.getSnapshotDiffReport(foo, "s1", "s2");
+
+      Assert.assertEquals("/foo", diffReport.getSnapshotRoot());
+      Assert.assertEquals("s1", diffReport.getFromSnapshot());
+      Assert.assertEquals("s2", diffReport.getLaterSnapshotName());
+      DiffReportEntry entry0 =
+          new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes(""));
+      DiffReportEntry entry1 =
+          new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("file0"));
+      DiffReportEntry entry2 =
+          new DiffReportEntry(DiffType.DELETE, DFSUtil.string2Bytes("file1"));
+      DiffReportEntry entry3 =
+          new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("file2"),
+              DFSUtil.string2Bytes("file4"));
+      DiffReportEntry entry4 =
+          new DiffReportEntry(DiffType.CREATE, DFSUtil.string2Bytes("file3"));
+      Assert.assertTrue(diffReport.getDiffList().contains(entry0));
+      Assert.assertTrue(diffReport.getDiffList().contains(entry1));
+      Assert.assertTrue(diffReport.getDiffList().contains(entry2));
+      Assert.assertTrue(diffReport.getDiffList().contains(entry3));
+      Assert.assertTrue(diffReport.getDiffList().contains(entry4));
+      Assert.assertEquals(diffReport.getDiffList().size(), 5);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   @Test
   public void testWebHdfsCreateNonRecursive() throws IOException, URISyntaxException {
     MiniDFSCluster cluster = null;

+ 10 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java

@@ -370,17 +370,22 @@ public class TaskAttemptListenerImpl extends CompositeService
         TypeConverter.toYarn(taskAttemptID);
 
     AMFeedback feedback = new AMFeedback();
+    feedback.setTaskFound(true);
+
     AtomicReference<TaskAttemptStatus> lastStatusRef =
         attemptIdToStatus.get(yarnAttemptID);
     if (lastStatusRef == null) {
-      LOG.error("Status update was called with illegal TaskAttemptId: "
-          + yarnAttemptID);
-      feedback.setTaskFound(false);
+      // The task is not known, but it could be in the process of tearing
+      // down gracefully or receiving a thread dump signal. Tolerate unknown
+      // tasks as long as they have unregistered recently.
+      if (!taskHeartbeatHandler.hasRecentlyUnregistered(yarnAttemptID)) {
+        LOG.error("Status update was called with illegal TaskAttemptId: "
+            + yarnAttemptID);
+        feedback.setTaskFound(false);
+      }
       return feedback;
     }
 
-    feedback.setTaskFound(true);
-
     // Propagating preemption to the task if TASK_PREEMPTION is enabled
     if (getConfig().getBoolean(MRJobConfig.TASK_PREEMPTION, false)
         && preemptionPolicy.isPreempted(yarnAttemptID)) {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -1255,7 +1255,7 @@ public class MRAppMaster extends CompositeService {
 
     boolean initFailed = false;
     if (!errorHappenedShutDown) {
-      // create a job event for job intialization
+      // create a job event for job initialization
       JobEvent initJobEvent = new JobEvent(job.getID(), JobEventType.JOB_INIT);
       // Send init to the job (this does NOT trigger job execution)
       // This is a synchronous call, not an event through dispatcher. We want

+ 47 - 20
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java

@@ -71,12 +71,14 @@ public class TaskHeartbeatHandler extends AbstractService {
   private Thread lostTaskCheckerThread;
   private volatile boolean stopped;
   private long taskTimeOut;
+  private long unregisterTimeOut;
   private int taskTimeOutCheckInterval = 30 * 1000; // 30 seconds.
 
   private final EventHandler eventHandler;
   private final Clock clock;
   
   private ConcurrentMap<TaskAttemptId, ReportTime> runningAttempts;
+  private ConcurrentMap<TaskAttemptId, ReportTime> recentlyUnregisteredAttempts;
 
   public TaskHeartbeatHandler(EventHandler eventHandler, Clock clock,
       int numThreads) {
@@ -85,6 +87,8 @@ public class TaskHeartbeatHandler extends AbstractService {
     this.clock = clock;
     runningAttempts =
       new ConcurrentHashMap<TaskAttemptId, ReportTime>(16, 0.75f, numThreads);
+    recentlyUnregisteredAttempts =
+        new ConcurrentHashMap<TaskAttemptId, ReportTime>(16, 0.75f, numThreads);
   }
 
   @Override
@@ -92,6 +96,8 @@ public class TaskHeartbeatHandler extends AbstractService {
     super.serviceInit(conf);
     taskTimeOut = conf.getLong(
         MRJobConfig.TASK_TIMEOUT, MRJobConfig.DEFAULT_TASK_TIMEOUT_MILLIS);
+    unregisterTimeOut = conf.getLong(MRJobConfig.TASK_EXIT_TIMEOUT,
+        MRJobConfig.TASK_EXIT_TIMEOUT_DEFAULT);
 
     // enforce task timeout is at least twice as long as task report interval
     long taskProgressReportIntervalMillis = MRJobConfUtil.
@@ -140,6 +146,12 @@ public class TaskHeartbeatHandler extends AbstractService {
 
   public void unregister(TaskAttemptId attemptID) {
     runningAttempts.remove(attemptID);
+    recentlyUnregisteredAttempts.put(attemptID,
+        new ReportTime(clock.getTime()));
+  }
+
+  public boolean hasRecentlyUnregistered(TaskAttemptId attemptID) {
+    return recentlyUnregisteredAttempts.containsKey(attemptID);
   }
 
   private class PingChecker implements Runnable {
@@ -147,27 +159,9 @@ public class TaskHeartbeatHandler extends AbstractService {
     @Override
     public void run() {
       while (!stopped && !Thread.currentThread().isInterrupted()) {
-        Iterator<Map.Entry<TaskAttemptId, ReportTime>> iterator =
-            runningAttempts.entrySet().iterator();
-
-        // avoid calculating current time everytime in loop
         long currentTime = clock.getTime();
-
-        while (iterator.hasNext()) {
-          Map.Entry<TaskAttemptId, ReportTime> entry = iterator.next();
-          boolean taskTimedOut = (taskTimeOut > 0) &&
-              (currentTime > (entry.getValue().getLastProgress() + taskTimeOut));
-           
-          if(taskTimedOut) {
-            // task is lost, remove from the list and raise lost event
-            iterator.remove();
-            eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(entry
-                .getKey(), "AttemptID:" + entry.getKey().toString()
-                + " Timed out after " + taskTimeOut / 1000 + " secs"));
-            eventHandler.handle(new TaskAttemptEvent(entry.getKey(),
-                TaskAttemptEventType.TA_TIMED_OUT));
-          }
-        }
+        checkRunning(currentTime);
+        checkRecentlyUnregistered(currentTime);
         try {
           Thread.sleep(taskTimeOutCheckInterval);
         } catch (InterruptedException e) {
@@ -176,6 +170,39 @@ public class TaskHeartbeatHandler extends AbstractService {
         }
       }
     }
+
+    private void checkRunning(long currentTime) {
+      Iterator<Map.Entry<TaskAttemptId, ReportTime>> iterator =
+          runningAttempts.entrySet().iterator();
+
+      while (iterator.hasNext()) {
+        Map.Entry<TaskAttemptId, ReportTime> entry = iterator.next();
+        boolean taskTimedOut = (taskTimeOut > 0) &&
+            (currentTime > (entry.getValue().getLastProgress() + taskTimeOut));
+
+        if(taskTimedOut) {
+          // task is lost, remove from the list and raise lost event
+          iterator.remove();
+          eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(entry
+              .getKey(), "AttemptID:" + entry.getKey().toString()
+              + " Timed out after " + taskTimeOut / 1000 + " secs"));
+          eventHandler.handle(new TaskAttemptEvent(entry.getKey(),
+              TaskAttemptEventType.TA_TIMED_OUT));
+        }
+      }
+    }
+
+    private void checkRecentlyUnregistered(long currentTime) {
+      Iterator<ReportTime> iterator =
+          recentlyUnregisteredAttempts.values().iterator();
+      while (iterator.hasNext()) {
+        ReportTime unregisteredTime = iterator.next();
+        if (currentTime >
+            unregisteredTime.getLastProgress() + unregisterTimeOut) {
+          iterator.remove();
+        }
+      }
+    }
   }
 
   @VisibleForTesting

+ 50 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java

@@ -17,6 +17,7 @@
 */
 package org.apache.hadoop.mapred;
 
+import com.google.common.base.Supplier;
 import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapreduce.checkpoint.EnumCounter;
 
@@ -51,11 +52,13 @@ import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
 import org.apache.hadoop.mapreduce.v2.app.rm.preemption.CheckpointAMPreemptionPolicy;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.junit.After;
 import org.junit.Test;
@@ -488,14 +491,57 @@ public class TestTaskAttemptListenerImpl {
   }
 
   @Test
-  public void testStatusUpdateFromUnregisteredTask()
-      throws IOException, InterruptedException{
+  public void testStatusUpdateFromUnregisteredTask() throws Exception {
     configureMocks();
-    startListener(false);
+    ControlledClock clock = new ControlledClock();
+    clock.setTime(0);
+    doReturn(clock).when(appCtx).getClock();
+
+    final TaskAttemptListenerImpl tal = new TaskAttemptListenerImpl(appCtx,
+        secret, rmHeartbeatHandler, policy) {
+      @Override
+      protected void startRpcServer() {
+        // Empty
+      }
+      @Override
+      protected void stopRpcServer() {
+        // Empty
+      }
+    };
 
-    AMFeedback feedback = listener.statusUpdate(attemptID, firstReduceStatus);
+    Configuration conf = new Configuration();
+    conf.setLong(MRJobConfig.TASK_TIMEOUT_CHECK_INTERVAL_MS, 1);
+    tal.init(conf);
+    tal.start();
 
+    AMFeedback feedback = tal.statusUpdate(attemptID, firstReduceStatus);
     assertFalse(feedback.getTaskFound());
+    tal.registerPendingTask(task, wid);
+    tal.registerLaunchedTask(attemptId, wid);
+    feedback = tal.statusUpdate(attemptID, firstReduceStatus);
+    assertTrue(feedback.getTaskFound());
+
+    // verify attempt is still reported as found if recently unregistered
+    tal.unregister(attemptId, wid);
+    feedback = tal.statusUpdate(attemptID, firstReduceStatus);
+    assertTrue(feedback.getTaskFound());
+
+    // verify attempt is not found if not recently unregistered
+    long unregisterTimeout = conf.getLong(MRJobConfig.TASK_EXIT_TIMEOUT,
+        MRJobConfig.TASK_EXIT_TIMEOUT_DEFAULT);
+    clock.setTime(unregisterTimeout + 1);
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        try {
+          AMFeedback response =
+              tal.statusUpdate(attemptID, firstReduceStatus);
+          return !response.getTaskFound();
+        } catch (Exception e) {
+          throw new RuntimeException("status update failed", e);
+        }
+      }
+    }, 10, 10000);
   }
 
   private void configureMocks() {

+ 38 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java

@@ -23,6 +23,7 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
+import com.google.common.base.Supplier;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -30,10 +31,12 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.junit.Assert;
 import org.junit.Test;
@@ -105,6 +108,41 @@ public class TestTaskHeartbeatHandler {
     verifyTaskTimeoutConfig(conf, expectedTimeout);
   }
 
+  @Test
+  public void testTaskUnregistered() throws Exception {
+    EventHandler mockHandler = mock(EventHandler.class);
+    ControlledClock clock = new ControlledClock();
+    clock.setTime(0);
+    final TaskHeartbeatHandler hb =
+        new TaskHeartbeatHandler(mockHandler, clock, 1);
+    Configuration conf = new Configuration();
+    conf.setInt(MRJobConfig.TASK_TIMEOUT_CHECK_INTERVAL_MS, 1);
+    hb.init(conf);
+    hb.start();
+    try {
+      ApplicationId appId = ApplicationId.newInstance(0l, 5);
+      JobId jobId = MRBuilderUtils.newJobId(appId, 4);
+      TaskId tid = MRBuilderUtils.newTaskId(jobId, 3, TaskType.MAP);
+      final TaskAttemptId taid = MRBuilderUtils.newTaskAttemptId(tid, 2);
+      Assert.assertFalse(hb.hasRecentlyUnregistered(taid));
+      hb.register(taid);
+      Assert.assertFalse(hb.hasRecentlyUnregistered(taid));
+      hb.unregister(taid);
+      Assert.assertTrue(hb.hasRecentlyUnregistered(taid));
+      long unregisterTimeout = conf.getLong(MRJobConfig.TASK_EXIT_TIMEOUT,
+          MRJobConfig.TASK_EXIT_TIMEOUT_DEFAULT);
+      clock.setTime(unregisterTimeout + 1);
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          return !hb.hasRecentlyUnregistered(taid);
+        }
+      }, 10, 10000);
+    } finally {
+      hb.stop();
+    }
+  }
+
   /**
    * Test if task timeout is set properly in response to the configuration of
    * the task progress report interval.

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java

@@ -98,7 +98,7 @@ public class JobHistory extends AbstractService implements HistoryContext {
     try {
       hsManager.initExisting();
     } catch (IOException e) {
-      throw new YarnRuntimeException("Failed to intialize existing directories", e);
+      throw new YarnRuntimeException("Failed to initialize existing directories", e);
     }
 
     storage = createHistoryStorage();

+ 39 - 45
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java

@@ -18,18 +18,6 @@
 
 package org.apache.hadoop.mapred;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.ByteArrayInputStream;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.zip.Inflater;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
@@ -42,16 +30,26 @@ import org.apache.hadoop.io.compress.zlib.ZlibFactory;
 import org.apache.hadoop.util.LineReader;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.junit.After;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@Ignore
+import java.io.ByteArrayInputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.zip.Inflater;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test class for concatenated {@link CompressionInputStream}.
+ */
 public class TestConcatenatedCompressedInput {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestConcatenatedCompressedInput.class);
-  private static int MAX_LENGTH = 10000;
   private static JobConf defaultConf = new JobConf();
   private static FileSystem localFs = null;
 
@@ -85,13 +83,15 @@ public class TestConcatenatedCompressedInput {
   public void after() {
     ZlibFactory.loadNativeZLib();
   }
+
+  private static final String DEFAULT_WORK_DIR = "target/test-classes/testdata";
   private static Path workDir = localFs.makeQualified(new Path(
-      System.getProperty("test.build.data", "/tmp"),
+      System.getProperty("test.build.data", DEFAULT_WORK_DIR),
       "TestConcatenatedCompressedInput"));
 
   private static LineReader makeStream(String str) throws IOException {
-    return new LineReader(new ByteArrayInputStream(str.getBytes("UTF-8")),
-                          defaultConf);
+    return new LineReader(new ByteArrayInputStream(
+            str.getBytes("UTF-8")), defaultConf);
   }
 
   private static void writeFile(FileSystem fs, Path name,
@@ -190,7 +190,8 @@ public class TestConcatenatedCompressedInput {
 
     // copy prebuilt (correct!) version of concat.gz to HDFS
     final String fn = "concat" + gzip.getDefaultExtension();
-    Path fnLocal = new Path(System.getProperty("test.concat.data", "/tmp"), fn);
+    Path fnLocal = new Path(
+            System.getProperty("test.concat.data", DEFAULT_WORK_DIR), fn);
     Path fnHDFS  = new Path(workDir, fn);
     localFs.copyFromLocalFile(fnLocal, fnHDFS);
 
@@ -227,7 +228,7 @@ public class TestConcatenatedCompressedInput {
   @Test
   public void testPrototypeInflaterGzip() throws IOException {
     CompressionCodec gzip = new GzipCodec();  // used only for file extension
-    localFs.delete(workDir, true);            // localFs = FileSystem instance
+    localFs.delete(workDir, true); // localFs = FileSystem instance
 
     System.out.println(COLOR_BR_BLUE + "testPrototypeInflaterGzip() using " +
       "non-native/Java Inflater and manual gzip header/trailer parsing" +
@@ -235,7 +236,8 @@ public class TestConcatenatedCompressedInput {
 
     // copy prebuilt (correct!) version of concat.gz to HDFS
     final String fn = "concat" + gzip.getDefaultExtension();
-    Path fnLocal = new Path(System.getProperty("test.concat.data", "/tmp"), fn);
+    Path fnLocal = new Path(
+            System.getProperty("test.concat.data", DEFAULT_WORK_DIR), fn);
     Path fnHDFS  = new Path(workDir, fn);
     localFs.copyFromLocalFile(fnLocal, fnHDFS);
 
@@ -326,14 +328,16 @@ public class TestConcatenatedCompressedInput {
 
     // copy single-member test file to HDFS
     String fn1 = "testConcatThenCompress.txt" + gzip.getDefaultExtension();
-    Path fnLocal1 = new Path(System.getProperty("test.concat.data","/tmp"),fn1);
+    Path fnLocal1 = new Path(
+            System.getProperty("test.concat.data", DEFAULT_WORK_DIR), fn1);
     Path fnHDFS1  = new Path(workDir, fn1);
     localFs.copyFromLocalFile(fnLocal1, fnHDFS1);
 
     // copy multiple-member test file to HDFS
     // (actually in "seekable gzip" format, a la JIRA PIG-42)
     String fn2 = "testCompressThenConcat.txt" + gzip.getDefaultExtension();
-    Path fnLocal2 = new Path(System.getProperty("test.concat.data","/tmp"),fn2);
+    Path fnLocal2 = new Path(
+            System.getProperty("test.concat.data", DEFAULT_WORK_DIR), fn2);
     Path fnHDFS2  = new Path(workDir, fn2);
     localFs.copyFromLocalFile(fnLocal2, fnHDFS2);
 
@@ -439,7 +443,8 @@ public class TestConcatenatedCompressedInput {
     InputSplit[] splits = format.getSplits(jConf, 100);
     assertEquals("compressed splits == 2", 2, splits.length);
     FileSplit tmp = (FileSplit) splits[0];
-    if (tmp.getPath().getName().equals("testCompressThenConcat.txt.gz")) {
+    if (tmp.getPath()
+            .getName().equals("testdata/testCompressThenConcat.txt.gz")) {
       System.out.println("  (swapping)");
       splits[0] = splits[1];
       splits[1] = tmp;
@@ -481,7 +486,8 @@ public class TestConcatenatedCompressedInput {
 
     // copy prebuilt (correct!) version of concat.bz2 to HDFS
     final String fn = "concat" + bzip2.getDefaultExtension();
-    Path fnLocal = new Path(System.getProperty("test.concat.data", "/tmp"), fn);
+    Path fnLocal = new Path(
+            System.getProperty("test.concat.data", DEFAULT_WORK_DIR), fn);
     Path fnHDFS  = new Path(workDir, fn);
     localFs.copyFromLocalFile(fnLocal, fnHDFS);
 
@@ -531,13 +537,15 @@ public class TestConcatenatedCompressedInput {
 
     // copy single-member test file to HDFS
     String fn1 = "testConcatThenCompress.txt" + bzip2.getDefaultExtension();
-    Path fnLocal1 = new Path(System.getProperty("test.concat.data","/tmp"),fn1);
+    Path fnLocal1 = new Path(
+            System.getProperty("test.concat.data", DEFAULT_WORK_DIR), fn1);
     Path fnHDFS1  = new Path(workDir, fn1);
     localFs.copyFromLocalFile(fnLocal1, fnHDFS1);
 
     // copy multiple-member test file to HDFS
     String fn2 = "testCompressThenConcat.txt" + bzip2.getDefaultExtension();
-    Path fnLocal2 = new Path(System.getProperty("test.concat.data","/tmp"),fn2);
+    Path fnLocal2 = new Path(
+            System.getProperty("test.concat.data", DEFAULT_WORK_DIR), fn2);
     Path fnHDFS2  = new Path(workDir, fn2);
     localFs.copyFromLocalFile(fnLocal2, fnHDFS2);
 
@@ -549,21 +557,6 @@ public class TestConcatenatedCompressedInput {
     assertEquals("concat bytes available", 2567, in1.available());
     assertEquals("concat bytes available", 3056, in2.available());
 
-/*
-    // FIXME
-    // The while-loop below dies at the beginning of the 2nd concatenated
-    // member (after 17 lines successfully read) with:
-    //
-    //   java.io.IOException: bad block header
-    //   at org.apache.hadoop.io.compress.bzip2.CBZip2InputStream.initBlock(
-    //   CBZip2InputStream.java:527)
-    //
-    // It is not critical to concatenated-gzip support, HADOOP-6835, so it's
-    // simply commented out for now (and HADOOP-6852 filed).  If and when the
-    // latter issue is resolved--perhaps by fixing an error here--this code
-    // should be reenabled.  Note that the doMultipleBzip2BufferSizes() test
-    // below uses the same testCompressThenConcat.txt.bz2 file but works fine.
-
     CompressionInputStream cin2 = bzip2.createInputStream(in2);
     LineReader in = new LineReader(cin2);
     Text out = new Text();
@@ -578,7 +571,6 @@ public class TestConcatenatedCompressedInput {
                  5346, totalBytes);
     assertEquals("total uncompressed lines in concatenated test file",
                  84, lineNum);
- */
 
     // test CBZip2InputStream with lots of different input-buffer sizes
     doMultipleBzip2BufferSizes(jobConf);
@@ -645,7 +637,8 @@ public class TestConcatenatedCompressedInput {
 
   // this tests both files (testCompressThenConcat, testConcatThenCompress); all
   // should work with existing Java bzip2 decoder and any future native version
-  private static void doSingleBzip2BufferSize(JobConf jConf) throws IOException {
+  private static void doSingleBzip2BufferSize(JobConf jConf)
+          throws IOException {
     TextInputFormat format = new TextInputFormat();
     format.configure(jConf);
     format.setMinSplitSize(5500);  // work around 256-byte/22-splits issue
@@ -654,7 +647,8 @@ public class TestConcatenatedCompressedInput {
     InputSplit[] splits = format.getSplits(jConf, 100);
     assertEquals("compressed splits == 2", 2, splits.length);
     FileSplit tmp = (FileSplit) splits[0];
-    if (tmp.getPath().getName().equals("testCompressThenConcat.txt.gz")) {
+    if (tmp.getPath()
+            .getName().equals("testdata/testCompressThenConcat.txt.gz")) {
       System.out.println("  (swapping)");
       splits[0] = splits[1];
       splits[1] = tmp;

+ 8 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java

@@ -183,6 +183,14 @@ public class TestTextInputFormat {
     // corner case when we have byte alignment and position of stream are same
     verifyPartitions(471507, 218, file, codec, conf);
     verifyPartitions(473608, 110, file, codec, conf);
+
+    // corner case when split size is small and position of stream is before
+    // the first BZip2 block
+    verifyPartitions(100, 20, file, codec, conf);
+    verifyPartitions(100, 25, file, codec, conf);
+    verifyPartitions(100, 30, file, codec, conf);
+    verifyPartitions(100, 50, file, codec, conf);
+    verifyPartitions(100, 100, file, codec, conf);
   }
 
   // Test a corner case when position of stream is right after BZip2 marker

BIN
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/resources/testdata/concat.bz2


BIN
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/resources/testdata/concat.gz


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