Browse Source

Merge branch 'trunk' into HDFS-7240

 Conflicts:
	hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
Anu Engineer 7 years ago
parent
commit
5f16cdbd28
100 changed files with 4221 additions and 762 deletions
  1. 49 22
      dev-support/bin/create-release
  2. 6 1
      hadoop-common-project/hadoop-common/src/main/bin/hadoop
  3. 36 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderTokenIssuer.java
  4. 18 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/package-info.java
  5. 41 20
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
  6. 15 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  7. 39 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java
  8. 70 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
  9. 34 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
  10. 5 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java
  11. 34 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
  12. 85 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
  13. 22 7
      hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
  14. 1 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
  15. 1 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
  16. 86 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java
  17. 31 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/permission/TestFsPermission.java
  18. 128 16
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
  19. 8 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
  20. 126 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/StatUtils.java
  21. 36 0
      hadoop-common-project/hadoop-common/src/test/scripts/hadoop_shell_execname.bats
  22. 31 0
      hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
  23. 21 112
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  24. 53 30
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
  25. 51 24
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  26. 190 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java
  27. 4 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
  28. 262 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/SnapshotDiffReportGenerator.java
  29. 29 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  30. 283 247
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
  31. 159 49
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
  32. 180 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsNamedFileStatus.java
  33. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
  34. 160 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReportListing.java
  35. 18 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/package-info.java
  36. 24 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  37. 162 22
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  38. 36 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  39. 12 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
  40. 26 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
  41. 106 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestHdfsFileStatusMethods.java
  42. 1 4
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
  43. 0 7
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  44. 1 0
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  45. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
  46. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  47. 6 38
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  48. 22 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  49. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
  50. 44 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  51. 10 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReconstructionWork.java
  52. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java
  53. 143 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java
  54. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java
  55. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/ActiveNamenodeResolver.java
  56. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java
  57. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
  58. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
  59. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
  60. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java
  61. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
  62. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java
  63. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
  64. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
  65. 11 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
  66. 2 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/CachedRecordStore.java
  67. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java
  68. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
  69. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java
  70. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java
  71. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreZooKeeperImpl.java
  72. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MembershipStoreImpl.java
  73. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
  74. 24 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
  75. 21 26
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
  76. 94 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  77. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  78. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  79. 135 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
  80. 207 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotDiffListingInfo.java
  81. 28 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
  82. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
  83. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  84. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  85. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
  86. 38 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
  87. 23 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithKMS.java
  88. 8 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java
  89. 103 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
  90. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java
  91. 87 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java
  92. 107 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
  93. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
  94. 41 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
  95. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
  96. 19 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java
  97. 116 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
  98. 36 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/scripts/hadoop_shell_execname.bats
  99. 7 1
      hadoop-mapreduce-project/bin/mapred
  100. 65 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java

+ 49 - 22
dev-support/bin/create-release

@@ -202,6 +202,8 @@ function set_defaults
     | head -1 \
     | sed  -e 's|^ *<version>||' -e 's|</version>.*$||')
 
+  DEPLOY=false
+
   DOCKER=false
   DOCKERCACHE=false
   DOCKERFILE="${BASEDIR}/dev-support/docker/Dockerfile"
@@ -283,6 +285,7 @@ function usage
 {
   echo "--artifactsdir=[path]   Path to use to store release bits"
   echo "--asfrelease            Make an ASF release"
+  echo "--deploy                Deploy Maven artifacts using ~/.m2/settings.xml"
   echo "--docker                Use Hadoop's Dockerfile for guaranteed environment"
   echo "--dockercache           Use a Docker-private maven cache"
   echo "--logdir=[path]         Path to store logs"
@@ -304,10 +307,14 @@ function option_parse
         ASFRELEASE=true
         NATIVE=true
         SIGN=true
+        DEPLOY=true
       ;;
       --artifactsdir=*)
         ARTIFACTS_DIR=${i#*=}
       ;;
+      --deploy)
+        DEPLOY=true
+      ;;
       --docker)
         DOCKER=true
       ;;
@@ -360,6 +367,11 @@ function option_parse
     fi
   fi
 
+  if [[ "${DEPLOY}" = true && ! -f "${HOME}/.m2/settings.xml" ]]; then
+    hadoop_error "ERROR: No ~/.m2/settings.xml file, cannot deploy Maven artifacts."
+    exit 1
+  fi
+
   DOCKERCMD=$(command -v docker)
   if [[ "${DOCKER}" = true && -z "${DOCKERCMD}" ]]; then
       hadoop_error "ERROR: docker binary not found. Disabling docker mode."
@@ -452,6 +464,11 @@ function dockermode
     extrad+=("-v" "${lines}:${lines}")
   fi
 
+  if [[ "${DEPLOY}" = true ]]; then
+    modp+=("--deploy")
+    extrad+=("-v" "${HOME}/.m2/settings.xml:/home/${user_name}/.m2/settings.xml")
+  fi
+
   if [[ "${DOCKERCACHE}" = true ]]; then
     modp+=("--mvncache=/maven")
   else
@@ -540,14 +557,32 @@ function makearelease
     signflags=("-Psign" "-Dgpg.useagent=true" "-Dgpg.executable=${GPG}")
   fi
 
+  local target="install"
+  if [[ "${DEPLOY}" = true ]]; then
+    target="deploy"
+  fi
+
   # Create SRC and BIN tarballs for release,
   # shellcheck disable=SC2046
-  run_and_redirect "${LOGDIR}/mvn_install.log" \
-    "${MVN}" "${MVN_ARGS[@]}" install \
+  run_and_redirect "${LOGDIR}/mvn_${target}.log" \
+    "${MVN}" "${MVN_ARGS[@]}" ${target} \
       -Pdist,src,yarn-ui \
       "${signflags[@]}" \
       -DskipTests -Dtar $(hadoop_native_flags)
 
+  # Stage BIN tarball
+  run cd "${BASEDIR}"
+  run mv \
+    "${BASEDIR}/hadoop-dist/target/hadoop-${HADOOP_VERSION}.tar.gz" \
+    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz"
+
+  # Stage SRC tarball
+  run mv \
+    "${BASEDIR}/hadoop-dist/target/hadoop-${HADOOP_VERSION}-src.tar.gz" \
+    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-src.tar.gz"
+
+  big_console_header "Maven Site"
+
   if [[ "${SECURITYRELEASE}" = true ]]; then
     DOCFLAGS="-Pdocs"
     hadoop_error "WARNING: Skipping automatic changelog and release notes generation due to --security"
@@ -555,7 +590,6 @@ function makearelease
     DOCFLAGS="-Preleasedocs,docs"
   fi
 
-
   # Create site for release
   # we need to do install again so that jdiff and
   # a few other things get registered in the maven
@@ -568,13 +602,17 @@ function makearelease
       -Pdist,src \
       "${DOCFLAGS}"
 
-  big_console_header "Staging the release"
-
+  # Create the site tarball
   run mv "${BASEDIR}/target/staging/hadoop-project" "${BASEDIR}/target/r${HADOOP_VERSION}/"
   run cd "${BASEDIR}/target/"
   run tar czpf "hadoop-site-${HADOOP_VERSION}.tar.gz" "r${HADOOP_VERSION}"/*
   run cd "${BASEDIR}"
 
+  # Stage SITE tarball
+  run mv \
+     "${BASEDIR}/target/hadoop-site-${HADOOP_VERSION}.tar.gz" \
+    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-site.tar.gz"
+
   # Stage RAT report
   #shellcheck disable=SC2038
   find . -name rat.txt | xargs -I% cat % > "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-rat.txt"
@@ -586,25 +624,14 @@ function makearelease
         "${ARTIFACTS_DIR}/${i}.md"
   done
 
-  # Prepare and stage BIN tarball
-  run cd "${BASEDIR}/hadoop-dist/target/"
-  run tar -xzpf "hadoop-${HADOOP_VERSION}.tar.gz"
+  # We need to fixup the BIN tarball at the end to contain the site docs.
+  run cd "${ARTIFACTS_DIR}"
+  run tar -xzpf "hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz"
+  run mkdir -p "hadoop-${HADOOP_VERSION}/share/doc/hadoop/"
   run cp -r "${BASEDIR}/target/r${HADOOP_VERSION}"/* "hadoop-${HADOOP_VERSION}/share/doc/hadoop/"
-  run tar -czpf "hadoop-${HADOOP_VERSION}.tar.gz" "hadoop-${HADOOP_VERSION}"
-  run cd "${BASEDIR}"
-  run mv \
-    "${BASEDIR}/hadoop-dist/target/hadoop-${HADOOP_VERSION}.tar.gz" \
-    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz"
-
-  # Stage SRC tarball
-  run mv \
-    "${BASEDIR}/hadoop-dist/target/hadoop-${HADOOP_VERSION}-src.tar.gz" \
-    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-src.tar.gz"
+  run tar -czpf "hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz" "hadoop-${HADOOP_VERSION}"
+  run rm -rf "hadoop-${HADOOP_VERSION}"
 
-  # Stage SITE tarball
-  run mv \
-     "${BASEDIR}/target/hadoop-site-${HADOOP_VERSION}.tar.gz" \
-    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-site.tar.gz"
 }
 
 function signartifacts

+ 6 - 1
hadoop-common-project/hadoop-common/src/main/bin/hadoop

@@ -15,8 +15,9 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+# The name of the script being executed.
+HADOOP_SHELL_EXECNAME="hadoop"
 MYNAME="${BASH_SOURCE-$0}"
-HADOOP_SHELL_EXECNAME="${MYNAME##*/}"
 
 ## @description  build up the hadoop command's usage text.
 ## @audience     public
@@ -124,6 +125,10 @@ function hadoopcmd_case
       echo "HADOOP_TOOLS_HOME='${HADOOP_TOOLS_HOME}'"
       echo "HADOOP_TOOLS_DIR='${HADOOP_TOOLS_DIR}'"
       echo "HADOOP_TOOLS_LIB_JARS_DIR='${HADOOP_TOOLS_LIB_JARS_DIR}'"
+      if [[ -n "${QATESTMODE}" ]]; then
+        echo "MYNAME=${MYNAME}"
+        echo "HADOOP_SHELL_EXECNAME=${HADOOP_SHELL_EXECNAME}"
+      fi
       exit 0
     ;;
     fs)

+ 36 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderTokenIssuer.java

@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * File systems that support Encryption Zones have to implement this interface.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface KeyProviderTokenIssuer {
+
+  KeyProvider getKeyProvider() throws IOException;
+
+  URI getKeyProviderUri() throws IOException;
+}

+ 18 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/package-info.java

@@ -0,0 +1,18 @@
+/**
+ * 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.crypto.key;

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

@@ -56,15 +56,36 @@ public class FileStatus implements Writable, Comparable<Object>,
   private Path symlink;
   private Set<AttrFlags> attr;
 
-  private enum AttrFlags {
+  /**
+   * Flags for entity attributes.
+   */
+  public enum AttrFlags {
+    /** ACL information available for this entity. */
     HAS_ACL,
+    /** Entity is encrypted. */
     HAS_CRYPT,
+    /** Entity is stored erasure-coded. */
     HAS_EC,
-    SNAPSHOT_ENABLED
+    /** Snapshot capability enabled. */
+    SNAPSHOT_ENABLED,
   }
-  private static final Set<AttrFlags> NONE = Collections.<AttrFlags>emptySet();
-  private static Set<AttrFlags> flags(boolean acl, boolean crypt, boolean ec) {
-    if (!(acl || crypt || ec)) {
+
+  /**
+   * Shared, empty set of attributes (a common case for FileStatus).
+   */
+  public static final Set<AttrFlags> NONE = Collections.<AttrFlags>emptySet();
+
+  /**
+   * Convert boolean attributes to a set of flags.
+   * @param acl   See {@link AttrFlags#HAS_ACL}.
+   * @param crypt See {@link AttrFlags#HAS_CRYPT}.
+   * @param ec    See {@link AttrFlags#HAS_EC}.
+   * @param sn    See {@link AttrFlags#SNAPSHOT_ENABLED}.
+   * @return converted set of flags.
+   */
+  public static Set<AttrFlags> attributes(boolean acl, boolean crypt,
+                                          boolean ec, boolean sn) {
+    if (!(acl || crypt || ec || sn)) {
       return NONE;
     }
     EnumSet<AttrFlags> ret = EnumSet.noneOf(AttrFlags.class);
@@ -77,6 +98,9 @@ public class FileStatus implements Writable, Comparable<Object>,
     if (ec) {
       ret.add(AttrFlags.HAS_EC);
     }
+    if (sn) {
+      ret.add(AttrFlags.SNAPSHOT_ENABLED);
+    }
     return ret;
   }
 
@@ -117,6 +141,15 @@ public class FileStatus implements Writable, Comparable<Object>,
       long blocksize, long modification_time, long access_time,
       FsPermission permission, String owner, String group, Path symlink,
       Path path, boolean hasAcl, boolean isEncrypted, boolean isErasureCoded) {
+    this(length, isdir, block_replication, blocksize, modification_time,
+        access_time, permission, owner, group, symlink, path,
+        attributes(hasAcl, isEncrypted, isErasureCoded, false));
+  }
+
+  public FileStatus(long length, boolean isdir, int block_replication,
+      long blocksize, long modification_time, long access_time,
+      FsPermission permission, String owner, String group, Path symlink,
+      Path path, Set<AttrFlags> attr) {
     this.length = length;
     this.isdir = isdir;
     this.block_replication = (short)block_replication;
@@ -136,7 +169,7 @@ public class FileStatus implements Writable, Comparable<Object>,
     this.group = (group == null) ? "" : group;
     this.symlink = symlink;
     this.path = path;
-    attr = flags(hasAcl, isEncrypted, isErasureCoded);
+    this.attr = attr;
 
     // The variables isdir and symlink indicate the type:
     // 1. isdir implies directory, in which case symlink must be null.
@@ -340,19 +373,6 @@ public class FileStatus implements Writable, Comparable<Object>,
     this.group = (group == null) ? "" :  group;
   }
 
-  /**
-   * Sets Snapshot enabled flag.
-   *
-   * @param isSnapShotEnabled When true, SNAPSHOT_ENABLED flag is set
-   */
-  public void setSnapShotEnabledFlag(boolean isSnapShotEnabled) {
-    if (isSnapShotEnabled) {
-      attr.add(AttrFlags.SNAPSHOT_ENABLED);
-    } else {
-      attr.remove(AttrFlags.SNAPSHOT_ENABLED);
-    }
-  }
-
   /**
    * @return The contents of the symbolic link.
    */
@@ -480,7 +500,8 @@ public class FileStatus implements Writable, Comparable<Object>,
     setGroup(other.getGroup());
     setSymlink((other.isSymlink() ? other.getSymlink() : null));
     setPath(other.getPath());
-    attr = flags(other.hasAcl(), other.isEncrypted(), other.isErasureCoded());
+    attr = attributes(other.hasAcl(), other.isEncrypted(),
+        other.isErasureCoded(), other.isSnapshotEnabled());
     assert (isDirectory() && getSymlink() == null) || !isDirectory();
   }
 

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

@@ -951,6 +951,21 @@ public abstract class FileSystem extends Configured implements Closeable {
         IO_FILE_BUFFER_SIZE_DEFAULT));
   }
 
+  /**
+   * Open an FSDataInputStream matching the PathHandle instance. The
+   * implementation may encode metadata in PathHandle to address the
+   * resource directly and verify that the resource referenced
+   * satisfies constraints specified at its construciton.
+   * @param fd PathHandle object returned by the FS authority.
+   * @throws IOException IO failure
+   * @throws UnsupportedOperationException If {@link #open(PathHandle, int)}
+   *                                       not overridden by subclass
+   */
+  public FSDataInputStream open(PathHandle fd) throws IOException {
+    return open(fd, getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
+        IO_FILE_BUFFER_SIZE_DEFAULT));
+  }
+
   /**
    * Open an FSDataInputStream matching the PathHandle instance. The
    * implementation may encode metadata in PathHandle to address the

+ 39 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs;
 
 import java.io.IOException;
+import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -115,9 +116,35 @@ public class LocatedFileStatus extends FileStatus {
       Path symlink, Path path,
       boolean hasAcl, boolean isEncrypted, boolean isErasureCoded,
       BlockLocation[] locations) {
-    super(length, isdir, block_replication, blocksize, modification_time,
+    this(length, isdir, block_replication, blocksize, modification_time,
         access_time, permission, owner, group, symlink, path,
-        hasAcl, isEncrypted, isErasureCoded);
+        attributes(hasAcl, isEncrypted, isErasureCoded, false), locations);
+    this.locations = locations;
+  }
+
+  /**
+   * Constructor.
+   *
+   * @param length a file's length
+   * @param isdir if the path is a directory
+   * @param block_replication the file's replication factor
+   * @param blocksize a file's block size
+   * @param modification_time a file's modification time
+   * @param access_time a file's access time
+   * @param permission a file's permission
+   * @param owner a file's owner
+   * @param group a file's group
+   * @param symlink symlink if the path is a symbolic link
+   * @param path the path's qualified name
+   * @param attr Attribute flags (See {@link FileStatus.AttrFlags}).
+   * @param locations a file's block locations
+   */
+  public LocatedFileStatus(long length, boolean isdir, int block_replication,
+      long blocksize, long modification_time, long access_time,
+      FsPermission permission, String owner, String group, Path symlink,
+      Path path, Set<AttrFlags> attr, BlockLocation[] locations) {
+    super(length, isdir, block_replication, blocksize, modification_time,
+        access_time, permission, owner, group, symlink, path, attr);
     this.locations = locations;
   }
 
@@ -135,7 +162,16 @@ public class LocatedFileStatus extends FileStatus {
   public BlockLocation[] getBlockLocations() {
     return locations;
   }
-  
+
+  /**
+   * Hook for subclasses to lazily set block locations. The {@link #locations}
+   * field should be null before this is called.
+   * @param locations Block locations for this instance.
+   */
+  protected void setBlockLocations(BlockLocation[] locations) {
+    this.locations = locations;
+  }
+
   /**
    * Compare this FileStatus to another FileStatus
    * @param   o the FileStatus to be compared.

+ 70 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -696,11 +696,34 @@ public class RawLocalFileSystem extends FileSystem {
       return super.getGroup();
     }
 
+    /**
+     * Load file permission information (UNIX symbol rwxrwxrwx, sticky bit info).
+     *
+     * To improve peformance, give priority to native stat() call. First try get
+     * permission information by using native JNI call then fall back to use non
+     * native (ProcessBuilder) call in case native lib is not loaded or native
+     * call is not successful
+     */
+    private synchronized void loadPermissionInfo() {
+      if (!isPermissionLoaded() && NativeIO.isAvailable()) {
+        try {
+          loadPermissionInfoByNativeIO();
+        } catch (IOException ex) {
+          LOG.debug("Native call failed", ex);
+        }
+      }
+
+      if (!isPermissionLoaded()) {
+        loadPermissionInfoByNonNativeIO();
+      }
+    }
+
     /// loads permissions, owner, and group from `ls -ld`
-    private void loadPermissionInfo() {
+    @VisibleForTesting
+    void loadPermissionInfoByNonNativeIO() {
       IOException e = null;
       try {
-        String output = FileUtil.execCommand(new File(getPath().toUri()), 
+        String output = FileUtil.execCommand(new File(getPath().toUri()),
             Shell.getGetPermissionCommand());
         StringTokenizer t =
             new StringTokenizer(output, Shell.TOKEN_SEPARATOR_REGEX);
@@ -716,16 +739,16 @@ public class RawLocalFileSystem extends FileSystem {
         t.nextToken();
 
         String owner = t.nextToken();
+        String group = t.nextToken();
         // If on windows domain, token format is DOMAIN\\user and we want to
         // extract only the user name
+        // same as to the group name
         if (Shell.WINDOWS) {
-          int i = owner.indexOf('\\');
-          if (i != -1)
-            owner = owner.substring(i + 1);
+          owner = removeDomain(owner);
+          group = removeDomain(group);
         }
         setOwner(owner);
-
-        setGroup(t.nextToken());
+        setGroup(group);
       } catch (Shell.ExitCodeException ioe) {
         if (ioe.getExitCode() != 1) {
           e = ioe;
@@ -745,6 +768,46 @@ public class RawLocalFileSystem extends FileSystem {
       }
     }
 
+    // In Windows, domain name is added.
+    // For example, given machine name (domain name) dname, user name i, then
+    // the result for user is dname\\i and for group is dname\\None. So we need
+    // remove domain name as follows:
+    // DOMAIN\\user => user, DOMAIN\\group => group
+    private String removeDomain(String str) {
+      int index = str.indexOf("\\");
+      if (index != -1) {
+        str = str.substring(index + 1);
+      }
+      return str;
+    }
+
+    // loads permissions, owner, and group from `ls -ld`
+    // but use JNI to more efficiently get file mode (permission, owner, group)
+    // by calling file stat() in *nix or some similar calls in Windows
+    @VisibleForTesting
+    void loadPermissionInfoByNativeIO() throws IOException {
+      Path path = getPath();
+      String pathName = path.toUri().getPath();
+      // remove leading slash for Windows path
+      if (Shell.WINDOWS && pathName.startsWith("/")) {
+        pathName = pathName.substring(1);
+      }
+      try {
+        NativeIO.POSIX.Stat stat = NativeIO.POSIX.getStat(pathName);
+        String owner = stat.getOwner();
+        String group = stat.getGroup();
+        int mode = stat.getMode();
+        setOwner(owner);
+        setGroup(group);
+        setPermission(new FsPermission(mode));
+      } catch (IOException e) {
+        setOwner(null);
+        setGroup(null);
+        setPermission(null);
+        throw e;
+      }
+    }
+
     @Override
     public void write(DataOutput out) throws IOException {
       if (!isPermissionLoaded()) {

+ 34 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java

@@ -90,6 +90,40 @@ public class FsPermission implements Writable, Serializable,
    */
   public FsPermission(short mode) { fromShort(mode); }
 
+  /**
+   * Construct by the given mode.
+   *
+   * octal mask is applied.
+   *
+   *<pre>
+   *              before mask     after mask    file type   sticky bit
+   *
+   *    octal     100644            644         file          no
+   *    decimal    33188            420
+   *
+   *    octal     101644           1644         file          yes
+   *    decimal    33700           1420
+   *
+   *    octal      40644            644         directory     no
+   *    decimal    16804            420
+   *
+   *    octal      41644           1644         directory     yes
+   *    decimal    17316           1420
+   *</pre>
+   *
+   * 100644 becomes 644 while 644 remains as 644
+   *
+   * @param mode Mode is supposed to come from the result of native stat() call.
+   *             It contains complete permission information: rwxrwxrwx, sticky
+   *             bit, whether it is a directory or a file, etc. Upon applying
+   *             mask, only permission and sticky bit info will be kept because
+   *             they are the only parts to be used for now.
+   * @see #FsPermission(short mode)
+   */
+  public FsPermission(int mode) {
+    this((short)(mode & 01777));
+  }
+
   /**
    * Copy constructor
    * 

+ 5 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java

@@ -96,12 +96,11 @@ public final class PBHelper {
     int flags = proto.getFlags();
     FileStatus fileStatus = new FileStatus(length, isdir, blockReplication,
         blocksize, mtime, atime, permission, owner, group, symlink, path,
-        (flags & FileStatusProto.Flags.HAS_ACL_VALUE) != 0,
-        (flags & FileStatusProto.Flags.HAS_CRYPT_VALUE) != 0,
-        (flags & FileStatusProto.Flags.HAS_EC_VALUE) != 0);
-
-    fileStatus.setSnapShotEnabledFlag((flags & FileStatusProto.Flags
-        .SNAPSHOT_ENABLED_VALUE) != 0);
+        FileStatus.attributes(
+          (flags & FileStatusProto.Flags.HAS_ACL_VALUE) != 0,
+          (flags & FileStatusProto.Flags.HAS_CRYPT_VALUE) != 0,
+          (flags & FileStatusProto.Flags.HAS_EC_VALUE) != 0,
+          (flags & FileStatusProto.Flags.SNAPSHOT_ENABLED_VALUE) != 0));
     return fileStatus;
   }
 

+ 34 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.SecureIOUtils.AlreadyExistsException;
 import org.apache.hadoop.util.NativeCodeLoader;
@@ -221,6 +222,8 @@ public class NativeIO {
     public static native FileDescriptor open(String path, int flags, int mode) throws IOException;
     /** Wrapper around fstat(2) */
     private static native Stat fstat(FileDescriptor fd) throws IOException;
+    /** Wrapper around stat(2). */
+    private static native Stat stat(String path) throws IOException;
 
     /** Native chmod implementation. On UNIX, it is a wrapper around chmod(2) */
     private static native void chmodImpl(String path, int mode) throws IOException;
@@ -428,6 +431,37 @@ public class NativeIO {
       return stat;
     }
 
+    /**
+     * Return the file stat for a file path.
+     *
+     * @param path  file path
+     * @return  the file stat
+     * @throws IOException  thrown if there is an IO error while obtaining the
+     * file stat
+     */
+    public static Stat getStat(String path) throws IOException {
+      if (path == null) {
+        String errMessage = "Path is null";
+        LOG.warn(errMessage);
+        throw new IOException(errMessage);
+      }
+      Stat stat = null;
+      try {
+        if (!Shell.WINDOWS) {
+          stat = stat(path);
+          stat.owner = getName(IdCache.USER, stat.ownerId);
+          stat.group = getName(IdCache.GROUP, stat.groupId);
+        } else {
+          stat = stat(path);
+        }
+      } catch (NativeIOException nioe) {
+        LOG.warn("NativeIO.getStat error ({}): {} -- file path: {}",
+            nioe.getErrorCode(), nioe.getMessage(), path);
+        throw new PathIOException(path, nioe);
+      }
+      return stat;
+    }
+
     private static String getName(IdCache domain, int id) throws IOException {
       Map<Integer, CachedName> idNameCache = (domain == IdCache.USER)
         ? USER_ID_NAME_CACHE : GROUP_ID_NAME_CACHE;

+ 85 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c

@@ -383,7 +383,92 @@ cleanup:
 #endif
 }
 
+/*
+ * Class:     org_apache_hadoop_io_nativeio_NativeIO_POSIX
+ * Method:    stat
+ * Signature: (Ljava/lang/String;)Lorg/apache/hadoop/io/nativeio/NativeIO$POSIX$Stat;
+ * public static native Stat stat(String path);
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT jobject JNICALL
+Java_org_apache_hadoop_io_nativeio_NativeIO_00024POSIX_stat(
+  JNIEnv *env, jclass clazz, jstring j_path)
+{
+#ifdef UNIX
+  jobject ret = NULL;
+
+  const char *c_path = (*env)->GetStringUTFChars(env, j_path, NULL);
+  if (c_path == NULL) {
+    goto cleanup;
+  }
+
+  struct stat s;
+  int rc = stat(c_path, &s);
+  if (rc != 0) {
+    throw_ioe(env, errno);
+    goto cleanup;
+  }
+
+  // Construct result
+  ret = (*env)->NewObject(env, stat_clazz, stat_ctor,
+    (jint)s.st_uid, (jint)s.st_gid, (jint)s.st_mode);
+
+cleanup:
+  if (c_path != NULL) {
+    (*env)->ReleaseStringUTFChars(env, j_path, c_path);
+  }
+  return ret;
+#endif
 
+#ifdef WINDOWS
+  LPWSTR owner = NULL;
+  LPWSTR group = NULL;
+  int mode = 0;
+  jstring jstr_owner = NULL;
+  jstring jstr_group = NULL;
+  int rc;
+  jobject ret = NULL;
+
+  LPCWSTR path = (LPCWSTR) (*env)->GetStringChars(env, j_path, NULL);
+  if (path == NULL) {
+    goto cleanup;
+  }
+
+  rc = FindFileOwnerAndPermission(path, TRUE, &owner, &group, &mode);
+  if (rc != ERROR_SUCCESS) {
+    throw_ioe(env, rc);
+    goto cleanup;
+  }
+
+  jstr_owner = (*env)->NewString(env, owner, (jsize) wcslen(owner));
+  if (jstr_owner == NULL) goto cleanup;
+
+  jstr_group = (*env)->NewString(env, group, (jsize) wcslen(group));
+  if (jstr_group == NULL) goto cleanup;
+
+  ret = (*env)->NewObject(env, stat_clazz, stat_ctor2,
+    jstr_owner, jstr_group, (jint)mode);
+
+cleanup:
+  if (path != NULL)
+    (*env)->ReleaseStringChars(env, j_path, (const jchar*) path);
+
+  if (ret == NULL) {
+    if (jstr_owner != NULL)
+      (*env)->ReleaseStringChars(env, jstr_owner, owner);
+
+    if (jstr_group != NULL)
+      (*env)->ReleaseStringChars(env, jstr_group, group);
+  }
+
+  LocalFree(owner);
+  LocalFree(group);
+
+  return ret;
+#endif
+}
 
 /**
  * public static native void posix_fadvise(

+ 22 - 7
hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md

@@ -117,13 +117,7 @@ Compatibility types
 
 Developers SHOULD annotate all Hadoop interfaces and classes with the
 @InterfaceAudience and @InterfaceStability annotations to describe the
-intended audience and stability. Annotations may be at the package, class, or
-member variable or method level. Member variable and method annotations SHALL
-override class annotations, and class annotations SHALL override package
-annotations. A package, class, or member variable or method that is not
-annotated SHALL be interpreted as implicitly
-[Private](./InterfaceClassification.html#Private) and
-[Unstable](./InterfaceClassification.html#Unstable).
+intended audience and stability.
 
 * @InterfaceAudience captures the intended audience. Possible values are
 [Public](./InterfaceClassification.html#Public) (for end users and external
@@ -134,6 +128,27 @@ etc.), and [Private](./InterfaceClassification.html#Private)
 * @InterfaceStability describes what types of interface changes are permitted. Possible values are [Stable](./InterfaceClassification.html#Stable), [Evolving](./InterfaceClassification.html#Evolving), and [Unstable](./InterfaceClassification.html#Unstable).
 * @Deprecated notes that the package, class, or member variable or method could potentially be removed in the future and should not be used.
 
+Annotations MAY be applied at the package, class, or method level. If a method
+has no privacy or stability annotation, it SHALL inherit its intended audience
+or stability level from the class to which it belongs. If a class has no
+privacy or stability annotation, it SHALL inherit its intended audience or
+stability level from the package to which it belongs. If a package has no
+privacy or stability annotation, it SHALL be assumed to be
+[Private](./InterfaceClassification.html#Private) and
+[Unstable](./InterfaceClassification.html#Unstable),
+respectively.
+
+In the event that an element's audience or stability annotation conflicts with
+the corresponding annotation of its parent (whether explicit or inherited), the
+element's audience or stability (respectively) SHALL be determined by the
+more restrictive annotation. For example, if a
+[Private](./InterfaceClassification.html#Private) method is contained
+in a [Public](./InterfaceClassification.html#Public) class, then the method
+SHALL be treated as [Private](./InterfaceClassification.html#Private). If a
+[Public](./InterfaceClassification.html#Public) method is contained in a
+[Private](./InterfaceClassification.html#Private) class, the method SHALL be
+treated as [Private](./InterfaceClassification.html#Private).
+
 #### Use Cases
 
 * [Public](./InterfaceClassification.html#Public)-[Stable](./InterfaceClassification.html#Stable) API compatibility is required to ensure end-user programs and downstream projects continue to work without modification.

+ 1 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java

@@ -79,6 +79,7 @@ public class TestFilterFileSystem {
 
     public boolean mkdirs(Path f);
     public FSDataInputStream open(Path f);
+    public FSDataInputStream open(PathHandle f);
     public FSDataOutputStream create(Path f);
     public FSDataOutputStream create(Path f, boolean overwrite);
     public FSDataOutputStream create(Path f, Progressable progress);

+ 1 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java

@@ -80,6 +80,7 @@ public class TestHarFileSystem {
 
     public boolean mkdirs(Path f);
     public FSDataInputStream open(Path f);
+    public FSDataInputStream open(PathHandle f);
     public FSDataOutputStream create(Path f);
     public FSDataOutputStream create(Path f, boolean overwrite);
     public FSDataOutputStream create(Path f, Progressable progress);

+ 86 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java

@@ -17,10 +17,18 @@
  */
 package org.apache.hadoop.fs;
 
+import java.io.File;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.StatUtils;
 import org.apache.hadoop.util.Shell;
+
 import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -78,4 +86,81 @@ public class TestRawLocalFileSystemContract extends FileSystemContractBaseTest {
   protected boolean filesystemIsCaseSensitive() {
     return !(Shell.WINDOWS || Shell.MAC);
   }
-}
+
+  // cross-check getPermission using both native/non-native
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testPermission() throws Exception {
+    Path testDir = getTestBaseDir();
+    String testFilename = "teststat2File";
+    Path path = new Path(testDir, testFilename);
+
+    RawLocalFileSystem rfs = new RawLocalFileSystem();
+    Configuration conf = new Configuration();
+    rfs.initialize(rfs.getUri(), conf);
+    rfs.createNewFile(path);
+
+    File file = rfs.pathToFile(path);
+    long defaultBlockSize = rfs.getDefaultBlockSize(path);
+
+    //
+    // test initial permission
+    //
+    RawLocalFileSystem.DeprecatedRawLocalFileStatus fsNIO =
+      new RawLocalFileSystem.DeprecatedRawLocalFileStatus(
+          file, defaultBlockSize, rfs);
+    fsNIO.loadPermissionInfoByNativeIO();
+    RawLocalFileSystem.DeprecatedRawLocalFileStatus fsnonNIO =
+        new RawLocalFileSystem.DeprecatedRawLocalFileStatus(
+            file, defaultBlockSize, rfs);
+    fsnonNIO.loadPermissionInfoByNonNativeIO();
+
+    assertEquals(fsNIO.getOwner(), fsnonNIO.getOwner());
+    assertEquals(fsNIO.getGroup(), fsnonNIO.getGroup());
+    assertEquals(fsNIO.getPermission(), fsnonNIO.getPermission());
+
+    LOG.info("owner: {}, group: {}, permission: {}, isSticky: {}",
+        fsNIO.getOwner(), fsNIO.getGroup(), fsNIO.getPermission(),
+        fsNIO.getPermission().getStickyBit());
+
+    //
+    // test normal chmod - no sticky bit
+    //
+    StatUtils.setPermissionFromProcess("644", file.getPath());
+    fsNIO.loadPermissionInfoByNativeIO();
+    fsnonNIO.loadPermissionInfoByNonNativeIO();
+    assertEquals(fsNIO.getPermission(), fsnonNIO.getPermission());
+    assertEquals(644, fsNIO.getPermission().toOctal());
+    assertFalse(fsNIO.getPermission().getStickyBit());
+    assertFalse(fsnonNIO.getPermission().getStickyBit());
+
+    //
+    // test sticky bit
+    // unfortunately, cannot be done in Windows environments
+    //
+    if (!Shell.WINDOWS) {
+      //
+      // add sticky bit
+      //
+      StatUtils.setPermissionFromProcess("1644", file.getPath());
+      fsNIO.loadPermissionInfoByNativeIO();
+      fsnonNIO.loadPermissionInfoByNonNativeIO();
+      assertEquals(fsNIO.getPermission(), fsnonNIO.getPermission());
+      assertEquals(1644, fsNIO.getPermission().toOctal());
+      assertEquals(true, fsNIO.getPermission().getStickyBit());
+      assertEquals(true, fsnonNIO.getPermission().getStickyBit());
+
+      //
+      // remove sticky bit
+      //
+      StatUtils.setPermissionFromProcess("-t", file.getPath());
+      fsNIO.loadPermissionInfoByNativeIO();
+      fsnonNIO.loadPermissionInfoByNonNativeIO();
+      assertEquals(fsNIO.getPermission(), fsnonNIO.getPermission());
+      assertEquals(644, fsNIO.getPermission().toOctal());
+      assertEquals(false, fsNIO.getPermission().getStickyBit());
+      assertEquals(false, fsnonNIO.getPermission().getStickyBit());
+    }
+  }
+
+}

+ 31 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/permission/TestFsPermission.java

@@ -264,6 +264,37 @@ public class TestFsPermission {
            msg.contains("octal or symbolic");
   }
 
+  /**
+   *  test FsPermission(int) constructor.
+   */
+  @Test
+  public void testIntPermission() {
+    // Octal           Decimals        Masked OCT      Masked DEC
+    // 100644          33188           644             420
+    // 101644          33700           1644            932
+    // 40644           16804           644             420
+    // 41644           17316           1644            932
+    // 644             420             644             420
+    // 1644            932             1644            932
+
+    int[][] permission_mask_maps = {
+      // Octal                 Decimal    Unix Symbolic
+      { 0100644,  0644, 0 },   // 33188    -rw-r--
+      { 0101644, 01644, 1 },   // 33700    -rw-r-t
+      { 040644,   0644, 0 },   // 16804    drw-r--
+      { 041644,  01644, 1 }    // 17316    drw-r-t
+    };
+
+    for (int[] permission_mask_map : permission_mask_maps) {
+      int original_permission_value = permission_mask_map[0];
+      int masked_permission_value = permission_mask_map[1];
+      boolean hasStickyBit = permission_mask_map[2] == 1;
+      FsPermission fsPermission = new FsPermission(original_permission_value);
+      assertEquals(masked_permission_value, fsPermission.toShort());
+      assertEquals(hasStickyBit, fsPermission.getStickyBit());
+    }
+  }
+
   // Symbolic umask list is generated in linux shell using by the command:
   // umask 0; umask <octal number>; umask -S
   static final String[][] SYMBOLIC = new String[][] {

+ 128 - 16
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java

@@ -29,36 +29,44 @@ import java.nio.MappedByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel.MapMode;
 import java.util.Random;
-import java.util.concurrent.atomic.AtomicReference;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assume.*;
-import static org.junit.Assert.*;
-import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows;
-import static org.apache.hadoop.test.PlatformAssumptions.assumeWindows;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.commons.io.FileUtils;
+
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.test.StatUtils;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.util.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.*;
 import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.Stat.*;
+import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows;
+import static org.apache.hadoop.test.PlatformAssumptions.assumeWindows;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assume.*;
+import static org.junit.Assert.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestNativeIO {
   static final Logger LOG = LoggerFactory.getLogger(TestNativeIO.class);
@@ -163,6 +171,110 @@ public class TestNativeIO {
     }
   }
 
+  @Test (timeout = 30000)
+  public void testStat() throws Exception {
+    Configuration conf = new Configuration();
+    FileSystem fileSystem = FileSystem.getLocal(conf).getRawFileSystem();
+    Path path = new Path(TEST_DIR.getPath(), "teststat2");
+    fileSystem.createNewFile(path);
+    String testFilePath = path.toString();
+
+    try {
+      doStatTest(testFilePath);
+      LOG.info("testStat() is successful.");
+    } finally {
+      ContractTestUtils.cleanup("cleanup test file: " + path.toString(),
+          fileSystem, path);
+    }
+  }
+
+  private boolean doStatTest(String testFilePath) throws Exception {
+    NativeIO.POSIX.Stat stat = NativeIO.POSIX.getStat(testFilePath);
+    String owner = stat.getOwner();
+    String group = stat.getGroup();
+    int mode = stat.getMode();
+
+    // direct check with System
+    String expectedOwner = System.getProperty("user.name");
+    assertEquals(expectedOwner, owner);
+    assertNotNull(group);
+    assertTrue(!group.isEmpty());
+
+    // cross check with ProcessBuilder
+    StatUtils.Permission expected =
+        StatUtils.getPermissionFromProcess(testFilePath);
+    StatUtils.Permission permission =
+        new StatUtils.Permission(owner, group, new FsPermission(mode));
+
+    assertEquals(expected.getOwner(), permission.getOwner());
+    assertEquals(expected.getGroup(), permission.getGroup());
+    assertEquals(expected.getFsPermission(), permission.getFsPermission());
+
+    LOG.info("Load permission test is successful for path: {}, stat: {}",
+        testFilePath, stat);
+    LOG.info("On mask, stat is owner: {}, group: {}, permission: {}",
+        owner, group, permission.getFsPermission().toOctal());
+    return true;
+  }
+
+  @Test
+  public void testStatOnError() throws Exception {
+    final String testNullFilePath = null;
+    LambdaTestUtils.intercept(IOException.class,
+            "Path is null",
+            () -> NativeIO.POSIX.getStat(testNullFilePath));
+
+    final String testInvalidFilePath = "C:\\nonexisting_path\\nonexisting_file";
+    LambdaTestUtils.intercept(IOException.class,
+            PathIOException.class.getName(),
+            () -> NativeIO.POSIX.getStat(testInvalidFilePath));
+  }
+
+  @Test (timeout = 30000)
+  public void testMultiThreadedStat() throws Exception {
+    Configuration conf = new Configuration();
+    FileSystem fileSystem = FileSystem.getLocal(conf).getRawFileSystem();
+    Path path = new Path(TEST_DIR.getPath(), "teststat2");
+    fileSystem.createNewFile(path);
+    String testFilePath = path.toString();
+
+    int numOfThreads = 10;
+    ExecutorService executorService =
+        Executors.newFixedThreadPool(numOfThreads);
+    executorService.awaitTermination(1000, TimeUnit.MILLISECONDS);
+    try {
+      for (int i = 0; i < numOfThreads; i++){
+        Future<Boolean> result =
+            executorService.submit(() -> doStatTest(testFilePath));
+        assertTrue(result.get());
+      }
+      LOG.info("testMultiThreadedStat() is successful.");
+    } finally {
+      executorService.shutdown();
+      ContractTestUtils.cleanup("cleanup test file: " + path.toString(),
+          fileSystem, path);
+    }
+  }
+
+  @Test
+  public void testMultiThreadedStatOnError() throws Exception {
+    final String testInvalidFilePath = "C:\\nonexisting_path\\nonexisting_file";
+
+    int numOfThreads = 10;
+    ExecutorService executorService =
+        Executors.newFixedThreadPool(numOfThreads);
+    for (int i = 0; i < numOfThreads; i++) {
+      try {
+        Future<Boolean> result =
+            executorService.submit(() -> doStatTest(testInvalidFilePath));
+        result.get();
+      } catch (Exception e) {
+        assertTrue(e.getCause() instanceof PathIOException);
+      }
+    }
+    executorService.shutdown();
+  }
+
   @Test (timeout = 30000)
   public void testSetFilePointer() throws Exception {
     assumeWindows();

+ 8 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java

@@ -641,10 +641,16 @@ public abstract class GenericTestUtils {
    * conditions.
    */
   public static class SleepAnswer implements Answer<Object> {
+    private final int minSleepTime;
     private final int maxSleepTime;
     private static Random r = new Random();
-    
+
     public SleepAnswer(int maxSleepTime) {
+      this(0, maxSleepTime);
+    }
+
+    public SleepAnswer(int minSleepTime, int maxSleepTime) {
+      this.minSleepTime = minSleepTime;
       this.maxSleepTime = maxSleepTime;
     }
     
@@ -652,7 +658,7 @@ public abstract class GenericTestUtils {
     public Object answer(InvocationOnMock invocation) throws Throwable {
       boolean interrupted = false;
       try {
-        Thread.sleep(r.nextInt(maxSleepTime));
+        Thread.sleep(r.nextInt(maxSleepTime) + minSleepTime);
       } catch (InterruptedException ie) {
         interrupted = true;
       }

+ 126 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/StatUtils.java

@@ -0,0 +1,126 @@
+/**
+ * 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.test;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Shell;
+
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.StringTokenizer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Helper class for stat/permission utility methods. Forks processes to query
+ * permission info.
+ */
+public class StatUtils {
+  public static class Permission {
+    private String owner;
+    private String group;
+    private FsPermission fsPermission;
+
+    public Permission(String owner, String group, FsPermission fsPermission) {
+      this.owner = owner;
+      this.group = group;
+      this.fsPermission = fsPermission;
+    }
+
+    public String getOwner() {
+      return owner;
+    }
+
+    public String getGroup() {
+      return group;
+    }
+
+    public FsPermission getFsPermission() {
+      return fsPermission;
+    }
+  }
+
+  public static Permission getPermissionFromProcess(String filePath)
+      throws Exception {
+    String[] shellCommand = Shell.getGetPermissionCommand();
+    String sPerm = getPermissionStringFromProcess(shellCommand, filePath);
+
+    StringTokenizer tokenizer =
+        new StringTokenizer(sPerm, Shell.TOKEN_SEPARATOR_REGEX);
+    String symbolicPermission = tokenizer.nextToken();
+    tokenizer.nextToken(); // skip hard link
+    String owner = tokenizer.nextToken();
+    String group = tokenizer.nextToken();
+    if (Shell.WINDOWS) {
+      owner = removeDomain(owner);
+      group = removeDomain(group);
+    }
+
+    Permission permission =
+        new Permission(owner, group, FsPermission.valueOf(symbolicPermission));
+
+    return permission;
+  }
+
+  public static void setPermissionFromProcess(String chmod, String filePath)
+      throws Exception {
+    setPermissionFromProcess(chmod, false, filePath);
+  }
+
+  public static void setPermissionFromProcess(String chmod, boolean recursive,
+      String filePath) throws Exception {
+    String[] shellCommand = Shell.getSetPermissionCommand(chmod, recursive);
+    getPermissionStringFromProcess(shellCommand, filePath);
+  }
+
+  private static String removeDomain(String str) {
+    int index = str.indexOf("\\");
+    if (index != -1) {
+      str = str.substring(index + 1);
+    }
+    return str;
+  }
+
+  private static String getPermissionStringFromProcess(String[] shellCommand,
+      String testFilePath) throws Exception {
+    List<String> cmd = new ArrayList(Arrays.asList(shellCommand));
+    cmd.add(testFilePath);
+
+    ProcessBuilder processBuilder = new ProcessBuilder(cmd);
+    Process process = processBuilder.start();
+
+    ExecutorService executorService = Executors.newSingleThreadExecutor();
+    executorService.awaitTermination(2000, TimeUnit.MILLISECONDS);
+    try {
+      Future<String> future =
+          executorService.submit(() -> new BufferedReader(
+              new InputStreamReader(process.getInputStream(),
+                  Charset.defaultCharset())).lines().findFirst().orElse(""));
+      return future.get();
+    } finally {
+      process.destroy();
+      executorService.shutdown();
+    }
+  }
+}

+ 36 - 0
hadoop-common-project/hadoop-common/src/test/scripts/hadoop_shell_execname.bats

@@ -0,0 +1,36 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+load hadoop-functions_test_helper
+
+# Setup minimal environment to invoke the 'hadoop' command.
+hadoopcommandsetup () {
+  export HADOOP_LIBEXEC_DIR="${TMP}/libexec"
+  export HADOOP_CONF_DIR="${TMP}/conf"
+  mkdir -p "${HADOOP_LIBEXEC_DIR}"
+  echo   ". \"${BATS_TEST_DIRNAME}/../../main/bin/hadoop-functions.sh\"" > "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh"
+  chmod a+rx "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh"
+}
+
+# Verify that the 'hadoop' command correctly infers MYNAME and
+# HADOOP_SHELL_EXECNAME
+@test "hadoop_shell_execname" {
+  hadoopcommandsetup
+  export QATESTMODE=unittest
+  run "${BATS_TEST_DIRNAME}/../../main/bin/hadoop" envvars
+  echo ">${output}<"
+  [[ ${output} =~ MYNAME=.*/hadoop ]]
+  [[ ${output} =~ HADOOP_SHELL_EXECNAME=hadoop ]]
+}

+ 31 - 0
hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml

@@ -19,6 +19,8 @@
       <Class name="org.apache.hadoop.hdfs.DFSPacket"/>
       <Class name="org.apache.hadoop.hdfs.protocol.LocatedStripedBlock"/>
       <Class name="org.apache.hadoop.hdfs.util.StripedBlockUtil$ChunkByteArray"/>
+      <Class name="org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing$DiffReportListingEntry"/>
+      <Class name="org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing"/>
     </Or>
     <Bug pattern="EI_EXPOSE_REP,EI_EXPOSE_REP2" />
   </Match>
@@ -67,4 +69,33 @@
   <Match>
     <Package name="org.apache.hadoop.hdfs.ozone.protocol.proto" />
   </Match>
+
+  <!-- BlockLocations are user-facing, but LocatedBlocks are not. -->
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus" />
+    <Field name="hdfsloc" />
+    <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED" />
+  </Match>
+
+  <!-- Hdfs*FileStatus are internal types. This "internal" state is not sensitive. -->
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.protocol.HdfsNamedFileStatus" />
+    <Method name="getLocalNameInBytes" />
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.protocol.HdfsNamedFileStatus" />
+    <Method name="getSymlinkInBytes" />
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus" />
+    <Method name="getLocalNameInBytes" />
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus" />
+    <Method name="getSymlinkInBytes" />
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
 </FindBugsFilter>

+ 21 - 112
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -17,13 +17,14 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_READAHEAD;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_LOCAL_INTERFACES;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_DEFAULT;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY;
 
@@ -56,18 +57,15 @@ import javax.net.SocketFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.CryptoCodec;
 import org.apache.hadoop.crypto.CryptoInputStream;
 import org.apache.hadoop.crypto.CryptoOutputStream;
-import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CacheFlag;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -137,10 +135,10 @@ import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.protocol.ReencryptionStatusIterator;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
@@ -169,7 +167,6 @@ import org.apache.hadoop.ipc.RpcNoSuchMethodException;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
@@ -205,9 +202,6 @@ import com.google.common.net.InetAddresses;
 public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     DataEncryptionKeyFactory {
   public static final Logger LOG = LoggerFactory.getLogger(DFSClient.class);
-  // 1 hour
-  public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L;
-  private static final String DFS_KMS_PREFIX = "dfs-kms-";
 
   private final Configuration conf;
   private final Tracer tracer;
@@ -240,6 +234,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
   private static volatile ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
   private final int smallBufferSize;
+  private final long serverDefaultsValidityPeriod;
 
   public DfsClientConf getConf() {
     return dfsClientConf;
@@ -371,6 +366,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
             null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_READS, false);
     Long readahead = (conf.get(DFS_CLIENT_CACHE_READAHEAD) == null) ?
         null : conf.getLong(DFS_CLIENT_CACHE_READAHEAD, 0);
+    this.serverDefaultsValidityPeriod =
+            conf.getLong(DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_KEY,
+      DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_DEFAULT);
     Boolean writeDropBehind =
         (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES) == null) ?
             null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES, false);
@@ -663,7 +661,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     checkOpen();
     long now = Time.monotonicNow();
     if ((serverDefaults == null) ||
-        (now - serverDefaultsLastUpdate > SERVER_DEFAULTS_VALIDITY_PERIOD)) {
+        (now - serverDefaultsLastUpdate > serverDefaultsValidityPeriod)) {
       serverDefaults = namenode.getServerDefaults();
       serverDefaultsLastUpdate = now;
     }
@@ -932,55 +930,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  /**
-   * Obtain the crypto protocol version from the provided FileEncryptionInfo,
-   * checking to see if this version is supported by.
-   *
-   * @param feInfo FileEncryptionInfo
-   * @return CryptoProtocolVersion from the feInfo
-   * @throws IOException if the protocol version is unsupported.
-   */
-  private static CryptoProtocolVersion getCryptoProtocolVersion(
-      FileEncryptionInfo feInfo) throws IOException {
-    final CryptoProtocolVersion version = feInfo.getCryptoProtocolVersion();
-    if (!CryptoProtocolVersion.supports(version)) {
-      throw new IOException("Client does not support specified " +
-          "CryptoProtocolVersion " + version.getDescription() + " version " +
-          "number" + version.getVersion());
-    }
-    return version;
-  }
-
-  /**
-   * Obtain a CryptoCodec based on the CipherSuite set in a FileEncryptionInfo
-   * and the available CryptoCodecs configured in the Configuration.
-   *
-   * @param conf   Configuration
-   * @param feInfo FileEncryptionInfo
-   * @return CryptoCodec
-   * @throws IOException if no suitable CryptoCodec for the CipherSuite is
-   *                     available.
-   */
-  private static CryptoCodec getCryptoCodec(Configuration conf,
-      FileEncryptionInfo feInfo) throws IOException {
-    final CipherSuite suite = feInfo.getCipherSuite();
-    if (suite.equals(CipherSuite.UNKNOWN)) {
-      throw new IOException("NameNode specified unknown CipherSuite with ID "
-          + suite.getUnknownValue() + ", cannot instantiate CryptoCodec.");
-    }
-    final CryptoCodec codec = CryptoCodec.getInstance(conf, suite);
-    if (codec == null) {
-      throw new UnknownCipherSuiteException(
-          "No configuration found for the cipher suite "
-              + suite.getConfigSuffix() + " prefixed with "
-              + HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
-              + ". Please see the example configuration "
-              + "hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE "
-              + "at core-default.xml for details.");
-    }
-    return codec;
-  }
-
   /**
    * Wraps the stream in a CryptoInputStream if the underlying file is
    * encrypted.
@@ -991,8 +940,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     if (feInfo != null) {
       // File is encrypted, wrap the stream in a crypto stream.
       // Currently only one version, so no special logic based on the version #
-      getCryptoProtocolVersion(feInfo);
-      final CryptoCodec codec = getCryptoCodec(conf, feInfo);
+      HdfsKMSUtil.getCryptoProtocolVersion(feInfo);
+      final CryptoCodec codec = HdfsKMSUtil.getCryptoCodec(conf, feInfo);
       final KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
       final CryptoInputStream cryptoIn =
           new CryptoInputStream(dfsis, codec, decrypted.getMaterial(),
@@ -1023,8 +972,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     if (feInfo != null) {
       // File is encrypted, wrap the stream in a crypto stream.
       // Currently only one version, so no special logic based on the version #
-      getCryptoProtocolVersion(feInfo);
-      final CryptoCodec codec = getCryptoCodec(conf, feInfo);
+      HdfsKMSUtil.getCryptoProtocolVersion(feInfo);
+      final CryptoCodec codec = HdfsKMSUtil.getCryptoCodec(conf, feInfo);
       KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
       final CryptoOutputStream cryptoOut =
           new CryptoOutputStream(dfsos, codec,
@@ -2136,14 +2085,16 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   /**
    * Get the difference between two snapshots, or between a snapshot and the
    * current tree of a directory.
-   * @see ClientProtocol#getSnapshotDiffReport(String, String, String)
+   * @see ClientProtocol#getSnapshotDiffReportListing
    */
-  public SnapshotDiffReport getSnapshotDiffReport(String snapshotDir,
-      String fromSnapshot, String toSnapshot) throws IOException {
+  public SnapshotDiffReportListing getSnapshotDiffReportListing(
+      String snapshotDir, String fromSnapshot, String toSnapshot,
+      byte[] startPath, int index) throws IOException {
     checkOpen();
     try (TraceScope ignored = tracer.newScope("getSnapshotDiffReport")) {
-      return namenode.getSnapshotDiffReport(snapshotDir,
-          fromSnapshot, toSnapshot);
+      return namenode
+          .getSnapshotDiffReportListing(snapshotDir, fromSnapshot, toSnapshot,
+              startPath, index);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
     }
@@ -2977,51 +2928,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     return HEDGED_READ_METRIC;
   }
 
-  /**
-   * Returns a key to map namenode uri to key provider uri.
-   * Tasks will lookup this key to find key Provider.
-   */
-  public Text getKeyProviderMapKey() {
-    return new Text(DFS_KMS_PREFIX + namenodeUri.getScheme()
-        +"://" + namenodeUri.getAuthority());
-  }
-
-  /**
-   * The key provider uri is searched in the following order.
-   * 1. If there is a mapping in Credential's secrets map for namenode uri.
-   * 2. From namenode getServerDefaults rpc.
-   * 3. Finally fallback to local conf.
-   * @return keyProviderUri if found from either of above 3 cases,
-   * null otherwise
-   * @throws IOException
-   */
   URI getKeyProviderUri() throws IOException {
-    URI keyProviderUri = null;
-    // Lookup the secret in credentials object for namenodeuri.
-    Credentials credentials = ugi.getCredentials();
-    byte[] keyProviderUriBytes = credentials.getSecretKey(getKeyProviderMapKey());
-    if(keyProviderUriBytes != null) {
-      keyProviderUri =
-          URI.create(DFSUtilClient.bytes2String(keyProviderUriBytes));
-      return keyProviderUri;
-    }
-
-    // Query the namenode for the key provider uri.
-    FsServerDefaults serverDefaults = getServerDefaults();
-    if (serverDefaults.getKeyProviderUri() != null) {
-      if (!serverDefaults.getKeyProviderUri().isEmpty()) {
-        keyProviderUri = URI.create(serverDefaults.getKeyProviderUri());
-      }
-      return keyProviderUri;
-    }
-
-    // Last thing is to trust its own conf to be backwards compatible.
-    String keyProviderUriStr = conf.getTrimmed(
-        CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
-    if (keyProviderUriStr != null && !keyProviderUriStr.isEmpty()) {
-      keyProviderUri = URI.create(keyProviderUriStr);
-    }
-    return keyProviderUri;
+    return HdfsKMSUtil.getKeyProviderUri(ugi, namenodeUri,
+        getServerDefaults().getKeyProviderUri(), conf);
   }
 
   public KeyProvider getKeyProvider() throws IOException {

+ 53 - 30
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java

@@ -23,7 +23,6 @@ import com.google.common.collect.Maps;
 import com.google.common.primitives.SignedBytes;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
@@ -54,7 +53,6 @@ import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.KMSUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -89,6 +87,7 @@ import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.Arrays;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_CLIENT_TCPNODELAY_DEFAULT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_CLIENT_TCPNODELAY_KEY;
@@ -124,6 +123,56 @@ public class DFSUtilClient {
     return bytes2String(bytes, 0, bytes.length);
   }
 
+  /**
+   * Converts a byte array to array of arrays of bytes
+   * on byte separator.
+   */
+  public static byte[][] bytes2byteArray(byte[] bytes) {
+    return bytes2byteArray(bytes, bytes.length, (byte)Path.SEPARATOR_CHAR);
+  }
+  /**
+   * Splits first len bytes in bytes to array of arrays of bytes
+   * on byte separator.
+   * @param bytes the byte array to split
+   * @param len the number of bytes to split
+   * @param separator the delimiting byte
+   */
+  public static byte[][] bytes2byteArray(byte[] bytes, int len,
+      byte separator) {
+    Preconditions.checkPositionIndex(len, bytes.length);
+    if (len == 0) {
+      return new byte[][]{null};
+    }
+    // Count the splits. Omit multiple separators and the last one by
+    // peeking at prior byte.
+    int splits = 0;
+    for (int i = 1; i < len; i++) {
+      if (bytes[i-1] == separator && bytes[i] != separator) {
+        splits++;
+      }
+    }
+    if (splits == 0 && bytes[0] == separator) {
+      return new byte[][]{null};
+    }
+    splits++;
+    byte[][] result = new byte[splits][];
+    int nextIndex = 0;
+    // Build the splits.
+    for (int i = 0; i < splits; i++) {
+      int startIndex = nextIndex;
+      // find next separator in the bytes.
+      while (nextIndex < len && bytes[nextIndex] != separator) {
+        nextIndex++;
+      }
+      result[i] = (nextIndex > 0)
+          ? Arrays.copyOfRange(bytes, startIndex, nextIndex)
+          : DFSUtilClient.EMPTY_BYTES; // reuse empty bytes for root.
+      do { // skip over separators.
+        nextIndex++;
+      } while (nextIndex < len && bytes[nextIndex] == separator);
+    }
+    return result;
+  }
   /** Return used as percentage of capacity */
   public static float getPercentUsed(long used, long capacity) {
     return capacity <= 0 ? 100 : (used * 100.0f)/capacity;
@@ -277,11 +326,9 @@ public class DFSUtilClient {
    * Given a list of path components returns a byte array
    */
   public static byte[] byteArray2bytes(byte[][] pathComponents) {
-    if (pathComponents.length == 0) {
+    if (pathComponents.length == 0 ||  (pathComponents.length == 1
+        && (pathComponents[0] == null || pathComponents[0].length == 0))) {
       return EMPTY_BYTES;
-    } else if (pathComponents.length == 1
-        && (pathComponents[0] == null || pathComponents[0].length == 0)) {
-      return new byte[]{(byte) Path.SEPARATOR_CHAR};
     }
     int length = 0;
     for (int i = 0; i < pathComponents.length; i++) {
@@ -547,30 +594,6 @@ public class DFSUtilClient {
     return new ReconfigurationProtocolTranslatorPB(addr, ticket, conf, factory);
   }
 
-  private static String keyProviderUriKeyName =
-      CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH;
-
-  /**
-   * Set the key provider uri configuration key name for creating key providers.
-   * @param keyName The configuration key name.
-   */
-  public static void setKeyProviderUriKeyName(final String keyName) {
-    keyProviderUriKeyName = keyName;
-  }
-
-  /**
-   * Creates a new KeyProvider from the given Configuration.
-   *
-   * @param conf Configuration
-   * @return new KeyProvider, or null if no provider was found.
-   * @throws IOException if the KeyProvider is improperly specified in
-   *                             the Configuration
-   */
-  public static KeyProvider createKeyProvider(
-      final Configuration conf) throws IOException {
-    return KMSUtil.createKeyProvider(conf, keyProviderUriKeyName);
-  }
-
   public static Peer peerFromSocket(Socket socket)
       throws IOException {
     Peer peer;

+ 51 - 24
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -21,11 +21,13 @@ package org.apache.hadoop.hdfs;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import org.apache.commons.collections.list.TreeList;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.CacheFlag;
@@ -90,12 +92,16 @@ import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing.DiffReportListingEntry;
+import org.apache.hadoop.hdfs.client.impl.SnapshotDiffReportGenerator;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.Progressable;
 
 import javax.annotation.Nonnull;
@@ -119,7 +125,8 @@ import java.util.stream.Collectors;
  *****************************************************************/
 @InterfaceAudience.LimitedPrivate({ "MapReduce", "HBase" })
 @InterfaceStability.Unstable
-public class DistributedFileSystem extends FileSystem {
+public class DistributedFileSystem extends FileSystem
+    implements KeyProviderTokenIssuer {
   private Path workingDir;
   private URI uri;
   private String homeDirPrefix =
@@ -1971,19 +1978,46 @@ public class DistributedFileSystem extends FileSystem {
     }.resolve(this, absF);
   }
 
+  private SnapshotDiffReport getSnapshotDiffReportInternal(
+      final String snapshotDir, final String fromSnapshot,
+      final String toSnapshot) throws IOException {
+    byte[] startPath = DFSUtilClient.EMPTY_BYTES;
+    int index = -1;
+    SnapshotDiffReportGenerator snapshotDiffReport;
+    List<DiffReportListingEntry> modifiedList = new TreeList();
+    List<DiffReportListingEntry> createdList = new ChunkedArrayList<>();
+    List<DiffReportListingEntry> deletedList = new ChunkedArrayList<>();
+    SnapshotDiffReportListing report;
+    do {
+      report = dfs.getSnapshotDiffReportListing(snapshotDir, fromSnapshot,
+          toSnapshot, startPath, index);
+      startPath = report.getLastPath();
+      index = report.getLastIndex();
+      modifiedList.addAll(report.getModifyList());
+      createdList.addAll(report.getCreateList());
+      deletedList.addAll(report.getDeleteList());
+    } while (!(Arrays.equals(startPath, DFSUtilClient.EMPTY_BYTES)
+        && index == -1));
+    snapshotDiffReport =
+        new SnapshotDiffReportGenerator(snapshotDir, fromSnapshot, toSnapshot,
+            report.getIsFromEarlier(), modifiedList, createdList, deletedList);
+    return snapshotDiffReport.generateReport();
+  }
+
   /**
    * Get the difference between two snapshots, or between a snapshot and the
    * current tree of a directory.
    *
-   * @see DFSClient#getSnapshotDiffReport(String, String, String)
+   * @see DFSClient#getSnapshotDiffReportListing
    */
   public SnapshotDiffReport getSnapshotDiffReport(final Path snapshotDir,
       final String fromSnapshot, final String toSnapshot) throws IOException {
     Path absF = fixRelativePart(snapshotDir);
     return new FileSystemLinkResolver<SnapshotDiffReport>() {
       @Override
-      public SnapshotDiffReport doCall(final Path p) throws IOException {
-        return dfs.getSnapshotDiffReport(getPathName(p), fromSnapshot,
+      public SnapshotDiffReport doCall(final Path p)
+          throws IOException {
+        return getSnapshotDiffReportInternal(getPathName(p), fromSnapshot,
             toSnapshot);
       }
 
@@ -2572,29 +2606,22 @@ public class DistributedFileSystem extends FileSystem {
     }.resolve(this, absF);
   }
 
+  @Override
+  public URI getKeyProviderUri() throws IOException {
+    return dfs.getKeyProviderUri();
+  }
+
+  @Override
+  public KeyProvider getKeyProvider() throws IOException {
+    return dfs.getKeyProvider();
+  }
+
   @Override
   public Token<?>[] addDelegationTokens(
       final String renewer, Credentials credentials) throws IOException {
     Token<?>[] tokens = super.addDelegationTokens(renewer, credentials);
-    URI keyProviderUri = dfs.getKeyProviderUri();
-    if (keyProviderUri != null) {
-      KeyProviderDelegationTokenExtension keyProviderDelegationTokenExtension =
-          KeyProviderDelegationTokenExtension.
-              createKeyProviderDelegationTokenExtension(dfs.getKeyProvider());
-      Token<?>[] kpTokens = keyProviderDelegationTokenExtension.
-          addDelegationTokens(renewer, credentials);
-      credentials.addSecretKey(dfs.getKeyProviderMapKey(),
-          DFSUtilClient.string2Bytes(keyProviderUri.toString()));
-      if (tokens != null && kpTokens != null) {
-        Token<?>[] all = new Token<?>[tokens.length + kpTokens.length];
-        System.arraycopy(tokens, 0, all, 0, tokens.length);
-        System.arraycopy(kpTokens, 0, all, tokens.length, kpTokens.length);
-        tokens = all;
-      } else {
-        tokens = (tokens != null) ? tokens : kpTokens;
-      }
-    }
-    return tokens;
+    return HdfsKMSUtil.addDelegationTokensForKeyProvider(
+        this, renewer, credentials, uri, tokens);
   }
 
   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {

+ 190 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java

@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CipherSuite;
+import org.apache.hadoop.crypto.CryptoCodec;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
+import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.KMSUtil;
+
+/**
+ * Utility class for key provider related methods in hdfs client package.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class HdfsKMSUtil {
+  private static final String DFS_KMS_PREFIX = "dfs-kms-";
+  private static String keyProviderUriKeyName =
+      CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH;
+
+  private HdfsKMSUtil() { /* Hidden constructor */ }
+
+  /**
+   * Creates a new KeyProvider from the given Configuration.
+   *
+   * @param conf Configuration
+   * @return new KeyProvider, or null if no provider was found.
+   * @throws IOException if the KeyProvider is improperly specified in
+   *                             the Configuration
+   */
+  public static KeyProvider createKeyProvider(
+      final Configuration conf) throws IOException {
+    return KMSUtil.createKeyProvider(conf, keyProviderUriKeyName);
+  }
+
+  public static Token<?>[] addDelegationTokensForKeyProvider(
+      KeyProviderTokenIssuer kpTokenIssuer, final String renewer,
+      Credentials credentials, URI namenodeUri, Token<?>[] tokens)
+          throws IOException {
+    KeyProvider keyProvider = kpTokenIssuer.getKeyProvider();
+    if (keyProvider != null) {
+      KeyProviderDelegationTokenExtension keyProviderDelegationTokenExtension
+          = KeyProviderDelegationTokenExtension.
+              createKeyProviderDelegationTokenExtension(keyProvider);
+      Token<?>[] kpTokens = keyProviderDelegationTokenExtension.
+          addDelegationTokens(renewer, credentials);
+      credentials.addSecretKey(getKeyProviderMapKey(namenodeUri),
+          DFSUtilClient.string2Bytes(
+              kpTokenIssuer.getKeyProviderUri().toString()));
+      if (tokens != null && kpTokens != null) {
+        Token<?>[] all = new Token<?>[tokens.length + kpTokens.length];
+        System.arraycopy(tokens, 0, all, 0, tokens.length);
+        System.arraycopy(kpTokens, 0, all, tokens.length, kpTokens.length);
+        tokens = all;
+      } else {
+        tokens = (tokens != null) ? tokens : kpTokens;
+      }
+    }
+    return tokens;
+  }
+
+  /**
+   * Obtain the crypto protocol version from the provided FileEncryptionInfo,
+   * checking to see if this version is supported by.
+   *
+   * @param feInfo FileEncryptionInfo
+   * @return CryptoProtocolVersion from the feInfo
+   * @throws IOException if the protocol version is unsupported.
+   */
+  public static CryptoProtocolVersion getCryptoProtocolVersion(
+      FileEncryptionInfo feInfo) throws IOException {
+    final CryptoProtocolVersion version = feInfo.getCryptoProtocolVersion();
+    if (!CryptoProtocolVersion.supports(version)) {
+      throw new IOException("Client does not support specified " +
+          "CryptoProtocolVersion " + version.getDescription() + " version " +
+          "number" + version.getVersion());
+    }
+    return version;
+  }
+
+  /**
+   * Obtain a CryptoCodec based on the CipherSuite set in a FileEncryptionInfo
+   * and the available CryptoCodecs configured in the Configuration.
+   *
+   * @param conf   Configuration
+   * @param feInfo FileEncryptionInfo
+   * @return CryptoCodec
+   * @throws IOException if no suitable CryptoCodec for the CipherSuite is
+   *                     available.
+   */
+  public static CryptoCodec getCryptoCodec(Configuration conf,
+      FileEncryptionInfo feInfo) throws IOException {
+    final CipherSuite suite = feInfo.getCipherSuite();
+    if (suite.equals(CipherSuite.UNKNOWN)) {
+      throw new IOException("NameNode specified unknown CipherSuite with ID "
+          + suite.getUnknownValue() + ", cannot instantiate CryptoCodec.");
+    }
+    final CryptoCodec codec = CryptoCodec.getInstance(conf, suite);
+    if (codec == null) {
+      throw new UnknownCipherSuiteException(
+          "No configuration found for the cipher suite "
+              + suite.getConfigSuffix() + " prefixed with "
+              + HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
+              + ". Please see the example configuration "
+              + "hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE "
+              + "at core-default.xml for details.");
+    }
+    return codec;
+  }
+
+  /**
+   * The key provider uri is searched in the following order.
+   * 1. If there is a mapping in Credential's secrets map for namenode uri.
+   * 2. From namenode getServerDefaults call.
+   * 3. Finally fallback to local conf.
+   * @return keyProviderUri if found from either of above 3 cases,
+   * null otherwise
+   * @throws IOException
+   */
+  public static URI getKeyProviderUri(UserGroupInformation ugi,
+      URI namenodeUri, String keyProviderUriStr, Configuration conf)
+          throws IOException {
+    URI keyProviderUri = null;
+    // Lookup the secret in credentials object for namenodeuri.
+    Credentials credentials = ugi.getCredentials();
+    byte[] keyProviderUriBytes =
+        credentials.getSecretKey(getKeyProviderMapKey(namenodeUri));
+    if(keyProviderUriBytes != null) {
+      keyProviderUri =
+          URI.create(DFSUtilClient.bytes2String(keyProviderUriBytes));
+      return keyProviderUri;
+    }
+
+    if (keyProviderUriStr != null) {
+      if (!keyProviderUriStr.isEmpty()) {
+        keyProviderUri = URI.create(keyProviderUriStr);
+      }
+      return keyProviderUri;
+    }
+
+    // Last thing is to trust its own conf to be backwards compatible.
+    String keyProviderUriFromConf = conf.getTrimmed(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
+    if (keyProviderUriFromConf != null && !keyProviderUriFromConf.isEmpty()) {
+      keyProviderUri = URI.create(keyProviderUriFromConf);
+    }
+    return keyProviderUri;
+  }
+
+  /**
+   * Returns a key to map namenode uri to key provider uri.
+   * Tasks will lookup this key to find key Provider.
+   */
+  public static Text getKeyProviderMapKey(URI namenodeUri) {
+    return new Text(DFS_KMS_PREFIX + namenodeUri.getScheme()
+        +"://" + namenodeUri.getAuthority());
+  }
+}

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

@@ -112,6 +112,10 @@ public interface HdfsClientConfigKeys {
   String  DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY =
       "dfs.client.max.block.acquire.failures";
   int     DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT = 3;
+  String  DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_KEY =
+      "dfs.client.server-defaults.validity.period.ms";
+  long    DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_DEFAULT =
+      TimeUnit.HOURS.toMillis(1);
   String  DFS_CHECKSUM_TYPE_KEY = "dfs.checksum.type";
   String  DFS_CHECKSUM_TYPE_DEFAULT = "CRC32C";
   String  DFS_BYTES_PER_CHECKSUM_KEY = "dfs.bytes-per-checksum";

+ 262 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/SnapshotDiffReportGenerator.java

@@ -0,0 +1,262 @@
+/**
+ * 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.client.impl;
+
+import java.util.*;
+
+import com.google.common.primitives.SignedBytes;
+
+import org.apache.hadoop.util.ChunkedArrayList;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing.DiffReportListingEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+/**
+ * This class represents to end users the difference between two snapshots of
+ * the same directory, or the difference between a snapshot of the directory and
+ * its current state. Instead of capturing all the details of the diff, this
+ * class only lists where the changes happened and their types.
+ */
+public class SnapshotDiffReportGenerator {
+  /**
+   * Compare two inodes based on their full names.
+   */
+  public static final Comparator<DiffReportListingEntry> INODE_COMPARATOR =
+      new Comparator<DiffReportListingEntry>() {
+        @Override
+        public int compare(DiffReportListingEntry left,
+            DiffReportListingEntry right) {
+          final Comparator<byte[]> cmp =
+              SignedBytes.lexicographicalComparator();
+          //source path can never be null
+          final byte[][] l = left.getSourcePath();
+          final byte[][] r = right.getSourcePath();
+          if (l.length == 1 && l[0] == null) {
+            return -1;
+          } else if (r.length == 1 && r[0] == null) {
+            return 1;
+          } else {
+            for (int i = 0; i < l.length && i < r.length; i++) {
+              final int diff = cmp.compare(l[i], r[i]);
+              if (diff != 0) {
+                return diff;
+              }
+            }
+            return l.length == r.length ? 0 : l.length > r.length ? 1 : -1;
+          }
+        }
+      };
+
+  static class RenameEntry {
+    private byte[][] sourcePath;
+    private byte[][] targetPath;
+
+    void setSource(byte[][] srcPath) {
+      this.sourcePath = srcPath;
+    }
+
+    void setTarget(byte[][] target) {
+      this.targetPath = target;
+    }
+
+    boolean isRename() {
+      return sourcePath != null && targetPath != null;
+    }
+
+    byte[][] getSourcePath() {
+      return sourcePath;
+    }
+
+    byte[][] getTargetPath() {
+      return targetPath;
+    }
+  }
+
+  /*
+   * A class represnting the diff in a directory between two given snapshots
+   * in two lists: createdList and deleted list.
+   */
+  static class ChildrenDiff {
+    private final List<DiffReportListingEntry> createdList;
+    private final List<DiffReportListingEntry> deletedList;
+
+    ChildrenDiff(List<DiffReportListingEntry> createdList,
+        List<DiffReportListingEntry> deletedList) {
+      this.createdList = createdList != null ? createdList :
+          Collections.emptyList();
+      this.deletedList = deletedList != null ? deletedList :
+          Collections.emptyList();
+    }
+
+    public List<DiffReportListingEntry> getCreatedList() {
+      return createdList;
+    }
+
+    public List<DiffReportListingEntry> getDeletedList() {
+      return deletedList;
+    }
+  }
+
+  /**
+   * snapshot root full path.
+   */
+  private final String snapshotRoot;
+
+  /**
+   * start point of the diff.
+   */
+  private final String fromSnapshot;
+
+  /**
+   * end point of the diff.
+   */
+  private final String toSnapshot;
+
+  /**
+   * Flag to indicate the diff is calculated from older to newer snapshot
+   * or not.
+   */
+  private final boolean isFromEarlier;
+
+  /**
+   * A map capturing the detailed difference about file creation/deletion.
+   * Each key indicates a directory inode whose children have been changed
+   * between the two snapshots, while its associated value is a
+   * {@link ChildrenDiff} storing the changes (creation/deletion) happened to
+   * the children (files).
+   */
+  private final Map<Long, ChildrenDiff> dirDiffMap =
+      new HashMap<>();
+
+  private final Map<Long, RenameEntry> renameMap =
+      new HashMap<>();
+
+  private List<DiffReportListingEntry> mlist = null;
+  private List<DiffReportListingEntry> clist = null;
+  private List<DiffReportListingEntry> dlist = null;
+
+  public SnapshotDiffReportGenerator(String snapshotRoot, String fromSnapshot,
+      String toSnapshot, boolean isFromEarlier,
+      List<DiffReportListingEntry> mlist, List<DiffReportListingEntry> clist,
+      List<DiffReportListingEntry> dlist) {
+    this.snapshotRoot = snapshotRoot;
+    this.fromSnapshot = fromSnapshot;
+    this.toSnapshot = toSnapshot;
+    this.isFromEarlier = isFromEarlier;
+    this.mlist =
+        mlist != null ? mlist : Collections.emptyList();
+    this.clist =
+        clist != null ? clist : Collections.emptyList();
+    this.dlist =
+        dlist != null ? dlist : Collections.emptyList();
+  }
+
+  private RenameEntry getEntry(long inodeId) {
+    RenameEntry entry = renameMap.get(inodeId);
+    if (entry == null) {
+      entry = new RenameEntry();
+      renameMap.put(inodeId, entry);
+    }
+    return entry;
+  }
+
+  public void generateReportList() {
+    mlist.sort(INODE_COMPARATOR);
+    for (DiffReportListingEntry created : clist) {
+      ChildrenDiff entry = dirDiffMap.get(created.getDirId());
+      if (entry == null) {
+        List<DiffReportListingEntry> createdList = new ChunkedArrayList<>();
+        createdList.add(created);
+        ChildrenDiff list = new ChildrenDiff(createdList, null);
+        dirDiffMap.put(created.getDirId(), list);
+      } else {
+        dirDiffMap.get(created.getDirId()).getCreatedList().add(created);
+      }
+      if (created.isReference()) {
+        RenameEntry renameEntry = getEntry(created.getFileId());
+        if (renameEntry.getTargetPath() != null) {
+          renameEntry.setTarget(created.getSourcePath());
+        }
+      }
+    }
+    for (DiffReportListingEntry deleted : dlist) {
+      ChildrenDiff entry = dirDiffMap.get(deleted.getDirId());
+      if (entry == null || (entry.getDeletedList().isEmpty())) {
+        ChildrenDiff list;
+        List<DiffReportListingEntry> deletedList = new ChunkedArrayList<>();
+        deletedList.add(deleted);
+        if (entry == null) {
+          list = new ChildrenDiff(null, deletedList);
+        } else {
+          list = new ChildrenDiff(entry.getCreatedList(), deletedList);
+        }
+        dirDiffMap.put(deleted.getDirId(), list);
+      } else {
+        entry.getDeletedList().add(deleted);
+      }
+      if (deleted.isReference()) {
+        RenameEntry renameEntry = getEntry(deleted.getFileId());
+        renameEntry.setTarget(deleted.getTargetPath());
+        renameEntry.setSource(deleted.getSourcePath());
+      }
+    }
+  }
+
+  public SnapshotDiffReport generateReport() {
+    List<DiffReportEntry> diffReportList = new ChunkedArrayList<>();
+    generateReportList();
+    for (DiffReportListingEntry modified : mlist) {
+      diffReportList.add(
+          new DiffReportEntry(DiffType.MODIFY, modified.getSourcePath(), null));
+      if (modified.isReference()
+          && dirDiffMap.get(modified.getDirId()) != null) {
+        List<DiffReportEntry> subList = generateReport(modified);
+        diffReportList.addAll(subList);
+      }
+    }
+    return new SnapshotDiffReport(snapshotRoot, fromSnapshot, toSnapshot,
+        diffReportList);
+  }
+
+  private List<DiffReportEntry> generateReport(
+      DiffReportListingEntry modified) {
+    List<DiffReportEntry> diffReportList = new ChunkedArrayList<>();
+    ChildrenDiff list = dirDiffMap.get(modified.getDirId());
+    for (DiffReportListingEntry created : list.getCreatedList()) {
+      RenameEntry entry = renameMap.get(created.getFileId());
+      if (entry == null || !entry.isRename()) {
+        diffReportList.add(new DiffReportEntry(
+            isFromEarlier ? DiffType.CREATE : DiffType.DELETE,
+            created.getSourcePath()));
+      }
+    }
+    for (DiffReportListingEntry deleted : list.getDeletedList()) {
+      RenameEntry entry = renameMap.get(deleted.getFileId());
+      if (entry != null && entry.isRename()) {
+        diffReportList.add(new DiffReportEntry(DiffType.RENAME,
+            isFromEarlier ? entry.getSourcePath() : entry.getTargetPath(),
+            isFromEarlier ? entry.getTargetPath() : entry.getSourcePath()));
+      } else {
+        diffReportList.add(new DiffReportEntry(
+            isFromEarlier ? DiffType.DELETE : DiffType.CREATE,
+            deleted.getSourcePath()));
+      }
+    }
+    return diffReportList;
+  }
+}

+ 29 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -1288,6 +1288,35 @@ public interface ClientProtocol {
   SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
       String fromSnapshot, String toSnapshot) throws IOException;
 
+  /**
+   * Get the difference between two snapshots, or between a snapshot and the
+   * current tree of a directory.
+   *
+   * @param snapshotRoot
+   *          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.
+   * @param startPath
+   *          path relative to the snapshottable root directory from where the
+   *          snapshotdiff computation needs to start across multiple rpc calls
+   * @param index
+   *           index in the created or deleted list of the directory at which
+   *           the snapshotdiff computation stopped during the last rpc call
+   *           as the no of entries exceeded the snapshotdiffentry limit. -1
+   *           indicates, the snapshotdiff compuatation needs to start right
+   *           from the startPath provided.
+   * @return The difference report represented as a {@link SnapshotDiffReport}.
+   * @throws IOException on error
+   */
+  @Idempotent
+  SnapshotDiffReportListing getSnapshotDiffReportListing(String snapshotRoot,
+      String fromSnapshot, String toSnapshot, byte[] startPath, int index)
+      throws IOException;
+
   /**
    * Add a CacheDirective to the CacheManager.
    *

+ 283 - 247
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java

@@ -18,271 +18,40 @@
 package org.apache.hadoop.hdfs.protocol;
 
 import java.io.IOException;
+import java.io.ObjectInputValidation;
+import java.io.Serializable;
 import java.net.URI;
 import java.util.Arrays;
 import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileStatus.AttrFlags;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.io.Writable;
 
-/** Interface that represents the over the wire information for a file.
+/**
+ * HDFS metadata for an entity in the filesystem.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class HdfsFileStatus extends FileStatus {
-
-  private static final long serialVersionUID = 0x126eb82a;
-
-  // local name of the inode that's encoded in java UTF8
-  private byte[] uPath;
-  private byte[] uSymlink; // symlink target encoded in java UTF8/null
-  private final long fileId;
-  private final FileEncryptionInfo feInfo;
-  private final ErasureCodingPolicy ecPolicy;
+public interface HdfsFileStatus
+    extends Writable, Comparable<Object>, Serializable, ObjectInputValidation {
 
-  // Used by dir, not including dot and dotdot. Always zero for a regular file.
-  private final int childrenNum;
-  private final byte storagePolicy;
+  byte[] EMPTY_NAME = new byte[0];
 
-  public static final byte[] EMPTY_NAME = new byte[0];
-
-  /**
-   * Set of features potentially active on an instance.
-   */
-  public enum Flags {
+  /** Set of features potentially active on an instance. */
+  enum Flags {
     HAS_ACL,
     HAS_CRYPT,
     HAS_EC,
     SNAPSHOT_ENABLED
   }
-  private final EnumSet<Flags> flags;
-
-  /**
-   * Constructor.
-   * @param length the number of bytes the file has
-   * @param isdir if the path is a directory
-   * @param replication the replication factor
-   * @param blocksize the block size
-   * @param mtime modification time
-   * @param atime access time
-   * @param permission permission
-   * @param owner the owner of the path
-   * @param group the group of the path
-   * @param symlink symlink target encoded in java UTF8 or null
-   * @param path the local name in java UTF8 encoding the same as that in-memory
-   * @param fileId the file id
-   * @param childrenNum the number of children. Used by directory.
-   * @param feInfo the file's encryption info
-   * @param storagePolicy ID which specifies storage policy
-   * @param ecPolicy the erasure coding policy
-   */
-  protected HdfsFileStatus(long length, boolean isdir, int replication,
-                         long blocksize, long mtime, long atime,
-                         FsPermission permission, EnumSet<Flags> flags,
-                         String owner, String group,
-                         byte[] symlink, byte[] path, long fileId,
-                         int childrenNum, FileEncryptionInfo feInfo,
-                         byte storagePolicy, ErasureCodingPolicy ecPolicy) {
-    super(length, isdir, replication, blocksize, mtime,
-        atime, convert(isdir, symlink != null, permission, flags),
-        owner, group, null, null,
-        flags.contains(Flags.HAS_ACL), flags.contains(Flags.HAS_CRYPT),
-        flags.contains(Flags.HAS_EC));
-    this.flags = flags;
-    this.uSymlink = symlink;
-    this.uPath = path;
-    this.fileId = fileId;
-    this.childrenNum = childrenNum;
-    this.feInfo = feInfo;
-    this.storagePolicy = storagePolicy;
-    this.ecPolicy = ecPolicy;
-  }
-
-  /**
-   * Set redundant flags for compatibility with existing applications.
-   */
-  protected static FsPermission convert(boolean isdir, boolean symlink,
-      FsPermission p, EnumSet<Flags> f) {
-    if (p instanceof FsPermissionExtension) {
-      // verify flags are set consistently
-      assert p.getAclBit() == f.contains(HdfsFileStatus.Flags.HAS_ACL);
-      assert p.getEncryptedBit() == f.contains(HdfsFileStatus.Flags.HAS_CRYPT);
-      assert p.getErasureCodedBit() == f.contains(HdfsFileStatus.Flags.HAS_EC);
-      return p;
-    }
-    if (null == p) {
-      if (isdir) {
-        p = FsPermission.getDirDefault();
-      } else if (symlink) {
-        p = FsPermission.getDefault();
-      } else {
-        p = FsPermission.getFileDefault();
-      }
-    }
-    return new FsPermissionExtension(p, f.contains(Flags.HAS_ACL),
-        f.contains(Flags.HAS_CRYPT), f.contains(Flags.HAS_EC));
-  }
-
-  @Override
-  public boolean isSymlink() {
-    return uSymlink != null;
-  }
-
-  @Override
-  public boolean hasAcl() {
-    return flags.contains(Flags.HAS_ACL);
-  }
-
-  @Override
-  public boolean isEncrypted() {
-    return flags.contains(Flags.HAS_CRYPT);
-  }
-
-  @Override
-  public boolean isErasureCoded() {
-    return flags.contains(Flags.HAS_EC);
-  }
-
-  /**
-   * Check if the local name is empty.
-   * @return true if the name is empty
-   */
-  public final boolean isEmptyLocalName() {
-    return uPath.length == 0;
-  }
-
-  /**
-   * Get the string representation of the local name.
-   * @return the local name in string
-   */
-  public final String getLocalName() {
-    return DFSUtilClient.bytes2String(uPath);
-  }
-
-  /**
-   * Get the Java UTF8 representation of the local name.
-   * @return the local name in java UTF8
-   */
-  public final byte[] getLocalNameInBytes() {
-    return uPath;
-  }
-
-  /**
-   * Get the string representation of the full path name.
-   * @param parent the parent path
-   * @return the full path in string
-   */
-  public final String getFullName(final String parent) {
-    if (isEmptyLocalName()) {
-      return parent;
-    }
-
-    StringBuilder fullName = new StringBuilder(parent);
-    if (!parent.endsWith(Path.SEPARATOR)) {
-      fullName.append(Path.SEPARATOR);
-    }
-    fullName.append(getLocalName());
-    return fullName.toString();
-  }
-
-  /**
-   * Get the full path.
-   * @param parent the parent path
-   * @return the full path
-   */
-  public final Path getFullPath(final Path parent) {
-    if (isEmptyLocalName()) {
-      return parent;
-    }
-
-    return new Path(parent, getLocalName());
-  }
-
-  @Override
-  public Path getSymlink() throws IOException {
-    if (isSymlink()) {
-      return new Path(DFSUtilClient.bytes2String(uSymlink));
-    }
-    throw new IOException("Path " + getPath() + " is not a symbolic link");
-  }
-
-  @Override
-  public void setSymlink(Path sym) {
-    uSymlink = DFSUtilClient.string2Bytes(sym.toString());
-  }
-
-  /**
-   * Opaque referant for the symlink, to be resolved at the client.
-   */
-  public final byte[] getSymlinkInBytes() {
-    return uSymlink;
-  }
-
-  public final long getFileId() {
-    return fileId;
-  }
-
-  public final FileEncryptionInfo getFileEncryptionInfo() {
-    return feInfo;
-  }
-
-  /**
-   * Get the erasure coding policy if it's set.
-   * @return the erasure coding policy
-   */
-  public ErasureCodingPolicy getErasureCodingPolicy() {
-    return ecPolicy;
-  }
-
-  public final int getChildrenNum() {
-    return childrenNum;
-  }
-
-  /** @return the storage policy id */
-  public final byte getStoragePolicy() {
-    return storagePolicy;
-  }
-
-  /**
-   * Check if directory is Snapshot enabled or not.
-   *
-   * @return true if directory is snapshot enabled
-   */
-  public boolean isSnapshotEnabled() {
-    return flags.contains(Flags.SNAPSHOT_ENABLED);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    // satisfy findbugs
-    return super.equals(o);
-  }
-
-  @Override
-  public int hashCode() {
-    // satisfy findbugs
-    return super.hashCode();
-  }
-
-  /**
-   * Resolve the short name of the Path given the URI, parent provided. This
-   * FileStatus reference will not contain a valid Path until it is resolved
-   * by this method.
-   * @param defaultUri FileSystem to fully qualify HDFS path.
-   * @param parent Parent path of this element.
-   * @return Reference to this instance.
-   */
-  public final FileStatus makeQualified(URI defaultUri, Path parent) {
-    // fully-qualify path
-    setPath(getFullPath(parent).makeQualified(defaultUri, null));
-    return this; // API compatibility
-
-  }
 
   /**
    * Builder class for HdfsFileStatus instances. Note default values for
@@ -290,7 +59,7 @@ public class HdfsFileStatus extends FileStatus {
    */
   @InterfaceAudience.Private
   @InterfaceStability.Unstable
-  public static class Builder {
+  class Builder {
     // Changing default values will affect cases where values are not
     // specified. Be careful!
     private long length                    = 0L;
@@ -311,6 +80,7 @@ public class HdfsFileStatus extends FileStatus {
     private byte storagePolicy             =
         HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
     private ErasureCodingPolicy ecPolicy   = null;
+    private LocatedBlocks locations        = null;
 
     /**
      * Set the length of the entity (default = 0).
@@ -489,14 +259,280 @@ public class HdfsFileStatus extends FileStatus {
       return this;
     }
 
+    /**
+     * Set the block locations for this entity (default = null).
+     * @param locations HDFS locations
+     *       (see {@link HdfsLocatedFileStatus#makeQualifiedLocated(URI, Path)})
+     * @return This Builder instance
+     */
+    public Builder locations(LocatedBlocks locations) {
+      this.locations = locations;
+      return this;
+    }
+
     /**
      * @return An {@link HdfsFileStatus} instance from these parameters.
      */
     public HdfsFileStatus build() {
-      return new HdfsFileStatus(length, isdir, replication, blocksize,
-          mtime, atime, permission, flags, owner, group, symlink, path, fileId,
-          childrenNum, feInfo, storagePolicy, ecPolicy);
+      if (null == locations && !isdir && null == symlink) {
+        return new HdfsNamedFileStatus(length, isdir, replication, blocksize,
+            mtime, atime, permission, flags, owner, group, symlink, path,
+            fileId, childrenNum, feInfo, storagePolicy, ecPolicy);
+      }
+      return new HdfsLocatedFileStatus(length, isdir, replication, blocksize,
+          mtime, atime, permission, flags, owner, group, symlink, path,
+          fileId, childrenNum, feInfo, storagePolicy, ecPolicy, locations);
+    }
+
+  }
+
+  ///////////////////
+  // HDFS-specific //
+  ///////////////////
+
+  /**
+   * Inode ID for this entity, if a file.
+   * @return inode ID.
+   */
+  long getFileId();
+
+  /**
+   * Get metadata for encryption, if present.
+   * @return the {@link FileEncryptionInfo} for this stream, or null if not
+   *         encrypted.
+   */
+  FileEncryptionInfo getFileEncryptionInfo();
+
+  /**
+   * Check if the local name is empty.
+   * @return true if the name is empty
+   */
+  default boolean isEmptyLocalName() {
+    return getLocalNameInBytes().length == 0;
+  }
+
+  /**
+   * Get the string representation of the local name.
+   * @return the local name in string
+   */
+  default String getLocalName() {
+    return DFSUtilClient.bytes2String(getLocalNameInBytes());
+  }
+
+  /**
+   * Get the Java UTF8 representation of the local name.
+   * @return the local name in java UTF8
+   */
+  byte[] getLocalNameInBytes();
+
+  /**
+   * Get the string representation of the full path name.
+   * @param parent the parent path
+   * @return the full path in string
+   */
+  default String getFullName(String parent) {
+    if (isEmptyLocalName()) {
+      return parent;
+    }
+
+    StringBuilder fullName = new StringBuilder(parent);
+    if (!parent.endsWith(Path.SEPARATOR)) {
+      fullName.append(Path.SEPARATOR);
+    }
+    fullName.append(getLocalName());
+    return fullName.toString();
+  }
+
+  /**
+   * Get the full path.
+   * @param parent the parent path
+   * @return the full path
+   */
+  default Path getFullPath(Path parent) {
+    if (isEmptyLocalName()) {
+      return parent;
+    }
+
+    return new Path(parent, getLocalName());
+  }
+
+  /**
+   * Opaque referant for the symlink, to be resolved at the client.
+   */
+  byte[] getSymlinkInBytes();
+
+  /**
+   * @return number of children for this inode.
+   */
+  int getChildrenNum();
+
+  /**
+   * Get the erasure coding policy if it's set.
+   * @return the erasure coding policy
+   */
+  ErasureCodingPolicy getErasureCodingPolicy();
+
+  /** @return the storage policy id */
+  byte getStoragePolicy();
+
+  /**
+   * Resolve the short name of the Path given the URI, parent provided. This
+   * FileStatus reference will not contain a valid Path until it is resolved
+   * by this method.
+   * @param defaultUri FileSystem to fully qualify HDFS path.
+   * @param parent Parent path of this element.
+   * @return Reference to this instance.
+   */
+  default FileStatus makeQualified(URI defaultUri, Path parent) {
+    // fully-qualify path
+    setPath(getFullPath(parent).makeQualified(defaultUri, null));
+    return (FileStatus) this; // API compatibility
+  }
+
+  ////////////////////////////
+  // FileStatus "overrides" //
+  ////////////////////////////
+
+  /**
+   * See {@link FileStatus#getPath()}.
+   */
+  Path getPath();
+  /**
+   * See {@link FileStatus#setPath(Path)}.
+   */
+  void setPath(Path p);
+  /**
+   * See {@link FileStatus#getLen()}.
+   */
+  long getLen();
+  /**
+   * See {@link FileStatus#isFile()}.
+   */
+  boolean isFile();
+  /**
+   * See {@link FileStatus#isDirectory()}.
+   */
+  boolean isDirectory();
+  /**
+   * See {@link FileStatus#isDir()}.
+   */
+  boolean isDir();
+  /**
+   * See {@link FileStatus#isSymlink()}.
+   */
+  boolean isSymlink();
+  /**
+   * See {@link FileStatus#getBlockSize()}.
+   */
+  long getBlockSize();
+  /**
+   * See {@link FileStatus#getReplication()}.
+   */
+  short getReplication();
+  /**
+   * See {@link FileStatus#getModificationTime()}.
+   */
+  long getModificationTime();
+  /**
+   * See {@link FileStatus#getAccessTime()}.
+   */
+  long getAccessTime();
+  /**
+   * See {@link FileStatus#getPermission()}.
+   */
+  FsPermission getPermission();
+  /**
+   * See {@link FileStatus#setPermission(FsPermission)}.
+   */
+  void setPermission(FsPermission permission);
+  /**
+   * See {@link FileStatus#getOwner()}.
+   */
+  String getOwner();
+  /**
+   * See {@link FileStatus#setOwner(String)}.
+   */
+  void setOwner(String owner);
+  /**
+   * See {@link FileStatus#getGroup()}.
+   */
+  String getGroup();
+  /**
+   * See {@link FileStatus#setGroup(String)}.
+   */
+  void setGroup(String group);
+  /**
+   * See {@link FileStatus#hasAcl()}.
+   */
+  boolean hasAcl();
+  /**
+   * See {@link FileStatus#isEncrypted()}.
+   */
+  boolean isEncrypted();
+  /**
+   * See {@link FileStatus#isErasureCoded()}.
+   */
+  boolean isErasureCoded();
+  /**
+   * See {@link FileStatus#isSnapshotEnabled()}.
+   */
+  boolean isSnapshotEnabled();
+  /**
+   * See {@link FileStatus#getSymlink()}.
+   */
+  Path getSymlink() throws IOException;
+  /**
+   * See {@link FileStatus#setSymlink(Path sym)}.
+   */
+  void setSymlink(Path sym);
+  /**
+   * See {@link FileStatus#compareTo(FileStatus)}.
+   */
+  int compareTo(FileStatus stat);
+
+  /**
+   * Set redundant flags for compatibility with existing applications.
+   */
+  static FsPermission convert(boolean isdir, boolean symlink,
+                              FsPermission p, Set<Flags> f) {
+    if (p instanceof FsPermissionExtension) {
+      // verify flags are set consistently
+      assert p.getAclBit() == f.contains(HdfsFileStatus.Flags.HAS_ACL);
+      assert p.getEncryptedBit() == f.contains(HdfsFileStatus.Flags.HAS_CRYPT);
+      assert p.getErasureCodedBit() == f.contains(HdfsFileStatus.Flags.HAS_EC);
+      return p;
+    }
+    if (null == p) {
+      if (isdir) {
+        p = FsPermission.getDirDefault();
+      } else if (symlink) {
+        p = FsPermission.getDefault();
+      } else {
+        p = FsPermission.getFileDefault();
+      }
+    }
+    return new FsPermissionExtension(p, f.contains(Flags.HAS_ACL),
+        f.contains(Flags.HAS_CRYPT), f.contains(Flags.HAS_EC));
+  }
+
+  static Set<AttrFlags> convert(Set<Flags> flags) {
+    if (flags.isEmpty()) {
+      return FileStatus.NONE;
+    }
+    EnumSet<AttrFlags> attr = EnumSet.noneOf(AttrFlags.class);
+    if (flags.contains(Flags.HAS_ACL)) {
+      attr.add(AttrFlags.HAS_ACL);
+    }
+    if (flags.contains(Flags.HAS_EC)) {
+      attr.add(AttrFlags.HAS_EC);
+    }
+    if (flags.contains(Flags.HAS_CRYPT)) {
+      attr.add(AttrFlags.HAS_CRYPT);
+    }
+    if (flags.contains(Flags.SNAPSHOT_ENABLED)) {
+      attr.add(AttrFlags.SNAPSHOT_ENABLED);
     }
+    return attr;
   }
 
 }

+ 159 - 49
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import java.io.IOException;
 import java.net.URI;
 import java.util.EnumSet;
 
@@ -29,71 +30,150 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 
 /**
- * Interface that represents the over the wire information
- * including block locations for a file.
+ * HDFS metadata for an entity in the filesystem with locations. Note that
+ * symlinks and directories are returned as {@link HdfsLocatedFileStatus} for
+ * backwards compatibility.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class HdfsLocatedFileStatus extends HdfsFileStatus {
+public class HdfsLocatedFileStatus
+    extends LocatedFileStatus implements HdfsFileStatus {
+  private static final long serialVersionUID = 0x126eb82a;
 
-  private static final long serialVersionUID = 0x23c73328;
+  // local name of the inode that's encoded in java UTF8
+  private byte[] uPath;
+  private byte[] uSymlink; // symlink target encoded in java UTF8/null
+  private final long fileId;
+  private final FileEncryptionInfo feInfo;
+  private final ErasureCodingPolicy ecPolicy;
 
-  /**
-   * Left transient, because {@link #makeQualifiedLocated(URI,Path)}
-   * is the user-facing type.
-   */
-  private transient LocatedBlocks locations;
+  // Used by dir, not including dot and dotdot. Always zero for a regular file.
+  private final int childrenNum;
+  private final byte storagePolicy;
+
+  // BlockLocations[] is the user-facing type
+  private transient LocatedBlocks hdfsloc;
 
   /**
-   * Constructor
-   *
-   * @param length size
-   * @param isdir if this is directory
-   * @param block_replication the file's replication factor
-   * @param blocksize the file's block size
-   * @param modification_time most recent modification time
-   * @param access_time most recent access time
+   * Constructor.
+   * @param length the number of bytes the file has
+   * @param isdir if the path is a directory
+   * @param replication the replication factor
+   * @param blocksize the block size
+   * @param mtime modification time
+   * @param atime access time
    * @param permission permission
-   * @param owner owner
-   * @param group group
-   * @param symlink symbolic link
-   * @param path local path name in java UTF8 format
+   * @param owner the owner of the path
+   * @param group the group of the path
+   * @param symlink symlink target encoded in java UTF8 or null
+   * @param path the local name in java UTF8 encoding the same as that in-memory
    * @param fileId the file id
-   * @param locations block locations
-   * @param feInfo file encryption info
+   * @param childrenNum the number of children. Used by directory.
+   * @param feInfo the file's encryption info
+   * @param storagePolicy ID which specifies storage policy
+   * @param ecPolicy the erasure coding policy
+   * @param hdfsloc block locations
    */
-  public HdfsLocatedFileStatus(long length, boolean isdir,
-      int block_replication, long blocksize, long modification_time,
-      long access_time, FsPermission permission, EnumSet<Flags> flags,
-      String owner, String group, byte[] symlink, byte[] path, long fileId,
-      LocatedBlocks locations, int childrenNum, FileEncryptionInfo feInfo,
-      byte storagePolicy, ErasureCodingPolicy ecPolicy) {
-    super(length, isdir, block_replication, blocksize, modification_time,
-        access_time, permission, flags, owner, group, symlink, path, fileId,
-        childrenNum, feInfo, storagePolicy, ecPolicy);
-    this.locations = locations;
+  HdfsLocatedFileStatus(long length, boolean isdir, int replication,
+                        long blocksize, long mtime, long atime,
+                        FsPermission permission, EnumSet<Flags> flags,
+                        String owner, String group,
+                        byte[] symlink, byte[] path, long fileId,
+                        int childrenNum, FileEncryptionInfo feInfo,
+                        byte storagePolicy, ErasureCodingPolicy ecPolicy,
+                        LocatedBlocks hdfsloc) {
+    super(length, isdir, replication, blocksize, mtime, atime,
+        HdfsFileStatus.convert(isdir, symlink != null, permission, flags),
+        owner, group, null, null, HdfsFileStatus.convert(flags),
+        null);
+    this.uSymlink = symlink;
+    this.uPath = path;
+    this.fileId = fileId;
+    this.childrenNum = childrenNum;
+    this.feInfo = feInfo;
+    this.storagePolicy = storagePolicy;
+    this.ecPolicy = ecPolicy;
+    this.hdfsloc = hdfsloc;
   }
 
-  public LocatedBlocks getBlockLocations() {
-    return locations;
+  @Override // visibility
+  public void setOwner(String owner) {
+    super.setOwner(owner);
+  }
+
+  @Override // visibility
+  public void setGroup(String group) {
+    super.setOwner(group);
+  }
+
+  @Override
+  public boolean isSymlink() {
+    return uSymlink != null;
+  }
+
+  @Override
+  public Path getSymlink() throws IOException {
+    if (isSymlink()) {
+      return new Path(DFSUtilClient.bytes2String(getSymlinkInBytes()));
+    }
+    throw new IOException("Path " + getPath() + " is not a symbolic link");
+  }
+
+  @Override // visibility
+  public void setPermission(FsPermission permission) {
+    super.setPermission(permission);
   }
 
   /**
-   * This function is used to transform the underlying HDFS LocatedBlocks to
-   * BlockLocations.
-   *
-   * The returned BlockLocation will have different formats for replicated
-   * and erasure coded file.
-   * Please refer to
-   * {@link org.apache.hadoop.fs.FileSystem#getFileBlockLocations
-   * (FileStatus, long, long)}
-   * for examples.
+   * Get the Java UTF8 representation of the local name.
+   * @return the local name in java UTF8
    */
-  public final LocatedFileStatus makeQualifiedLocated(URI defaultUri,
-      Path path) {
-    makeQualified(defaultUri, path);
-    return new LocatedFileStatus(this,
-        DFSUtilClient.locatedBlocks2Locations(getBlockLocations()));
+  @Override
+  public byte[] getLocalNameInBytes() {
+    return uPath;
+  }
+
+  @Override
+  public void setSymlink(Path sym) {
+    uSymlink = DFSUtilClient.string2Bytes(sym.toString());
+  }
+
+  /**
+   * Opaque referant for the symlink, to be resolved at the client.
+   */
+  @Override
+  public byte[] getSymlinkInBytes() {
+    return uSymlink;
+  }
+
+  @Override
+  public long getFileId() {
+    return fileId;
+  }
+
+  @Override
+  public FileEncryptionInfo getFileEncryptionInfo() {
+    return feInfo;
+  }
+
+  /**
+   * Get the erasure coding policy if it's set.
+   * @return the erasure coding policy
+   */
+  @Override
+  public ErasureCodingPolicy getErasureCodingPolicy() {
+    return ecPolicy;
+  }
+
+  @Override
+  public int getChildrenNum() {
+    return childrenNum;
+  }
+
+  /** @return the storage policy id */
+  @Override
+  public byte getStoragePolicy() {
+    return storagePolicy;
   }
 
   @Override
@@ -107,4 +187,34 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
     // satisfy findbugs
     return super.hashCode();
   }
+
+  /**
+   * Get block locations for this entity, in HDFS format.
+   * See {@link #makeQualifiedLocated(URI, Path)}.
+   * See {@link DFSUtilClient#locatedBlocks2Locations(LocatedBlocks)}.
+   * @return block locations
+   */
+  public LocatedBlocks getLocatedBlocks() {
+    return hdfsloc;
+  }
+
+  /**
+   * This function is used to transform the underlying HDFS LocatedBlocks to
+   * BlockLocations. This method must be invoked before
+   * {@link #getBlockLocations()}.
+   *
+   * The returned BlockLocation will have different formats for replicated
+   * and erasure coded file.
+   * Please refer to
+   * {@link org.apache.hadoop.fs.FileSystem#getFileBlockLocations
+   * (FileStatus, long, long)}
+   * for examples.
+   */
+  public LocatedFileStatus makeQualifiedLocated(URI defaultUri, Path path) {
+    makeQualified(defaultUri, path);
+    setBlockLocations(
+        DFSUtilClient.locatedBlocks2Locations(getLocatedBlocks()));
+    return this;
+  }
+
 }

+ 180 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsNamedFileStatus.java

@@ -0,0 +1,180 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+
+import java.io.IOException;
+import java.util.Set;
+
+/**
+ * HDFS metadata for an entity in the filesystem without locations. Note that
+ * symlinks and directories are returned as {@link HdfsLocatedFileStatus} for
+ * backwards compatibility.
+ */
+public class HdfsNamedFileStatus extends FileStatus implements HdfsFileStatus {
+
+  // local name of the inode that's encoded in java UTF8
+  private byte[] uPath;
+  private byte[] uSymlink; // symlink target encoded in java UTF8/null
+  private final long fileId;
+  private final FileEncryptionInfo feInfo;
+  private final ErasureCodingPolicy ecPolicy;
+
+  // Used by dir, not including dot and dotdot. Always zero for a regular file.
+  private final int childrenNum;
+  private final byte storagePolicy;
+
+  /**
+   * Constructor.
+   * @param length the number of bytes the file has
+   * @param isdir if the path is a directory
+   * @param replication the replication factor
+   * @param blocksize the block size
+   * @param mtime modification time
+   * @param atime access time
+   * @param permission permission
+   * @param owner the owner of the path
+   * @param group the group of the path
+   * @param symlink symlink target encoded in java UTF8 or null
+   * @param path the local name in java UTF8 encoding the same as that in-memory
+   * @param fileId the file id
+   * @param childrenNum the number of children. Used by directory.
+   * @param feInfo the file's encryption info
+   * @param storagePolicy ID which specifies storage policy
+   * @param ecPolicy the erasure coding policy
+   */
+  HdfsNamedFileStatus(long length, boolean isdir, int replication,
+                      long blocksize, long mtime, long atime,
+                      FsPermission permission, Set<Flags> flags,
+                      String owner, String group,
+                      byte[] symlink, byte[] path, long fileId,
+                      int childrenNum, FileEncryptionInfo feInfo,
+                      byte storagePolicy, ErasureCodingPolicy ecPolicy) {
+    super(length, isdir, replication, blocksize, mtime, atime,
+        HdfsFileStatus.convert(isdir, symlink != null, permission, flags),
+        owner, group, null, null,
+        HdfsFileStatus.convert(flags));
+    this.uSymlink = symlink;
+    this.uPath = path;
+    this.fileId = fileId;
+    this.childrenNum = childrenNum;
+    this.feInfo = feInfo;
+    this.storagePolicy = storagePolicy;
+    this.ecPolicy = ecPolicy;
+  }
+
+  @Override
+  public void setOwner(String owner) {
+    super.setOwner(owner);
+  }
+
+  @Override
+  public void setGroup(String group) {
+    super.setOwner(group);
+  }
+
+  @Override
+  public boolean isSymlink() {
+    return uSymlink != null;
+  }
+
+  @Override
+  public Path getSymlink() throws IOException {
+    if (isSymlink()) {
+      return new Path(DFSUtilClient.bytes2String(getSymlinkInBytes()));
+    }
+    throw new IOException("Path " + getPath() + " is not a symbolic link");
+  }
+
+  @Override
+  public void setPermission(FsPermission permission) {
+    super.setPermission(permission);
+  }
+
+  /**
+   * Get the Java UTF8 representation of the local name.
+   *
+   * @return the local name in java UTF8
+   */
+  @Override
+  public byte[] getLocalNameInBytes() {
+    return uPath;
+  }
+
+  @Override
+  public void setSymlink(Path sym) {
+    uSymlink = DFSUtilClient.string2Bytes(sym.toString());
+  }
+
+  /**
+   * Opaque referant for the symlink, to be resolved at the client.
+   */
+  @Override
+  public byte[] getSymlinkInBytes() {
+    return uSymlink;
+  }
+
+  @Override
+  public long getFileId() {
+    return fileId;
+  }
+
+  @Override
+  public FileEncryptionInfo getFileEncryptionInfo() {
+    return feInfo;
+  }
+
+  /**
+   * Get the erasure coding policy if it's set.
+   *
+   * @return the erasure coding policy
+   */
+  @Override
+  public ErasureCodingPolicy getErasureCodingPolicy() {
+    return ecPolicy;
+  }
+
+  @Override
+  public int getChildrenNum() {
+    return childrenNum;
+  }
+
+  /** @return the storage policy id */
+  @Override
+  public byte getStoragePolicy() {
+    return storagePolicy;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    // satisfy findbugs
+    return super.equals(o);
+  }
+
+  @Override
+  public int hashCode() {
+    // satisfy findbugs
+    return super.hashCode();
+  }
+
+}

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

@@ -192,6 +192,7 @@ public class LocatedBlocks {
         + "\n  underConstruction=" + underConstruction
         + "\n  blocks=" + blocks
         + "\n  lastLocatedBlock=" + lastLocatedBlock
-        + "\n  isLastBlockComplete=" + isLastBlockComplete + "}";
+        + "\n  isLastBlockComplete=" + isLastBlockComplete
+        + "\n  ecPolicy=" + ecPolicy + "}";
   }
 }

+ 160 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReportListing.java

@@ -0,0 +1,160 @@
+/**
+ * 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.protocol;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.curator.shaded.com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hdfs.DFSUtilClient;
+
+/**
+ * This class represents to  the difference between two snapshots of
+ * the same directory, or the difference between a snapshot of the directory and
+ * its current state. This Class serves the purpose of collecting diff entries
+ * in 3 lists : created, deleted and modified list combined size of which is set
+ * by dfs.snapshotdiff-report.limit over one rpc call to the namenode.
+ */
+public class SnapshotDiffReportListing {
+  /**
+   * Representing the full path and diff type of a file/directory where changes
+   * have happened.
+   */
+  public static class DiffReportListingEntry {
+    /**
+     * The type of the difference.
+     */
+    private final long fileId;
+    private final long dirId;
+    private final boolean isReference;
+    /**
+     * The relative path (related to the snapshot root) of 1) the file/directory
+     * where changes have happened, or 2) the source file/dir of a rename op.
+     * or 3) target file/dir for a rename op.
+     */
+    private final byte[][] sourcePath;
+    private final byte[][] targetPath;
+
+    public DiffReportListingEntry(long dirId, long fileId, byte[][] sourcePath,
+        boolean isReference, byte[][] targetPath) {
+      Preconditions.checkNotNull(sourcePath);
+      this.dirId = dirId;
+      this.fileId = fileId;
+      this.sourcePath = sourcePath;
+      this.isReference = isReference;
+      this.targetPath = targetPath;
+    }
+
+    public DiffReportListingEntry(long dirId, long fileId, byte[] sourcePath,
+        boolean isReference, byte[] targetpath) {
+      Preconditions.checkNotNull(sourcePath);
+      this.dirId = dirId;
+      this.fileId = fileId;
+      this.sourcePath = DFSUtilClient.bytes2byteArray(sourcePath);
+      this.isReference = isReference;
+      this.targetPath =
+          targetpath == null ? null : DFSUtilClient.bytes2byteArray(targetpath);
+    }
+
+    public long getDirId() {
+      return dirId;
+    }
+
+    public long getFileId() {
+      return fileId;
+    }
+
+    public byte[][] getSourcePath() {
+      return sourcePath;
+    }
+
+    public byte[][] getTargetPath() {
+      return targetPath;
+    }
+
+    public boolean isReference() {
+      return isReference;
+    }
+  }
+
+  /** store the starting path to process across RPC's for snapshot diff. */
+  private final byte[] lastPath;
+
+  private final int lastIndex;
+
+  private final boolean isFromEarlier;
+
+  /** list of diff. */
+  private final List<DiffReportListingEntry> modifyList;
+
+  private final List<DiffReportListingEntry> createList;
+
+  private final List<DiffReportListingEntry> deleteList;
+
+  public SnapshotDiffReportListing() {
+    this.modifyList = Collections.emptyList();
+    this.createList = Collections.emptyList();
+    this.deleteList = Collections.emptyList();
+    this.lastPath = DFSUtilClient.string2Bytes("");
+    this.lastIndex = -1;
+    this.isFromEarlier = false;
+  }
+
+  public SnapshotDiffReportListing(byte[] startPath,
+      List<DiffReportListingEntry> modifiedEntryList,
+      List<DiffReportListingEntry> createdEntryList,
+      List<DiffReportListingEntry> deletedEntryList, int index,
+      boolean isFromEarlier) {
+    this.modifyList = modifiedEntryList;
+    this.createList = createdEntryList;
+    this.deleteList = deletedEntryList;
+    this.lastPath =
+        startPath != null ? startPath : DFSUtilClient.string2Bytes("");
+    this.lastIndex = index;
+    this.isFromEarlier = isFromEarlier;
+  }
+
+  public List<DiffReportListingEntry> getModifyList() {
+    return modifyList;
+  }
+
+  public List<DiffReportListingEntry> getCreateList() {
+    return createList;
+  }
+
+  public List<DiffReportListingEntry> getDeleteList() {
+    return deleteList;
+  }
+
+  /**
+   * @return {@link #lastPath}
+   */
+  public byte[] getLastPath() {
+    return lastPath;
+  }
+
+  public int getLastIndex() {
+    return lastIndex;
+  }
+
+  public boolean getIsFromEarlier() {
+    return isFromEarlier;
+  }
+
+}

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/package-info.java

@@ -0,0 +1,18 @@
+/**
+ * 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.protocol;

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -79,6 +79,7 @@ import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
@@ -133,6 +134,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetQuo
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
@@ -1205,6 +1208,27 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
 
+  @Override
+  public SnapshotDiffReportListing getSnapshotDiffReportListing(
+      String snapshotRoot, String fromSnapshot, String toSnapshot,
+      byte[] startPath, int index) throws IOException {
+    GetSnapshotDiffReportListingRequestProto req =
+        GetSnapshotDiffReportListingRequestProto.newBuilder()
+            .setSnapshotRoot(snapshotRoot).setFromSnapshot(fromSnapshot)
+            .setToSnapshot(toSnapshot).setCursor(
+            HdfsProtos.SnapshotDiffReportCursorProto.newBuilder()
+                .setStartPath(PBHelperClient.getByteString(startPath))
+                .setIndex(index).build()).build();
+    try {
+      GetSnapshotDiffReportListingResponseProto result =
+          rpcProxy.getSnapshotDiffReportListing(null, req);
+
+      return PBHelperClient.convert(result.getDiffReport());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
   @Override
   public long addCacheDirective(CacheDirectiveInfo directive,
       EnumSet<CacheFlag> flags) throws IOException {

+ 162 - 22
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -99,6 +99,8 @@ import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing.DiffReportListingEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
@@ -169,6 +171,8 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.QuotaUsageProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReencryptionInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RollingUpgradeStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportListingEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportListingProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
@@ -1489,6 +1493,61 @@ public class PBHelperClient {
         .toByteArray() : null);
   }
 
+  public static SnapshotDiffReportListing convert(
+      SnapshotDiffReportListingProto reportProto) {
+    if (reportProto == null) {
+      return null;
+    }
+    List<SnapshotDiffReportListingEntryProto> modifyList =
+        reportProto.getModifiedEntriesList();
+    List<DiffReportListingEntry> modifiedEntries = new ChunkedArrayList<>();
+    for (SnapshotDiffReportListingEntryProto entryProto : modifyList) {
+      DiffReportListingEntry entry = convert(entryProto);
+      if (entry != null) {
+        modifiedEntries.add(entry);
+      }
+    }
+    List<SnapshotDiffReportListingEntryProto> createList =
+        reportProto.getCreatedEntriesList();
+    List<DiffReportListingEntry> createdEntries = new ChunkedArrayList<>();
+    for (SnapshotDiffReportListingEntryProto entryProto : createList) {
+      DiffReportListingEntry entry = convert(entryProto);
+      if (entry != null) {
+        createdEntries.add(entry);
+      }
+    }
+    List<SnapshotDiffReportListingEntryProto> deletedList =
+        reportProto.getDeletedEntriesList();
+    List<DiffReportListingEntry> deletedEntries = new ChunkedArrayList<>();
+    for (SnapshotDiffReportListingEntryProto entryProto : deletedList) {
+      DiffReportListingEntry entry = convert(entryProto);
+      if (entry != null) {
+        deletedEntries.add(entry);
+      }
+    }
+    byte[] startPath = reportProto.getCursor().getStartPath().toByteArray();
+    boolean isFromEarlier = reportProto.getIsFromEarlier();
+
+    int index = reportProto.getCursor().getIndex();
+    return new SnapshotDiffReportListing(startPath, modifiedEntries,
+        createdEntries, deletedEntries, index, isFromEarlier);
+  }
+
+  public static DiffReportListingEntry convert(
+      SnapshotDiffReportListingEntryProto entry) {
+    if (entry == null) {
+      return null;
+    }
+    long dirId = entry.getDirId();
+    long fileId = entry.getFileId();
+    boolean isReference = entry.getIsReference();
+    byte[] sourceName = entry.getFullpath().toByteArray();
+    byte[] targetName =
+        entry.hasTargetPath() ? entry.getTargetPath().toByteArray() : null;
+    return new DiffReportListingEntry(dirId, fileId, sourceName, isReference,
+        targetName);
+  }
+
   public static SnapshottableDirectoryStatus[] convert(
       SnapshottableDirectoryListingProto sdlp) {
     if (sdlp == null)
@@ -1585,23 +1644,36 @@ public class PBHelperClient {
     EnumSet<HdfsFileStatus.Flags> flags = fs.hasFlags()
         ? convertFlags(fs.getFlags())
         : convertFlags(fs.getPermission());
-    return new HdfsLocatedFileStatus(
-        fs.getLength(), fs.getFileType().equals(FileType.IS_DIR),
-        fs.getBlockReplication(), fs.getBlocksize(),
-        fs.getModificationTime(), fs.getAccessTime(),
-        convert(fs.getPermission()),
-        flags,
-        fs.getOwner(), fs.getGroup(),
-        fs.getFileType().equals(FileType.IS_SYMLINK) ?
-            fs.getSymlink().toByteArray() : null,
-        fs.getPath().toByteArray(),
-        fs.hasFileId()? fs.getFileId(): HdfsConstants.GRANDFATHER_INODE_ID,
-        fs.hasLocations() ? convert(fs.getLocations()) : null,
-        fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
-        fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
-        fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
-            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
-        fs.hasEcPolicy() ? convertErasureCodingPolicy(fs.getEcPolicy()) : null);
+    return new HdfsFileStatus.Builder()
+        .length(fs.getLength())
+        .isdir(fs.getFileType().equals(FileType.IS_DIR))
+        .replication(fs.getBlockReplication())
+        .blocksize(fs.getBlocksize())
+        .mtime(fs.getModificationTime())
+        .atime(fs.getAccessTime())
+        .perm(convert(fs.getPermission()))
+        .flags(flags)
+        .owner(fs.getOwner())
+        .group(fs.getGroup())
+        .symlink(FileType.IS_SYMLINK.equals(fs.getFileType())
+            ? fs.getSymlink().toByteArray()
+            : null)
+        .path(fs.getPath().toByteArray())
+        .fileId(fs.hasFileId()
+            ? fs.getFileId()
+            : HdfsConstants.GRANDFATHER_INODE_ID)
+        .locations(fs.hasLocations() ? convert(fs.getLocations()) : null)
+        .children(fs.hasChildrenNum() ? fs.getChildrenNum() : -1)
+        .feInfo(fs.hasFileEncryptionInfo()
+            ? convert(fs.getFileEncryptionInfo())
+            : null)
+        .storagePolicy(fs.hasStoragePolicy()
+            ? (byte) fs.getStoragePolicy()
+            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED)
+        .ecPolicy(fs.hasEcPolicy()
+            ? convertErasureCodingPolicy(fs.getEcPolicy())
+            : null)
+        .build();
   }
 
   private static EnumSet<HdfsFileStatus.Flags> convertFlags(int flags) {
@@ -1864,10 +1936,10 @@ public class PBHelperClient {
     if (dl == null)
       return null;
     List<HdfsFileStatusProto> partList =  dl.getPartialListingList();
-    return new DirectoryListing(partList.isEmpty() ?
-        new HdfsLocatedFileStatus[0] :
-        convert(partList.toArray(new HdfsFileStatusProto[partList.size()])),
-        dl.getRemainingEntries());
+    return new DirectoryListing(partList.isEmpty()
+        ? new HdfsFileStatus[0]
+        : convert(partList.toArray(new HdfsFileStatusProto[partList.size()])),
+                  dl.getRemainingEntries());
   }
 
   public static HdfsFileStatus[] convert(HdfsFileStatusProto[] fs) {
@@ -2163,7 +2235,7 @@ public class PBHelperClient {
     }
     if (fs instanceof HdfsLocatedFileStatus) {
       final HdfsLocatedFileStatus lfs = (HdfsLocatedFileStatus) fs;
-      LocatedBlocks locations = lfs.getBlockLocations();
+      LocatedBlocks locations = lfs.getLocatedBlocks();
       if (locations != null) {
         builder.setLocations(convert(locations));
       }
@@ -2495,6 +2567,74 @@ public class PBHelperClient {
     return builder.build();
   }
 
+  public static SnapshotDiffReportListingEntryProto convert(
+      DiffReportListingEntry entry) {
+    if (entry == null) {
+      return null;
+    }
+    ByteString sourcePath = getByteString(
+        entry.getSourcePath() == null ? DFSUtilClient.EMPTY_BYTES :
+            DFSUtilClient.byteArray2bytes(entry.getSourcePath()));
+    long dirId = entry.getDirId();
+    long fileId = entry.getFileId();
+    boolean isReference = entry.isReference();
+    ByteString targetPath = getByteString(
+        entry.getTargetPath() == null ? DFSUtilClient.EMPTY_BYTES :
+            DFSUtilClient.byteArray2bytes(entry.getTargetPath()));
+    SnapshotDiffReportListingEntryProto.Builder builder =
+        SnapshotDiffReportListingEntryProto.newBuilder().setFullpath(sourcePath)
+            .setDirId(dirId).setFileId(fileId).setIsReference(isReference)
+            .setTargetPath(targetPath);
+    return builder.build();
+  }
+
+  public static SnapshotDiffReportListingProto convert(
+      SnapshotDiffReportListing report) {
+    if (report == null) {
+      return null;
+    }
+    ByteString startPath = getByteString(
+        report.getLastPath() == null ? DFSUtilClient.EMPTY_BYTES :
+            report.getLastPath());
+    List<DiffReportListingEntry> modifiedEntries = report.getModifyList();
+    List<DiffReportListingEntry> createdEntries = report.getCreateList();
+    List<DiffReportListingEntry> deletedEntries = report.getDeleteList();
+    List<SnapshotDiffReportListingEntryProto> modifiedEntryProtos =
+        new ChunkedArrayList<>();
+    for (DiffReportListingEntry entry : modifiedEntries) {
+      SnapshotDiffReportListingEntryProto entryProto = convert(entry);
+      if (entryProto != null) {
+        modifiedEntryProtos.add(entryProto);
+      }
+    }
+    List<SnapshotDiffReportListingEntryProto> createdEntryProtos =
+        new ChunkedArrayList<>();
+    for (DiffReportListingEntry entry : createdEntries) {
+      SnapshotDiffReportListingEntryProto entryProto = convert(entry);
+      if (entryProto != null) {
+        createdEntryProtos.add(entryProto);
+      }
+    }
+    List<SnapshotDiffReportListingEntryProto> deletedEntryProtos =
+        new ChunkedArrayList<>();
+    for (DiffReportListingEntry entry : deletedEntries) {
+      SnapshotDiffReportListingEntryProto entryProto = convert(entry);
+      if (entryProto != null) {
+        deletedEntryProtos.add(entryProto);
+      }
+    }
+
+    return SnapshotDiffReportListingProto.newBuilder()
+        .addAllModifiedEntries(modifiedEntryProtos)
+        .addAllCreatedEntries(createdEntryProtos)
+        .addAllDeletedEntries(deletedEntryProtos)
+        .setIsFromEarlier(report.getIsFromEarlier())
+        .setCursor(HdfsProtos.SnapshotDiffReportCursorProto.newBuilder()
+        .setStartPath(startPath)
+        .setIndex(report.getLastIndex()).build())
+        .build();
+  }
+
   public static SnapshotDiffReportProto convert(SnapshotDiffReport report) {
     if (report == null) {
       return null;

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

@@ -56,6 +56,8 @@ import com.fasterxml.jackson.databind.ObjectReader;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.input.BoundedInputStream;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.ContentSummary;
@@ -84,6 +86,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtilClient;
+import org.apache.hadoop.hdfs.HdfsKMSUtil;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
@@ -100,6 +103,7 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
@@ -107,6 +111,7 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenSelector;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
+import org.apache.hadoop.util.KMSUtil;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
@@ -120,7 +125,7 @@ import com.google.common.collect.Lists;
 /** A FileSystem for HDFS over the web. */
 public class WebHdfsFileSystem extends FileSystem
     implements DelegationTokenRenewer.Renewable,
-    TokenAspect.TokenManagementDelegator {
+    TokenAspect.TokenManagementDelegator, KeyProviderTokenIssuer {
   public static final Logger LOG = LoggerFactory
       .getLogger(WebHdfsFileSystem.class);
   /** WebHdfs version. */
@@ -1633,6 +1638,13 @@ public class WebHdfsFileSystem extends FileSystem
   }
 
   @Override
+  public Token<?>[] addDelegationTokens(String renewer,
+      Credentials credentials) throws IOException {
+    Token<?>[] tokens = super.addDelegationTokens(renewer, credentials);
+    return HdfsKMSUtil.addDelegationTokensForKeyProvider(this, renewer,
+        credentials, getUri(), tokens);
+  }
+
   public BlockLocation[] getFileBlockLocations(final FileStatus status,
       final long offset, final long length) throws IOException {
     if (status == null) {
@@ -1822,6 +1834,29 @@ public class WebHdfsFileSystem extends FileSystem
     this.retryPolicy = rp;
   }
 
+
+  @Override
+  public URI getKeyProviderUri() throws IOException {
+    String keyProviderUri = null;
+    try {
+      keyProviderUri = getServerDefaults().getKeyProviderUri();
+    } catch (UnsupportedOperationException e) {
+      // This means server doesn't supports GETSERVERDEFAULTS call.
+      // Do nothing, let keyProviderUri = null.
+    }
+    return HdfsKMSUtil.getKeyProviderUri(ugi, getUri(), keyProviderUri,
+        getConf());
+  }
+
+  @Override
+  public KeyProvider getKeyProvider() throws IOException {
+    URI keyProviderUri = getKeyProviderUri();
+    if (keyProviderUri == null) {
+      return null;
+    }
+    return KMSUtil.createKeyProviderFromUri(getConf(), keyProviderUri);
+  }
+
   /**
    * This class is used for opening, reading, and seeking files while using the
    * WebHdfsFileSystem. This class will invoke the retry policy when performing

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

@@ -297,6 +297,16 @@ message GetSnapshotDiffReportResponseProto {
   required SnapshotDiffReportProto diffReport = 1;
 }
 
+message GetSnapshotDiffReportListingRequestProto {
+  required string snapshotRoot = 1;
+  required string fromSnapshot = 2;
+  required string toSnapshot = 3;
+  optional SnapshotDiffReportCursorProto cursor = 4;
+}
+
+message GetSnapshotDiffReportListingResponseProto {
+  required SnapshotDiffReportListingProto diffReport = 1;
+}
 message RenewLeaseRequestProto {
   required string clientName = 1;
 }
@@ -913,6 +923,8 @@ service ClientNamenodeProtocol {
       returns(DeleteSnapshotResponseProto);
   rpc getSnapshotDiffReport(GetSnapshotDiffReportRequestProto)
       returns(GetSnapshotDiffReportResponseProto);
+  rpc getSnapshotDiffReportListing(GetSnapshotDiffReportListingRequestProto)
+      returns(GetSnapshotDiffReportListingResponseProto);
   rpc isFileClosed(IsFileClosedRequestProto)
       returns(IsFileClosedResponseProto);
   rpc modifyAclEntries(ModifyAclEntriesRequestProto)

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

@@ -530,6 +530,32 @@ message SnapshotDiffReportProto {
   repeated SnapshotDiffReportEntryProto diffReportEntries = 4;
 }
 
+/**
+ * Snapshot diff report listing entry
+ */
+message SnapshotDiffReportListingEntryProto {
+  required bytes fullpath = 1;
+  required uint64 dirId = 2;
+  required bool isReference = 3;
+  optional bytes targetPath = 4;
+  optional uint64 fileId = 5;
+}
+
+message SnapshotDiffReportCursorProto {
+  required bytes startPath = 1;
+  required int32 index = 2 [default = -1];
+}
+/**
+ * Snapshot diff report listing
+ */
+message SnapshotDiffReportListingProto {
+  // full path of the directory where snapshots were taken
+  repeated SnapshotDiffReportListingEntryProto modifiedEntries = 1;
+  repeated SnapshotDiffReportListingEntryProto createdEntries = 2;
+  repeated SnapshotDiffReportListingEntryProto deletedEntries = 3;
+  required bool isFromEarlier = 4;
+  optional SnapshotDiffReportCursorProto cursor = 5;
+}
 /**
  * Block information
  *

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

@@ -0,0 +1,106 @@
+/**
+ * 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.protocol;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Set;
+import java.util.stream.Stream;
+import static java.util.stream.Collectors.joining;
+import static java.util.stream.Collectors.toSet;
+
+import org.apache.hadoop.fs.FileStatus;
+
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Unit test verifying that {@link HdfsFileStatus} is a superset of
+ * {@link FileStatus}.
+ */
+public class TestHdfsFileStatusMethods {
+
+  @Test
+  public void testInterfaceSuperset() {
+    Set<MethodSignature>  fsM = signatures(FileStatus.class);
+    Set<MethodSignature> hfsM = signatures(HdfsFileStatus.class);
+    hfsM.addAll(Stream.of(HdfsFileStatus.class.getInterfaces())
+        .flatMap(i -> Stream.of(i.getDeclaredMethods()))
+        .map(MethodSignature::new)
+        .collect(toSet()));
+    // HdfsFileStatus is not a concrete type
+    hfsM.addAll(signatures(Object.class));
+    assertTrue(fsM.removeAll(hfsM));
+    // verify that FileStatus is a subset of HdfsFileStatus
+    assertEquals(fsM.stream()
+            .map(MethodSignature::toString)
+            .collect(joining("\n")),
+        Collections.EMPTY_SET, fsM);
+  }
+
+  /** Map non-static, declared methods for this class to signatures. */
+  private static Set<MethodSignature> signatures(Class<?> c) {
+    return Stream.of(c.getDeclaredMethods())
+        .filter(m -> !Modifier.isStatic(m.getModifiers()))
+        .map(MethodSignature::new)
+        .collect(toSet());
+  }
+
+  private static class MethodSignature {
+    private final String name;
+    private final Type rval;
+    private final Type[] param;
+    MethodSignature(Method m) {
+      name = m.getName();
+      rval = m.getGenericReturnType();
+      param = m.getParameterTypes();
+    }
+    @Override
+    public int hashCode() {
+      return name.hashCode();
+    }
+    /**
+     * Methods are equal iff they have the same name, return type, and params
+     * (non-generic).
+     */
+    @Override
+    public boolean equals(Object o) {
+      if (!(o instanceof MethodSignature)) {
+        return false;
+      }
+      MethodSignature s = (MethodSignature) o;
+      return name.equals(s.name) &&
+          rval.equals(s.rval) &&
+          Arrays.equals(param, s.param);
+    }
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append(rval).append(" ").append(name).append("(")
+        .append(Stream.of(param)
+            .map(Type::toString).collect(joining(",")))
+        .append(")");
+      return sb.toString();
+    }
+  }
+
+}

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

@@ -1085,10 +1085,7 @@ public class HttpFSFileSystem extends FileSystem
           new FsPermissionExtension(permission, aBit, eBit, ecBit);
       FileStatus fileStatus = new FileStatus(len, FILE_TYPE.DIRECTORY == type,
           replication, blockSize, mTime, aTime, deprecatedPerm, owner, group,
-          null, path, aBit, eBit, ecBit);
-      if (seBit) {
-        fileStatus.setSnapShotEnabledFlag(seBit);
-      }
+          null, path, FileStatus.attributes(aBit, eBit, ecBit, seBit));
       return fileStatus;
     } else {
       return new FileStatus(len, FILE_TYPE.DIRECTORY == type,

+ 0 - 7
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -269,13 +269,6 @@
         <Method name="visitFile" />
         <Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
     </Match>
-    <!-- HdfsFileStatus is user-facing, but HdfsLocatedFileStatus is not.
-         Defensible compatibility choices over time create odd corners. -->
-    <Match>
-        <Class name="org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus" />
-        <Field name="locations" />
-        <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED" />
-    </Match>
     <Match>
        <Class name="org.apache.hadoop.hdfs.server.namenode.NNUpgradeUtil$1" />
        <Method name="visitFile" />

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -425,6 +425,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/webapps/hdfs/robots.txt</exclude>
             <exclude>src/main/webapps/journal/robots.txt</exclude>
             <exclude>src/main/webapps/secondary/robots.txt</exclude>
+            <exclude>src/main/webapps/router/robots.txt</exclude>
             <exclude>src/contrib/**</exclude>
             <exclude>src/site/resources/images/*</exclude>
             <exclude>src/main/webapps/static/bootstrap-3.0.2/**</exclude>

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -15,8 +15,9 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+# The name of the script being executed.
+HADOOP_SHELL_EXECNAME="hdfs"
 MYNAME="${BASH_SOURCE-$0}"
-HADOOP_SHELL_EXECNAME="${MYNAME##*/}"
 
 ## @description  build up the hdfs command's usage text.
 ## @audience     public
@@ -147,6 +148,10 @@ function hdfscmd_case
       echo "HADOOP_TOOLS_HOME='${HADOOP_TOOLS_HOME}'"
       echo "HADOOP_TOOLS_DIR='${HADOOP_TOOLS_DIR}'"
       echo "HADOOP_TOOLS_LIB_JARS_DIR='${HADOOP_TOOLS_LIB_JARS_DIR}'"
+      if [[ -n "${QATESTMODE}" ]]; then
+        echo "MYNAME=${MYNAME}"
+        echo "HADOOP_SHELL_EXECNAME=${HADOOP_SHELL_EXECNAME}"
+      fi
       exit 0
     ;;
     ec)
@@ -316,4 +321,4 @@ fi
 hadoop_subcommand_opts "${HADOOP_SHELL_EXECNAME}" "${HADOOP_SUBCMD}"
 
 # everything is in globals at this point, so call the generic handler
-hadoop_generic_java_subcmd_handler
+hadoop_generic_java_subcmd_handler

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

@@ -381,6 +381,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT_DEFAULT =
       true;
 
+  public static final String
+      DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT  =
+      "dfs.namenode.snapshotdiff.listing.limit";
+  public static final int
+      DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT_DEFAULT = 1000;
   // Whether to enable datanode's stale state detection and usage for reads
   public static final String DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY = "dfs.namenode.avoid.read.stale.datanode";
   public static final boolean DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_DEFAULT = false;

+ 6 - 38
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -352,7 +352,8 @@ public class DFSUtil {
   public static byte[][] getPathComponents(String path) {
     // avoid intermediate split to String[]
     final byte[] bytes = string2Bytes(path);
-    return bytes2byteArray(bytes, bytes.length, (byte)Path.SEPARATOR_CHAR);
+    return DFSUtilClient
+        .bytes2byteArray(bytes, bytes.length, (byte) Path.SEPARATOR_CHAR);
   }
 
   /**
@@ -372,42 +373,9 @@ public class DFSUtil {
    * @param len the number of bytes to split
    * @param separator the delimiting byte
    */
-  public static byte[][] bytes2byteArray(byte[] bytes,
-                                         int len,
-                                         byte separator) {
-    Preconditions.checkPositionIndex(len, bytes.length);
-    if (len == 0) {
-      return new byte[][]{null};
-    }
-    // Count the splits. Omit multiple separators and the last one by
-    // peeking at prior byte.
-    int splits = 0;
-    for (int i = 1; i < len; i++) {
-      if (bytes[i-1] == separator && bytes[i] != separator) {
-        splits++;
-      }
-    }
-    if (splits == 0 && bytes[0] == separator) {
-      return new byte[][]{null};
-    }
-    splits++;
-    byte[][] result = new byte[splits][];
-    int nextIndex = 0;
-    // Build the splits.
-    for (int i = 0; i < splits; i++) {
-      int startIndex = nextIndex;
-      // find next separator in the bytes.
-      while (nextIndex < len && bytes[nextIndex] != separator) {
-        nextIndex++;
-      }
-      result[i] = (nextIndex > 0)
-          ? Arrays.copyOfRange(bytes, startIndex, nextIndex)
-          : DFSUtilClient.EMPTY_BYTES; // reuse empty bytes for root.
-      do { // skip over separators.
-        nextIndex++;
-      } while (nextIndex < len && bytes[nextIndex] == separator);
-    }
-    return result;
+  public static byte[][] bytes2byteArray(byte[] bytes, int len,
+      byte separator) {
+    return DFSUtilClient.bytes2byteArray(bytes, len, separator);
   }
 
   /**
@@ -1707,7 +1675,7 @@ public class DFSUtil {
    */
   public static KeyProviderCryptoExtension createKeyProviderCryptoExtension(
       final Configuration conf) throws IOException {
-    KeyProvider keyProvider = DFSUtilClient.createKeyProvider(conf);
+    KeyProvider keyProvider = HdfsKMSUtil.createKeyProvider(conf);
     if (keyProvider == null) {
       return null;
     }

+ 22 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
@@ -143,6 +144,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSer
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
@@ -1245,6 +1248,25 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     }
   }
 
+  @Override
+  public GetSnapshotDiffReportListingResponseProto getSnapshotDiffReportListing(
+      RpcController controller,
+      GetSnapshotDiffReportListingRequestProto request)
+      throws ServiceException {
+    try {
+      SnapshotDiffReportListing report = server
+          .getSnapshotDiffReportListing(request.getSnapshotRoot(),
+              request.getFromSnapshot(), request.getToSnapshot(),
+              request.getCursor().getStartPath().toByteArray(),
+              request.getCursor().getIndex());
+              //request.getStartPath(), request.getIndex());
+      return GetSnapshotDiffReportListingResponseProto.newBuilder()
+          .setDiffReport(PBHelperClient.convert(report)).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
   @Override
   public IsFileClosedResponseProto isFileClosed(
       RpcController controller, IsFileClosedRequestProto request) 

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java

@@ -498,6 +498,12 @@ public class QuorumJournalManager implements JournalManager {
         // than committedTxnId. This ensures the consistency.
         if (onlyDurableTxns && inProgressOk) {
           endTxId = Math.min(endTxId, committedTxnId);
+          if (endTxId < remoteLog.getStartTxId()) {
+            LOG.warn("Found endTxId (" + endTxId + ") that is less than " +
+                "the startTxId (" + remoteLog.getStartTxId() +
+                ") - setting it to startTxId.");
+            endTxId = remoteLog.getStartTxId();
+          }
         }
 
         EditLogInputStream elis = EditLogFileInputStream.fromUrl(

+ 44 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -164,6 +164,8 @@ public class BlockManager implements BlockStatsMXBean {
   private static final String QUEUE_REASON_FUTURE_GENSTAMP =
     "generation stamp is in the future";
 
+  private static final long BLOCK_RECOVERY_TIMEOUT_MULTIPLIER = 30;
+
   private final Namesystem namesystem;
 
   private final BlockManagerSafeMode bmSafeMode;
@@ -353,6 +355,9 @@ public class BlockManager implements BlockStatsMXBean {
   @VisibleForTesting
   final PendingReconstructionBlocks pendingReconstruction;
 
+  /** Stores information about block recovery attempts. */
+  private final PendingRecoveryBlocks pendingRecoveryBlocks;
+
   /** The maximum number of replicas allowed for a block */
   public final short maxReplication;
   /**
@@ -549,6 +554,12 @@ public class BlockManager implements BlockStatsMXBean {
     }
     this.minReplicationToBeInMaintenance = (short)minMaintenanceR;
 
+    long heartbeatIntervalSecs = conf.getTimeDuration(
+        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
+        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS);
+    long blockRecoveryTimeout = getBlockRecoveryTimeout(heartbeatIntervalSecs);
+    pendingRecoveryBlocks = new PendingRecoveryBlocks(blockRecoveryTimeout);
+
     this.blockReportLeaseManager = new BlockReportLeaseManager(conf);
 
     bmSafeMode = new BlockManagerSafeMode(this, namesystem, haEnabled, conf);
@@ -1367,12 +1378,11 @@ public class BlockManager implements BlockStatsMXBean {
               b.getStorageIDs());
         }
         sb.setBlockTokens(blockTokens);
-      } else {
-        b.setBlockToken(blockTokenSecretManager.generateToken(
-            NameNode.getRemoteUser().getShortUserName(),
-            b.getBlock(), EnumSet.of(mode), b.getStorageTypes(),
-            b.getStorageIDs()));
       }
+      b.setBlockToken(blockTokenSecretManager.generateToken(
+          NameNode.getRemoteUser().getShortUserName(),
+          b.getBlock(), EnumSet.of(mode), b.getStorageTypes(),
+          b.getStorageIDs()));
     }
   }
 
@@ -1825,8 +1835,6 @@ public class BlockManager implements BlockStatsMXBean {
       }
 
       // choose replication targets: NOT HOLDING THE GLOBAL LOCK
-      // It is costly to extract the filename for which chooseTargets is called,
-      // so for now we pass in the block collection itself.
       final BlockPlacementPolicy placementPolicy =
           placementPolicies.getPolicy(rw.getBlock().getBlockType());
       rw.chooseTargets(placementPolicy, storagePolicySuite, excludedNodes);
@@ -4738,6 +4746,25 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
 
+  /**
+   * Notification of a successful block recovery.
+   * @param block for which the recovery succeeded
+   */
+  public void successfulBlockRecovery(BlockInfo block) {
+    pendingRecoveryBlocks.remove(block);
+  }
+
+  /**
+   * Checks whether a recovery attempt has been made for the given block.
+   * If so, checks whether that attempt has timed out.
+   * @param b block for which recovery is being attempted
+   * @return true if no recovery attempt has been made or
+   *         the previous attempt timed out
+   */
+  public boolean addBlockRecoveryAttempt(BlockInfo b) {
+    return pendingRecoveryBlocks.add(b);
+  }
+
   @VisibleForTesting
   public void flushBlockOps() throws IOException {
     runBlockOp(new Callable<Void>(){
@@ -4865,4 +4892,14 @@ public class BlockManager implements BlockStatsMXBean {
     }
     return i;
   }
+
+  private static long getBlockRecoveryTimeout(long heartbeatIntervalSecs) {
+    return TimeUnit.SECONDS.toMillis(heartbeatIntervalSecs *
+        BLOCK_RECOVERY_TIMEOUT_MULTIPLIER);
+  }
+
+  @VisibleForTesting
+  public void setBlockRecoveryTimeout(long blockRecoveryTimeout) {
+    pendingRecoveryBlocks.setRecoveryTimeoutInterval(blockRecoveryTimeout);
+  }
 }

+ 10 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReconstructionWork.java

@@ -32,7 +32,8 @@ import java.util.Set;
 abstract class BlockReconstructionWork {
   private final BlockInfo block;
 
-  private final BlockCollection bc;
+  private final String srcPath;
+  private final byte storagePolicyID;
 
   /**
    * An erasure coding reconstruction task has multiple source nodes.
@@ -57,7 +58,8 @@ abstract class BlockReconstructionWork {
       int additionalReplRequired,
       int priority) {
     this.block = block;
-    this.bc = bc;
+    this.srcPath = bc.getName();
+    this.storagePolicyID = bc.getStoragePolicyID();
     this.srcNodes = srcNodes;
     this.containingNodes = containingNodes;
     this.liveReplicaStorages = liveReplicaStorages;
@@ -94,8 +96,12 @@ abstract class BlockReconstructionWork {
     return srcNodes;
   }
 
-  BlockCollection getBc() {
-    return bc;
+  public String getSrcPath() {
+    return srcPath;
+  }
+
+  public byte getStoragePolicyID() {
+    return storagePolicyID;
   }
 
   List<DatanodeStorageInfo> getLiveReplicaStorages() {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java

@@ -58,10 +58,10 @@ class ErasureCodingWork extends BlockReconstructionWork {
       Set<Node> excludedNodes) {
     // TODO: new placement policy for EC considering multiple writers
     DatanodeStorageInfo[] chosenTargets = blockplacement.chooseTarget(
-        getBc().getName(), getAdditionalReplRequired(), getSrcNodes()[0],
+        getSrcPath(), getAdditionalReplRequired(), getSrcNodes()[0],
         getLiveReplicaStorages(), false, excludedNodes,
         getBlock().getNumBytes(),
-        storagePolicySuite.getPolicy(getBc().getStoragePolicyID()), null);
+        storagePolicySuite.getPolicy(getStoragePolicyID()), null);
     setTargets(chosenTargets);
   }
 

+ 143 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java

@@ -0,0 +1,143 @@
+/**
+ * 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.blockmanagement;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdfs.util.LightWeightHashSet;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * PendingRecoveryBlocks tracks recovery attempts for each block and their
+ * timeouts to ensure we do not have multiple recoveries at the same time
+ * and retry only after the timeout for a recovery has expired.
+ */
+class PendingRecoveryBlocks {
+  private static final Logger LOG = BlockManager.LOG;
+
+  /** List of recovery attempts per block and the time they expire. */
+  private final LightWeightHashSet<BlockRecoveryAttempt> recoveryTimeouts =
+      new LightWeightHashSet<>();
+
+  /** The timeout for issuing a block recovery again.
+   * (it should be larger than the time to recover a block)
+   */
+  private long recoveryTimeoutInterval;
+
+  PendingRecoveryBlocks(long timeout) {
+    this.recoveryTimeoutInterval = timeout;
+  }
+
+  /**
+   * Remove recovery attempt for the given block.
+   * @param block whose recovery attempt to remove.
+   */
+  synchronized void remove(BlockInfo block) {
+    recoveryTimeouts.remove(new BlockRecoveryAttempt(block));
+  }
+
+  /**
+   * Checks whether a recovery attempt has been made for the given block.
+   * If so, checks whether that attempt has timed out.
+   * @param block block for which recovery is being attempted
+   * @return true if no recovery attempt has been made or
+   *         the previous attempt timed out
+   */
+  synchronized boolean add(BlockInfo block) {
+    boolean added = false;
+    long curTime = getTime();
+    BlockRecoveryAttempt recoveryAttempt =
+        recoveryTimeouts.getElement(new BlockRecoveryAttempt(block));
+
+    if (recoveryAttempt == null) {
+      BlockRecoveryAttempt newAttempt = new BlockRecoveryAttempt(
+          block, curTime + recoveryTimeoutInterval);
+      added = recoveryTimeouts.add(newAttempt);
+    } else if (recoveryAttempt.hasTimedOut(curTime)) {
+      // Previous attempt timed out, reset the timeout
+      recoveryAttempt.setTimeout(curTime + recoveryTimeoutInterval);
+      added = true;
+    } else {
+      long timeoutIn = TimeUnit.MILLISECONDS.toSeconds(
+          recoveryAttempt.timeoutAt - curTime);
+      LOG.info("Block recovery attempt for " + block + " rejected, as the " +
+          "previous attempt times out in " + timeoutIn + " seconds.");
+    }
+    return added;
+  }
+
+  /**
+   * Check whether the given block is under recovery.
+   * @param b block for which to check
+   * @return true if the given block is being recovered
+   */
+  synchronized boolean isUnderRecovery(BlockInfo b) {
+    BlockRecoveryAttempt recoveryAttempt =
+        recoveryTimeouts.getElement(new BlockRecoveryAttempt(b));
+    return recoveryAttempt != null;
+  }
+
+  long getTime() {
+    return Time.monotonicNow();
+  }
+
+  @VisibleForTesting
+  synchronized void setRecoveryTimeoutInterval(long recoveryTimeoutInterval) {
+    this.recoveryTimeoutInterval = recoveryTimeoutInterval;
+  }
+
+  /**
+   * Tracks timeout for block recovery attempt of a given block.
+   */
+  private static class BlockRecoveryAttempt {
+    private final BlockInfo blockInfo;
+    private long timeoutAt;
+
+    private BlockRecoveryAttempt(BlockInfo blockInfo) {
+      this(blockInfo, 0);
+    }
+
+    BlockRecoveryAttempt(BlockInfo blockInfo, long timeoutAt) {
+      this.blockInfo = blockInfo;
+      this.timeoutAt = timeoutAt;
+    }
+
+    boolean hasTimedOut(long currentTime) {
+      return currentTime > timeoutAt;
+    }
+
+    void setTimeout(long newTimeoutAt) {
+      this.timeoutAt = newTimeoutAt;
+    }
+
+    @Override
+    public int hashCode() {
+      return blockInfo.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj instanceof BlockRecoveryAttempt) {
+        return this.blockInfo.equals(((BlockRecoveryAttempt) obj).blockInfo);
+      }
+      return false;
+    }
+  }
+}

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java

@@ -44,10 +44,10 @@ class ReplicationWork extends BlockReconstructionWork {
         : "At least 1 source node should have been selected";
     try {
       DatanodeStorageInfo[] chosenTargets = blockplacement.chooseTarget(
-          getBc().getName(), getAdditionalReplRequired(), getSrcNodes()[0],
+          getSrcPath(), getAdditionalReplRequired(), getSrcNodes()[0],
           getLiveReplicaStorages(), false, excludedNodes,
           getBlock().getNumBytes(),
-          storagePolicySuite.getPolicy(getBc().getStoragePolicyID()),
+          storagePolicySuite.getPolicy(getStoragePolicyID()),
           null);
       setTargets(chosenTargets);
     } finally {

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

@@ -94,7 +94,7 @@ public interface ActiveNamenodeResolver {
    * @return True if the node was registered and successfully committed to the
    *         data store.
    * @throws IOException Throws exception if the namenode could not be
-   *           registered.
+   *         registered.
    */
   boolean registerNamenode(NamenodeStatusReport report) throws IOException;
 
@@ -103,7 +103,8 @@ public interface ActiveNamenodeResolver {
    * federation.
    *
    * @return List of name spaces in the federation
-   * @throws Throws exception if the namespace list is not available.
+   * @throws IOException Throws exception if the namespace list is not
+   *         available.
    */
   Set<FederationNamespaceInfo> getNamespaces() throws IOException;
 
@@ -111,7 +112,7 @@ public interface ActiveNamenodeResolver {
    * Assign a unique identifier for the parent router service.
    * Required to report the status to the namenode resolver.
    *
-   * @param router Unique string identifier for the router.
+   * @param routerId Unique string identifier for the router.
    */
   void setRouterId(String routerId);
 }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java

@@ -270,8 +270,7 @@ public class MembershipNamenodeResolver
    * (if showStandby) 3) Most recently updated UNAVAILABLE registration (if
    * showUnavailable). EXPIRED registrations are ignored.
    *
-   * @param query The select query for NN registrations.
-   * @param excludes List of NNs to exclude from matching results.
+   * @param request The select query for NN registrations.
    * @param addUnavailable include UNAVAILABLE registrations.
    * @param addExpired include EXPIRED registrations.
    * @return List of memberships or null if no registrations that

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

@@ -207,7 +207,7 @@ public class MountTableResolver
   /**
    * Invalidates all cache entries below this path. It requires the write lock.
    *
-   * @param src Source path.
+   * @param path Source path.
    */
   private void invalidateLocationCache(final String path) {
     if (locationCache.isEmpty()) {
@@ -449,7 +449,8 @@ public class MountTableResolver
   /**
    * Build a location for this result beneath the discovered mount point.
    *
-   * @param result Tree node search result.
+   * @param path Path to build for.
+   * @param entry Mount table entry.
    * @return PathLocation containing the namespace, local path.
    */
   private static PathLocation buildLocation(

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

@@ -289,9 +289,9 @@ public class NamenodeStatusReport {
    * @param numFiles Number of files.
    * @param numBlocks Total number of blocks.
    * @param numBlocksMissing Number of missing blocks.
-   * @param numOfBlocksPendingReplication Number of blocks pending replication.
-   * @param numOfBlocksUnderReplicated Number of blocks under replication.
-   * @param numOfBlocksPendingDeletion Number of blocks pending deletion.
+   * @param numBlocksPendingReplication Number of blocks pending replication.
+   * @param numBlocksUnderReplicated Number of blocks under replication.
+   * @param numBlocksPendingDeletion Number of blocks pending deletion.
    */
   public void setNamesystemInfo(long available, long total,
       long numFiles, long numBlocks, long numBlocksMissing,
@@ -385,4 +385,4 @@ public class NamenodeStatusReport {
     return String.format("%s-%s:%s",
         nameserviceId, namenodeId, serviceAddress);
   }
-}
+}

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

@@ -93,8 +93,6 @@ public class ConnectionManager {
    * Creates a proxy client connection pool manager.
    *
    * @param config Configuration for the connections.
-   * @param minPoolSize Min size of the connection pool.
-   * @param maxPoolSize Max size of the connection pool.
    */
   public ConnectionManager(Configuration config) {
     this.conf = config;
@@ -428,4 +426,4 @@ public class ConnectionManager {
       this.interrupt();
     }
   }
-}
+}

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

@@ -171,7 +171,7 @@ public class ConnectionPool {
   /**
    * Add a connection to the current pool. It uses a Copy-On-Write approach.
    *
-   * @param conns New connections to add to the pool.
+   * @param conn New connection to add to the pool.
    */
   public synchronized void addConnection(ConnectionContext conn) {
     List<ConnectionContext> tmpConnections = new ArrayList<>(this.connections);
@@ -334,4 +334,4 @@ public class ConnectionPool {
     ConnectionContext connection = new ConnectionContext(clientProxy);
     return connection;
   }
-}
+}

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

@@ -155,7 +155,7 @@ public final class FederationUtil {
    * Creates an instance of an ActiveNamenodeResolver from the configuration.
    *
    * @param conf Configuration that defines the namenode resolver class.
-   * @param obj Context object passed to class constructor.
+   * @param stateStore State store passed to class constructor.
    * @return New active namenode resolver.
    */
   public static ActiveNamenodeResolver newActiveNamenodeResolver(

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java

@@ -89,8 +89,8 @@ public class NamenodeHeartbeatService extends PeriodicService {
   /**
    * Create a new Namenode status updater.
    * @param resolver Namenode resolver service to handle NN registration.
-   * @param nameserviceId Identifier of the nameservice.
-   * @param namenodeId Identifier of the namenode in HA.
+   * @param nsId Identifier of the nameservice.
+   * @param nnId Identifier of the namenode in HA.
    */
   public NamenodeHeartbeatService(
       ActiveNamenodeResolver resolver, String nsId, String nnId) {
@@ -320,7 +320,7 @@ public class NamenodeHeartbeatService extends PeriodicService {
 
   /**
    * Get the parameters for a Namenode from JMX and add them to the report.
-   * @param webAddress Web interface of the Namenode to monitor.
+   * @param address Web interface of the Namenode to monitor.
    * @param report Namenode status report to update with JMX data.
    */
   private void updateJMXParameters(

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java

@@ -267,7 +267,7 @@ public class Router extends CompositeService {
   /**
    * Set the current RPC socket for the router.
    *
-   * @param rpcAddress RPC address.
+   * @param address RPC address.
    */
   protected void setRpcServerAddress(InetSocketAddress address) {
     this.rpcAddress = address;
@@ -310,7 +310,7 @@ public class Router extends CompositeService {
   /**
    * Set the current Admin socket for the router.
    *
-   * @param adminAddress Admin RPC address.
+   * @param address Admin RPC address.
    */
   protected void setAdminServerAddress(InetSocketAddress address) {
     this.adminAddress = address;
@@ -513,7 +513,7 @@ public class Router extends CompositeService {
   /**
    * Sets a unique ID for this router.
    *
-   * @param router Identifier of the Router.
+   * @param id Identifier of the Router.
    */
   public void setRouterId(String id) {
     this.routerId = id;

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

@@ -264,7 +264,7 @@ public class RouterRpcClient {
   /**
    * If we should retry the RPC call.
    *
-   * @param ex Exception reported.
+   * @param ioe IOException reported.
    * @param retryCount Number of retries.
    * @return Retry decision.
    * @throws IOException Original exception if the retry policy generates one.
@@ -675,8 +675,8 @@ public class RouterRpcClient {
   /**
    * Checks if a result matches the required result class.
    *
-   * @param expectedResultClass Required result class, null to skip the check.
-   * @param result The result to check.
+   * @param expectedClass Required result class, null to skip the check.
+   * @param clazz The result to check.
    * @return True if the result is an instance of the required class or if the
    *         expected class is null.
    */
@@ -693,8 +693,8 @@ public class RouterRpcClient {
   /**
    * Checks if a result matches the expected value.
    *
-   * @param expectedResultValue The expected value, null to skip the check.
-   * @param result The result to check.
+   * @param expectedValue The expected value, null to skip the check.
+   * @param value The result to check.
    * @return True if the result is equals to the expected value or if the
    *         expected value is null.
    */
@@ -717,7 +717,7 @@ public class RouterRpcClient {
    *
    * @param <T> The type of the remote location.
    * @param locations List of remote locations to call concurrently.
-   * @param remoteMethod The remote method and parameters to invoke.
+   * @param method The remote method and parameters to invoke.
    * @param requireResponse If true an exception will be thrown if all calls do
    *          not complete. If false exceptions are ignored and all data results
    *          successfully received are returned.
@@ -740,12 +740,12 @@ public class RouterRpcClient {
    * RemoteException or IOException.
    *
    * @param locations List of remote locations to call concurrently.
-   * @param remoteMethod The remote method and parameters to invoke.
+   * @param method The remote method and parameters to invoke.
    * @param requireResponse If true an exception will be thrown if all calls do
    *          not complete. If false exceptions are ignored and all data results
    *          successfully received are returned.
    * @param standby If the requests should go to the standby namenodes too.
-   * @param timeoutMs Timeout for each individual call.
+   * @param timeOutMs Timeout for each individual call.
    * @return Result of invoking the method per subcluster: nsId -> result.
    * @throws IOException If requiredResponse=true and any of the calls throw an
    *           exception.
@@ -877,7 +877,7 @@ public class RouterRpcClient {
    * Get a prioritized list of NNs that share the same nameservice ID (in the
    * same namespace). NNs that are reported as ACTIVE will be first in the list.
    *
-   * @param nameserviceId The nameservice ID for the namespace.
+   * @param nsId The nameservice ID for the namespace.
    * @return A prioritized list of NNs to use for communication.
    * @throws IOException If a NN cannot be located for the nameservice ID.
    */
@@ -898,7 +898,7 @@ public class RouterRpcClient {
    * Get a prioritized list of NNs that share the same block pool ID (in the
    * same namespace). NNs that are reported as ACTIVE will be first in the list.
    *
-   * @param blockPoolId The blockpool ID for the namespace.
+   * @param bpId The blockpool ID for the namespace.
    * @return A prioritized list of NNs to use for communication.
    * @throws IOException If a NN cannot be located for the block pool ID.
    */
@@ -929,4 +929,4 @@ public class RouterRpcClient {
     FederationNamenodeContext namenode = namenodes.get(0);
     return namenode.getNameserviceId();
   }
-}
+}

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

@@ -92,6 +92,7 @@ import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
@@ -1390,7 +1391,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
   /**
    * Aggregate content summaries for each subcluster.
    *
-   * @param results Collection of individual summaries.
+   * @param summaries Collection of individual summaries.
    * @return Aggregated content summary.
    */
   private ContentSummary aggregateContentSummary(
@@ -1508,6 +1509,14 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     return null;
   }
 
+  @Override // ClientProtocol
+  public SnapshotDiffReportListing getSnapshotDiffReportListing(
+      String snapshotRoot, String earlierSnapshotName, String laterSnapshotName,
+      byte[] startPath, int index) throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
   @Override // ClientProtocol
   public long addCacheDirective(CacheDirectiveInfo path,
       EnumSet<CacheFlag> flags) throws IOException {
@@ -1999,7 +2008,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
    *
    * @param name Name of the mount point.
    * @param childrenNum Number of children.
-   * @param dates Map with the dates.
+   * @param date Map with the dates.
    * @return New HDFS file status representing a mount point.
    */
   private HdfsFileStatus getMountPointStatus(

+ 2 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/CachedRecordStore.java

@@ -86,7 +86,7 @@ public abstract class CachedRecordStore<R extends BaseRecord>
    *
    * @param clazz Class of the record to store.
    * @param driver State Store driver.
-   * @param override If the entries should be override if they expire
+   * @param over If the entries should be override if they expire
    */
   protected CachedRecordStore(
       Class<R> clazz, StateStoreDriver driver, boolean over) {
@@ -167,7 +167,6 @@ public abstract class CachedRecordStore<R extends BaseRecord>
    * expired state.
    *
    * @param query RecordQueryResult containing the data to be inspected.
-   * @param clazz Type of objects contained in the query.
    * @throws IOException
    */
   public void overrideExpiredRecords(QueryResult<R> query) throws IOException {
@@ -194,9 +193,7 @@ public abstract class CachedRecordStore<R extends BaseRecord>
    * Updates the state store with any record overrides we detected, such as an
    * expired state.
    *
-   * @param driver State store driver for the data store.
    * @param record Record record to be updated.
-   * @param clazz Type of data record.
    * @throws IOException
    */
   public void overrideExpiredRecord(R record) throws IOException {
@@ -242,4 +239,4 @@ public abstract class CachedRecordStore<R extends BaseRecord>
       this.readLock.unlock();
     }
   }
-}
+}

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

@@ -80,7 +80,7 @@ public abstract class RecordStore<R extends BaseRecord> {
   /**
    * Build a state store API implementation interface.
    *
-   * @param interfaceClass The specific interface implementation to create
+   * @param clazz The specific interface implementation to create
    * @param driver The {@link StateStoreDriver} implementation in use.
    * @return An initialized instance of the specified state store API
    *         implementation.

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

@@ -128,9 +128,10 @@ public abstract class StateStoreDriver implements StateStoreRecordOperations {
   /**
    * Initialize storage for a single record class.
    *
-   * @param name String reference of the record class to initialize, used to
-   *             construct paths and file names for the record. Determined by
-   *             configuration settings for the specific driver.
+   * @param className String reference of the record class to initialize,
+   *                  used to construct paths and file names for the record.
+   *                  Determined by configuration settings for the specific
+   *                  driver.
    * @param clazz Record type corresponding to the provided name.
    * @return True if successful, false otherwise.
    */
@@ -199,4 +200,4 @@ public abstract class StateStoreDriver implements StateStoreRecordOperations {
     }
     return hostname;
   }
-}
+}

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java

@@ -192,7 +192,7 @@ public abstract class StateStoreFileBaseImpl
    * Read all lines from a file and deserialize into the desired record type.
    *
    * @param reader Open handle for the file.
-   * @param recordClass Record class to create.
+   * @param clazz Record class to create.
    * @param includeDates True if dateModified/dateCreated are serialized.
    * @return List of records.
    * @throws IOException
@@ -250,7 +250,7 @@ public abstract class StateStoreFileBaseImpl
   /**
    * Overwrite the existing data with a new data set.
    *
-   * @param list List of records to write.
+   * @param records List of records to write.
    * @param writer BufferedWriter stream to write to.
    * @return If the records were succesfully written.
    */
@@ -426,4 +426,4 @@ public abstract class StateStoreFileBaseImpl
     boolean status = writeAll(emptyList, clazz);
     return status;
   }
-}
+}

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java

@@ -98,7 +98,7 @@ public class StateStoreFileSystemImpl extends StateStoreFileBaseImpl {
   /**
    * Get the folder path for the record class' data.
    *
-   * @param cls Data record class.
+   * @param clazz Data record class.
    * @return Path of the folder containing the record class' data files.
    */
   private Path getPathForClass(Class<? extends BaseRecord> clazz) {
@@ -175,4 +175,4 @@ public class StateStoreFileSystemImpl extends StateStoreFileBaseImpl {
       return null;
     }
   }
-}
+}

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreZooKeeperImpl.java

@@ -309,7 +309,8 @@ public class StateStoreZooKeeperImpl extends StateStoreSerializableImpl {
   /**
    * Creates a record from a string returned by ZooKeeper.
    *
-   * @param source Object from ZooKeeper.
+   * @param data The data to write.
+   * @param stat Stat of the data record to create.
    * @param clazz The data record type to create.
    * @return The created record.
    * @throws IOException
@@ -321,4 +322,4 @@ public class StateStoreZooKeeperImpl extends StateStoreSerializableImpl {
     record.setDateModified(stat.getMtime());
     return record;
   }
-}
+}

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MembershipStoreImpl.java

@@ -267,8 +267,7 @@ public class MembershipStoreImpl
    * majority value 2) Otherwise the first sorted entry in the set of all
    * entries
    *
-   * @param entries - Collection of state store record objects of the same type
-   * @param fieldName - Field name for the value to compare
+   * @param records - Collection of state store record objects of the same type
    * @return record that is most representative of the field name
    */
   private MembershipState getRepresentativeQuorum(
@@ -308,4 +307,4 @@ public class MembershipStoreImpl
       return null;
     }
   }
-}
+}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java

@@ -395,7 +395,7 @@ public class Mover {
           status.getReplication());
 
       final ErasureCodingPolicy ecPolicy = status.getErasureCodingPolicy();
-      final LocatedBlocks locatedBlocks = status.getBlockLocations();
+      final LocatedBlocks locatedBlocks = status.getLocatedBlocks();
       final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete();
       List<LocatedBlock> lbs = locatedBlocks.getLocatedBlocks();
       for (int i = 0; i < lbs.size(); i++) {

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
@@ -164,6 +165,29 @@ class FSDirSnapshotOp {
     return diffs;
   }
 
+  static SnapshotDiffReportListing getSnapshotDiffReportListing(FSDirectory fsd,
+      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);
+      if (fsd.isPermissionEnabled()) {
+        checkSubtreeReadPermission(fsd, pc, path, fromSnapshot);
+        checkSubtreeReadPermission(fsd, pc, path, toSnapshot);
+      }
+      diffs = snapshotManager
+          .diff(iip, path, fromSnapshot, toSnapshot, startPath, index,
+              snapshotDiffReportLimit);
+    } catch (Exception e) {
+      throw e;
+    } finally {
+      fsd.readUnlock();
+    }
+    return diffs;
+  }
   /** Get a collection of full snapshot paths given file and snapshot dir.
    * @param lsf a list of snapshottable features
    * @param file full path of the file

+ 21 - 26
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java

@@ -261,7 +261,7 @@ class FSDirStatAndListingOp {
             // This helps to prevent excessively large response payloads.
             // Approximate #locations with locatedBlockCount() * repl_factor
             LocatedBlocks blks =
-                ((HdfsLocatedFileStatus)listing[i]).getBlockLocations();
+                ((HdfsLocatedFileStatus)listing[i]).getLocatedBlocks();
             locationBudget -= (blks == null) ? 0 :
                blks.locatedBlockCount() * listing[i].getReplication();
         }
@@ -486,31 +486,26 @@ class FSDirStatAndListingOp {
       String owner, String group, byte[] symlink, byte[] path, long fileId,
       int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy,
       ErasureCodingPolicy ecPolicy, LocatedBlocks locations) {
-    if (locations == null) {
-      return new HdfsFileStatus.Builder()
-          .length(length)
-          .isdir(isdir)
-          .replication(replication)
-          .blocksize(blocksize)
-          .mtime(mtime)
-          .atime(atime)
-          .perm(permission)
-          .flags(flags)
-          .owner(owner)
-          .group(group)
-          .symlink(symlink)
-          .path(path)
-          .fileId(fileId)
-          .children(childrenNum)
-          .feInfo(feInfo)
-          .storagePolicy(storagePolicy)
-          .ecPolicy(ecPolicy)
-          .build();
-    } else {
-      return new HdfsLocatedFileStatus(length, isdir, replication, blocksize,
-          mtime, atime, permission, flags, owner, group, symlink, path,
-          fileId, locations, childrenNum, feInfo, storagePolicy, ecPolicy);
-    }
+    return new HdfsFileStatus.Builder()
+        .length(length)
+        .isdir(isdir)
+        .replication(replication)
+        .blocksize(blocksize)
+        .mtime(mtime)
+        .atime(atime)
+        .perm(permission)
+        .flags(flags)
+        .owner(owner)
+        .group(group)
+        .symlink(symlink)
+        .path(path)
+        .fileId(fileId)
+        .children(childrenNum)
+        .feInfo(feInfo)
+        .storagePolicy(storagePolicy)
+        .ecPolicy(ecPolicy)
+        .locations(locations)
+        .build();
   }
 
   private static ContentSummary getContentSummaryInt(FSDirectory fsd,

+ 94 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -87,6 +87,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROU
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT_DEFAULT;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 
@@ -95,6 +97,8 @@ import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.server.namenode.metrics.ReplicatedBlocksMBean;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.apache.hadoop.util.Time.now;
@@ -211,7 +215,6 @@ import org.apache.hadoop.hdfs.protocol.RollingUpgradeException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -426,6 +429,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private final UserGroupInformation fsOwner;
   private final String supergroup;
   private final boolean standbyShouldCheckpoint;
+  private final int snapshotDiffReportLimit;
 
   /** Interval between each check of lease to release. */
   private final long leaseRecheckIntervalMs;
@@ -761,6 +765,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                                  DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
       this.isPermissionEnabled = conf.getBoolean(DFS_PERMISSIONS_ENABLED_KEY,
                                                  DFS_PERMISSIONS_ENABLED_DEFAULT);
+      this.snapshotDiffReportLimit =
+          conf.getInt(DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT,
+              DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT_DEFAULT);
+
       LOG.info("fsOwner             = " + fsOwner);
       LOG.info("supergroup          = " + supergroup);
       LOG.info("isPermissionEnabled = " + isPermissionEnabled);
@@ -3310,25 +3318,30 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             + "Removed empty last block and closed file " + src);
         return true;
       }
-      // start recovery of the last block for this file
-      long blockRecoveryId = nextGenerationStamp(
-          blockManager.isLegacyBlock(lastBlock));
-      lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
-      if(copyOnTruncate) {
-        lastBlock.setGenerationStamp(blockRecoveryId);
-      } else if(truncateRecovery) {
-        recoveryBlock.setGenerationStamp(blockRecoveryId);
-      }
-      uc.initializeBlockRecovery(lastBlock, blockRecoveryId, true);
-      leaseManager.renewLease(lease);
-      // Cannot close file right now, since the last block requires recovery.
-      // This may potentially cause infinite loop in lease recovery
-      // if there are no valid replicas on data-nodes.
-      NameNode.stateChangeLog.warn(
-                "DIR* NameSystem.internalReleaseLease: " +
+      // Start recovery of the last block for this file
+      // Only do so if there is no ongoing recovery for this block,
+      // or the previous recovery for this block timed out.
+      if (blockManager.addBlockRecoveryAttempt(lastBlock)) {
+        long blockRecoveryId = nextGenerationStamp(
+            blockManager.isLegacyBlock(lastBlock));
+        if(copyOnTruncate) {
+          lastBlock.setGenerationStamp(blockRecoveryId);
+        } else if(truncateRecovery) {
+          recoveryBlock.setGenerationStamp(blockRecoveryId);
+        }
+        uc.initializeBlockRecovery(lastBlock, blockRecoveryId, true);
+
+        // Cannot close file right now, since the last block requires recovery.
+        // This may potentially cause infinite loop in lease recovery
+        // if there are no valid replicas on data-nodes.
+        NameNode.stateChangeLog.warn(
+            "DIR* NameSystem.internalReleaseLease: " +
                 "File " + src + " has not been closed." +
-               " Lease recovery is in progress. " +
+                " Lease recovery is in progress. " +
                 "RecoveryId = " + blockRecoveryId + " for block " + lastBlock);
+      }
+      lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
+      leaseManager.renewLease(lease);
       break;
     }
     return false;
@@ -3596,6 +3609,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         // If this commit does not want to close the file, persist blocks
         FSDirWriteFileOp.persistBlocks(dir, src, iFile, false);
       }
+      blockManager.successfulBlockRecovery(storedBlock);
     } finally {
       writeUnlock("commitBlockSynchronization");
     }
@@ -6364,16 +6378,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Get the difference between two snapshots (or between a snapshot and the
    * current status) of a snapshottable directory.
-   * 
+   *
    * @param path The full path of the snapshottable directory.
    * @param fromSnapshot Name of the snapshot to calculate the diff from. Null
    *          or empty string indicates the current tree.
    * @param toSnapshot Name of the snapshot to calculated the diff to. Null or
    *          empty string indicates the current tree.
-   * @return A report about the difference between {@code fromSnapshot} and 
-   *         {@code toSnapshot}. Modified/deleted/created/renamed files and 
-   *         directories belonging to the snapshottable directories are listed 
-   *         and labeled as M/-/+/R respectively. 
+   * @return A report about the difference between {@code fromSnapshot} and
+   *         {@code toSnapshot}. Modified/deleted/created/renamed files and
+   *         directories belonging to the snapshottable directories are listed
+   *         and labeled as M/-/+/R respectively.
    * @throws IOException
    */
   SnapshotDiffReport getSnapshotDiffReport(String path,
@@ -6403,6 +6417,63 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         toSnapshotRoot, null);
     return diffs;
   }
+
+  /**
+   * Get the difference between two snapshots (or between a snapshot and the
+   * current status) of a snapshottable directory.
+   *
+   * @param path The full path of the snapshottable directory.
+   * @param fromSnapshot Name of the snapshot to calculate the diff from. Null
+   *          or empty string indicates the current tree.
+   * @param toSnapshot Name of the snapshot to calculated the diff to. Null or
+   *          empty string indicates the current tree.
+   * @param startPath
+   *          path relative to the snapshottable root directory from where the
+   *          snapshotdiff computation needs to start across multiple rpc calls
+   * @param index
+   *           index in the created or deleted list of the directory at which
+   *           the snapshotdiff computation stopped during the last rpc call
+   *           as the no of entries exceeded the snapshotdiffentry limit. -1
+   *           indicates, the snapshotdiff compuatation needs to start right
+   *           from the startPath provided.
+   * @return A partial report about the difference between {@code fromSnapshot}
+   *         and {@code toSnapshot}. Modified/deleted/created/renamed files and
+   *         directories belonging to the snapshottable directories are listed
+   *         and labeled as M/-/+/R respectively.
+   * @throws IOException
+   */
+  SnapshotDiffReportListing getSnapshotDiffReportListing(String path,
+      String fromSnapshot, String toSnapshot, byte[] startPath, int index)
+      throws IOException {
+    final String operationName = "computeSnapshotDiff";
+    SnapshotDiffReportListing diffs = null;
+    checkOperation(OperationCategory.READ);
+    boolean success = false;
+    String fromSnapshotRoot =
+        (fromSnapshot == null || fromSnapshot.isEmpty()) ? path :
+            Snapshot.getSnapshotPath(path, fromSnapshot);
+    String toSnapshotRoot =
+        (toSnapshot == null || toSnapshot.isEmpty()) ? path :
+            Snapshot.getSnapshotPath(path, toSnapshot);
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      diffs = FSDirSnapshotOp
+          .getSnapshotDiffReportListing(dir, snapshotManager, path,
+              fromSnapshot, toSnapshot, startPath, index,
+              snapshotDiffReportLimit);
+      success = true;
+    } catch (AccessControlException ace) {
+      logAuditEvent(success, operationName, fromSnapshotRoot, toSnapshotRoot,
+          null);
+      throw ace;
+    } finally {
+      readUnlock(operationName);
+    }
+    logAuditEvent(success, operationName, fromSnapshotRoot, toSnapshotRoot,
+        null);
+    return diffs;
+  }
   
   /**
    * Delete a snapshot of a snapshottable directory

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -33,9 +33,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
@@ -1058,6 +1060,18 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
       assert toDelete != null : "toDelete is null";
       toDelete.delete();
       toDeleteList.add(toDelete);
+      // If the file is being truncated
+      // the copy-on-truncate block should also be collected for deletion
+      BlockUnderConstructionFeature uc = toDelete.getUnderConstructionFeature();
+      if(uc == null) {
+        return;
+      }
+      Block truncateBlock = uc.getTruncateBlock();
+      if(truncateBlock == null || truncateBlock.equals(toDelete)) {
+        return;
+      }
+      assert truncateBlock instanceof BlockInfo : "should be BlockInfo";
+      addDeleteBlock((BlockInfo) truncateBlock);
     }
 
     public void addUpdateReplicationFactor(BlockInfo block, short targetRepl) {

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -121,6 +121,7 @@ import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
@@ -1862,6 +1863,18 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     return report;
   }
 
+  @Override // ClientProtocol
+  public SnapshotDiffReportListing getSnapshotDiffReportListing(
+      String snapshotRoot, String earlierSnapshotName, String laterSnapshotName,
+      byte[] startPath, int index) throws IOException {
+    checkNNStartup();
+    SnapshotDiffReportListing report = namesystem
+        .getSnapshotDiffReportListing(snapshotRoot, earlierSnapshotName,
+            laterSnapshotName, startPath, index);
+    metrics.incrSnapshotDiffReportOps();
+    return report;
+  }
+
   @Override // ClientProtocol
   public long addCacheDirective(
       CacheDirectiveInfo path, EnumSet<CacheFlag> flags) throws IOException {

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

@@ -24,10 +24,12 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
+import java.util.Arrays;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
@@ -284,6 +286,54 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
     return diffs;
   }
 
+  /**
+   * Compute the difference between two snapshots (or a snapshot and the current
+   * directory) of the directory. The diff calculation can be scoped to either
+   * the snapshot root or any descendant directory under the snapshot root.
+   *
+   * @param snapshotRootDir the snapshot root directory
+   * @param snapshotDiffScopeDir the descendant directory under snapshot root
+   *          to scope the diff calculation to.
+   * @param from The name of the start point of the comparison. Null indicating
+   *          the current tree.
+   * @param to The name of the end point. Null indicating the current tree.
+   * @param startPath
+   *           path relative to the snapshottable root directory from where the
+   *           snapshotdiff computation needs to start across multiple rpc calls
+   * @param index
+   *           index in the created or deleted list of the directory at which
+   *           the snapshotdiff computation stopped during the last rpc call
+   *           as the no of entries exceeded the snapshotdiffentry limit. -1
+   *           indicates, the snapshotdiff computation needs to start right
+   *           from the startPath provided.
+   *
+   * @return The difference between the start/end points.
+   * @throws SnapshotException If there is no snapshot matching the starting
+   *           point, or if endSnapshotName is not null but cannot be identified
+   *           as a previous snapshot.
+   */
+  SnapshotDiffListingInfo computeDiff(final INodeDirectory snapshotRootDir,
+      final INodeDirectory snapshotDiffScopeDir, final String from,
+      final String to, byte[] startPath, int index,
+      int snapshotDiffReportEntriesLimit) throws SnapshotException {
+    Preconditions.checkArgument(
+        snapshotDiffScopeDir.isDescendantOfSnapshotRoot(snapshotRootDir));
+    Snapshot fromSnapshot = getSnapshotByName(snapshotRootDir, from);
+    Snapshot toSnapshot = getSnapshotByName(snapshotRootDir, to);
+    boolean toProcess = Arrays.equals(startPath, DFSUtilClient.EMPTY_BYTES);
+    byte[][] resumePath = DFSUtilClient.bytes2byteArray(startPath);
+    if (from.equals(to)) {
+      return null;
+    }
+    SnapshotDiffListingInfo diffs =
+        new SnapshotDiffListingInfo(snapshotRootDir, snapshotDiffScopeDir,
+            fromSnapshot, toSnapshot, snapshotDiffReportEntriesLimit);
+    diffs.setLastIndex(index);
+    computeDiffRecursively(snapshotDiffScopeDir, snapshotDiffScopeDir,
+        new ArrayList<byte[]>(), diffs, resumePath, 0, toProcess);
+    return diffs;
+  }
+
   /**
    * Find the snapshot matching the given name.
    *
@@ -367,12 +417,96 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
     }
   }
 
+  /**
+   * Recursively compute the difference between snapshots under a given
+   * directory/file partially.
+   * @param snapshotDir The directory where snapshots were taken. Can be a
+   *                    snapshot root directory or any descendant directory
+   *                    under snapshot root directory.
+   * @param node The directory/file under which the diff is computed.
+   * @param parentPath Relative path (corresponding to the snapshot root) of
+   *                   the node's parent.
+   * @param diffReport data structure used to store the diff.
+   * @param resume  path from where to resume the snapshotdiff computation
+   *                    in one rpc call
+   * @param level       indicates the level of the directory tree rooted at
+   *                    snapshotRoot.
+   * @param processFlag indicates that the dir/file where the snapshotdiff
+   *                    computation has to start is processed or not.
+   */
+  private boolean computeDiffRecursively(final INodeDirectory snapshotDir,
+       INode node, List<byte[]> parentPath, SnapshotDiffListingInfo diffReport,
+       final byte[][] resume, int level, boolean processFlag) {
+    final Snapshot earlier = diffReport.getEarlier();
+    final Snapshot later = diffReport.getLater();
+    byte[][] relativePath = parentPath.toArray(new byte[parentPath.size()][]);
+    if (!processFlag && level == resume.length
+        && Arrays.equals(resume[resume.length - 1], node.getLocalNameBytes())) {
+      processFlag = true;
+    }
+
+    if (node.isDirectory()) {
+      final ChildrenDiff diff = new ChildrenDiff();
+      INodeDirectory dir = node.asDirectory();
+      if (processFlag) {
+        DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
+        if (sf != null) {
+          boolean change =
+              sf.computeDiffBetweenSnapshots(earlier, later, diff, dir);
+          if (change) {
+            if (!diffReport.addDirDiff(dir.getId(), relativePath, diff)) {
+              return false;
+            }
+          }
+        }
+      }
+
+      ReadOnlyList<INode> children = dir.getChildrenList(earlier.getId());
+      boolean iterate = false;
+      for (INode child : children) {
+        final byte[] name = child.getLocalNameBytes();
+        if (!processFlag && !iterate && !Arrays.equals(resume[level], name)) {
+          continue;
+        }
+        iterate = true;
+        level = level + 1;
+        boolean toProcess = diff.searchIndex(ListType.DELETED, name) < 0;
+        if (!toProcess && child instanceof INodeReference.WithName) {
+          byte[][] renameTargetPath = findRenameTargetPath(snapshotDir,
+              (WithName) child, Snapshot.getSnapshotId(later));
+          if (renameTargetPath != null) {
+            toProcess = true;
+          }
+        }
+        if (toProcess) {
+          parentPath.add(name);
+          processFlag = computeDiffRecursively(snapshotDir, child, parentPath,
+              diffReport, resume, level, processFlag);
+          parentPath.remove(parentPath.size() - 1);
+          if (!processFlag) {
+            return false;
+          }
+        }
+      }
+    } else if (node.isFile() && node.asFile().isWithSnapshot() && processFlag) {
+      INodeFile file = node.asFile();
+      boolean change = file.getFileWithSnapshotFeature()
+          .changedBetweenSnapshots(file, earlier, later);
+      if (change) {
+        if (!diffReport.addFileDiff(file, relativePath)) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
   /**
    * We just found a deleted WithName node as the source of a rename operation.
    * However, we should include it in our snapshot diff report as rename only
    * if the rename target is also under the same snapshottable directory.
    */
-  private byte[][] findRenameTargetPath(final INodeDirectory snapshotRoot,
+  public byte[][] findRenameTargetPath(final INodeDirectory snapshotRoot,
       INodeReference.WithName wn, final int snapshotId) {
     INode inode = wn.getReferredINode();
     final LinkedList<byte[]> ancestors = Lists.newLinkedList();

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

@@ -0,0 +1,207 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.snapshot;
+
+import java.util.List;
+import java.util.ListIterator;
+
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing.DiffReportListingEntry;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.INodeReference;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.ChildrenDiff;
+import org.apache.hadoop.hdfs.util.Diff.ListType;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.util.ChunkedArrayList;
+
+/**
+ * A class describing the difference between snapshots of a snapshottable
+ * directory where the difference is limited by dfs.snapshotDiff-report.limit.
+ */
+
+class SnapshotDiffListingInfo {
+  private final int maxEntries;
+
+  /** The root directory of the snapshots. */
+  private final INodeDirectory snapshotRoot;
+  /**
+   *  The scope directory under which snapshot diff is calculated.
+   */
+  private final INodeDirectory snapshotDiffScopeDir;
+  /** The starting point of the difference. */
+  private final Snapshot from;
+  /** The end point of the difference. */
+  private final Snapshot to;
+
+  /** The path of the file to start for computing the snapshot diff. */
+  private byte[] lastPath = DFSUtilClient.EMPTY_BYTES;
+
+  private int lastIndex = -1;
+
+  /*
+   * A list containing all the modified entries between the given snapshots
+   * within a single rpc call.
+   */
+  private final List<DiffReportListingEntry> modifiedList =
+      new ChunkedArrayList<>();
+
+  private final List<DiffReportListingEntry> createdList =
+      new ChunkedArrayList<>();
+
+  private final List<DiffReportListingEntry> deletedList =
+      new ChunkedArrayList<>();
+
+  SnapshotDiffListingInfo(INodeDirectory snapshotRootDir,
+      INodeDirectory snapshotDiffScopeDir, Snapshot start, Snapshot end,
+      int snapshotDiffReportLimit) {
+    Preconditions.checkArgument(
+        snapshotRootDir.isSnapshottable() && snapshotDiffScopeDir
+            .isDescendantOfSnapshotRoot(snapshotRootDir));
+    this.snapshotRoot = snapshotRootDir;
+    this.snapshotDiffScopeDir = snapshotDiffScopeDir;
+    this.from = start;
+    this.to = end;
+    this.maxEntries = snapshotDiffReportLimit;
+  }
+
+  boolean addDirDiff(long dirId, byte[][] parent, ChildrenDiff diff) {
+    final Snapshot laterSnapshot = getLater();
+    if (lastIndex == -1) {
+      if (getTotalEntries() < maxEntries) {
+        modifiedList.add(new DiffReportListingEntry(
+            dirId, dirId, parent, true, null));
+      } else {
+        setLastPath(parent);
+        setLastIndex(-1);
+        return false;
+      }
+    }
+
+    if (lastIndex == -1 || lastIndex < diff.getList(ListType.CREATED).size()) {
+      ListIterator<INode> iterator = lastIndex != -1 ?
+          diff.getList(ListType.CREATED).listIterator(lastIndex)
+          : diff.getList(ListType.CREATED).listIterator();
+      while (iterator.hasNext()) {
+        if (getTotalEntries() < maxEntries) {
+          INode created = iterator.next();
+          byte[][] path = newPath(parent, created.getLocalNameBytes());
+          createdList.add(new DiffReportListingEntry(dirId, created.getId(),
+              path, created.isReference(), null));
+        } else {
+          setLastPath(parent);
+          setLastIndex(iterator.nextIndex());
+          return false;
+        }
+      }
+      setLastIndex(-1);
+    }
+
+    if (lastIndex == -1 || lastIndex >= diff.getList(ListType.CREATED).size()) {
+      int size = diff.getList(ListType.DELETED).size();
+      ListIterator<INode> iterator = lastIndex != -1 ?
+          diff.getList(ListType.DELETED).listIterator(lastIndex - size)
+          : diff.getList(ListType.DELETED).listIterator();
+      while (iterator.hasNext()) {
+        if (getTotalEntries() < maxEntries) {
+          final INode d = iterator.next();
+          byte[][] path = newPath(parent, d.getLocalNameBytes());
+          byte[][] target = findRenameTargetPath(d, laterSnapshot);
+          final DiffReportListingEntry e = target != null ?
+              new DiffReportListingEntry(dirId, d.getId(), path, true, target) :
+              new DiffReportListingEntry(dirId, d.getId(), path, false, null);
+          deletedList.add(e);
+        } else {
+          setLastPath(parent);
+          setLastIndex(size + iterator.nextIndex());
+          return false;
+        }
+      }
+      setLastIndex(-1);
+    }
+    return true;
+  }
+
+  private byte[][] findRenameTargetPath(INode deleted, Snapshot laterSnapshot) {
+    if (deleted instanceof INodeReference.WithName) {
+      return snapshotRoot.getDirectorySnapshottableFeature()
+          .findRenameTargetPath(snapshotDiffScopeDir,
+              (INodeReference.WithName) deleted,
+              Snapshot.getSnapshotId(laterSnapshot));
+    }
+    return null;
+  }
+
+  private static byte[][] newPath(byte[][] parent, byte[] name) {
+    byte[][] fullPath = new byte[parent.length + 1][];
+    System.arraycopy(parent, 0, fullPath, 0, parent.length);
+    fullPath[fullPath.length - 1] = name;
+    return fullPath;
+  }
+
+  Snapshot getEarlier() {
+    return isFromEarlier()? from: to;
+  }
+
+  Snapshot getLater() {
+    return isFromEarlier()? to: from;
+  }
+
+
+  public void setLastPath(byte[][] lastPath) {
+    this.lastPath = DFSUtilClient.byteArray2bytes(lastPath);
+  }
+
+  public void setLastIndex(int idx) {
+    this.lastIndex = idx;
+  }
+
+  boolean addFileDiff(INodeFile file, byte[][] relativePath) {
+    if (getTotalEntries() < maxEntries) {
+      modifiedList.add(new DiffReportListingEntry(file.getId(),
+          file.getId(), relativePath,false, null));
+    } else {
+      setLastPath(relativePath);
+      return false;
+    }
+    return true;
+  }
+  /** @return True if {@link #from} is earlier than {@link #to} */
+  boolean isFromEarlier() {
+    return Snapshot.ID_COMPARATOR.compare(from, to) < 0;
+  }
+
+
+  private int getTotalEntries() {
+    return createdList.size() + modifiedList.size() + deletedList.size();
+  }
+
+  /**
+   * Generate a {@link SnapshotDiffReportListing} based on detailed diff
+   * information.
+   *
+   * @return A {@link SnapshotDiffReportListing} describing the difference
+   */
+  public SnapshotDiffReportListing generateReport() {
+    return new SnapshotDiffReportListing(lastPath, modifiedList, createdList,
+        deletedList, lastIndex, isFromEarlier());
+  }
+}

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

@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshotInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@@ -466,6 +467,33 @@ public class SnapshotManager implements SnapshotStatsMXBean {
     return diffs != null ? diffs.generateReport() : new SnapshotDiffReport(
         snapshotPath, from, to, Collections.<DiffReportEntry> emptyList());
   }
+
+  /**
+   * Compute the partial difference between two snapshots of a directory,
+   * or between a snapshot of the directory and its current tree.
+   */
+  public SnapshotDiffReportListing diff(final INodesInPath iip,
+      final String snapshotPath, final String from, final String to,
+      byte[] startPath, int index, int snapshotDiffReportLimit)
+      throws IOException {
+    // Find the source root directory path where the snapshots were taken.
+    // All the check for path has been included in the valueOf method.
+    INodeDirectory snapshotRootDir;
+    if (this.snapshotDiffAllowSnapRootDescendant) {
+      snapshotRootDir = getSnapshottableAncestorDir(iip);
+    } else {
+      snapshotRootDir = getSnapshottableRoot(iip);
+    }
+    Preconditions.checkNotNull(snapshotRootDir);
+    INodeDirectory snapshotDescendantDir = INodeDirectory.valueOf(
+        iip.getLastINode(), snapshotPath);
+    final SnapshotDiffListingInfo diffs =
+        snapshotRootDir.getDirectorySnapshottableFeature()
+            .computeDiff(snapshotRootDir, snapshotDescendantDir, from, to,
+                startPath, index, snapshotDiffReportLimit);
+    return diffs != null ? diffs.generateReport() :
+        new SnapshotDiffReportListing();
+  }
   
   public void clearSnapshottableDirs() {
     snapshottables.clear();

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.tools;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -190,14 +191,13 @@ public class DFSZKFailoverController extends ZKFailoverController {
         new HdfsConfiguration(), args);
     DFSZKFailoverController zkfc = DFSZKFailoverController.create(
         parser.getConfiguration());
-    int retCode = 0;
     try {
-      retCode = zkfc.run(parser.getRemainingArgs());
+      System.exit(zkfc.run(parser.getRemainingArgs()));
     } catch (Throwable t) {
       LOG.fatal("DFSZKFailOverController exiting due to earlier exception "
           + t);
+      terminate(1, t);
     }
-    System.exit(retCode);
   }
 
   @Override

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

@@ -2282,6 +2282,16 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.client.server-defaults.validity.period.ms</name>
+  <value>3600000</value>
+  <description>
+    The amount of milliseconds after which cached server defaults are updated.
+
+    By default this parameter is set to 1 hour.
+  </description>
+</property>
+
 <property>
   <name>dfs.namenode.enable.retrycache</name>
   <value>true</value>
@@ -4322,6 +4332,17 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.snapshotdiff.listing.limit</name>
+  <value>1000</value>
+  <description>
+    Limit the number of entries generated by getSnapshotDiffReportListing within
+    one rpc call to the namenode.If less or equal to zero, at most
+    DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT_DEFAULT (= 1000) will be sent
+    across to the client within one rpc call.
+  </description>
+</property>
+
 <property>
   <name>dfs.pipeline.ecn</name>
   <value>false</value>

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -3108,6 +3108,16 @@ public class MiniDFSCluster implements AutoCloseable {
     // Wait for new namenode to get registrations from all the datanodes
     waitActive(nnIndex);
   }
+
+  /**
+   * Sets the timeout for re-issuing a block recovery.
+   */
+  public void setBlockRecoveryTimeout(long timeout) {
+    for (int nnIndex = 0; nnIndex < getNumNameNodes(); nnIndex++) {
+      getNamesystem(nnIndex).getBlockManager().setBlockRecoveryTimeout(
+          timeout);
+    }
+  }
   
   protected void setupDatanodeAddress(
       int i, Configuration dnConf, boolean setupHostsFile,

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java

@@ -1096,7 +1096,7 @@ public class TestBlockStoragePolicy {
                                   int replicaNum, StorageType... types) {
     List<StorageType> typeList = Lists.newArrayList();
     Collections.addAll(typeList, types);
-    LocatedBlocks lbs = status.getBlockLocations();
+    LocatedBlocks lbs = status.getLocatedBlocks();
     Assert.assertEquals(blockNum, lbs.getLocatedBlocks().size());
     for (LocatedBlock lb : lbs.getLocatedBlocks()) {
       Assert.assertEquals(replicaNum, lb.getStorageTypes().length);

+ 38 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java

@@ -1705,7 +1705,8 @@ public class TestEncryptionZones {
     Credentials credentials = new Credentials();
     // Key provider uri should be in the secret map of credentials object with
     // namenode uri as key
-    Text lookUpKey = client.getKeyProviderMapKey();
+    Text lookUpKey = HdfsKMSUtil.getKeyProviderMapKey(
+        cluster.getFileSystem().getUri());
     credentials.addSecretKey(lookUpKey,
         DFSUtilClient.string2Bytes(dummyKeyProvider));
     client.ugi.addCredentials(credentials);
@@ -1856,7 +1857,8 @@ public class TestEncryptionZones {
         CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
     DFSClient client = cluster.getFileSystem().getClient();
     Credentials credentials = new Credentials();
-    Text lookUpKey = client.getKeyProviderMapKey();
+    Text lookUpKey = HdfsKMSUtil.
+        getKeyProviderMapKey(cluster.getFileSystem().getUri());
     credentials.addSecretKey(lookUpKey,
         DFSUtilClient.string2Bytes(getKeyProviderURI()));
     client.ugi.addCredentials(credentials);
@@ -1920,4 +1922,38 @@ public class TestEncryptionZones {
         dfsAdmin.listEncryptionZones().hasNext());
   }
 
+  /**
+  * This test returns mocked kms token when
+  * {@link WebHdfsFileSystem#addDelegationTokens(String, Credentials)} method
+  * is called.
+  * @throws Exception
+  */
+  @Test
+  public void addMockKmsToken() throws Exception {
+    UserGroupInformation.createRemoteUser("JobTracker");
+    WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+        WebHdfsConstants.WEBHDFS_SCHEME);
+    KeyProvider keyProvider = Mockito.mock(KeyProvider.class, withSettings()
+        .extraInterfaces(DelegationTokenExtension.class,
+         CryptoExtension.class));
+    Mockito.when(keyProvider.getConf()).thenReturn(conf);
+    byte[] testIdentifier = "Test identifier for delegation token".getBytes();
+
+    Token<?> testToken = new Token(testIdentifier, new byte[0],
+        new Text("kms-dt"), new Text());
+    Mockito.when(((DelegationTokenExtension) keyProvider)
+        .addDelegationTokens(anyString(), (Credentials) any()))
+        .thenReturn(new Token<?>[] {testToken});
+
+    WebHdfsFileSystem webfsSpy = Mockito.spy(webfs);
+    Mockito.doReturn(keyProvider).when(webfsSpy).getKeyProvider();
+
+    Credentials creds = new Credentials();
+    final Token<?>[] tokens =
+        webfsSpy.addDelegationTokens("JobTracker", creds);
+
+    Assert.assertEquals(2, tokens.length);
+    Assert.assertEquals(tokens[1], testToken);
+    Assert.assertEquals(1, creds.numberOfTokens());
+  }
 }

+ 23 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithKMS.java

@@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue;
 
 import com.google.common.base.Supplier;
 import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
+import org.apache.hadoop.crypto.key.kms.KMSDelegationToken;
 import org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider;
 import org.apache.hadoop.crypto.key.kms.server.MiniKMS;
 import org.apache.hadoop.security.Credentials;
@@ -28,6 +29,9 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Assert;
@@ -134,4 +138,23 @@ public class TestEncryptionZonesWithKMS extends TestEncryptionZones {
       }
     }, 1000, 60000);
   }
+
+  /**
+   * This method fetches the kms delegation token
+   * for {@link WebHdfsFileSystem}.
+   * @throws Exception
+   */
+  @Test
+  public void addDelegationTokenFromWebhdfsFileSystem() throws Exception {
+    UserGroupInformation.createRemoteUser("JobTracker");
+    WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystem(
+        conf, WebHdfsConstants.WEBHDFS_SCHEME);
+    Credentials creds = new Credentials();
+    final Token<?>[] tokens = webfs.addDelegationTokens("JobTracker", creds);
+
+    Assert.assertEquals(2, tokens.length);
+    Assert.assertEquals(KMSDelegationToken.TOKEN_KIND_STR,
+        tokens[1].getKind().toString());
+    Assert.assertEquals(2, creds.numberOfTokens());
+  }
 }

+ 8 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -33,8 +34,12 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.slf4j.event.Level;
 
 import java.io.IOException;
+import java.util.Random;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY;
 
 /**
  * This test serves a prototype to demo the idea proposed so far. It creates two
@@ -77,6 +82,7 @@ public class TestFileChecksum {
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
         false);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
+    conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     Path ecPath = new Path(ecDir);
     cluster.getFileSystem().mkdir(ecPath, FsPermission.getDirDefault());
@@ -89,6 +95,7 @@ public class TestFileChecksum {
     bytesPerCRC = conf.getInt(
         HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY,
         HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT);
+    GenericTestUtils.setLogLevel(FileChecksumHelper.LOG, Level.DEBUG);
   }
 
   @After
@@ -518,7 +525,7 @@ public class TestFileChecksum {
 
     LocatedBlock locatedBlock = locatedBlocks.get(0);
     DatanodeInfo[] datanodes = locatedBlock.getLocations();
-    DatanodeInfo chosenDn = datanodes[0];
+    DatanodeInfo chosenDn = datanodes[new Random().nextInt(datanodes.length)];
 
     int idx = 0;
     for (DataNode dn : cluster.getDataNodes()) {

+ 103 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -23,6 +23,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY;
@@ -38,6 +39,7 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assume.assumeTrue;
+import static org.mockito.Mockito.doReturn;
 
 import java.io.BufferedReader;
 import java.io.ByteArrayOutputStream;
@@ -51,6 +53,7 @@ import java.net.URI;
 import java.net.UnknownHostException;
 import java.security.PrivilegedExceptionAction;
 import java.util.EnumSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -179,6 +182,106 @@ public class TestFileCreation {
     }
   }
 
+  /**
+   * Test that server default values are cached on the client size
+   * and are stale after namenode update.
+   */
+  @Test
+  public void testServerDefaultsWithCaching()
+      throws IOException, InterruptedException {
+    // Create cluster with an explicit block size param
+    Configuration clusterConf = new HdfsConfiguration();
+    long originalBlockSize = DFS_BLOCK_SIZE_DEFAULT * 2;
+    clusterConf.setLong(DFS_BLOCK_SIZE_KEY, originalBlockSize);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(clusterConf)
+        .numDataNodes(0)
+        .build();
+    cluster.waitActive();
+    // Set a spy namesystem inside the namenode and return it
+    FSNamesystem spyNamesystem =
+        NameNodeAdapter.spyOnNamesystem(cluster.getNameNode());
+    InetSocketAddress nameNodeAddr = cluster.getNameNode().getNameNodeAddress();
+    try {
+      // Create a dfs client and set a long enough validity interval
+      Configuration clientConf = new HdfsConfiguration();
+      clientConf.setLong(DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_KEY,
+          TimeUnit.MINUTES.toMillis(1));
+      DFSClient dfsClient = new DFSClient(nameNodeAddr, clientConf);
+      FsServerDefaults defaults = dfsClient.getServerDefaults();
+      assertEquals(originalBlockSize, defaults.getBlockSize());
+
+      // Update the namenode with a new parameter
+      long updatedDefaultBlockSize = DFS_BLOCK_SIZE_DEFAULT * 3;
+      FsServerDefaults newDefaults =
+          new FsServerDefaults(updatedDefaultBlockSize,
+              defaults.getBytesPerChecksum(), defaults.getWritePacketSize(),
+              defaults.getReplication(), defaults.getFileBufferSize(),
+              defaults.getEncryptDataTransfer(), defaults.getTrashInterval(),
+              defaults.getChecksumType(), defaults.getKeyProviderUri(),
+              defaults.getDefaultStoragePolicyId());
+      doReturn(newDefaults).when(spyNamesystem).getServerDefaults();
+
+      // The value is stale
+      Thread.sleep(1);
+      defaults = dfsClient.getServerDefaults();
+      assertEquals(originalBlockSize, defaults.getBlockSize());
+
+      // Another client reads the updated value correctly
+      DFSClient newDfsClient = new DFSClient(nameNodeAddr, clientConf);
+      defaults = newDfsClient.getServerDefaults();
+      assertEquals(updatedDefaultBlockSize, defaults.getBlockSize());
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test that server defaults are updated on the client after cache expiration.
+   */
+  @Test
+  public void testServerDefaultsWithMinimalCaching()
+      throws IOException, InterruptedException {
+    // Create cluster with an explicit block size param
+    Configuration clusterConf = new HdfsConfiguration();
+    long originalBlockSize = DFS_BLOCK_SIZE_DEFAULT * 2;
+    clusterConf.setLong(DFS_BLOCK_SIZE_KEY, originalBlockSize);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(clusterConf)
+        .numDataNodes(0)
+        .build();
+    cluster.waitActive();
+    // Set a spy namesystem inside the namenode and return it
+    FSNamesystem spyNamesystem =
+        NameNodeAdapter.spyOnNamesystem(cluster.getNameNode());
+    InetSocketAddress nameNodeAddr = cluster.getNameNode().getNameNodeAddress();
+    try {
+      // Create a dfs client and set a minimal validity interval
+      Configuration clientConf = new HdfsConfiguration();
+      // Invalidate cache in at most 1 ms, see DfsClient#getServerDefaults
+      clientConf.setLong(DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_KEY, 0L);
+      DFSClient dfsClient = new DFSClient(nameNodeAddr, clientConf);
+      FsServerDefaults defaults = dfsClient.getServerDefaults();
+      assertEquals(originalBlockSize, defaults.getBlockSize());
+
+      // Update the namenode with a new FsServerDefaults
+      long updatedDefaultBlockSize = DFS_BLOCK_SIZE_DEFAULT * 3;
+      FsServerDefaults newDefaults =
+          new FsServerDefaults(updatedDefaultBlockSize,
+              defaults.getBytesPerChecksum(), defaults.getWritePacketSize(),
+              defaults.getReplication(), defaults.getFileBufferSize(),
+              defaults.getEncryptDataTransfer(), defaults.getTrashInterval(),
+              defaults.getChecksumType(), defaults.getKeyProviderUri(),
+              defaults.getDefaultStoragePolicyId());
+      doReturn(newDefaults).when(spyNamesystem).getServerDefaults();
+
+      Thread.sleep(1);
+      defaults = dfsClient.getServerDefaults();
+      // Value is updated correctly
+      assertEquals(updatedDefaultBlockSize, defaults.getBlockSize());
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   @Test
   public void testFileCreation() throws IOException {
     checkFileCreation(null, false);

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

@@ -110,7 +110,7 @@ public class TestFileStatusSerialization {
       dib.reset(dob.getData(), 0, dob.getLength());
       FileStatus fstat = new FileStatus();
       fstat.readFields(dib);
-      checkFields(stat, fstat);
+      checkFields((FileStatus) stat, fstat);
 
       // FsPermisisonExtension used for HdfsFileStatus, not FileStatus,
       // attribute flags should still be preserved
@@ -133,7 +133,7 @@ public class TestFileStatusSerialization {
     try (ObjectInputStream ois = new ObjectInputStream(bais)) {
       FileStatus deser = (FileStatus) ois.readObject();
       assertEquals(hs, deser);
-      checkFields(hs, deser);
+      checkFields((FileStatus) hs, deser);
     }
   }
 
@@ -168,8 +168,8 @@ public class TestFileStatusSerialization {
       byte[] dst = fsp.toByteArray();
       HdfsFileStatusProto hsp2 = HdfsFileStatusProto.parseFrom(dst);
       assertEquals(hsp, hsp2);
-      FileStatus hstat  = PBHelperClient.convert(hsp);
-      FileStatus hstat2 = PBHelperClient.convert(hsp2);
+      FileStatus hstat  = (FileStatus) PBHelperClient.convert(hsp);
+      FileStatus hstat2 = (FileStatus) PBHelperClient.convert(hsp2);
       checkFields(hstat, hstat2);
     }
   }

+ 87 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java

@@ -0,0 +1,87 @@
+/**
+ * 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.blockmanagement;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * This class contains unit tests for PendingRecoveryBlocks.java functionality.
+ */
+public class TestPendingRecoveryBlocks {
+
+  private PendingRecoveryBlocks pendingRecoveryBlocks;
+  private final long recoveryTimeout = 1000L;
+
+  private final BlockInfo blk1 = getBlock(1);
+  private final BlockInfo blk2 = getBlock(2);
+  private final BlockInfo blk3 = getBlock(3);
+
+  @Before
+  public void setUp() {
+    pendingRecoveryBlocks =
+        Mockito.spy(new PendingRecoveryBlocks(recoveryTimeout));
+  }
+
+  BlockInfo getBlock(long blockId) {
+    return new BlockInfoContiguous(new Block(blockId), (short) 0);
+  }
+
+  @Test
+  public void testAddDifferentBlocks() {
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+    assertTrue(pendingRecoveryBlocks.isUnderRecovery(blk1));
+    assertTrue(pendingRecoveryBlocks.add(blk2));
+    assertTrue(pendingRecoveryBlocks.isUnderRecovery(blk2));
+    assertTrue(pendingRecoveryBlocks.add(blk3));
+    assertTrue(pendingRecoveryBlocks.isUnderRecovery(blk3));
+  }
+
+  @Test
+  public void testAddAndRemoveBlocks() {
+    // Add blocks
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+    assertTrue(pendingRecoveryBlocks.add(blk2));
+
+    // Remove blk1
+    pendingRecoveryBlocks.remove(blk1);
+
+    // Adding back blk1 should succeed
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+  }
+
+  @Test
+  public void testAddBlockWithPreviousRecoveryTimedOut() {
+    // Add blk
+    Mockito.doReturn(0L).when(pendingRecoveryBlocks).getTime();
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+
+    // Should fail, has not timed out yet
+    Mockito.doReturn(recoveryTimeout / 2).when(pendingRecoveryBlocks).getTime();
+    assertFalse(pendingRecoveryBlocks.add(blk1));
+
+    // Should succeed after timing out
+    Mockito.doReturn(recoveryTimeout * 2).when(pendingRecoveryBlocks).getTime();
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+  }
+}

+ 107 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java

@@ -18,7 +18,10 @@
 
 package org.apache.hadoop.hdfs.server.datanode;
 
+import org.apache.hadoop.hdfs.AppendTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
+
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
@@ -43,6 +46,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Random;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
@@ -94,6 +98,7 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.GenericTestUtils.SleepAnswer;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
@@ -1035,4 +1040,106 @@ public class TestBlockRecovery {
       Assert.fail("Thread failure: " + failureReason);
     }
   }
+
+  /**
+   * Test for block recovery taking longer than the heartbeat interval.
+   */
+  @Test(timeout = 300000L)
+  public void testRecoverySlowerThanHeartbeat() throws Exception {
+    tearDown(); // Stop the Mocked DN started in startup()
+
+    SleepAnswer delayer = new SleepAnswer(3000, 6000);
+    testRecoveryWithDatanodeDelayed(delayer);
+  }
+
+  /**
+   * Test for block recovery timeout. All recovery attempts will be delayed
+   * and the first attempt will be lost to trigger recovery timeout and retry.
+   */
+  @Test(timeout = 300000L)
+  public void testRecoveryTimeout() throws Exception {
+    tearDown(); // Stop the Mocked DN started in startup()
+    final Random r = new Random();
+
+    // Make sure first commitBlockSynchronization call from the DN gets lost
+    // for the recovery timeout to expire and new recovery attempt
+    // to be started.
+    SleepAnswer delayer = new SleepAnswer(3000) {
+      private final AtomicBoolean callRealMethod = new AtomicBoolean();
+
+      @Override
+      public Object answer(InvocationOnMock invocation) throws Throwable {
+        boolean interrupted = false;
+        try {
+          Thread.sleep(r.nextInt(3000) + 6000);
+        } catch (InterruptedException ie) {
+          interrupted = true;
+        }
+        try {
+          if (callRealMethod.get()) {
+            return invocation.callRealMethod();
+          }
+          callRealMethod.set(true);
+          return null;
+        } finally {
+          if (interrupted) {
+            Thread.currentThread().interrupt();
+          }
+        }
+      }
+    };
+    testRecoveryWithDatanodeDelayed(delayer);
+  }
+
+  private void testRecoveryWithDatanodeDelayed(
+      GenericTestUtils.SleepAnswer recoveryDelayer) throws Exception {
+    Configuration configuration = new HdfsConfiguration();
+    configuration.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    MiniDFSCluster cluster = null;
+
+    try {
+      cluster = new MiniDFSCluster.Builder(configuration)
+          .numDataNodes(2).build();
+      cluster.waitActive();
+      final FSNamesystem ns = cluster.getNamesystem();
+      final NameNode nn = cluster.getNameNode();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      cluster.setBlockRecoveryTimeout(TimeUnit.SECONDS.toMillis(15));
+
+      // Create a file and never close the output stream to trigger recovery
+      FSDataOutputStream out = dfs.create(new Path("/testSlowRecovery"),
+          (short) 2);
+      out.write(AppendTestUtil.randomBytes(0, 4096));
+      out.hsync();
+
+      List<DataNode> dataNodes = cluster.getDataNodes();
+      for (DataNode datanode : dataNodes) {
+        DatanodeProtocolClientSideTranslatorPB nnSpy =
+            InternalDataNodeTestUtils.spyOnBposToNN(datanode, nn);
+
+        Mockito.doAnswer(recoveryDelayer).when(nnSpy).
+            commitBlockSynchronization(
+                Mockito.any(ExtendedBlock.class), Mockito.anyInt(),
+                Mockito.anyLong(), Mockito.anyBoolean(),
+                Mockito.anyBoolean(), Mockito.any(DatanodeID[].class),
+                Mockito.any(String[].class));
+      }
+
+      // Make sure hard lease expires to trigger replica recovery
+      cluster.setLeasePeriod(100L, 100L);
+
+      // Wait for recovery to succeed
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          return ns.getCompleteBlocksTotal() > 0;
+        }
+      }, 300, 300000);
+
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java

@@ -320,7 +320,7 @@ public class TestStorageMover {
       }
       final List<StorageType> types = policy.chooseStorageTypes(
           status.getReplication());
-      for(LocatedBlock lb : fileStatus.getBlockLocations().getLocatedBlocks()) {
+      for(LocatedBlock lb : fileStatus.getLocatedBlocks().getLocatedBlocks()) {
         final Mover.StorageTypeDiff diff = new Mover.StorageTypeDiff(types,
             lb.getStorageTypes());
         Assert.assertTrue(fileStatus.getFullName(parent.toString())

+ 41 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java

@@ -60,6 +60,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
+import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
@@ -1155,6 +1156,46 @@ public class TestFileTruncate {
     fs.delete(parent, true);
   }
 
+  /**
+   * While rolling upgrade is in-progress the test truncates a file
+   * such that copy-on-truncate is triggered, then deletes the file,
+   * and makes sure that no blocks involved in truncate are hanging around.
+   */
+  @Test
+  public void testTruncateWithRollingUpgrade() throws Exception {
+    final DFSAdmin dfsadmin = new DFSAdmin(cluster.getConfiguration(0));
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    //start rolling upgrade
+    dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    int status = dfsadmin.run(new String[]{"-rollingUpgrade", "prepare"});
+    assertEquals("could not prepare for rolling upgrade", 0, status);
+    dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
+    Path dir = new Path("/testTruncateWithRollingUpgrade");
+    fs.mkdirs(dir);
+    final Path p = new Path(dir, "file");
+    final byte[] data = new byte[3];
+    ThreadLocalRandom.current().nextBytes(data);
+    writeContents(data, data.length, p);
+
+    assertEquals("block num should 1", 1,
+        cluster.getNamesystem().getFSDirectory().getBlockManager()
+            .getTotalBlocks());
+
+    final boolean isReady = fs.truncate(p, 2);
+    assertFalse("should be copy-on-truncate", isReady);
+    assertEquals("block num should 2", 2,
+        cluster.getNamesystem().getFSDirectory().getBlockManager()
+            .getTotalBlocks());
+    fs.delete(p, true);
+
+    assertEquals("block num should 0", 0,
+        cluster.getNamesystem().getFSDirectory().getBlockManager()
+            .getTotalBlocks());
+    status = dfsadmin.run(new String[]{"-rollingUpgrade", "finalize"});
+    assertEquals("could not finalize rolling upgrade", 0, status);
+  }
+
   static void writeContents(byte[] contents, int fileLength, Path p)
       throws IOException {
     FSDataOutputStream out = fs.create(p, true, BLOCK_SIZE, REPLICATION,

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

@@ -25,6 +25,7 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
@@ -278,12 +279,13 @@ public class TestPipelinesFailover {
     // Disable permissions so that another user can recover the lease.
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false);
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    
+
     FSDataOutputStream stm = null;
     final MiniDFSCluster cluster = newMiniCluster(conf, 3);
     try {
       cluster.waitActive();
       cluster.transitionToActive(0);
+      cluster.setBlockRecoveryTimeout(TimeUnit.SECONDS.toMillis(1));
       Thread.sleep(500);
 
       LOG.info("Starting with NN 0 active");

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java

@@ -309,6 +309,25 @@ public class TestStandbyInProgressTail {
     assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test3", true));
   }
 
+  @Test
+  public void testNonUniformConfig() throws Exception {
+    // Test case where some NNs (in this case the active NN) in the cluster
+    // do not have in-progress tailing enabled.
+    Configuration newConf = cluster.getNameNode(0).getConf();
+    newConf.setBoolean(
+        DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY,
+        false);
+    cluster.restartNameNode(0);
+    cluster.transitionToActive(0);
+
+    cluster.getNameNode(0).getRpcServer().mkdirs("/test",
+        FsPermission.createImmutable((short) 0755), true);
+    cluster.getNameNode(0).getRpcServer().rollEdits();
+
+    cluster.getNameNode(1).getNamesystem().getEditLogTailer().doTailEdits();
+    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
+  }
+
   /**
    * Check that no edits files are present in the given storage dirs.
    */

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

@@ -90,6 +90,7 @@ public class TestSnapshotDiffReport {
     conf.setBoolean(
         DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT,
         true);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT, 3);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
         .format(true).build();
     cluster.waitActive();
@@ -1293,4 +1294,119 @@ public class TestSnapshotDiffReport {
 
     assertAtimeNotEquals(filePostSS, root, "s2", "s3");
   }
+
+  /**
+   * Tests to verfy the diff report with maximum SnapsdiffReportEntries limit
+   * over an rpc being set to 3.
+   * @throws Exception
+   */
+  @Test
+  public void testDiffReportWithRpcLimit() throws Exception {
+    final Path root = new Path("/");
+    hdfs.mkdirs(root);
+    for (int i = 1; i < 4; i++) {
+      final Path path = new Path(root, "dir" + i);
+      hdfs.mkdirs(path);
+    }
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
+    for (int i = 1; i < 4; 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, "s1");
+    verifyDiffReport(root, "s0", "s1",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir1/file1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir1/file2")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir1/file3")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir2")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir2/file1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir2/file2")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir2/file3")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir3")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir3/file1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir3/file2")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir3/file3")));
+  }
+
+  @Test
+  public void testDiffReportWithRpcLimit2() 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");
+    verifyDiffReport(root, "s0", "s1",
+        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")));
+  }
 }

+ 36 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/scripts/hadoop_shell_execname.bats

@@ -0,0 +1,36 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+load hdfs-functions_test_helper
+
+# Setup minimal environment to invoke the 'hdfs' command.
+hdfscommandsetup () {
+  export HADOOP_LIBEXEC_DIR="${TMP}/libexec"
+  export HADOOP_CONF_DIR="${TMP}/conf"
+  mkdir -p "${HADOOP_LIBEXEC_DIR}"
+  echo   ". \"${BATS_TEST_DIRNAME}/../../../../../hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh\"" > "${HADOOP_LIBEXEC_DIR}/hdfs-config.sh"
+  chmod a+rx "${HADOOP_LIBEXEC_DIR}/hdfs-config.sh"
+}
+
+# Verify that the 'hdfs' command correctly infers MYNAME and
+# HADOOP_SHELL_EXECNAME
+@test "hadoop_shell_execname" {
+  hdfscommandsetup
+  export QATESTMODE=unittest
+  run "${BATS_TEST_DIRNAME}/../../main/bin/hdfs" envvars
+  echo ">${output}<"
+  [[ ${output} =~ MYNAME=.*/hdfs ]]
+  [[ ${output} =~ HADOOP_SHELL_EXECNAME=hdfs ]]
+}

+ 7 - 1
hadoop-mapreduce-project/bin/mapred

@@ -15,8 +15,10 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+# The name of the script being executed.
+HADOOP_SHELL_EXECNAME="mapred"
 MYNAME="${BASH_SOURCE-$0}"
-HADOOP_SHELL_EXECNAME="${MYNAME##*/}"
+
 
 ## @description  build up the mapred command's usage text.
 ## @audience     public
@@ -32,6 +34,7 @@ function hadoop_usage
   hadoop_add_subcommand "pipes" client "run a Pipes job"
   hadoop_add_subcommand "queue" client "get information regarding JobQueues"
   hadoop_add_subcommand "sampler" client "sampler"
+  hadoop_add_subcommand "frameworkuploader" admin "mapreduce framework upload"
   hadoop_add_subcommand "version" client "print the version"
   hadoop_generate_usage "${HADOOP_SHELL_EXECNAME}" true
 }
@@ -92,6 +95,9 @@ function mapredcmd_case
     sampler)
       HADOOP_CLASSNAME=org.apache.hadoop.mapred.lib.InputSampler
     ;;
+    frameworkuploader)
+      HADOOP_CLASSNAME=org.apache.hadoop.mapred.uploader.FrameworkUploader
+    ;;
     version)
       HADOOP_CLASSNAME=org.apache.hadoop.util.VersionInfo
     ;;

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

@@ -22,9 +22,11 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -36,6 +38,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
 import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
@@ -58,6 +61,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * This class is responsible for talking to the task umblical.
  * It also converts all the old data structures
@@ -66,7 +71,6 @@ import org.slf4j.LoggerFactory;
  * This class HAS to be in this package to access package private 
  * methods/classes.
  */
-@SuppressWarnings({"unchecked"})
 public class TaskAttemptListenerImpl extends CompositeService 
     implements TaskUmbilicalProtocol, TaskAttemptListener {
 
@@ -84,6 +88,11 @@ public class TaskAttemptListenerImpl extends CompositeService
   private ConcurrentMap<WrappedJvmID, org.apache.hadoop.mapred.Task>
     jvmIDToActiveAttemptMap
       = new ConcurrentHashMap<WrappedJvmID, org.apache.hadoop.mapred.Task>();
+
+  private ConcurrentMap<TaskAttemptId,
+      AtomicReference<TaskAttemptStatus>> attemptIdToStatus
+        = new ConcurrentHashMap<>();
+
   private Set<WrappedJvmID> launchedJVMs = Collections
       .newSetFromMap(new ConcurrentHashMap<WrappedJvmID, Boolean>());
 
@@ -359,6 +368,13 @@ public class TaskAttemptListenerImpl extends CompositeService
     org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId yarnAttemptID =
         TypeConverter.toYarn(taskAttemptID);
 
+    AtomicReference<TaskAttemptStatus> lastStatusRef =
+        attemptIdToStatus.get(yarnAttemptID);
+    if (lastStatusRef == null) {
+      throw new IllegalStateException("Status update was called"
+          + " with illegal TaskAttemptId: " + yarnAttemptID);
+    }
+
     AMFeedback feedback = new AMFeedback();
     feedback.setTaskFound(true);
 
@@ -437,9 +453,8 @@ public class TaskAttemptListenerImpl extends CompositeService
 //    // isn't ever changed by the Task itself.
 //    taskStatus.getIncludeCounters();
 
-    context.getEventHandler().handle(
-        new TaskAttemptStatusUpdateEvent(taskAttemptStatus.id,
-            taskAttemptStatus));
+    coalesceStatusUpdate(yarnAttemptID, taskAttemptStatus, lastStatusRef);
+
     return feedback;
   }
 
@@ -520,6 +535,8 @@ public class TaskAttemptListenerImpl extends CompositeService
     launchedJVMs.add(jvmId);
 
     taskHeartbeatHandler.register(attemptID);
+
+    attemptIdToStatus.put(attemptID, new AtomicReference<>());
   }
 
   @Override
@@ -541,6 +558,8 @@ public class TaskAttemptListenerImpl extends CompositeService
 
     //unregister this attempt
     taskHeartbeatHandler.unregister(attemptID);
+
+    attemptIdToStatus.remove(attemptID);
   }
 
   @Override
@@ -563,4 +582,46 @@ public class TaskAttemptListenerImpl extends CompositeService
     preemptionPolicy.setCheckpointID(tid, cid);
   }
 
+  private void coalesceStatusUpdate(TaskAttemptId yarnAttemptID,
+      TaskAttemptStatus taskAttemptStatus,
+      AtomicReference<TaskAttemptStatus> lastStatusRef) {
+    boolean asyncUpdatedNeeded = false;
+    TaskAttemptStatus lastStatus = lastStatusRef.get();
+
+    if (lastStatus == null) {
+      lastStatusRef.set(taskAttemptStatus);
+      asyncUpdatedNeeded = true;
+    } else {
+      List<TaskAttemptId> oldFetchFailedMaps =
+          taskAttemptStatus.fetchFailedMaps;
+
+      // merge fetchFailedMaps from the previous update
+      if (lastStatus.fetchFailedMaps != null) {
+        if (taskAttemptStatus.fetchFailedMaps == null) {
+          taskAttemptStatus.fetchFailedMaps = lastStatus.fetchFailedMaps;
+        } else {
+          taskAttemptStatus.fetchFailedMaps.addAll(lastStatus.fetchFailedMaps);
+        }
+      }
+
+      if (!lastStatusRef.compareAndSet(lastStatus, taskAttemptStatus)) {
+        // update failed - async dispatcher has processed it in the meantime
+        taskAttemptStatus.fetchFailedMaps = oldFetchFailedMaps;
+        lastStatusRef.set(taskAttemptStatus);
+        asyncUpdatedNeeded = true;
+      }
+    }
+
+    if (asyncUpdatedNeeded) {
+      context.getEventHandler().handle(
+          new TaskAttemptStatusUpdateEvent(taskAttemptStatus.id,
+              lastStatusRef));
+    }
+  }
+
+  @VisibleForTesting
+  ConcurrentMap<TaskAttemptId,
+      AtomicReference<TaskAttemptStatus>> getAttemptIdToStatus() {
+    return attemptIdToStatus;
+  }
 }

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