Parcourir la source

Merge branch 'trunk' into HDFS-7240

Anu Engineer il y a 8 ans
Parent
commit
1dc5c7ebfc
100 fichiers modifiés avec 2426 ajouts et 713 suppressions
  1. 5 0
      .gitignore
  2. 98 0
      hadoop-common-project/hadoop-common/dev-support/jdiff-workaround.patch
  3. 11 0
      hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.7.2.xml
  4. 2 0
      hadoop-common-project/hadoop-common/pom.xml
  5. 2 2
      hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
  6. 4 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
  7. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java
  8. 113 14
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java
  9. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Options.java
  10. 0 26
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java
  11. 6 15
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
  12. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java
  13. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java
  14. 17 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
  15. 19 12
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java
  16. 5 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
  17. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallerContext.java
  18. 3 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSink.java
  19. 44 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java
  20. 10 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
  21. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NodeHealthScriptRunner.java
  22. 12 7
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  23. 1 1
      hadoop-common-project/hadoop-common/src/site/markdown/CredentialProviderAPI.md
  24. 8 3
      hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
  25. 4 4
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
  26. 5 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
  27. 3 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextUtilBase.java
  28. 32 14
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
  29. 75 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java
  30. 0 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
  31. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
  32. 92 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java
  33. 7 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNodeHealthScriptRunner.java
  34. 10 2
      hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
  35. 10 0
      hadoop-common-project/hadoop-kms/src/main/conf/kms-site.xml
  36. 107 93
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java
  37. 154 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuditLogger.java
  38. 4 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
  39. 1 4
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
  40. 96 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/SimpleKMSAuditLogger.java
  41. 39 4
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java
  42. 4 1
      hadoop-common-project/hadoop-kms/src/test/resources/log4j-kmsaudit.properties
  43. 21 214
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  44. 12 11
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
  45. 106 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ReadStatistics.java
  46. 215 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ReaderStrategy.java
  47. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
  48. 19 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeAdminProperties.java
  49. 23 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  50. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
  51. 6 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  52. 8 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  53. 16 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/IOUtilsClient.java
  54. 21 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
  55. 55 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  56. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
  57. 38 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StartAfterParam.java
  58. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
  59. 4 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
  60. 11 0
      hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.7.2.xml
  61. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  62. 13 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  63. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
  64. 23 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CombinedHostFileManager.java
  65. 26 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  66. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStats.java
  67. 80 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
  68. 27 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
  69. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostConfigManager.java
  70. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java
  71. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  72. 9 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  73. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  74. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeFaultInjector.java
  75. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  76. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
  77. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  78. 43 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
  79. 22 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
  80. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryComputationContext.java
  81. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
  82. 39 52
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java
  83. 23 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
  84. 39 43
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
  85. 7 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSymlinkOp.java
  86. 17 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
  87. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  88. 79 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  89. 19 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/HdfsAuditLogger.java
  90. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  91. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  92. 11 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  93. 42 33
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
  94. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java
  95. 21 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  96. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java
  97. 39 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  98. 58 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  99. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsSnapshots.md
  100. 105 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md

+ 5 - 0
.gitignore

@@ -15,6 +15,11 @@
 .settings
 target
 build
+
+# External tool builders
+*/.externalToolBuilders
+*/maven-eclipse.xml
+
 hadoop-common-project/hadoop-kms/downloads/
 hadoop-hdfs-project/hadoop-hdfs/downloads
 hadoop-hdfs-project/hadoop-hdfs-httpfs/downloads

+ 98 - 0
hadoop-common-project/hadoop-common/dev-support/jdiff-workaround.patch

@@ -0,0 +1,98 @@
+diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystem.java
+index a277abd..ed7c709 100644
+--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystem.java
++++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystem.java
+@@ -43,18 +43,6 @@
+   public abstract MetricsSystem init(String prefix);
+
+   /**
+-   * Register a metrics source
+-   * @param <T>   the actual type of the source object
+-   * @param source object to register
+-   * @param name  of the source. Must be unique or null (then extracted from
+-   *              the annotations of the source object.)
+-   * @param desc  the description of the source (or null. See above.)
+-   * @return the source object
+-   * @exception MetricsException
+-   */
+-  public abstract <T> T register(String name, String desc, T source);
+-
+-  /**
+    * Unregister a metrics source
+    * @param name of the source. This is the name you use to call register()
+    */
+@@ -77,18 +65,19 @@
+    */
+   @InterfaceAudience.Private
+   public abstract MetricsSource getSource(String name);
++
+
+   /**
+-   * Register a metrics sink
+-   * @param <T>   the type of the sink
+-   * @param sink  to register
+-   * @param name  of the sink. Must be unique.
+-   * @param desc  the description of the sink
+-   * @return the sink
++   * Register a metrics source
++   * @param <T>   the actual type of the source object
++   * @param source object to register
++   * @param name  of the source. Must be unique or null (then extracted from
++   *              the annotations of the source object.)
++   * @param desc  the description of the source (or null. See above.)
++   * @return the source object
+    * @exception MetricsException
+    */
+-  public abstract <T extends MetricsSink>
+-  T register(String name, String desc, T sink);
++  public abstract <T> T register(String name, String desc, T source);
+
+   /**
+    * Register a callback interface for JMX events
+diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
+index 6986edb..eeea81f 100644
+--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
++++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
+@@ -270,27 +270,6 @@ void registerSource(String name, String desc, MetricsSource source) {
+     LOG.debug("Registered source "+ name);
+   }
+
+-  @Override public synchronized <T extends MetricsSink>
+-  T register(final String name, final String description, final T sink) {
+-    LOG.debug(name +", "+ description);
+-    if (allSinks.containsKey(name)) {
+-      LOG.warn("Sink "+ name +" already exists!");
+-      return sink;
+-    }
+-    allSinks.put(name, sink);
+-    if (config != null) {
+-      registerSink(name, description, sink);
+-    }
+-    // We want to re-register the sink to pick up new config
+-    // when the metrics system restarts.
+-    register(name, new AbstractCallback() {
+-      @Override public void postStart() {
+-        register(name, description, sink);
+-      }
+-    });
+-    return sink;
+-  }
+-
+   synchronized void registerSink(String name, String desc, MetricsSink sink) {
+     checkNotNull(config, "config");
+     MetricsConfig conf = sinkConfigs.get(name);
+diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestGangliaMetrics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestGangliaMetrics.java
+index c19d238..f8412f1 100644
+--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestGangliaMetrics.java
++++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestGangliaMetrics.java
+@@ -130,8 +130,8 @@ public void testTagsForPrefix() throws Exception {
+     GangliaMetricsTestHelper.setDatagramSocket(gsink31, mockds31);
+
+     // register the sinks
+-    ms.register("gsink30", "gsink30 desc", gsink30);
+-    ms.register("gsink31", "gsink31 desc", gsink31);
++    //ms.register("gsink30", "gsink30 desc", gsink30);
++    //ms.register("gsink31", "gsink31 desc", gsink31);
+     ms.publishMetricsNow(); // publish the metrics
+
+     ms.stop();

Fichier diff supprimé car celui-ci est trop grand
+ 11 - 0
hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.7.2.xml


+ 2 - 0
hadoop-common-project/hadoop-common/pom.xml

@@ -34,6 +34,7 @@
     <kdc.resource.dir>src/test/resources/kdc</kdc.resource.dir>
     <hadoop.component>common</hadoop.component>
     <is.hadoop.component>true</is.hadoop.component>
+    <is.hadoop.common.component>true</is.hadoop.common.component>
     <wsce.config.dir>../etc/hadoop</wsce.config.dir>
     <wsce.config.file>wsce-site.xml</wsce.config.file>
   </properties>
@@ -512,6 +513,7 @@
             <exclude>src/test/resources/test.har/_masterindex</exclude>
             <exclude>src/test/resources/test.har/part-0</exclude>
             <exclude>src/test/resources/javakeystoreprovider.password</exclude>
+            <exclude>dev-support/jdiff-workaround.patch</exclude>
           </excludes>
         </configuration>
       </plugin>

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh

@@ -596,7 +596,7 @@ function hadoop_basic_init
   HADOOP_ROOT_LOGGER=${HADOOP_ROOT_LOGGER:-${HADOOP_LOGLEVEL},console}
   HADOOP_DAEMON_ROOT_LOGGER=${HADOOP_DAEMON_ROOT_LOGGER:-${HADOOP_LOGLEVEL},RFA}
   HADOOP_SECURITY_LOGGER=${HADOOP_SECURITY_LOGGER:-INFO,NullAppender}
-  HADOOP_SSH_OPTS=${HADOOP_SSH_OPTS:-"-o BatchMode=yes -o StrictHostKeyChecking=no -o ConnectTimeout=10s"}
+  HADOOP_SSH_OPTS=${HADOOP_SSH_OPTS-"-o BatchMode=yes -o StrictHostKeyChecking=no -o ConnectTimeout=10s"}
   HADOOP_SECURE_LOG_DIR=${HADOOP_SECURE_LOG_DIR:-${HADOOP_LOG_DIR}}
   HADOOP_SECURE_PID_DIR=${HADOOP_SECURE_PID_DIR:-${HADOOP_PID_DIR}}
   HADOOP_SSH_PARALLEL=${HADOOP_SSH_PARALLEL:-10}
@@ -695,7 +695,7 @@ function hadoop_connect_to_hosts
   #
   # User can specify hostnames or a file where the hostnames are (not both)
   if [[ -n "${HADOOP_WORKERS}" && -n "${HADOOP_WORKER_NAMES}" ]] ; then
-    hadoop_error "ERROR: Both HADOOP_WORKERS and HADOOP_WORKER_NAME were defined. Aborting."
+    hadoop_error "ERROR: Both HADOOP_WORKERS and HADOOP_WORKER_NAMES were defined. Aborting."
     exit 1
   elif [[ -z "${HADOOP_WORKER_NAMES}" ]]; then
     if [[ -n "${HADOOP_WORKERS}" ]]; then

+ 4 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java

@@ -197,7 +197,6 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
                new ChecksumFSInputChecker(fs, file)) {
         checker.seek(position);
         nread = checker.read(b, off, len);
-        checker.close();
       }
       return nread;
     }
@@ -356,12 +355,14 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   @Override
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
-    throw new IOException("Not supported");
+    throw new UnsupportedOperationException("Append is not supported "
+        + "by ChecksumFileSystem");
   }
 
   @Override
   public boolean truncate(Path f, long newLength) throws IOException {
-    throw new IOException("Not supported");
+    throw new UnsupportedOperationException("Truncate is not supported "
+        + "by ChecksumFileSystem");
   }
 
   /**

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

@@ -179,7 +179,6 @@ public abstract class ChecksumFs extends FilterFs {
                new ChecksumFSInputChecker(fs, file)) {
         checker.seek(position);
         nread = checker.read(b, off, len);
-        checker.close();
       }
       return nread;
     }
@@ -297,7 +296,8 @@ public abstract class ChecksumFs extends FilterFs {
 
   @Override
   public boolean truncate(Path f, long newLength) throws IOException {
-    throw new IOException("Not supported");
+    throw new UnsupportedOperationException("Truncate is not supported "
+        + "by ChecksumFs");
   }
 
   /**

+ 113 - 14
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java

@@ -34,6 +34,11 @@ public class ContentSummary extends QuotaUsage implements Writable{
   private long length;
   private long fileCount;
   private long directoryCount;
+  // These fields are to track the snapshot-related portion of the values.
+  private long snapshotLength;
+  private long snapshotFileCount;
+  private long snapshotDirectoryCount;
+  private long snapshotSpaceConsumed;
 
   /** We don't use generics. Instead override spaceConsumed and other methods
       in order to keep backward compatibility. */
@@ -56,6 +61,26 @@ public class ContentSummary extends QuotaUsage implements Writable{
       return this;
     }
 
+    public Builder snapshotLength(long snapshotLength) {
+      this.snapshotLength = snapshotLength;
+      return this;
+    }
+
+    public Builder snapshotFileCount(long snapshotFileCount) {
+      this.snapshotFileCount = snapshotFileCount;
+      return this;
+    }
+
+    public Builder snapshotDirectoryCount(long snapshotDirectoryCount) {
+      this.snapshotDirectoryCount = snapshotDirectoryCount;
+      return this;
+    }
+
+    public Builder snapshotSpaceConsumed(long snapshotSpaceConsumed) {
+      this.snapshotSpaceConsumed = snapshotSpaceConsumed;
+      return this;
+    }
+
     @Override
     public Builder quota(long quota){
       super.quota(quota);
@@ -107,6 +132,10 @@ public class ContentSummary extends QuotaUsage implements Writable{
     private long length;
     private long fileCount;
     private long directoryCount;
+    private long snapshotLength;
+    private long snapshotFileCount;
+    private long snapshotDirectoryCount;
+    private long snapshotSpaceConsumed;
   }
 
   /** Constructor deprecated by ContentSummary.Builder*/
@@ -142,17 +171,37 @@ public class ContentSummary extends QuotaUsage implements Writable{
     this.length = builder.length;
     this.fileCount = builder.fileCount;
     this.directoryCount = builder.directoryCount;
+    this.snapshotLength = builder.snapshotLength;
+    this.snapshotFileCount = builder.snapshotFileCount;
+    this.snapshotDirectoryCount = builder.snapshotDirectoryCount;
+    this.snapshotSpaceConsumed = builder.snapshotSpaceConsumed;
   }
 
   /** @return the length */
   public long getLength() {return length;}
 
+  public long getSnapshotLength() {
+    return snapshotLength;
+  }
+
   /** @return the directory count */
   public long getDirectoryCount() {return directoryCount;}
 
+  public long getSnapshotDirectoryCount() {
+    return snapshotDirectoryCount;
+  }
+
   /** @return the file count */
   public long getFileCount() {return fileCount;}
 
+  public long getSnapshotFileCount() {
+    return snapshotFileCount;
+  }
+
+  public long getSnapshotSpaceConsumed() {
+    return snapshotSpaceConsumed;
+  }
+
   @Override
   @InterfaceAudience.Private
   public void write(DataOutput out) throws IOException {
@@ -180,9 +229,14 @@ public class ContentSummary extends QuotaUsage implements Writable{
     if (this == to) {
       return true;
     } else if (to instanceof ContentSummary) {
-      return getLength() == ((ContentSummary) to).getLength() &&
-          getFileCount() == ((ContentSummary) to).getFileCount() &&
-          getDirectoryCount() == ((ContentSummary) to).getDirectoryCount() &&
+      ContentSummary right = (ContentSummary) to;
+      return getLength() == right.getLength() &&
+          getFileCount() == right.getFileCount() &&
+          getDirectoryCount() == right.getDirectoryCount() &&
+          getSnapshotLength() == right.getSnapshotLength() &&
+          getSnapshotFileCount() == right.getSnapshotFileCount() &&
+          getSnapshotDirectoryCount() == right.getSnapshotDirectoryCount() &&
+          getSnapshotSpaceConsumed() == right.getSnapshotSpaceConsumed() &&
           super.equals(to);
     } else {
       return super.equals(to);
@@ -191,7 +245,9 @@ public class ContentSummary extends QuotaUsage implements Writable{
 
   @Override
   public int hashCode() {
-    long result = getLength() ^ getFileCount() ^ getDirectoryCount();
+    long result = getLength() ^ getFileCount() ^ getDirectoryCount()
+        ^ getSnapshotLength() ^ getSnapshotFileCount()
+        ^ getSnapshotDirectoryCount() ^ getSnapshotSpaceConsumed();
     return ((int) result) ^ super.hashCode();
   }
 
@@ -255,15 +311,14 @@ public class ContentSummary extends QuotaUsage implements Writable{
    * @param qOption a flag indicating if quota needs to be printed or not
    * @return the string representation of the object
   */
+  @Override
   public String toString(boolean qOption) {
     return toString(qOption, false);
   }
 
   /** Return the string representation of the object in the output format.
-   * if qOption is false, output directory count, file count, and content size;
-   * if qOption is true, output quota and remaining quota as well.
-   * if hOption is false file sizes are returned in bytes
-   * if hOption is true file sizes are returned in human readable 
+   * For description of the options,
+   * @see #toString(boolean, boolean, boolean, boolean, List)
    * 
    * @param qOption a flag indicating if quota needs to be printed or not
    * @param hOption a flag indicating if human readable output if to be used
@@ -273,10 +328,24 @@ public class ContentSummary extends QuotaUsage implements Writable{
     return toString(qOption, hOption, false, null);
   }
 
+  /** Return the string representation of the object in the output format.
+   * For description of the options,
+   * @see #toString(boolean, boolean, boolean, boolean, List)
+   *
+   * @param qOption a flag indicating if quota needs to be printed or not
+   * @param hOption a flag indicating if human readable output is to be used
+   * @param xOption a flag indicating if calculation from snapshots is to be
+   *                included in the output
+   * @return the string representation of the object
+   */
+  public String toString(boolean qOption, boolean hOption, boolean xOption) {
+    return toString(qOption, hOption, false, xOption, null);
+  }
+
   /**
    * Return the string representation of the object in the output format.
-   * if tOption is true, display the quota by storage types,
-   * Otherwise, same logic with #toString(boolean,boolean)
+   * For description of the options,
+   * @see #toString(boolean, boolean, boolean, boolean, List)
    *
    * @param qOption a flag indicating if quota needs to be printed or not
    * @param hOption a flag indicating if human readable output if to be used
@@ -286,6 +355,29 @@ public class ContentSummary extends QuotaUsage implements Writable{
    */
   public String toString(boolean qOption, boolean hOption,
                          boolean tOption, List<StorageType> types) {
+    return toString(qOption, hOption, tOption, false, types);
+  }
+
+  /** Return the string representation of the object in the output format.
+   * if qOption is false, output directory count, file count, and content size;
+   * if qOption is true, output quota and remaining quota as well.
+   * if hOption is false, file sizes are returned in bytes
+   * if hOption is true, file sizes are returned in human readable
+   * if tOption is true, display the quota by storage types
+   * if tOption is false, same logic with #toString(boolean,boolean)
+   * if xOption is false, output includes the calculation from snapshots
+   * if xOption is true, output excludes the calculation from snapshots
+   *
+   * @param qOption a flag indicating if quota needs to be printed or not
+   * @param hOption a flag indicating if human readable output is to be used
+   * @param tOption a flag indicating if display quota by storage types
+   * @param xOption a flag indicating if calculation from snapshots is to be
+   *                included in the output
+   * @param types Storage types to display
+   * @return the string representation of the object
+   */
+  public String toString(boolean qOption, boolean hOption, boolean tOption,
+      boolean xOption, List<StorageType> types) {
     String prefix = "";
 
     if (tOption) {
@@ -296,10 +388,17 @@ public class ContentSummary extends QuotaUsage implements Writable{
       prefix = getQuotaUsage(hOption);
     }
 
-    return prefix + String.format(SUMMARY_FORMAT,
-        formatSize(directoryCount, hOption),
-        formatSize(fileCount, hOption),
-        formatSize(length, hOption));
+    if (xOption) {
+      return prefix + String.format(SUMMARY_FORMAT,
+          formatSize(directoryCount - snapshotDirectoryCount, hOption),
+          formatSize(fileCount - snapshotFileCount, hOption),
+          formatSize(length - snapshotLength, hOption));
+    } else {
+      return prefix + String.format(SUMMARY_FORMAT,
+          formatSize(directoryCount, hOption),
+          formatSize(fileCount, hOption),
+          formatSize(length, hOption));
+    }
   }
 
   /**

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

@@ -213,7 +213,8 @@ public final class Options {
    */
   public static enum Rename {
     NONE((byte) 0), // No options
-    OVERWRITE((byte) 1); // Overwrite the rename destination
+    OVERWRITE((byte) 1), // Overwrite the rename destination
+    TO_TRASH ((byte) 2); // Rename to trash
 
     private final byte code;
     

+ 0 - 26
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java

@@ -36,15 +36,6 @@ public abstract class TrashPolicy extends Configured {
   protected Path trash; // path to trash directory
   protected long deletionInterval; // deletion interval for Emptier
 
-  /**
-   * Used to setup the trash policy. Must be implemented by all TrashPolicy
-   * implementations.
-   * @param conf the configuration to be used
-   * @param fs the filesystem to be used
-   * @param home the home directory
-   */
-  public abstract void initialize(Configuration conf, FileSystem fs, Path home);
-
   /**
    * Used to setup the trash policy. Must be implemented by all TrashPolicy
    * implementations. Different from initialize(conf, fs, home), this one does
@@ -108,23 +99,6 @@ public abstract class TrashPolicy extends Configured {
    */
   public abstract Runnable getEmptier() throws IOException;
 
-  /**
-   * Get an instance of the configured TrashPolicy based on the value
-   * of the configuration parameter fs.trash.classname.
-   *
-   * @param conf the configuration to be used
-   * @param fs the file system to be used
-   * @param home the home directory
-   * @return an instance of TrashPolicy
-   */
-  public static TrashPolicy getInstance(Configuration conf, FileSystem fs, Path home) {
-    Class<? extends TrashPolicy> trashClass = conf.getClass(
-        "fs.trash.classname", TrashPolicyDefault.class, TrashPolicy.class);
-    TrashPolicy trash = ReflectionUtils.newInstance(trashClass, conf);
-    trash.initialize(conf, fs, home); // initialize TrashPolicy
-    return trash;
-  }
-
   /**
    * Get an instance of the configured TrashPolicy based on the value
    * of the configuration parameter fs.trash.classname.

+ 6 - 15
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java

@@ -75,17 +75,6 @@ public class TrashPolicyDefault extends TrashPolicy {
     initialize(conf, fs);
   }
 
-  @Override
-  public void initialize(Configuration conf, FileSystem fs, Path home) {
-    this.fs = fs;
-    this.deletionInterval = (long)(conf.getFloat(
-        FS_TRASH_INTERVAL_KEY, FS_TRASH_INTERVAL_DEFAULT)
-        * MSECS_PER_MINUTE);
-    this.emptierInterval = (long)(conf.getFloat(
-        FS_TRASH_CHECKPOINT_INTERVAL_KEY, FS_TRASH_CHECKPOINT_INTERVAL_DEFAULT)
-        * MSECS_PER_MINUTE);
-   }
-
   @Override
   public void initialize(Configuration conf, FileSystem fs) {
     this.fs = fs;
@@ -106,6 +95,7 @@ public class TrashPolicyDefault extends TrashPolicy {
     return deletionInterval != 0;
   }
 
+  @SuppressWarnings("deprecation")
   @Override
   public boolean moveToTrash(Path path) throws IOException {
     if (!isEnabled())
@@ -156,10 +146,11 @@ public class TrashPolicyDefault extends TrashPolicy {
           trashPath = new Path(orig + Time.now());
         }
         
-        if (fs.rename(path, trashPath)) {           // move to current trash
-          LOG.info("Moved: '" + path + "' to trash at: " + trashPath);
-          return true;
-        }
+        // move to current trash
+        fs.rename(path, trashPath,
+            Rename.TO_TRASH);
+        LOG.info("Moved: '" + path + "' to trash at: " + trashPath);
+        return true;
       } catch (IOException e) {
         cause = e;
       }

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java

@@ -288,7 +288,8 @@ public class FTPFileSystem extends FileSystem {
   @Override
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
-    throw new IOException("Not supported");
+    throw new UnsupportedOperationException("Append is not supported "
+        + "by FTPFileSystem");
   }
   
   /**

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java

@@ -77,7 +77,6 @@ public class SFTPFileSystem extends FileSystem {
       "Can't make directory for path \"%s\" under \"%s\".";
   public static final String E_DIR_NOTEMPTY = "Directory: %s is not empty.";
   public static final String E_FILE_CHECK_FAILED = "File check failed";
-  public static final String E_NOT_SUPPORTED = "Not supported";
   public static final String E_SPATH_NOTEXIST = "Source path %s does not exist";
   public static final String E_DPATH_EXIST =
       "Destination path %s already exist, cannot rename!";
@@ -578,7 +577,8 @@ public class SFTPFileSystem extends FileSystem {
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress)
       throws IOException {
-    throw new IOException(E_NOT_SUPPORTED);
+    throw new UnsupportedOperationException("Append is not supported "
+        + "by SFTPFileSystem");
   }
 
   /*

+ 17 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java

@@ -51,6 +51,8 @@ public class Count extends FsCommand {
   private static final String OPTION_HUMAN = "h";
   private static final String OPTION_HEADER = "v";
   private static final String OPTION_TYPE = "t";
+  // exclude snapshots from calculation. Only work on default columns.
+  private static final String OPTION_EXCLUDE_SNAPSHOT = "x";
   //return the quota, namespace count and disk space usage.
   private static final String OPTION_QUOTA_AND_USAGE = "u";
 
@@ -58,7 +60,8 @@ public class Count extends FsCommand {
   public static final String USAGE =
       "[-" + OPTION_QUOTA + "] [-" + OPTION_HUMAN + "] [-" + OPTION_HEADER
           + "] [-" + OPTION_TYPE + " [<storage type>]] [-" +
-          OPTION_QUOTA_AND_USAGE + "] <path> ...";
+          OPTION_QUOTA_AND_USAGE + "] [-" + OPTION_EXCLUDE_SNAPSHOT
+          + "] <path> ...";
   public static final String DESCRIPTION =
       "Count the number of directories, files and bytes under the paths\n" +
           "that match the specified file pattern.  The output columns are:\n" +
@@ -72,6 +75,8 @@ public class Count extends FsCommand {
           "The -" + OPTION_HUMAN +
           " option shows file sizes in human readable format.\n" +
           "The -" + OPTION_HEADER + " option displays a header line.\n" +
+          "The -" + OPTION_EXCLUDE_SNAPSHOT + " option excludes snapshots " +
+          "from being calculated. \n" +
           "The -" + OPTION_TYPE + " option displays quota by storage types.\n" +
           "It must be used with -" + OPTION_QUOTA + " option.\n" +
           "If a comma-separated list of storage types is given after the -" +
@@ -87,6 +92,7 @@ public class Count extends FsCommand {
   private boolean showQuotabyType;
   private List<StorageType> storageTypes = null;
   private boolean showQuotasAndUsageOnly;
+  private boolean excludeSnapshots;
 
   /** Constructor */
   public Count() {}
@@ -106,7 +112,8 @@ public class Count extends FsCommand {
   @Override
   protected void processOptions(LinkedList<String> args) {
     CommandFormat cf = new CommandFormat(1, Integer.MAX_VALUE,
-        OPTION_QUOTA, OPTION_HUMAN, OPTION_HEADER, OPTION_QUOTA_AND_USAGE);
+        OPTION_QUOTA, OPTION_HUMAN, OPTION_HEADER, OPTION_QUOTA_AND_USAGE,
+        OPTION_EXCLUDE_SNAPSHOT);
     cf.addOptionWithValue(OPTION_TYPE);
     cf.parse(args);
     if (args.isEmpty()) { // default path is the current working directory
@@ -115,6 +122,7 @@ public class Count extends FsCommand {
     showQuotas = cf.getOpt(OPTION_QUOTA);
     humanReadable = cf.getOpt(OPTION_HUMAN);
     showQuotasAndUsageOnly = cf.getOpt(OPTION_QUOTA_AND_USAGE);
+    excludeSnapshots = cf.getOpt(OPTION_EXCLUDE_SNAPSHOT);
 
     if (showQuotas || showQuotasAndUsageOnly) {
       String types = cf.getOptValue(OPTION_TYPE);
@@ -125,6 +133,11 @@ public class Count extends FsCommand {
       } else {
         showQuotabyType = false;
       }
+      if (excludeSnapshots) {
+        out.println(OPTION_QUOTA + " or " + OPTION_QUOTA_AND_USAGE + " option "
+            + "is given, the -x option is ignored.");
+        excludeSnapshots = false;
+      }
     }
 
     if (cf.getOpt(OPTION_HEADER)) {
@@ -163,7 +176,8 @@ public class Count extends FsCommand {
           storageTypes) + src);
     } else {
       ContentSummary summary = src.fs.getContentSummary(src.path);
-      out.println(summary.toString(showQuotas, isHumanReadable()) + src);
+      out.println(summary.
+          toString(showQuotas, isHumanReadable(), excludeSnapshots) + src);
     }
   }
 

+ 19 - 12
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java

@@ -107,27 +107,30 @@ class FsUsage extends FsCommand {
   /** show disk usage */
   public static class Du extends FsUsage {
     public static final String NAME = "du";
-    public static final String USAGE = "[-s] [-h] <path> ...";
+    public static final String USAGE = "[-s] [-h] [-x] <path> ...";
     public static final String DESCRIPTION =
-    "Show the amount of space, in bytes, used by the files that " +
-    "match the specified file pattern. The following flags are optional:\n" +
-    "-s: Rather than showing the size of each individual file that" +
-    " matches the pattern, shows the total (summary) size.\n" +
-    "-h: Formats the sizes of files in a human-readable fashion" +
-    " rather than a number of bytes.\n\n" +
-    "Note that, even without the -s option, this only shows size summaries " +
-    "one level deep into a directory.\n\n" +
-    "The output is in the form \n" + 
-    "\tsize\tdisk space consumed\tname(full path)\n";
+        "Show the amount of space, in bytes, used by the files that match " +
+            "the specified file pattern. The following flags are optional:\n" +
+            "-s: Rather than showing the size of each individual file that" +
+            " matches the pattern, shows the total (summary) size.\n" +
+            "-h: Formats the sizes of files in a human-readable fashion" +
+            " rather than a number of bytes.\n" +
+            "-x: Excludes snapshots from being counted.\n\n" +
+            "Note that, even without the -s option, this only shows size " +
+            "summaries one level deep into a directory.\n\n" +
+            "The output is in the form \n" +
+            "\tsize\tdisk space consumed\tname(full path)\n";
 
     protected boolean summary = false;
+    private boolean excludeSnapshots = false;
     
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
-      CommandFormat cf = new CommandFormat(0, Integer.MAX_VALUE, "h", "s");
+      CommandFormat cf = new CommandFormat(0, Integer.MAX_VALUE, "h", "s", "x");
       cf.parse(args);
       humanReadable = cf.getOpt("h");
       summary = cf.getOpt("s");
+      excludeSnapshots = cf.getOpt("x");
       if (args.isEmpty()) args.add(Path.CUR_DIR);
     }
 
@@ -156,6 +159,10 @@ class FsUsage extends FsCommand {
       ContentSummary contentSummary = item.fs.getContentSummary(item.path);
       long length = contentSummary.getLength();
       long spaceConsumed = contentSummary.getSpaceConsumed();
+      if (excludeSnapshots) {
+        length -= contentSummary.getSnapshotLength();
+        spaceConsumed -= contentSummary.getSnapshotSpaceConsumed();
+      }
       usagesTable.addRow(formatSize(length), formatSize(spaceConsumed), item);
     }
   }

+ 5 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java

@@ -351,9 +351,11 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
     if (retryInfo.isFail()) {
       // fail.
       if (retryInfo.action.reason != null) {
-        LOG.warn("Exception while invoking call #" + callId + " "
-            + proxyDescriptor.getProxyInfo().getString(method.getName())
-            + ". Not retrying because " + retryInfo.action.reason, e);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Exception while invoking call #" + callId + " "
+              + proxyDescriptor.getProxyInfo().getString(method.getName())
+              + ". Not retrying because " + retryInfo.action.reason, e);
+        }
       }
       throw e;
     }

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallerContext.java

@@ -33,7 +33,7 @@ import java.util.Arrays;
  * This class is immutable.
  */
 @InterfaceAudience.LimitedPrivate({"HBase", "HDFS", "Hive", "MapReduce",
-    "Pig", "YARN"})
+    "Pig", "Spark", "YARN"})
 @InterfaceStability.Evolving
 public final class CallerContext {
   public static final Charset SIGNATURE_ENCODING = StandardCharsets.UTF_8;

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSink.java

@@ -485,10 +485,10 @@ public class RollingFileSystemSink implements MetricsSink, Closeable {
 
     try {
       fs.append(basePath);
+    } catch (UnsupportedOperationException ex) {
+      canAppend = false;
     } catch (IOException ex) {
-      if (ex.getMessage().equals("Not supported")) {
-        canAppend = false;
-      }
+      // Ignore. The operation is supported.
     }
 
     return canAppend;

+ 44 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java

@@ -361,6 +361,7 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
             }
           }
         }, listenerThreadPool);
+        loadFromZKCache(false);
       }
     } catch (Exception e) {
       throw new IOException("Could not start PathChildrenCache for keys", e);
@@ -389,6 +390,7 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
             }
           }
         }, listenerThreadPool);
+        loadFromZKCache(true);
       }
     } catch (Exception e) {
       throw new IOException("Could not start PathChildrenCache for tokens", e);
@@ -396,6 +398,43 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
     super.startThreads();
   }
 
+  /**
+   * Load the PathChildrenCache into the in-memory map. Possible caches to be
+   * loaded are keyCache and tokenCache.
+   *
+   * @param isTokenCache true if loading tokenCache, false if loading keyCache.
+   */
+  private void loadFromZKCache(final boolean isTokenCache) {
+    final String cacheName = isTokenCache ? "token" : "key";
+    LOG.info("Starting to load {} cache.", cacheName);
+    final List<ChildData> children;
+    if (isTokenCache) {
+      children = tokenCache.getCurrentData();
+    } else {
+      children = keyCache.getCurrentData();
+    }
+
+    int count = 0;
+    for (ChildData child : children) {
+      try {
+        if (isTokenCache) {
+          processTokenAddOrUpdate(child);
+        } else {
+          processKeyAddOrUpdate(child.getData());
+        }
+      } catch (Exception e) {
+        LOG.info("Ignoring node {} because it failed to load.",
+            child.getPath());
+        LOG.debug("Failure exception:", e);
+        ++count;
+      }
+    }
+    if (count > 0) {
+      LOG.warn("Ignored {} nodes while loading {} cache.", count, cacheName);
+    }
+    LOG.info("Loaded {} cache.", cacheName);
+  }
+
   private void processKeyAddOrUpdate(byte[] data) throws IOException {
     ByteArrayInputStream bin = new ByteArrayInputStream(data);
     DataInputStream din = new DataInputStream(bin);
@@ -890,4 +929,9 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
   public ExecutorService getListenerThreadPool() {
     return listenerThreadPool;
   }
+
+  @VisibleForTesting
+  DelegationTokenInformation getTokenInfoFromMemory(TokenIdent ident) {
+    return currentTokens.get(ident);
+  }
 }

+ 10 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java

@@ -39,6 +39,8 @@ import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.http.NameValuePair;
 import org.apache.http.client.utils.URLEncodedUtils;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
@@ -81,6 +83,9 @@ public class DelegationTokenAuthenticationFilter
   private static final String ERROR_EXCEPTION_JSON = "exception";
   private static final String ERROR_MESSAGE_JSON = "message";
 
+  private static final Logger LOG = LoggerFactory.getLogger(
+          DelegationTokenAuthenticationFilter.class);
+
   /**
    * Sets an external <code>DelegationTokenSecretManager</code> instance to
    * manage creation and verification of Delegation Tokens.
@@ -261,6 +266,11 @@ public class DelegationTokenAuthenticationFilter
             HttpExceptionUtils.createServletExceptionResponse(response,
                 HttpServletResponse.SC_FORBIDDEN, ex);
             requestCompleted = true;
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Authentication exception: " + ex.getMessage(), ex);
+            } else {
+              LOG.warn("Authentication exception: " + ex.getMessage());
+            }
           }
         }
       }

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NodeHealthScriptRunner.java

@@ -106,6 +106,7 @@ public class NodeHealthScriptRunner extends AbstractService {
         shexec.execute();
       } catch (ExitCodeException e) {
         // ignore the exit code of the script
+        exceptionStackTrace = StringUtils.stringifyException(e);
         status = HealthCheckerExitStatus.FAILED_WITH_EXIT_CODE;
         // On Windows, we will not hit the Stream closed IOException
         // thrown by stdout buffered reader for timeout event.
@@ -162,7 +163,7 @@ public class NodeHealthScriptRunner extends AbstractService {
         setHealthStatus(false, exceptionStackTrace);
         break;
       case FAILED_WITH_EXIT_CODE:
-        setHealthStatus(true, "", now);
+        setHealthStatus(false, exceptionStackTrace);
         break;
       case FAILED:
         setHealthStatus(false, shexec.getOutput());

+ 12 - 7
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -877,22 +877,27 @@
 <property>
   <name>fs.s3a.aws.credentials.provider</name>
   <description>
-    Class name of a credentials provider that implements
-    com.amazonaws.auth.AWSCredentialsProvider.  Omit if using access/secret keys
-    or another authentication mechanism.  The specified class must provide an
-    accessible constructor accepting java.net.URI and
-    org.apache.hadoop.conf.Configuration, or an accessible default constructor.
+    Comma-separated class names of credential provider classes which implement
+    com.amazonaws.auth.AWSCredentialsProvider.
+
+    These are loaded and queried in sequence for a valid set of credentials.
+    Each listed class must provide either an accessible constructor accepting
+    java.net.URI and org.apache.hadoop.conf.Configuration, or an accessible
+    default constructor.
+
     Specifying org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider allows
     anonymous access to a publicly accessible S3 bucket without any credentials.
     Please note that allowing anonymous access to an S3 bucket compromises
-    security and therefore is unsuitable for most use cases.  It can be useful
+    security and therefore is unsuitable for most use cases. It can be useful
     for accessing public data sets without requiring AWS credentials.
   </description>
 </property>
 
 <property>
   <name>fs.s3a.session.token</name>
-  <description>The session token used with temporary credentials. Used only with provider org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider.</description>
+  <description>Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider
+    as one of the providers.
+  </description>
 </property>
 
 <property>

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

@@ -119,7 +119,7 @@ See the command options detail in the [Commands Manual](CommandsManual.html#cred
 
 Utilizing the credential command will often be for provisioning a password or secret to a particular credential store provider. In order to explicitly indicate which provider store to use the `-provider` option should be used.
 
-Example: `hadoop credential create ssl.server.keystore.password jceks://file/tmp/test.jceks`
+Example: `hadoop credential create ssl.server.keystore.password -provider jceks://file/tmp/test.jceks`
 
 In order to indicate a particular provider type and location, the user must provide the `hadoop.security.credential.provider.path` configuration element in core-site.xml or use the command line option `-provider` on each of the credential management commands. This provider path is a comma-separated list of URLs that indicates the type and location of a list of providers that should be consulted. For example, the following path: `user:///,jceks://file/tmp/test.jceks,jceks://hdfs@nn1.example.com/my/path/test.jceks` indicates that the current user's credentials file should be consulted through the User Provider, that the local file located at `/tmp/test.jceks` is a Java Keystore Provider and that the file located within HDFS at `nn1.example.com/my/path/test.jceks` is also a store for a Java Keystore Provider.
 

+ 8 - 3
hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md

@@ -132,10 +132,12 @@ Similar to get command, except that the destination is restricted to a local fil
 count
 -----
 
-Usage: `hadoop fs -count [-q] [-h] [-v] [-t [<storage type>]] [-u] <paths> `
+Usage: `hadoop fs -count [-q] [-h] [-v] [-x] [-t [<storage type>]] [-u] <paths> `
 
 Count the number of directories, files and bytes under the paths that match the specified file pattern. Get the quota and the usage. The output columns with -count are: DIR\_COUNT, FILE\_COUNT, CONTENT\_SIZE, PATHNAME
 
+The -u and -q options control what columns the output contains.  -q means show quotas, -u limits the output to show quotas and usage only.
+
 The output columns with -count -q are: QUOTA, REMAINING\_QUOTA, SPACE\_QUOTA, REMAINING\_SPACE\_QUOTA, DIR\_COUNT, FILE\_COUNT, CONTENT\_SIZE, PATHNAME
 
 The output columns with -count -u are: QUOTA, REMAINING\_QUOTA, SPACE\_QUOTA, REMAINING\_SPACE\_QUOTA
@@ -146,6 +148,8 @@ The -h option shows sizes in human readable format.
 
 The -v option displays a header line.
 
+The -x option excludes snapshots from the result calculation. Without the -x option (default), the result is always calculated from all INodes, including all snapshots under the given path. The -x option is ignored if -u or -q option is given.
+
 Example:
 
 * `hadoop fs -count hdfs://nn1.example.com/file1 hdfs://nn2.example.com/file2`
@@ -211,14 +215,15 @@ Example:
 du
 ----
 
-Usage: `hadoop fs -du [-s] [-h] URI [URI ...]`
+Usage: `hadoop fs -du [-s] [-h] [-x] URI [URI ...]`
 
 Displays sizes of files and directories contained in the given directory or the length of a file in case its just a file.
 
 Options:
 
-* The -s option will result in an aggregate summary of file lengths being displayed, rather than the individual files.
+* The -s option will result in an aggregate summary of file lengths being displayed, rather than the individual files. Without the -s option, calculation is done by going 1-level deep from the given path.
 * The -h option will format file sizes in a "human-readable" fashion (e.g 64.0m instead of 67108864)
+* The -x option will exclude snapshots from the result calculation. Without the -x option (default), the result is always calculated from all INodes, including all snapshots under the given path.
 
 The du returns three columns with the following format:
 

+ 4 - 4
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md

@@ -611,7 +611,7 @@ this precondition fails.
 
 ### `FSDataOutputStream append(Path p, int bufferSize, Progressable progress)`
 
-Implementations MAY throw `UnsupportedOperationException`.
+Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
 #### Preconditions
 
@@ -630,7 +630,7 @@ by appending data to the existing list.
 
 ### `FSDataInputStream open(Path f, int bufferSize)`
 
-Implementations MAY throw `UnsupportedOperationException`.
+Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
 #### Preconditions
 
@@ -944,7 +944,7 @@ to the `DFSFileSystem` implementation is an ongoing matter for debate.
 Joins multiple blocks together to create a single file. This
 is a little-used operation currently implemented only by HDFS.
 
-Implementations MAY throw `UnsupportedOperationException`
+Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
 #### Preconditions
 
@@ -989,7 +989,7 @@ from specification.
 
 Truncate file `p` to the specified `newLength`.
 
-Implementations MAY throw `UnsupportedOperationException`.
+Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
 #### Preconditions
 

+ 5 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java

@@ -109,9 +109,11 @@ public abstract class FileContextMainOperationsBaseTest  {
   
   @After
   public void tearDown() throws Exception {
-    boolean del = fc.delete(new Path(fileContextTestHelper.getAbsoluteTestRootPath(fc), new Path("test")), true);
-    assertTrue(del);
-    fc.delete(localFsRootPath, true);
+    if (fc != null) {
+      boolean del = fc.delete(new Path(fileContextTestHelper.getAbsoluteTestRootPath(fc), new Path("test")), true);
+      assertTrue(del);
+      fc.delete(localFsRootPath, true);
+    }
   }
   
   

+ 3 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextUtilBase.java

@@ -63,7 +63,9 @@ public abstract class FileContextUtilBase {
 
   @After
   public void tearDown() throws Exception {
-    fc.delete(fileContextTestHelper.getTestRootPath(fc), true);
+    if (fc != null) {
+      fc.delete(fileContextTestHelper.getTestRootPath(fc), true);
+    }
   }
   
   @Test

+ 32 - 14
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java

@@ -172,22 +172,39 @@ public abstract class FileSystemContractBaseTest extends TestCase {
   }
 
   public void testMkdirsWithUmask() throws Exception {
-    if (fs.getScheme().equals("s3n")) {
-      // skip permission tests for S3FileSystem until HDFS-1333 is fixed.
-      return;
+    if (!isS3(fs)) {
+      Configuration conf = fs.getConf();
+      String oldUmask = conf.get(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY);
+      try {
+        conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, TEST_UMASK);
+        final Path dir = path("/test/newDir");
+        assertTrue(fs.mkdirs(dir, new FsPermission((short) 0777)));
+        FileStatus status = fs.getFileStatus(dir);
+        assertTrue(status.isDirectory());
+        assertEquals((short) 0715, status.getPermission().toShort());
+      } finally {
+        conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, oldUmask);
+      }
     }
-    Configuration conf = fs.getConf();
-    String oldUmask = conf.get(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY);
+  }
+
+  /**
+   * Skip permission tests for S3FileSystem until HDFS-1333 is fixed.
+   * Classes that do not implement {@link FileSystem#getScheme()} method
+   * (e.g {@link RawLocalFileSystem}) will throw an
+   * {@link UnsupportedOperationException}.
+   * @param fileSystem FileSystem object to determine if it is S3 or not
+   * @return true if S3 false in any other case
+   */
+  private boolean isS3(FileSystem fileSystem) {
     try {
-      conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, TEST_UMASK);
-      final Path dir = new Path("/test/newDir");
-      assertTrue(fs.mkdirs(dir, new FsPermission((short)0777)));
-      FileStatus status = fs.getFileStatus(dir);
-      assertTrue(status.isDirectory());
-      assertEquals((short)0715, status.getPermission().toShort());
-    } finally {
-      conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, oldUmask);
+      if (fileSystem.getScheme().equals("s3n")) {
+        return true;
+      }
+    } catch (UnsupportedOperationException e) {
+      LOG.warn("Unable to determine the schema of filesystem.");
     }
+    return false;
   }
 
   public void testGetFileStatusThrowsExceptionForNonExistentFile() 
@@ -480,7 +497,8 @@ public abstract class FileSystemContractBaseTest extends TestCase {
   }
   
   protected Path path(String pathString) {
-    return new Path(pathString).makeQualified(fs);
+    return new Path(pathString).makeQualified(fs.getUri(),
+        fs.getWorkingDirectory());
   }
   
   protected void createFile(Path path) throws IOException {

+ 75 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java

@@ -0,0 +1,75 @@
+/**
+ * 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.fs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Shell;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test filesystem contracts with {@link RawLocalFileSystem}.
+ * Root directory related tests from super class will work into target
+ * directory since we have no permission to write / on local filesystem.
+ */
+public class TestRawLocalFileSystemContract extends FileSystemContractBaseTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRawLocalFileSystemContract.class);
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    fs = FileSystem.getLocal(conf).getRawFileSystem();
+  }
+
+  /**
+   * Actually rename is supported in RawLocalFileSystem but
+   * it works different as the other filesystems. Short term we do not test it.
+   * Please check HADOOP-13082.
+   * @return true if rename supported so rename related tests will run
+   */
+  @Override
+  protected boolean renameSupported() {
+    return false;
+  }
+
+  @Override
+  public String getDefaultWorkingDirectory() {
+    return fs.getWorkingDirectory().toUri().getPath();
+  }
+
+  @Override
+  protected Path path(String pathString) {
+    // For testWorkingDirectory
+    if (pathString.equals(getDefaultWorkingDirectory()) ||
+        pathString.equals(".") || pathString.equals("..")) {
+      return super.path(pathString);
+    }
+
+    return new Path(GenericTestUtils.getTempPath(pathString)).
+        makeQualified(fs.getUri(), fs.getWorkingDirectory());
+  }
+
+  @Override
+  protected boolean filesystemIsCaseSensitive() {
+    return !(Shell.WINDOWS || Shell.MAC);
+  }
+}

+ 0 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java

@@ -691,10 +691,6 @@ public class TestTrash extends TestCase {
   public static class TestTrashPolicy extends TrashPolicy {
     public TestTrashPolicy() { }
 
-    @Override
-    public void initialize(Configuration conf, FileSystem fs, Path home) {
-    }
-
     @Override
     public void initialize(Configuration conf, FileSystem fs) {
     }

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java

@@ -447,7 +447,7 @@ public class TestCount {
     Count count = new Count();
     String actual = count.getUsage();
     String expected =
-        "-count [-q] [-h] [-v] [-t [<storage type>]] [-u] <path> ...";
+        "-count [-q] [-h] [-v] [-t [<storage type>]] [-u] [-x] <path> ...";
     assertEquals("Count.getUsage", expected, actual);
   }
 
@@ -465,6 +465,7 @@ public class TestCount {
         + "      DIR_COUNT FILE_COUNT CONTENT_SIZE PATHNAME\n"
         + "The -h option shows file sizes in human readable format.\n"
         + "The -v option displays a header line.\n"
+        + "The -x option excludes snapshots from being calculated. \n"
         + "The -t option displays quota by storage types.\n"
         + "It must be used with -q option.\n"
         + "If a comma-separated list of storage types is given after the -t option, \n"
@@ -521,7 +522,7 @@ public class TestCount {
     }
 
     @Override
-    public String toString(boolean qOption, boolean hOption) {
+    public String toString(boolean qOption, boolean hOption, boolean xOption) {
       if (qOption) {
         if (hOption) {
           return (HUMAN + WITH_QUOTAS);

+ 92 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java

@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 
+import com.google.common.base.Supplier;
 import org.apache.curator.RetryPolicy;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -37,6 +38,7 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
@@ -44,12 +46,18 @@ import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.fail;
 
-import org.junit.Test;
 
 public class TestZKDelegationTokenSecretManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestZKDelegationTokenSecretManager.class);
 
   private static final int TEST_RETRIES = 2;
 
@@ -61,6 +69,9 @@ public class TestZKDelegationTokenSecretManager {
 
   private TestingServer zkServer;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setup() throws Exception {
     zkServer = new TestingServer();
@@ -382,4 +393,84 @@ public class TestZKDelegationTokenSecretManager {
     }
   }
 
+  @SuppressWarnings({ "unchecked" })
+  @Test
+  public void testNodesLoadedAfterRestart() throws Exception {
+    final String connectString = zkServer.getConnectString();
+    final Configuration conf = getSecretConf(connectString);
+    final int removeScan = 1;
+    // Set the remove scan interval to remove expired tokens
+    conf.setLong(DelegationTokenManager.REMOVAL_SCAN_INTERVAL, removeScan);
+    // Set the update interval to trigger background thread to run. The thread
+    // is hard-coded to sleep at least 5 seconds.
+    conf.setLong(DelegationTokenManager.UPDATE_INTERVAL, 5);
+    // Set token expire time to 5 seconds.
+    conf.setLong(DelegationTokenManager.RENEW_INTERVAL, 5);
+
+    DelegationTokenManager tm =
+        new DelegationTokenManager(conf, new Text("bla"));
+    tm.init();
+    Token<DelegationTokenIdentifier> token =
+        (Token<DelegationTokenIdentifier>) tm
+            .createToken(UserGroupInformation.getCurrentUser(), "good");
+    Assert.assertNotNull(token);
+    Token<DelegationTokenIdentifier> cancelled =
+        (Token<DelegationTokenIdentifier>) tm
+            .createToken(UserGroupInformation.getCurrentUser(), "cancelled");
+    Assert.assertNotNull(cancelled);
+    tm.verifyToken(token);
+    tm.verifyToken(cancelled);
+
+    // Cancel one token, verify it's gone
+    tm.cancelToken(cancelled, "cancelled");
+    final AbstractDelegationTokenSecretManager sm =
+        tm.getDelegationTokenSecretManager();
+    final ZKDelegationTokenSecretManager zksm =
+        (ZKDelegationTokenSecretManager) sm;
+    final AbstractDelegationTokenIdentifier idCancelled =
+        sm.decodeTokenIdentifier(cancelled);
+    LOG.info("Waiting for the cancelled token to be removed");
+
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        AbstractDelegationTokenSecretManager.DelegationTokenInformation dtinfo =
+            zksm.getTokenInfo(idCancelled);
+        return dtinfo == null;
+      }
+    }, 100, 5000);
+
+    // Fake a restart which launches a new tm
+    tm.destroy();
+    tm = new DelegationTokenManager(conf, new Text("bla"));
+    tm.init();
+    final AbstractDelegationTokenSecretManager smNew =
+        tm.getDelegationTokenSecretManager();
+    final ZKDelegationTokenSecretManager zksmNew =
+        (ZKDelegationTokenSecretManager) smNew;
+
+    // The cancelled token should be gone, and not loaded.
+    AbstractDelegationTokenIdentifier id =
+        smNew.decodeTokenIdentifier(cancelled);
+    AbstractDelegationTokenSecretManager.DelegationTokenInformation dtinfo =
+        zksmNew.getTokenInfo(id);
+    Assert.assertNull("canceled dt should be gone!", dtinfo);
+
+    // The good token should be loaded on startup, and removed after expiry.
+    id = smNew.decodeTokenIdentifier(token);
+    dtinfo = zksmNew.getTokenInfoFromMemory(id);
+    Assert.assertNotNull("good dt should be in memory!", dtinfo);
+
+    // Wait for the good token to expire.
+    Thread.sleep(5000);
+    final ZKDelegationTokenSecretManager zksm1 = zksmNew;
+    final AbstractDelegationTokenIdentifier id1 = id;
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        LOG.info("Waiting for the expired token to be removed...");
+        return zksm1.getTokenInfo(id1) == null;
+      }
+    }, 1000, 5000);
+  }
 }

+ 7 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNodeHealthScriptRunner.java

@@ -91,6 +91,7 @@ public class TestNodeHealthScriptRunner {
   public void testNodeHealthScript() throws Exception {
     String errorScript = "echo ERROR\n echo \"Tracker not healthy\"";
     String normalScript = "echo \"I am all fine\"";
+    String failWithExitCodeScript = "echo \"Not healthy\"; exit -1";
     String timeOutScript =
       Shell.WINDOWS ? "@echo off\nping -n 4 127.0.0.1 >nul\necho \"I am fine\""
       : "sleep 4\necho \"I am fine\"";
@@ -124,6 +125,12 @@ public class TestNodeHealthScriptRunner {
         nodeHealthScriptRunner.isHealthy());
     Assert.assertEquals("", nodeHealthScriptRunner.getHealthReport());
 
+    // Script which fails with exit code.
+    writeNodeHealthScriptFile(failWithExitCodeScript, true);
+    timerTask.run();
+    Assert.assertFalse("Node health status reported healthy",
+        nodeHealthScriptRunner.isHealthy());
+
     // Timeout script.
     writeNodeHealthScriptFile(timeOutScript, true);
     timerTask.run();

+ 10 - 2
hadoop-common-project/hadoop-common/src/test/resources/testConf.xml

@@ -200,7 +200,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-du \[-s\] \[-h\] &lt;path&gt; \.\.\. :\s*</expected-output>
+          <expected-output>^-du \[-s\] \[-h\] \[-x\] &lt;path&gt; \.\.\. :\s*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -226,6 +226,10 @@
           <type>RegexpComparator</type>
           <expected-output>\s*of bytes.\s*</expected-output>
         </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^\s*-x\s*Excludes snapshots from being counted.\s*</expected-output>
+        </comparator>
         <comparator>
           <type>RegexpComparator</type>
           <expected-output>^\s*Note that, even without the -s option, this only shows size summaries one level\s*</expected-output>
@@ -274,7 +278,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-count \[-q\] \[-h\] \[-v\] \[-t \[&lt;storage type&gt;\]\] \[-u\] &lt;path&gt; \.\.\. :( )*</expected-output>
+          <expected-output>^-count \[-q\] \[-h\] \[-v\] \[-t \[&lt;storage type&gt;\]\] \[-u\] \[-x\] &lt;path&gt; \.\.\. :( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -308,6 +312,10 @@
           <type>RegexpComparator</type>
           <expected-output>^( |\t)*The -v option displays a header line.( )*</expected-output>
         </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*The -x option excludes snapshots from being calculated.( )*</expected-output>
+        </comparator>
       </comparators>
     </test>
 

+ 10 - 0
hadoop-common-project/hadoop-kms/src/main/conf/kms-site.xml

@@ -170,4 +170,14 @@
     </description>
   </property>
 
+  <property>
+    <name>hadoop.kms.audit.logger</name>
+    <value>org.apache.hadoop.crypto.key.kms.server.SimpleKMSAuditLogger</value>
+    <description>
+      The audit logger for KMS. It is a comma-separated list of KMSAuditLogger
+      class names. Default is the text-format SimpleKMSAuditLogger only.
+      If this is not configured, default will be used.
+    </description>
+  </property>
+
 </configuration>

+ 107 - 93
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java

@@ -17,12 +17,18 @@
  */
 package org.apache.hadoop.crypto.key.kms.server;
 
+import static org.apache.hadoop.crypto.key.kms.server.KMSAuditLogger.AuditEvent;
+import static org.apache.hadoop.crypto.key.kms.server.KMSAuditLogger.OpStatus;
+
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Joiner;
 import com.google.common.base.Strings;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
@@ -31,67 +37,24 @@ import com.google.common.cache.RemovalNotification;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Set;
+import java.util.HashSet;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
 
 /**
- * Provides convenience methods for audit logging consistently the different
+ * Provides convenience methods for audit logging consisting different
  * types of events.
  */
 public class KMSAudit {
-
-  private static class AuditEvent {
-    private final AtomicLong accessCount = new AtomicLong(-1);
-    private final String keyName;
-    private final String user;
-    private final KMS.KMSOp op;
-    private final String extraMsg;
-    private final long startTime = System.currentTimeMillis();
-
-    private AuditEvent(String keyName, String user, KMS.KMSOp op, String msg) {
-      this.keyName = keyName;
-      this.user = user;
-      this.op = op;
-      this.extraMsg = msg;
-    }
-
-    public String getExtraMsg() {
-      return extraMsg;
-    }
-
-    public AtomicLong getAccessCount() {
-      return accessCount;
-    }
-
-    public String getKeyName() {
-      return keyName;
-    }
-
-    public String getUser() {
-      return user;
-    }
-
-    public KMS.KMSOp getOp() {
-      return op;
-    }
-
-    public long getStartTime() {
-      return startTime;
-    }
-  }
-
-  public static enum OpStatus {
-    OK, UNAUTHORIZED, UNAUTHENTICATED, ERROR;
-  }
-
-  private static Set<KMS.KMSOp> AGGREGATE_OPS_WHITELIST = Sets.newHashSet(
+  @VisibleForTesting
+  static final Set<KMS.KMSOp> AGGREGATE_OPS_WHITELIST = Sets.newHashSet(
     KMS.KMSOp.GET_KEY_VERSION, KMS.KMSOp.GET_CURRENT_KEY,
     KMS.KMSOp.DECRYPT_EEK, KMS.KMSOp.GENERATE_EEK
   );
@@ -102,17 +65,21 @@ public class KMSAudit {
 
   public static final String KMS_LOGGER_NAME = "kms-audit";
 
-  private static Logger AUDIT_LOG = LoggerFactory.getLogger(KMS_LOGGER_NAME);
+  private final static Logger LOG = LoggerFactory.getLogger(KMSAudit.class);
+  private final List<KMSAuditLogger> auditLoggers = new LinkedList<>();
 
   /**
    * Create a new KMSAudit.
    *
-   * @param windowMs Duplicate events within the aggregation window are quashed
-   *                 to reduce log traffic. A single message for aggregated
-   *                 events is printed at the end of the window, along with a
-   *                 count of the number of aggregated events.
+   * @param conf The configuration object.
    */
-  KMSAudit(long windowMs) {
+  KMSAudit(Configuration conf) {
+    // Duplicate events within the aggregation window are quashed
+    // to reduce log traffic. A single message for aggregated
+    // events is printed at the end of the window, along with a
+    // count of the number of aggregated events.
+    long windowMs = conf.getLong(KMSConfiguration.KMS_AUDIT_AGGREGATION_WINDOW,
+        KMSConfiguration.KMS_AUDIT_AGGREGATION_WINDOW_DEFAULT);
     cache = CacheBuilder.newBuilder()
         .expireAfterWrite(windowMs, TimeUnit.MILLISECONDS)
         .removalListener(
@@ -122,7 +89,7 @@ public class KMSAudit {
                   RemovalNotification<String, AuditEvent> entry) {
                 AuditEvent event = entry.getValue();
                 if (event.getAccessCount().get() > 0) {
-                  KMSAudit.this.logEvent(event);
+                  KMSAudit.this.logEvent(OpStatus.OK, event);
                   event.getAccessCount().set(0);
                   KMSAudit.this.cache.put(entry.getKey(), event);
                 }
@@ -136,87 +103,127 @@ public class KMSAudit {
         cache.cleanUp();
       }
     }, windowMs / 10, windowMs / 10, TimeUnit.MILLISECONDS);
+    initializeAuditLoggers(conf);
   }
 
-  private void logEvent(AuditEvent event) {
-    AUDIT_LOG.info(
-        "OK[op={}, key={}, user={}, accessCount={}, interval={}ms] {}",
-        event.getOp(), event.getKeyName(), event.getUser(),
-        event.getAccessCount().get(),
-        (System.currentTimeMillis() - event.getStartTime()),
-        event.getExtraMsg());
+  /**
+   * Read the KMSAuditLogger classes from configuration. If any loggers fail to
+   * load, a RumTimeException will be thrown.
+   *
+   * @param conf The configuration.
+   * @return Collection of KMSAudigLogger classes.
+   */
+  private Set<Class<? extends KMSAuditLogger>> getAuditLoggerClasses(
+      final Configuration conf) {
+    Set<Class<? extends KMSAuditLogger>> result = new HashSet<>();
+    // getTrimmedStringCollection will remove duplicates.
+    Collection<String> classes =
+        conf.getTrimmedStringCollection(KMSConfiguration.KMS_AUDIT_LOGGER_KEY);
+    if (classes.isEmpty()) {
+      LOG.info("No audit logger configured, using default.");
+      result.add(SimpleKMSAuditLogger.class);
+      return result;
+    }
+
+    for (String c : classes) {
+      try {
+        Class<?> cls = conf.getClassByName(c);
+        result.add(cls.asSubclass(KMSAuditLogger.class));
+      } catch (ClassNotFoundException cnfe) {
+        throw new RuntimeException("Failed to load " + c + ", please check "
+            + "configuration " + KMSConfiguration.KMS_AUDIT_LOGGER_KEY, cnfe);
+      }
+    }
+    return result;
   }
 
-  private void op(OpStatus opStatus, final KMS.KMSOp op, final String user,
-      final String key, final String extraMsg) {
+  /**
+   * Create a collection of KMSAuditLoggers from configuration, and initialize
+   * them. If any logger failed to be created or initialized, a RunTimeException
+   * is thrown.
+   */
+  private void initializeAuditLoggers(Configuration conf) {
+    Set<Class<? extends KMSAuditLogger>> classes = getAuditLoggerClasses(conf);
+    Preconditions
+        .checkState(!classes.isEmpty(), "Should have at least 1 audit logger.");
+    for (Class<? extends KMSAuditLogger> c : classes) {
+      final KMSAuditLogger logger = ReflectionUtils.newInstance(c, conf);
+      auditLoggers.add(logger);
+    }
+    for (KMSAuditLogger logger: auditLoggers) {
+      try {
+        LOG.info("Initializing audit logger {}", logger.getClass());
+        logger.initialize(conf);
+      } catch (Exception ex) {
+        throw new RuntimeException(
+            "Failed to initialize " + logger.getClass().getName(), ex);
+      }
+    }
+  }
+
+  private void logEvent(final OpStatus status, AuditEvent event) {
+    event.setEndTime(Time.now());
+    for (KMSAuditLogger logger: auditLoggers) {
+      logger.logAuditEvent(status, event);
+    }
+  }
+
+  private void op(final OpStatus opStatus, final KMS.KMSOp op,
+      final UserGroupInformation ugi, final String key, final String remoteHost,
+      final String extraMsg) {
+    final String user = ugi == null ? null: ugi.getShortUserName();
     if (!Strings.isNullOrEmpty(user) && !Strings.isNullOrEmpty(key)
         && (op != null)
         && AGGREGATE_OPS_WHITELIST.contains(op)) {
       String cacheKey = createCacheKey(user, key, op);
       if (opStatus == OpStatus.UNAUTHORIZED) {
         cache.invalidate(cacheKey);
-        AUDIT_LOG.info("UNAUTHORIZED[op={}, key={}, user={}] {}", op, key, user,
-            extraMsg);
+        logEvent(opStatus, new AuditEvent(op, ugi, key, remoteHost, extraMsg));
       } else {
         try {
           AuditEvent event = cache.get(cacheKey, new Callable<AuditEvent>() {
             @Override
             public AuditEvent call() throws Exception {
-              return new AuditEvent(key, user, op, extraMsg);
+              return new AuditEvent(op, ugi, key, remoteHost, extraMsg);
             }
           });
           // Log first access (initialized as -1 so
           // incrementAndGet() == 0 implies first access)
           if (event.getAccessCount().incrementAndGet() == 0) {
             event.getAccessCount().incrementAndGet();
-            logEvent(event);
+            logEvent(opStatus, event);
           }
         } catch (ExecutionException ex) {
           throw new RuntimeException(ex);
         }
       }
     } else {
-      List<String> kvs = new LinkedList<String>();
-      if (op != null) {
-        kvs.add("op=" + op);
-      }
-      if (!Strings.isNullOrEmpty(key)) {
-        kvs.add("key=" + key);
-      }
-      if (!Strings.isNullOrEmpty(user)) {
-        kvs.add("user=" + user);
-      }
-      if (kvs.size() == 0) {
-        AUDIT_LOG.info("{} {}", opStatus.toString(), extraMsg);
-      } else {
-        String join = Joiner.on(", ").join(kvs);
-        AUDIT_LOG.info("{}[{}] {}", opStatus.toString(), join, extraMsg);
-      }
+      logEvent(opStatus, new AuditEvent(op, ugi, key, remoteHost, extraMsg));
     }
   }
 
   public void ok(UserGroupInformation user, KMS.KMSOp op, String key,
       String extraMsg) {
-    op(OpStatus.OK, op, user.getShortUserName(), key, extraMsg);
+    op(OpStatus.OK, op, user, key, "Unknown", extraMsg);
   }
 
   public void ok(UserGroupInformation user, KMS.KMSOp op, String extraMsg) {
-    op(OpStatus.OK, op, user.getShortUserName(), null, extraMsg);
+    op(OpStatus.OK, op, user, null, "Unknown", extraMsg);
   }
 
   public void unauthorized(UserGroupInformation user, KMS.KMSOp op, String key) {
-    op(OpStatus.UNAUTHORIZED, op, user.getShortUserName(), key, "");
+    op(OpStatus.UNAUTHORIZED, op, user, key, "Unknown", "");
   }
 
   public void error(UserGroupInformation user, String method, String url,
       String extraMsg) {
-    op(OpStatus.ERROR, null, user.getShortUserName(), null, "Method:'" + method
+    op(OpStatus.ERROR, null, user, null, "Unknown", "Method:'" + method
         + "' Exception:'" + extraMsg + "'");
   }
 
   public void unauthenticated(String remoteHost, String method,
       String url, String extraMsg) {
-    op(OpStatus.UNAUTHENTICATED, null, null, null, "RemoteHost:"
+    op(OpStatus.UNAUTHENTICATED, null, null, null, remoteHost, "RemoteHost:"
         + remoteHost + " Method:" + method
         + " URL:" + url + " ErrorMsg:'" + extraMsg + "'");
   }
@@ -227,6 +234,13 @@ public class KMSAudit {
 
   public void shutdown() {
     executor.shutdownNow();
+    for (KMSAuditLogger logger : auditLoggers) {
+      try {
+        logger.cleanup();
+      } catch (Exception ex) {
+        LOG.error("Failed to cleanup logger {}", logger.getClass(), ex);
+      }
+    }
   }
 
   @VisibleForTesting

+ 154 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuditLogger.java

@@ -0,0 +1,154 @@
+/**
+ * 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.kms.server;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * Interface defining a KMS audit logger.
+ * <p>
+ * IMPORTANT WARNING: Audit logs should be strictly backwards-compatible,
+ * because there are usually parsing tools highly dependent on the audit log
+ * formatting. Different tools have different ways of parsing the audit log, so
+ * changing the audit log output in any way is considered incompatible,
+ * and will haunt the consumer tools / developers. Don't do it.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+interface KMSAuditLogger {
+  enum OpStatus {
+    OK, UNAUTHORIZED, UNAUTHENTICATED, ERROR;
+  }
+
+  /**
+   * Class defining an audit event.
+   */
+  class AuditEvent {
+    private final AtomicLong accessCount = new AtomicLong(-1);
+    private final KMS.KMSOp op;
+    private final String keyName;
+    private final String user;
+    private final String impersonator;
+    private final String remoteHost;
+    private final String extraMsg;
+    private final long startTime = System.currentTimeMillis();
+    private long endTime = startTime;
+
+    AuditEvent(KMS.KMSOp op, UserGroupInformation ugi, String keyName,
+        String remoteHost, String msg) {
+      this.keyName = keyName;
+      if (ugi == null) {
+        this.user = null;
+        this.impersonator = null;
+      } else {
+        this.user = ugi.getShortUserName();
+        if (ugi.getAuthenticationMethod()
+            == UserGroupInformation.AuthenticationMethod.PROXY) {
+          this.impersonator = ugi.getRealUser().getUserName();
+        } else {
+          this.impersonator = null;
+        }
+      }
+      this.remoteHost = remoteHost;
+      this.op = op;
+      this.extraMsg = msg;
+    }
+
+    public AtomicLong getAccessCount() {
+      return accessCount;
+    }
+
+    public KMS.KMSOp getOp() {
+      return op;
+    }
+
+    public String getKeyName() {
+      return keyName;
+    }
+
+    public String getUser() {
+      return user;
+    }
+
+    public String getImpersonator() {
+      return impersonator;
+    }
+
+    public String getRemoteHost() {
+      return remoteHost;
+    }
+
+    public String getExtraMsg() {
+      return extraMsg;
+    }
+
+    public long getStartTime() {
+      return startTime;
+    }
+
+    public long getEndTime() {
+      return endTime;
+    }
+
+    /**
+     * Set the time this audit event is finished.
+     */
+    void setEndTime(long endTime) {
+      this.endTime = endTime;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append("op=" + op).append(", keyName=" + keyName)
+          .append(", user=" + user).append(", impersonator=" + impersonator)
+          .append(", remoteHost=" + remoteHost)
+          .append(", extraMsg=" + extraMsg);
+      return sb.toString();
+    }
+  }
+
+  /**
+   * Clean up the audit logger.
+   *
+   * @throws IOException
+   */
+  void cleanup() throws IOException;
+
+  /**
+   * Initialize the audit logger.
+   *
+   * @param conf The configuration object.
+   * @throws IOException
+   */
+  void initialize(Configuration conf) throws IOException;
+
+  /**
+   * Log an audit event.
+   *
+   * @param status The status of the event.
+   * @param event  The audit event.
+   */
+  void logAuditEvent(final OpStatus status, final AuditEvent event);
+}

+ 4 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java

@@ -58,6 +58,10 @@ public class KMSConfiguration {
   // Delay for Audit logs that need aggregation
   public static final String KMS_AUDIT_AGGREGATION_WINDOW = CONFIG_PREFIX +
       "audit.aggregation.window.ms";
+
+  // KMS Audit logger classes to use
+  public static final String KMS_AUDIT_LOGGER_KEY = CONFIG_PREFIX +
+      "audit.logger";
   
   public static final boolean KEY_CACHE_ENABLE_DEFAULT = true;
   // 10 mins

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

@@ -147,10 +147,7 @@ public class KMSWebApp implements ServletContextListener {
       unauthenticatedCallsMeter = metricRegistry.register(
           UNAUTHENTICATED_CALLS_METER, new Meter());
 
-      kmsAudit =
-          new KMSAudit(kmsConf.getLong(
-              KMSConfiguration.KMS_AUDIT_AGGREGATION_WINDOW,
-              KMSConfiguration.KMS_AUDIT_AGGREGATION_WINDOW_DEFAULT));
+      kmsAudit = new KMSAudit(kmsConf);
 
       // this is required for the the JMXJsonServlet to work properly.
       // the JMXJsonServlet is behind the authentication filter,

+ 96 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/SimpleKMSAuditLogger.java

@@ -0,0 +1,96 @@
+/**
+ * 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.kms.server;
+
+import static org.apache.hadoop.crypto.key.kms.server.KMSAudit.KMS_LOGGER_NAME;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Strings;
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A simple text format audit logger. This is the default.
+ * <p>
+ * IMPORTANT WARNING: Audit logs should be strictly backwards-compatible,
+ * because there are usually parsing tools highly dependent on the audit log
+ * formatting. Different tools have different ways of parsing the audit log, so
+ * changing the audit log output in any way is considered incompatible,
+ * and will haunt the consumer tools / developers. Don't do it.
+ */
+class SimpleKMSAuditLogger implements KMSAuditLogger {
+  final private Logger auditLog = LoggerFactory.getLogger(KMS_LOGGER_NAME);
+
+  @Override
+  public void cleanup() throws IOException {
+  }
+
+  @Override
+  public void initialize(Configuration conf) throws IOException {
+  }
+
+  @Override
+  public void logAuditEvent(final OpStatus status, final AuditEvent event) {
+    if (!Strings.isNullOrEmpty(event.getUser()) && !Strings
+        .isNullOrEmpty(event.getKeyName()) && (event.getOp() != null)
+        && KMSAudit.AGGREGATE_OPS_WHITELIST.contains(event.getOp())) {
+      switch (status) {
+      case OK:
+        auditLog.info(
+            "{}[op={}, key={}, user={}, accessCount={}, interval={}ms] {}",
+            status, event.getOp(), event.getKeyName(), event.getUser(),
+            event.getAccessCount().get(),
+            (event.getEndTime() - event.getStartTime()), event.getExtraMsg());
+        break;
+      case UNAUTHORIZED:
+        logAuditSimpleFormat(status, event);
+        break;
+      default:
+        logAuditSimpleFormat(status, event);
+        break;
+      }
+    } else {
+      logAuditSimpleFormat(status, event);
+    }
+  }
+
+  private void logAuditSimpleFormat(final OpStatus status,
+      final AuditEvent event) {
+    final List<String> kvs = new LinkedList<>();
+    if (event.getOp() != null) {
+      kvs.add("op=" + event.getOp());
+    }
+    if (!Strings.isNullOrEmpty(event.getKeyName())) {
+      kvs.add("key=" + event.getKeyName());
+    }
+    if (!Strings.isNullOrEmpty(event.getUser())) {
+      kvs.add("user=" + event.getUser());
+    }
+    if (kvs.isEmpty()) {
+      auditLog.info("{} {}", status, event.getExtraMsg());
+    } else {
+      final String join = Joiner.on(", ").join(kvs);
+      auditLog.info("{}[{}] {}", status, join, event.getExtraMsg());
+    }
+  }
+}

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

@@ -23,10 +23,13 @@ import java.io.InputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.PrintStream;
+import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.kms.server.KMS.KMSOp;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.ThreadUtil;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.PropertyConfigurator;
@@ -37,6 +40,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
 import org.mockito.Mockito;
+import org.mockito.internal.util.reflection.Whitebox;
 
 public class TestKMSAudit {
 
@@ -45,7 +49,7 @@ public class TestKMSAudit {
   private FilterOut filterOut;
   private PrintStream capturedOut;
   
-  private KMSAudit kmsAudit; 
+  private KMSAudit kmsAudit;
 
   private static class FilterOut extends FilterOutputStream {
     public FilterOut(OutputStream out) {
@@ -71,9 +75,8 @@ public class TestKMSAudit {
         ThreadUtil.getResourceAsStream("log4j-kmsaudit.properties");
     PropertyConfigurator.configure(is);
     IOUtils.closeStream(is);
-
-    this.kmsAudit =
-        new KMSAudit(KMSConfiguration.KMS_AUDIT_AGGREGATION_WINDOW_DEFAULT);
+    Configuration conf = new Configuration();
+    this.kmsAudit = new KMSAudit(conf);
   }
 
   @After
@@ -175,4 +178,36 @@ public class TestKMSAudit {
             + "ERROR\\[user=luser\\] Method:'method' Exception:'testmsg'"
             + "UNAUTHENTICATED RemoteHost:remotehost Method:method URL:url ErrorMsg:'testmsg'"));
   }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testInitAuditLoggers() throws Exception {
+    // Default should be the simple logger
+    List<KMSAuditLogger> loggers = (List<KMSAuditLogger>) Whitebox
+        .getInternalState(kmsAudit, "auditLoggers");
+    Assert.assertEquals(1, loggers.size());
+    Assert.assertEquals(SimpleKMSAuditLogger.class, loggers.get(0).getClass());
+
+    // Explicitly configure the simple logger. Duplicates are ignored.
+    final Configuration conf = new Configuration();
+    conf.set(KMSConfiguration.KMS_AUDIT_LOGGER_KEY,
+        SimpleKMSAuditLogger.class.getName() + ", "
+            + SimpleKMSAuditLogger.class.getName());
+    final KMSAudit audit = new KMSAudit(conf);
+    loggers =
+        (List<KMSAuditLogger>) Whitebox.getInternalState(audit, "auditLoggers");
+    Assert.assertEquals(1, loggers.size());
+    Assert.assertEquals(SimpleKMSAuditLogger.class, loggers.get(0).getClass());
+
+    // If any loggers unable to load, init should fail.
+    conf.set(KMSConfiguration.KMS_AUDIT_LOGGER_KEY,
+        SimpleKMSAuditLogger.class.getName() + ",unknown");
+    try {
+      new KMSAudit(conf);
+      Assert.fail("loggers configured but invalid, init should fail.");
+    } catch (Exception ex) {
+      GenericTestUtils
+          .assertExceptionContains(KMSConfiguration.KMS_AUDIT_LOGGER_KEY, ex);
+    }
+  }
 }

+ 4 - 1
hadoop-common-project/hadoop-kms/src/test/resources/log4j-kmsaudit.properties

@@ -16,10 +16,13 @@
 # limitations under the License.
 #
 
+log4j.rootLogger=INFO, kms-audit
+
 # LOG Appender
 log4j.appender.kms-audit=org.apache.log4j.ConsoleAppender
 log4j.appender.kms-audit.Target=System.err
 log4j.appender.kms-audit.layout=org.apache.log4j.PatternLayout
 log4j.appender.kms-audit.layout.ConversionPattern=%m
 
-log4j.rootLogger=INFO, kms-audit
+# disable default logging in KMSAudit class
+log4j.logger.org.apache.hadoop.crypto.key.kms.server.KMSAudit=OFF

+ 21 - 214
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -71,6 +71,7 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
+import org.apache.hadoop.hdfs.util.IOUtilsClient;
 import org.apache.hadoop.io.ByteBufferPool;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
@@ -145,94 +146,6 @@ public class DFSInputStream extends FSInputStream
     return extendedReadBuffers;
   }
 
-  public static class ReadStatistics {
-    public ReadStatistics() {
-      clear();
-    }
-
-    public ReadStatistics(ReadStatistics rhs) {
-      this.totalBytesRead = rhs.getTotalBytesRead();
-      this.totalLocalBytesRead = rhs.getTotalLocalBytesRead();
-      this.totalShortCircuitBytesRead = rhs.getTotalShortCircuitBytesRead();
-      this.totalZeroCopyBytesRead = rhs.getTotalZeroCopyBytesRead();
-    }
-
-    /**
-     * @return The total bytes read.  This will always be at least as
-     * high as the other numbers, since it includes all of them.
-     */
-    public long getTotalBytesRead() {
-      return totalBytesRead;
-    }
-
-    /**
-     * @return The total local bytes read.  This will always be at least
-     * as high as totalShortCircuitBytesRead, since all short-circuit
-     * reads are also local.
-     */
-    public long getTotalLocalBytesRead() {
-      return totalLocalBytesRead;
-    }
-
-    /**
-     * @return The total short-circuit local bytes read.
-     */
-    public long getTotalShortCircuitBytesRead() {
-      return totalShortCircuitBytesRead;
-    }
-
-    /**
-     * @return The total number of zero-copy bytes read.
-     */
-    public long getTotalZeroCopyBytesRead() {
-      return totalZeroCopyBytesRead;
-    }
-
-    /**
-     * @return The total number of bytes read which were not local.
-     */
-    public long getRemoteBytesRead() {
-      return totalBytesRead - totalLocalBytesRead;
-    }
-
-    void addRemoteBytes(long amt) {
-      this.totalBytesRead += amt;
-    }
-
-    void addLocalBytes(long amt) {
-      this.totalBytesRead += amt;
-      this.totalLocalBytesRead += amt;
-    }
-
-    void addShortCircuitBytes(long amt) {
-      this.totalBytesRead += amt;
-      this.totalLocalBytesRead += amt;
-      this.totalShortCircuitBytesRead += amt;
-    }
-
-    void addZeroCopyBytes(long amt) {
-      this.totalBytesRead += amt;
-      this.totalLocalBytesRead += amt;
-      this.totalShortCircuitBytesRead += amt;
-      this.totalZeroCopyBytesRead += amt;
-    }
-
-    void clear() {
-      this.totalBytesRead = 0;
-      this.totalLocalBytesRead = 0;
-      this.totalShortCircuitBytesRead = 0;
-      this.totalZeroCopyBytesRead = 0;
-    }
-
-    private long totalBytesRead;
-
-    private long totalLocalBytesRead;
-
-    private long totalShortCircuitBytesRead;
-
-    private long totalZeroCopyBytesRead;
-  }
-
   /**
    * This variable tracks the number of failures since the start of the
    * most recent user-facing operation. That is to say, it should be reset
@@ -767,116 +680,11 @@ public class DFSInputStream extends FSInputStream
     return ( ret <= 0 ) ? -1 : (oneByteBuf[0] & 0xff);
   }
 
-  /**
-   * Wraps different possible read implementations so that readBuffer can be
-   * strategy-agnostic.
-   */
-  interface ReaderStrategy {
-    int doRead(BlockReader blockReader, int off, int len)
-        throws IOException;
-
-    /**
-     * Copy data from the src ByteBuffer into the read buffer.
-     * @param src The src buffer where the data is copied from
-     * @param offset Useful only when the ReadStrategy is based on a byte array.
-     *               Indicate the offset of the byte array for copy.
-     * @param length Useful only when the ReadStrategy is based on a byte array.
-     *               Indicate the length of the data to copy.
-     */
-    int copyFrom(ByteBuffer src, int offset, int length);
-  }
-
-  protected void updateReadStatistics(ReadStatistics readStatistics,
-        int nRead, BlockReader blockReader) {
-    if (nRead <= 0) return;
-    synchronized(infoLock) {
-      if (blockReader.isShortCircuit()) {
-        readStatistics.addShortCircuitBytes(nRead);
-      } else if (blockReader.getNetworkDistance() == 0) {
-        readStatistics.addLocalBytes(nRead);
-      } else {
-        readStatistics.addRemoteBytes(nRead);
-      }
-    }
-  }
-
-  /**
-   * Used to read bytes into a byte[]
-   */
-  private class ByteArrayStrategy implements ReaderStrategy {
-    final byte[] buf;
-
-    public ByteArrayStrategy(byte[] buf) {
-      this.buf = buf;
-    }
-
-    @Override
-    public int doRead(BlockReader blockReader, int off, int len)
-        throws IOException {
-      int nRead = blockReader.read(buf, off, len);
-      updateReadStatistics(readStatistics, nRead, blockReader);
-      dfsClient.updateFileSystemReadStats(blockReader.getNetworkDistance(),
-          nRead);
-      return nRead;
-    }
-
-    @Override
-    public int copyFrom(ByteBuffer src, int offset, int length) {
-      ByteBuffer writeSlice = src.duplicate();
-      writeSlice.get(buf, offset, length);
-      return length;
-    }
-  }
-
-  /**
-   * Used to read bytes into a user-supplied ByteBuffer
-   */
-  protected class ByteBufferStrategy implements ReaderStrategy {
-    final ByteBuffer buf;
-    ByteBufferStrategy(ByteBuffer buf) {
-      this.buf = buf;
-    }
-
-    @Override
-    public int doRead(BlockReader blockReader, int off, int len)
-        throws IOException {
-      int oldpos = buf.position();
-      int oldlimit = buf.limit();
-      boolean success = false;
-      try {
-        int ret = blockReader.read(buf);
-        success = true;
-        updateReadStatistics(readStatistics, ret, blockReader);
-        dfsClient.updateFileSystemReadStats(blockReader.getNetworkDistance(),
-            ret);
-        if (ret == 0) {
-          DFSClient.LOG.warn("zero");
-        }
-        return ret;
-      } finally {
-        if (!success) {
-          // Reset to original state so that retries work correctly.
-          buf.position(oldpos);
-          buf.limit(oldlimit);
-        }
-      }
-    }
-
-    @Override
-    public int copyFrom(ByteBuffer src, int offset, int length) {
-      ByteBuffer writeSlice = src.duplicate();
-      int remaining = Math.min(buf.remaining(), writeSlice.remaining());
-      writeSlice.limit(writeSlice.position() + remaining);
-      buf.put(writeSlice);
-      return remaining;
-    }
-  }
-
   /* This is a used by regular read() and handles ChecksumExceptions.
    * name readBuffer() is chosen to imply similarity to readBuffer() in
    * ChecksumFileSystem
    */
-  private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
+  private synchronized int readBuffer(ReaderStrategy reader, int len,
                                       CorruptedBlocks corruptedBlocks)
       throws IOException {
     IOException ioe;
@@ -892,7 +700,7 @@ public class DFSInputStream extends FSInputStream
     while (true) {
       // retry as many times as seekToNewSource allows.
       try {
-        return reader.doRead(blockReader, off, len);
+        return reader.readFromBlock(blockReader, len);
       } catch ( ChecksumException ce ) {
         DFSClient.LOG.warn("Found Checksum error for "
             + getCurrentBlock() + " from " + currentNode
@@ -927,13 +735,14 @@ public class DFSInputStream extends FSInputStream
     }
   }
 
-  protected synchronized int readWithStrategy(ReaderStrategy strategy, int off,
-      int len) throws IOException {
+  protected synchronized int readWithStrategy(ReaderStrategy strategy)
+      throws IOException {
     dfsClient.checkOpen();
     if (closed.get()) {
       throw new IOException("Stream closed");
     }
 
+    int len = strategy.getTargetLength();
     CorruptedBlocks corruptedBlocks = new CorruptedBlocks();
     failures = 0;
     if (pos < getFileLength()) {
@@ -952,7 +761,7 @@ public class DFSInputStream extends FSInputStream
                   locatedBlocks.getFileLength() - pos);
             }
           }
-          int result = readBuffer(strategy, off, realLen, corruptedBlocks);
+          int result = readBuffer(strategy, realLen, corruptedBlocks);
 
           if (result >= 0) {
             pos += result;
@@ -994,11 +803,12 @@ public class DFSInputStream extends FSInputStream
     if (len == 0) {
       return 0;
     }
-    ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
+    ReaderStrategy byteArrayReader =
+        new ByteArrayStrategy(buf, off, len, readStatistics, dfsClient);
     try (TraceScope scope =
              dfsClient.newReaderTraceScope("DFSInputStream#byteArrayRead",
                  src, getPos(), len)) {
-      int retLen = readWithStrategy(byteArrayReader, off, len);
+      int retLen = readWithStrategy(byteArrayReader);
       if (retLen < len) {
         dfsClient.addRetLenToReaderScope(scope, retLen);
       }
@@ -1008,12 +818,13 @@ public class DFSInputStream extends FSInputStream
 
   @Override
   public synchronized int read(final ByteBuffer buf) throws IOException {
-    ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
+    ReaderStrategy byteBufferReader =
+        new ByteBufferStrategy(buf, readStatistics, dfsClient);
     int reqLen = buf.remaining();
     try (TraceScope scope =
              dfsClient.newReaderTraceScope("DFSInputStream#byteBufferRead",
                  src, getPos(), reqLen)){
-      int retLen = readWithStrategy(byteBufferReader, 0, reqLen);
+      int retLen = readWithStrategy(byteBufferReader);
       if (retLen < reqLen) {
         dfsClient.addRetLenToReaderScope(scope, retLen);
       }
@@ -1221,7 +1032,7 @@ public class DFSInputStream extends FSInputStream
         reader = getBlockReader(block, startInBlk, len, datanode.addr,
             datanode.storageType, datanode.info);
         int nread = reader.readAll(buf, offset, len);
-        updateReadStatistics(readStatistics, nread, reader);
+        IOUtilsClient.updateReadStatistics(readStatistics, nread, reader);
         dfsClient.updateFileSystemReadStats(
             reader.getNetworkDistance(), nread);
         if (nread != len) {
@@ -1721,18 +1532,14 @@ public class DFSInputStream extends FSInputStream
    * Get statistics about the reads which this DFSInputStream has done.
    */
   public ReadStatistics getReadStatistics() {
-    synchronized(infoLock) {
-      return new ReadStatistics(readStatistics);
-    }
+    return new ReadStatistics(readStatistics);
   }
 
   /**
    * Clear statistics about the reads which this DFSInputStream has done.
    */
   public void clearReadStatistics() {
-    synchronized(infoLock) {
-      readStatistics.clear();
-    }
+    readStatistics.clear();
   }
 
   public FileEncryptionInfo getFileEncryptionInfo() {
@@ -1759,7 +1566,8 @@ public class DFSInputStream extends FSInputStream
       throws IOException {
     synchronized (infoLock) {
       this.cachingStrategy =
-          new CachingStrategy.Builder(this.cachingStrategy).setReadahead(readahead).build();
+          new CachingStrategy.Builder(this.cachingStrategy).
+              setReadahead(readahead).build();
     }
     closeCurrentBlockReaders();
   }
@@ -1769,7 +1577,8 @@ public class DFSInputStream extends FSInputStream
       throws IOException {
     synchronized (infoLock) {
       this.cachingStrategy =
-          new CachingStrategy.Builder(this.cachingStrategy).setDropBehind(dropBehind).build();
+          new CachingStrategy.Builder(this.cachingStrategy).
+              setDropBehind(dropBehind).build();
     }
     closeCurrentBlockReaders();
   }
@@ -1883,9 +1692,7 @@ public class DFSInputStream extends FSInputStream
       buffer.position((int)blockPos);
       buffer.limit((int)(blockPos + length));
       getExtendedReadBuffers().put(buffer, clientMmap);
-      synchronized (infoLock) {
-        readStatistics.addZeroCopyBytes(length);
-      }
+      readStatistics.addZeroCopyBytes(length);
       DFSClient.LOG.debug("readZeroCopy read {} bytes from offset {} via the "
           + "zero-copy read path.  blockEnd = {}", length, curPos, blockEnd);
       success = true;

+ 12 - 11
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java

@@ -359,11 +359,11 @@ public class DFSStripedInputStream extends DFSInputStream {
       ExtendedBlock currentBlock,
       CorruptedBlocks corruptedBlocks)
       throws IOException {
-    final int targetLength = strategy.buf.remaining();
+    final int targetLength = strategy.getTargetLength();
     int length = 0;
     try {
       while (length < targetLength) {
-        int ret = strategy.doRead(blockReader, 0, 0);
+        int ret = strategy.readFromBlock(blockReader);
         if (ret < 0) {
           throw new IOException("Unexpected EOS from the reader");
         }
@@ -425,13 +425,14 @@ public class DFSStripedInputStream extends DFSInputStream {
   }
 
   @Override
-  protected synchronized int readWithStrategy(ReaderStrategy strategy,
-      int off, int len) throws IOException {
+  protected synchronized int readWithStrategy(ReaderStrategy strategy)
+      throws IOException {
     dfsClient.checkOpen();
     if (closed.get()) {
       throw new IOException("Stream closed");
     }
 
+    int len = strategy.getTargetLength();
     CorruptedBlocks corruptedBlocks = new CorruptedBlocks();
     if (pos < getFileLength()) {
       try {
@@ -452,7 +453,7 @@ public class DFSStripedInputStream extends DFSInputStream {
           if (!curStripeRange.include(getOffsetInBlockGroup())) {
             readOneStripe(corruptedBlocks);
           }
-          int ret = copyToTargetBuf(strategy, off + result, realLen - result);
+          int ret = copyToTargetBuf(strategy, realLen - result);
           result += ret;
           pos += ret;
         }
@@ -470,16 +471,14 @@ public class DFSStripedInputStream extends DFSInputStream {
   /**
    * Copy the data from {@link #curStripeBuf} into the given buffer
    * @param strategy the ReaderStrategy containing the given buffer
-   * @param offset the offset of the given buffer. Used only when strategy is
-   *               a ByteArrayStrategy
    * @param length target length
    * @return number of bytes copied
    */
-  private int copyToTargetBuf(ReaderStrategy strategy, int offset, int length) {
+  private int copyToTargetBuf(ReaderStrategy strategy, int length) {
     final long offsetInBlk = getOffsetInBlockGroup();
     int bufOffset = getStripedBufOffset(offsetInBlk);
     curStripeBuf.position(bufOffset);
-    return strategy.copyFrom(curStripeBuf, offset,
+    return strategy.readFromBuffer(curStripeBuf,
         Math.min(length, curStripeBuf.remaining()));
   }
 
@@ -700,7 +699,8 @@ public class DFSStripedInputStream extends DFSInputStream {
 
     private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) {
       if (chunk.byteBuffer != null) {
-        ByteBufferStrategy strategy = new ByteBufferStrategy(chunk.byteBuffer);
+        ByteBufferStrategy strategy =
+            new ByteBufferStrategy(chunk.byteBuffer, readStatistics, dfsClient);
         return new ByteBufferStrategy[]{strategy};
       } else {
         ByteBufferStrategy[] strategies =
@@ -708,7 +708,8 @@ public class DFSStripedInputStream extends DFSInputStream {
         for (int i = 0; i < strategies.length; i++) {
           ByteBuffer buffer = ByteBuffer.wrap(chunk.byteArray.buf(),
               chunk.byteArray.getOffsets()[i], chunk.byteArray.getLengths()[i]);
-          strategies[i] = new ByteBufferStrategy(buffer);
+          strategies[i] =
+              new ByteBufferStrategy(buffer, readStatistics, dfsClient);
         }
         return strategies;
       }

+ 106 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ReadStatistics.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;
+
+/**
+ * A utility class that maintains statistics for reading.
+ */
+public class ReadStatistics {
+  private long totalBytesRead;
+  private long totalLocalBytesRead;
+  private long totalShortCircuitBytesRead;
+  private long totalZeroCopyBytesRead;
+
+  public ReadStatistics() {
+    clear();
+  }
+
+  public ReadStatistics(ReadStatistics rhs) {
+    this.totalBytesRead = rhs.getTotalBytesRead();
+    this.totalLocalBytesRead = rhs.getTotalLocalBytesRead();
+    this.totalShortCircuitBytesRead = rhs.getTotalShortCircuitBytesRead();
+    this.totalZeroCopyBytesRead = rhs.getTotalZeroCopyBytesRead();
+  }
+
+  /**
+   * @return The total bytes read.  This will always be at least as
+   * high as the other numbers, since it includes all of them.
+   */
+  public synchronized long getTotalBytesRead() {
+    return totalBytesRead;
+  }
+
+  /**
+   * @return The total local bytes read.  This will always be at least
+   * as high as totalShortCircuitBytesRead, since all short-circuit
+   * reads are also local.
+   */
+  public synchronized long getTotalLocalBytesRead() {
+    return totalLocalBytesRead;
+  }
+
+  /**
+   * @return The total short-circuit local bytes read.
+   */
+  public synchronized long getTotalShortCircuitBytesRead() {
+    return totalShortCircuitBytesRead;
+  }
+
+  /**
+   * @return The total number of zero-copy bytes read.
+   */
+  public synchronized long getTotalZeroCopyBytesRead() {
+    return totalZeroCopyBytesRead;
+  }
+
+  /**
+   * @return The total number of bytes read which were not local.
+   */
+  public synchronized long getRemoteBytesRead() {
+    return totalBytesRead - totalLocalBytesRead;
+  }
+
+  public synchronized void addRemoteBytes(long amt) {
+    this.totalBytesRead += amt;
+  }
+
+  public synchronized void addLocalBytes(long amt) {
+    this.totalBytesRead += amt;
+    this.totalLocalBytesRead += amt;
+  }
+
+  public synchronized void addShortCircuitBytes(long amt) {
+    this.totalBytesRead += amt;
+    this.totalLocalBytesRead += amt;
+    this.totalShortCircuitBytesRead += amt;
+  }
+
+  public synchronized void addZeroCopyBytes(long amt) {
+    this.totalBytesRead += amt;
+    this.totalLocalBytesRead += amt;
+    this.totalShortCircuitBytesRead += amt;
+    this.totalZeroCopyBytesRead += amt;
+  }
+
+  public synchronized void clear() {
+    this.totalBytesRead = 0;
+    this.totalLocalBytesRead = 0;
+    this.totalShortCircuitBytesRead = 0;
+    this.totalZeroCopyBytesRead = 0;
+  }
+}

+ 215 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ReaderStrategy.java

@@ -0,0 +1,215 @@
+/**
+ * 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 java.io.IOException;
+import java.nio.ByteBuffer;
+import static org.apache.hadoop.hdfs.util.IOUtilsClient.updateReadStatistics;
+
+/**
+ * Wraps different possible read implementations so that callers can be
+ * strategy-agnostic.
+ */
+interface ReaderStrategy {
+  /**
+   * Read from a block using the blockReader.
+   * @param blockReader
+   * @return number of bytes read
+   * @throws IOException
+   */
+  int readFromBlock(BlockReader blockReader) throws IOException;
+
+  /**
+   * Read from a block using the blockReader with desired length to read.
+   * @param blockReader
+   * @param length number of bytes desired to read, not ensured
+   * @return number of bytes read
+   * @throws IOException
+   */
+  int readFromBlock(BlockReader blockReader, int length) throws IOException;
+
+  /**
+   * Read or copy from a src buffer.
+   * @param src
+   * @return number of bytes copied
+   * Note: the position of the src buffer is not changed after the call
+   */
+  int readFromBuffer(ByteBuffer src);
+
+  /**
+   * Read or copy length of data bytes from a src buffer with desired length.
+   * @param src
+   * @return number of bytes copied
+   * Note: the position of the src buffer is not changed after the call
+   */
+  int readFromBuffer(ByteBuffer src, int length);
+
+  /**
+   * @return the target read buffer that reads data into.
+   */
+  ByteBuffer getReadBuffer();
+
+  /**
+   * @return the target length to read.
+   */
+  int getTargetLength();
+}
+
+/**
+ * Used to read bytes into a byte array buffer. Note it's not thread-safe
+ * and the behavior is not defined if concurrently operated.
+ */
+class ByteArrayStrategy implements ReaderStrategy {
+  private final DFSClient dfsClient;
+  private final ReadStatistics readStatistics;
+  private final byte[] readBuf;
+  private int offset;
+  private final int targetLength;
+
+  /**
+   * The constructor.
+   * @param readBuf target buffer to read into
+   * @param offset offset into the buffer
+   * @param targetLength target length of data
+   * @param readStatistics statistics counter
+   */
+  public ByteArrayStrategy(byte[] readBuf, int offset, int targetLength,
+                           ReadStatistics readStatistics,
+                           DFSClient dfsClient) {
+    this.readBuf = readBuf;
+    this.offset = offset;
+    this.targetLength = targetLength;
+    this.readStatistics = readStatistics;
+    this.dfsClient = dfsClient;
+  }
+
+  @Override
+  public ByteBuffer getReadBuffer() {
+    return ByteBuffer.wrap(readBuf, offset, targetLength);
+  }
+
+  @Override
+  public int getTargetLength() {
+    return targetLength;
+  }
+
+  @Override
+  public int readFromBlock(BlockReader blockReader) throws IOException {
+    return readFromBlock(blockReader, targetLength);
+  }
+
+  @Override
+  public int readFromBlock(BlockReader blockReader,
+                           int length) throws IOException {
+    int nRead = blockReader.read(readBuf, offset, length);
+    if (nRead > 0) {
+      updateReadStatistics(readStatistics, nRead, blockReader);
+      dfsClient.updateFileSystemReadStats(blockReader.getNetworkDistance(),
+          nRead);
+      offset += nRead;
+    }
+    return nRead;
+  }
+
+  @Override
+  public int readFromBuffer(ByteBuffer src) {
+    return readFromBuffer(src, src.remaining());
+  }
+
+  @Override
+  public int readFromBuffer(ByteBuffer src, int length) {
+    ByteBuffer dup = src.duplicate();
+    dup.get(readBuf, offset, length);
+    offset += length;
+    return length;
+  }
+}
+
+/**
+ * Used to read bytes into a user-supplied ByteBuffer. Note it's not thread-safe
+ * and the behavior is not defined if concurrently operated. When read operation
+ * is performed, the position of the underlying byte buffer will move forward as
+ * stated in ByteBufferReadable#read(ByteBuffer buf) method.
+ */
+class ByteBufferStrategy implements ReaderStrategy {
+  private final DFSClient dfsClient;
+  private final ReadStatistics readStatistics;
+  private final ByteBuffer readBuf;
+  private final int targetLength;
+
+  /**
+   * The constructor.
+   * @param readBuf target buffer to read into
+   * @param readStatistics statistics counter
+   */
+  ByteBufferStrategy(ByteBuffer readBuf,
+                     ReadStatistics readStatistics,
+                     DFSClient dfsClient) {
+    this.readBuf = readBuf;
+    this.targetLength = readBuf.remaining();
+    this.readStatistics = readStatistics;
+    this.dfsClient = dfsClient;
+  }
+
+  @Override
+  public ByteBuffer getReadBuffer() {
+    return readBuf;
+  }
+
+  @Override
+  public int readFromBlock(BlockReader blockReader) throws IOException {
+    return readFromBlock(blockReader, readBuf.remaining());
+  }
+
+  @Override
+  public int readFromBlock(BlockReader blockReader,
+                           int length) throws IOException {
+    ByteBuffer tmpBuf = readBuf.duplicate();
+    tmpBuf.limit(tmpBuf.position() + length);
+    int nRead = blockReader.read(tmpBuf);
+    // Only when data are read, update the position
+    if (nRead > 0) {
+      readBuf.position(readBuf.position() + nRead);
+      updateReadStatistics(readStatistics, nRead, blockReader);
+      dfsClient.updateFileSystemReadStats(blockReader.getNetworkDistance(),
+          nRead);
+    }
+
+    return nRead;
+  }
+
+  @Override
+  public int getTargetLength() {
+    return targetLength;
+  }
+
+  @Override
+  public int readFromBuffer(ByteBuffer src) {
+    return readFromBuffer(src, src.remaining());
+  }
+
+  @Override
+  public int readFromBuffer(ByteBuffer src, int length) {
+    ByteBuffer dup = src.duplicate();
+    int newLen = Math.min(readBuf.remaining(), dup.remaining());
+    newLen = Math.min(newLen, length);
+    dup.limit(dup.position() + newLen);
+    readBuf.put(dup);
+    return newLen;
+  }
+}

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

@@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.crypto.CryptoInputStream;
 import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.ReadStatistics;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -103,7 +104,7 @@ public class HdfsDataInputStream extends FSDataInputStream {
    * be higher than you would expect just by adding up the number of
    * bytes read through HdfsDataInputStream.
    */
-  public DFSInputStream.ReadStatistics getReadStatistics() {
+  public ReadStatistics getReadStatistics() {
     return getDFSInputStream().getReadStatistics();
   }
 

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeAdminProperties.java

@@ -33,6 +33,7 @@ public class DatanodeAdminProperties {
   private int port;
   private String upgradeDomain;
   private AdminStates adminState = AdminStates.NORMAL;
+  private long maintenanceExpireTimeInMS = Long.MAX_VALUE;
 
   /**
    * Return the host name of the datanode.
@@ -97,4 +98,22 @@ public class DatanodeAdminProperties {
   public void setAdminState(final AdminStates adminState) {
     this.adminState = adminState;
   }
+
+  /**
+   * Get the maintenance expiration time in milliseconds.
+   * @return the maintenance expiration time in milliseconds.
+   */
+  public long getMaintenanceExpireTimeInMS() {
+    return this.maintenanceExpireTimeInMS;
+  }
+
+  /**
+   * Get the maintenance expiration time in milliseconds.
+   * @param maintenanceExpireTimeInMS
+   *        the maintenance expiration time in milliseconds.
+   */
+  public void setMaintenanceExpireTimeInMS(
+      final long maintenanceExpireTimeInMS) {
+    this.maintenanceExpireTimeInMS = maintenanceExpireTimeInMS;
+  }
 }

+ 23 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -83,6 +83,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   }
 
   protected AdminStates adminState;
+  private long maintenanceExpireTimeInMS;
 
   public DatanodeInfo(DatanodeInfo from) {
     super(from);
@@ -499,17 +500,28 @@ public class DatanodeInfo extends DatanodeID implements Node {
   }
 
   /**
-   * Put a node to maintenance mode.
+   * Start the maintenance operation.
    */
   public void startMaintenance() {
-    adminState = AdminStates.ENTERING_MAINTENANCE;
+    this.adminState = AdminStates.ENTERING_MAINTENANCE;
   }
 
   /**
-   * Put a node to maintenance mode.
+   * Put a node directly to maintenance mode.
    */
   public void setInMaintenance() {
-    adminState = AdminStates.IN_MAINTENANCE;
+    this.adminState = AdminStates.IN_MAINTENANCE;
+  }
+
+  /**
+  * @param maintenanceExpireTimeInMS the time that the DataNode is in the
+  *        maintenance mode until in the unit of milliseconds.   */
+  public void setMaintenanceExpireTimeInMS(long maintenanceExpireTimeInMS) {
+    this.maintenanceExpireTimeInMS = maintenanceExpireTimeInMS;
+  }
+
+  public long getMaintenanceExpireTimeInMS() {
+    return this.maintenanceExpireTimeInMS;
   }
 
   /**
@@ -519,6 +531,9 @@ public class DatanodeInfo extends DatanodeID implements Node {
     adminState = null;
   }
 
+  public static boolean maintenanceNotExpired(long maintenanceExpireTimeInMS) {
+    return Time.monotonicNow() < maintenanceExpireTimeInMS;
+  }
   /**
    * Returns true if the node is is entering_maintenance
    */
@@ -541,6 +556,10 @@ public class DatanodeInfo extends DatanodeID implements Node {
         adminState == AdminStates.IN_MAINTENANCE);
   }
 
+  public boolean maintenanceExpired() {
+    return !maintenanceNotExpired(this.maintenanceExpireTimeInMS);
+  }
+
   public boolean isInService() {
     return getAdminState() == AdminStates.NORMAL;
   }

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

@@ -141,7 +141,7 @@ public final class HdfsConstants {
 
   // type of the datanode report
   public enum DatanodeReportType {
-    ALL, LIVE, DEAD, DECOMMISSIONING
+    ALL, LIVE, DEAD, DECOMMISSIONING, ENTERING_MAINTENANCE
   }
 
   public static final byte RS_6_3_POLICY_ID = 0;

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -523,16 +523,21 @@ public class ClientNamenodeProtocolTranslatorPB implements
   public void rename2(String src, String dst, Rename... options)
       throws IOException {
     boolean overwrite = false;
+    boolean toTrash = false;
     if (options != null) {
       for (Rename option : options) {
         if (option == Rename.OVERWRITE) {
           overwrite = true;
+        } else if (option == Rename.TO_TRASH) {
+          toTrash = true;
         }
       }
     }
     Rename2RequestProto req = Rename2RequestProto.newBuilder().
         setSrc(src).
-        setDst(dst).setOverwriteDest(overwrite).
+        setDst(dst).
+        setOverwriteDest(overwrite).
+        setMoveToTrash(toTrash).
         build();
     try {
       if (Client.isAsynchronousMode()) {

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -1466,6 +1466,10 @@ public class PBHelperClient {
     builder.length(cs.getLength()).
         fileCount(cs.getFileCount()).
         directoryCount(cs.getDirectoryCount()).
+        snapshotLength(cs.getSnapshotLength()).
+        snapshotFileCount(cs.getSnapshotFileCount()).
+        snapshotDirectoryCount(cs.getSnapshotDirectoryCount()).
+        snapshotSpaceConsumed(cs.getSnapshotSpaceConsumed()).
         quota(cs.getQuota()).
         spaceConsumed(cs.getSpaceConsumed()).
         spaceQuota(cs.getSpaceQuota());
@@ -2069,6 +2073,10 @@ public class PBHelperClient {
     builder.setLength(cs.getLength()).
         setFileCount(cs.getFileCount()).
         setDirectoryCount(cs.getDirectoryCount()).
+        setSnapshotLength(cs.getSnapshotLength()).
+        setSnapshotFileCount(cs.getSnapshotFileCount()).
+        setSnapshotDirectoryCount(cs.getSnapshotDirectoryCount()).
+        setSnapshotSpaceConsumed(cs.getSnapshotSpaceConsumed()).
         setQuota(cs.getQuota()).
         setSpaceConsumed(cs.getSpaceConsumed()).
         setSpaceQuota(cs.getSpaceQuota());

+ 16 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/IOUtilsClient.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.util;
 
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.ReadStatistics;
 import org.slf4j.Logger;
 
 import java.io.IOException;
@@ -43,4 +45,18 @@ public class IOUtilsClient {
     }
   }
 
+  public static void updateReadStatistics(ReadStatistics readStatistics,
+                                      int nRead, BlockReader blockReader) {
+    if (nRead <= 0) {
+      return;
+    }
+
+    if (blockReader.isShortCircuit()) {
+      readStatistics.addShortCircuitBytes(nRead);
+    } else if (blockReader.getNetworkDistance() == 0) {
+      readStatistics.addLocalBytes(nRead);
+    } else {
+      readStatistics.addRemoteBytes(nRead);
+    }
+  }
 }

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

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.web;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.hadoop.fs.ContentSummary;
@@ -33,6 +34,7 @@ import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -141,6 +143,25 @@ class JsonUtilClient {
         storagePolicy, null);
   }
 
+  static DirectoryListing toDirectoryListing(final Map<?, ?> json) {
+    if (json == null) {
+      return null;
+    }
+    final List<?> list = JsonUtilClient.getList(json,
+        "partialListing");
+
+    HdfsFileStatus[] partialListing = new HdfsFileStatus[list.size()];
+    int i = 0;
+    for (Object o : list) {
+      final Map<?, ?> m = (Map<?, ?>) o;
+      partialListing[i++] = toFileStatus(m, false);
+    }
+    int remainingEntries = getInt(json, "remainingEntries", -1);
+    Preconditions.checkState(remainingEntries != -1,
+        "remainingEntries was not set");
+    return new DirectoryListing(partialListing, remainingEntries);
+  }
+
   /** Convert a Json map to an ExtendedBlock object. */
   static ExtendedBlock toExtendedBlock(final Map<?, ?> m) {
     if (m == null) {

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

@@ -64,6 +64,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.hdfs.DFSOpsCountStatistics;
 import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
@@ -79,6 +80,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtilClient;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -106,6 +108,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
@@ -1499,6 +1502,58 @@ public class WebHdfsFileSystem extends FileSystem
     }.run();
   }
 
+  private static final byte[] EMPTY_ARRAY = new byte[] {};
+  private class DirListingIterator<T extends FileStatus> implements
+      RemoteIterator<T> {
+
+    private final Path path;
+    private DirectoryListing thisListing;
+    private int i = 0;
+    private byte[] prevKey = EMPTY_ARRAY;
+
+    DirListingIterator(Path path) {
+      this.path = path;
+    }
+
+    @Override
+    public boolean hasNext() throws IOException {
+      if (thisListing == null) {
+        fetchMore();
+      }
+      return i < thisListing.getPartialListing().length ||
+          thisListing.hasMore();
+    }
+
+    private void fetchMore() throws IOException {
+      thisListing = new FsPathResponseRunner<DirectoryListing>(
+          GetOpParam.Op.LISTSTATUS_BATCH,
+          path, new StartAfterParam(new String(prevKey, Charsets.UTF_8))) {
+        @Override
+        DirectoryListing decodeResponse(Map<?, ?> json) throws IOException {
+          return JsonUtilClient.toDirectoryListing(json);
+        }
+      }.run();
+      i = 0;
+      prevKey = thisListing.getLastName();
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public T next() throws IOException {
+      Preconditions.checkState(hasNext(), "No more items in iterator");
+      if (i == thisListing.getPartialListing().length) {
+        fetchMore();
+      }
+      return (T)makeQualified(thisListing.getPartialListing()[i++], path);
+    }
+  }
+
+  @Override
+  public RemoteIterator<FileStatus> listStatusIterator(final Path f)
+      throws FileNotFoundException, IOException {
+    return new DirListingIterator<>(f);
+  }
+
   @Override
   public Token<DelegationTokenIdentifier> getDelegationToken(
       final String renewer) throws IOException {

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

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

+ 38 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/StartAfterParam.java

@@ -0,0 +1,38 @@
+/**
+ * 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.web.resources;
+
+/**
+ * Used during batched ListStatus operations.
+ */
+public class StartAfterParam extends StringParam {
+
+  public static final String NAME = "startafter";
+  public static final String DEFAULT = "";
+
+  private static final Domain DOMAIN = new Domain(NAME, null);
+
+  public StartAfterParam(final String str) {
+    super(DOMAIN, str != null && !str.equals(DEFAULT) ? str : null);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

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

@@ -245,6 +245,7 @@ message Rename2RequestProto {
   required string src = 1;
   required string dst = 2;
   required bool overwriteDest = 3;
+  optional bool moveToTrash = 4;
 }
 
 message Rename2ResponseProto { // void response

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

@@ -139,6 +139,10 @@ message ContentSummaryProto {
   required uint64 spaceConsumed = 5;
   required uint64 spaceQuota = 6;
   optional StorageTypeQuotaInfosProto typeQuotaInfos = 7;
+  optional uint64 snapshotLength = 8;
+  optional uint64 snapshotFileCount = 9;
+  optional uint64 snapshotDirectoryCount = 10;
+  optional uint64 snapshotSpaceConsumed = 11;
 }
 
 /**

Fichier diff supprimé car celui-ci est trop grand
+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.7.2.xml


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

@@ -407,6 +407,15 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long
       DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT = 25;
 
+  // Threshold for how long namenode locks must be held for the
+  // event to be logged
+  public static final String  DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY =
+      "dfs.namenode.write-lock-reporting-threshold-ms";
+  public static final long    DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT = 1000L;
+  public static final String  DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY =
+      "dfs.namenode.read-lock-reporting-threshold-ms";
+  public static final long    DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_DEFAULT = 5000L;
+
   public static final String  DFS_UPGRADE_DOMAIN_FACTOR = "dfs.namenode.upgrade.domain.factor";
   public static final int DFS_UPGRADE_DOMAIN_FACTOR_DEFAULT = DFS_REPLICATION_DEFAULT;
 
@@ -482,6 +491,15 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_MOVER_MOVERTHREADS_DEFAULT = 1000;
   public static final String  DFS_MOVER_RETRY_MAX_ATTEMPTS_KEY = "dfs.mover.retry.max.attempts";
   public static final int     DFS_MOVER_RETRY_MAX_ATTEMPTS_DEFAULT = 10;
+  public static final String  DFS_MOVER_KEYTAB_ENABLED_KEY =
+      "dfs.mover.keytab.enabled";
+  public static final boolean DFS_MOVER_KEYTAB_ENABLED_DEFAULT = false;
+  public static final String  DFS_MOVER_ADDRESS_KEY = "dfs.mover.address";
+  public static final String  DFS_MOVER_ADDRESS_DEFAULT= "0.0.0.0:0";
+  public static final String  DFS_MOVER_KEYTAB_FILE_KEY =
+      "dfs.mover.keytab.file";
+  public static final String  DFS_MOVER_KERBEROS_PRINCIPAL_KEY =
+      "dfs.mover.kerberos.principal";
 
   public static final String  DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
   public static final int     DFS_DATANODE_DEFAULT_PORT = 9866;

+ 13 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.protocolPB;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
 
@@ -602,10 +603,21 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   @Override
   public Rename2ResponseProto rename2(RpcController controller,
       Rename2RequestProto req) throws ServiceException {
+    // resolve rename options
+    ArrayList<Rename> optionList = new ArrayList<Rename>();
+    if(req.getOverwriteDest()) {
+      optionList.add(Rename.OVERWRITE);
+    } else if(req.hasMoveToTrash()) {
+      optionList.add(Rename.TO_TRASH);
+    }
+
+    if(optionList.isEmpty()) {
+      optionList.add(Rename.NONE);
+    }
 
     try {
       server.rename2(req.getSrc(), req.getDst(), 
-          req.getOverwriteDest() ? Rename.OVERWRITE : Rename.NONE);
+          optionList.toArray(new Rename[optionList.size()]));
     } catch (IOException e) {
       throw new ServiceException(e);
     }   

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java

@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URL;
+import java.net.UnknownHostException;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -387,8 +388,12 @@ public class QuorumJournalManager implements JournalManager {
     
     List<InetSocketAddress> addrs = Lists.newArrayList();
     for (String addr : parts) {
-      addrs.add(NetUtils.createSocketAddr(
-          addr, DFSConfigKeys.DFS_JOURNALNODE_RPC_PORT_DEFAULT));
+      InetSocketAddress isa = NetUtils.createSocketAddr(
+          addr, DFSConfigKeys.DFS_JOURNALNODE_RPC_PORT_DEFAULT);
+      if (isa.isUnresolved()) {
+        throw new UnknownHostException(addr);
+      }
+      addrs.add(isa);
     }
     return addrs;
   }

+ 23 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CombinedHostFileManager.java

@@ -148,6 +148,24 @@ public class CombinedHostFileManager extends HostConfigManager {
       };
     }
 
+    synchronized long getMaintenanceExpireTimeInMS(
+        final InetSocketAddress address) {
+      Iterable<DatanodeAdminProperties> datanode = Iterables.filter(
+          allDNs.get(address.getAddress()),
+          new Predicate<DatanodeAdminProperties>() {
+            public boolean apply(DatanodeAdminProperties input) {
+              return input.getAdminState().equals(
+                  AdminStates.IN_MAINTENANCE) &&
+                  (input.getPort() == 0 ||
+                  input.getPort() == address.getPort());
+            }
+          });
+      // if DN isn't set to maintenance state, ignore MaintenanceExpireTimeInMS
+      // set in the config.
+      return datanode.iterator().hasNext() ?
+          datanode.iterator().next().getMaintenanceExpireTimeInMS() : 0;
+    }
+
     static class HostIterator extends UnmodifiableIterator<InetSocketAddress> {
       private final Iterator<Map.Entry<InetAddress,
           DatanodeAdminProperties>> it;
@@ -236,6 +254,11 @@ public class CombinedHostFileManager extends HostConfigManager {
     return hostProperties.getUpgradeDomain(dn.getResolvedAddress());
   }
 
+  @Override
+  public long getMaintenanceExpirationTimeInMS(DatanodeID dn) {
+    return hostProperties.getMaintenanceExpireTimeInMS(dn.getResolvedAddress());
+  }
+
   /**
    * Set the properties lists by the new instances. The
    * old instance is discarded.

+ 26 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -552,7 +552,7 @@ public class DatanodeManager {
 
 
   /** Get a datanode descriptor given corresponding DatanodeUUID */
-  DatanodeDescriptor getDatanode(final String datanodeUuid) {
+  public DatanodeDescriptor getDatanode(final String datanodeUuid) {
     if (datanodeUuid == null) {
       return null;
     }
@@ -902,10 +902,14 @@ public class DatanodeManager {
    *
    * @param nodeReg datanode
    */
-  void startDecommissioningIfExcluded(DatanodeDescriptor nodeReg) {
+  void startAdminOperationIfNecessary(DatanodeDescriptor nodeReg) {
+    long maintenanceExpireTimeInMS =
+        hostConfigManager.getMaintenanceExpirationTimeInMS(nodeReg);
     // If the registered node is in exclude list, then decommission it
     if (getHostConfigManager().isExcluded(nodeReg)) {
       decomManager.startDecommission(nodeReg);
+    } else if (nodeReg.maintenanceNotExpired(maintenanceExpireTimeInMS)) {
+      decomManager.startMaintenance(nodeReg, maintenanceExpireTimeInMS);
     }
   }
 
@@ -1017,7 +1021,7 @@ public class DatanodeManager {
           // also treat the registration message as a heartbeat
           heartbeatManager.register(nodeS);
           incrementVersionCount(nodeS.getSoftwareVersion());
-          startDecommissioningIfExcluded(nodeS);
+          startAdminOperationIfNecessary(nodeS);
           success = true;
         } finally {
           if (!success) {
@@ -1056,7 +1060,7 @@ public class DatanodeManager {
         heartbeatManager.addDatanode(nodeDescr);
         heartbeatManager.updateDnStat(nodeDescr);
         incrementVersionCount(nodeReg.getSoftwareVersion());
-        startDecommissioningIfExcluded(nodeDescr);
+        startAdminOperationIfNecessary(nodeDescr);
         success = true;
       } finally {
         if (!success) {
@@ -1122,9 +1126,14 @@ public class DatanodeManager {
       if (!hostConfigManager.isIncluded(node)) {
         node.setDisallowed(true); // case 2.
       } else {
-        if (hostConfigManager.isExcluded(node)) {
+        long maintenanceExpireTimeInMS =
+            hostConfigManager.getMaintenanceExpirationTimeInMS(node);
+        if (node.maintenanceNotExpired(maintenanceExpireTimeInMS)) {
+          decomManager.startMaintenance(node, maintenanceExpireTimeInMS);
+        } else if (hostConfigManager.isExcluded(node)) {
           decomManager.startDecommission(node); // case 3.
         } else {
+          decomManager.stopMaintenance(node);
           decomManager.stopDecommission(node); // case 4.
         }
       }
@@ -1157,7 +1166,12 @@ public class DatanodeManager {
     // A decommissioning DN may be "alive" or "dead".
     return getDatanodeListForReport(DatanodeReportType.DECOMMISSIONING);
   }
-  
+
+  /** @return list of datanodes that are entering maintenance. */
+  public List<DatanodeDescriptor> getEnteringMaintenanceNodes() {
+    return getDatanodeListForReport(DatanodeReportType.ENTERING_MAINTENANCE);
+  }
+
   /* Getter and Setter for stale DataNodes related attributes */
 
   /**
@@ -1342,6 +1356,9 @@ public class DatanodeManager {
     final boolean listDecommissioningNodes =
         type == DatanodeReportType.ALL ||
         type == DatanodeReportType.DECOMMISSIONING;
+    final boolean listEnteringMaintenanceNodes =
+        type == DatanodeReportType.ALL ||
+        type == DatanodeReportType.ENTERING_MAINTENANCE;
 
     ArrayList<DatanodeDescriptor> nodes;
     final HostSet foundNodes = new HostSet();
@@ -1353,10 +1370,12 @@ public class DatanodeManager {
       for (DatanodeDescriptor dn : datanodeMap.values()) {
         final boolean isDead = isDatanodeDead(dn);
         final boolean isDecommissioning = dn.isDecommissionInProgress();
+        final boolean isEnteringMaintenance = dn.isEnteringMaintenance();
 
         if (((listLiveNodes && !isDead) ||
             (listDeadNodes && isDead) ||
-            (listDecommissioningNodes && isDecommissioning)) &&
+            (listDecommissioningNodes && isDecommissioning) ||
+            (listEnteringMaintenanceNodes && isEnteringMaintenance)) &&
             hostConfigManager.isIncluded(dn)) {
           nodes.add(dn);
         }

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStats.java

@@ -47,7 +47,7 @@ class DatanodeStats {
 
   synchronized void add(final DatanodeDescriptor node) {
     xceiverCount += node.getXceiverCount();
-    if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
+    if (node.isInService()) {
       capacityUsed += node.getDfsUsed();
       blockPoolUsed += node.getBlockPoolUsed();
       nodesInService++;
@@ -56,7 +56,8 @@ class DatanodeStats {
       capacityRemaining += node.getRemaining();
       cacheCapacity += node.getCacheCapacity();
       cacheUsed += node.getCacheUsed();
-    } else if (!node.isDecommissioned()) {
+    } else if (node.isDecommissionInProgress() ||
+        node.isEnteringMaintenance()) {
       cacheCapacity += node.getCacheCapacity();
       cacheUsed += node.getCacheUsed();
     }
@@ -74,7 +75,7 @@ class DatanodeStats {
 
   synchronized void subtract(final DatanodeDescriptor node) {
     xceiverCount -= node.getXceiverCount();
-    if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
+    if (node.isInService()) {
       capacityUsed -= node.getDfsUsed();
       blockPoolUsed -= node.getBlockPoolUsed();
       nodesInService--;
@@ -83,7 +84,8 @@ class DatanodeStats {
       capacityRemaining -= node.getRemaining();
       cacheCapacity -= node.getCacheCapacity();
       cacheUsed -= node.getCacheUsed();
-    } else if (!node.isDecommissioned()) {
+    } else if (node.isDecommissionInProgress() ||
+        node.isEnteringMaintenance()) {
       cacheCapacity -= node.getCacheCapacity();
       cacheUsed -= node.getCacheUsed();
     }

+ 80 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java

@@ -86,8 +86,11 @@ public class DecommissionManager {
   private final ScheduledExecutorService executor;
 
   /**
-   * Map containing the decommission-in-progress datanodes that are being
-   * tracked so they can be be marked as decommissioned.
+   * Map containing the DECOMMISSION_INPROGRESS or ENTERING_MAINTENANCE
+   * datanodes that are being tracked so they can be be marked as
+   * DECOMMISSIONED or IN_MAINTENANCE. Even after the node is marked as
+   * IN_MAINTENANCE, the node remains in the map until
+   * maintenance expires checked during a monitor tick.
    * <p/>
    * This holds a set of references to the under-replicated blocks on the DN at
    * the time the DN is added to the map, i.e. the blocks that are preventing
@@ -102,12 +105,12 @@ public class DecommissionManager {
    * another check is done with the actual block map.
    */
   private final TreeMap<DatanodeDescriptor, AbstractList<BlockInfo>>
-      decomNodeBlocks;
+      outOfServiceNodeBlocks;
 
   /**
-   * Tracking a node in decomNodeBlocks consumes additional memory. To limit
-   * the impact on NN memory consumption, we limit the number of nodes in 
-   * decomNodeBlocks. Additional nodes wait in pendingNodes.
+   * Tracking a node in outOfServiceNodeBlocks consumes additional memory. To
+   * limit the impact on NN memory consumption, we limit the number of nodes in
+   * outOfServiceNodeBlocks. Additional nodes wait in pendingNodes.
    */
   private final Queue<DatanodeDescriptor> pendingNodes;
 
@@ -122,7 +125,7 @@ public class DecommissionManager {
     executor = Executors.newScheduledThreadPool(1,
         new ThreadFactoryBuilder().setNameFormat("DecommissionMonitor-%d")
             .setDaemon(true).build());
-    decomNodeBlocks = new TreeMap<>();
+    outOfServiceNodeBlocks = new TreeMap<>();
     pendingNodes = new LinkedList<>();
   }
 
@@ -222,13 +225,56 @@ public class DecommissionManager {
       }
       // Remove from tracking in DecommissionManager
       pendingNodes.remove(node);
-      decomNodeBlocks.remove(node);
+      outOfServiceNodeBlocks.remove(node);
     } else {
       LOG.trace("stopDecommission: Node {} in {}, nothing to do." +
           node, node.getAdminState());
     }
   }
 
+  /**
+   * Start maintenance of the specified datanode.
+   * @param node
+   */
+  @VisibleForTesting
+  public void startMaintenance(DatanodeDescriptor node,
+      long maintenanceExpireTimeInMS) {
+    // Even if the node is already in maintenance, we still need to adjust
+    // the expiration time.
+    node.setMaintenanceExpireTimeInMS(maintenanceExpireTimeInMS);
+    if (!node.isMaintenance()) {
+      // Update DN stats maintained by HeartbeatManager
+      hbManager.startMaintenance(node);
+      pendingNodes.add(node);
+    } else {
+      LOG.trace("startMaintenance: Node {} in {}, nothing to do." +
+          node, node.getAdminState());
+    }
+  }
+
+
+  /**
+   * Stop maintenance of the specified datanode.
+   * @param node
+   */
+  @VisibleForTesting
+  public void stopMaintenance(DatanodeDescriptor node) {
+    if (node.isMaintenance()) {
+      // Update DN stats maintained by HeartbeatManager
+      hbManager.stopMaintenance(node);
+
+      // TODO HDFS-9390 remove replicas from block maps
+      // or handle over replicated blocks.
+
+      // Remove from tracking in DecommissionManager
+      pendingNodes.remove(node);
+      outOfServiceNodeBlocks.remove(node);
+    } else {
+      LOG.trace("stopMaintenance: Node {} in {}, nothing to do." +
+          node, node.getAdminState());
+    }
+  }
+
   private void setDecommissioned(DatanodeDescriptor dn) {
     dn.setDecommissioned();
     LOG.info("Decommissioning complete for node {}", dn);
@@ -313,7 +359,7 @@ public class DecommissionManager {
 
   @VisibleForTesting
   public int getNumTrackedNodes() {
-    return decomNodeBlocks.size();
+    return outOfServiceNodeBlocks.size();
   }
 
   @VisibleForTesting
@@ -333,8 +379,8 @@ public class DecommissionManager {
      */
     private final int numBlocksPerCheck;
     /**
-     * The maximum number of nodes to track in decomNodeBlocks. A value of 0
-     * means no limit.
+     * The maximum number of nodes to track in outOfServiceNodeBlocks.
+     * A value of 0 means no limit.
      */
     private final int maxConcurrentTrackedNodes;
     /**
@@ -347,7 +393,7 @@ public class DecommissionManager {
      */
     private int numNodesChecked = 0;
     /**
-     * The last datanode in decomNodeBlocks that we've processed
+     * The last datanode in outOfServiceNodeBlocks that we've processed
      */
     private DatanodeDescriptor iterkey = new DatanodeDescriptor(new 
         DatanodeID("", "", "", 0, 0, 0, 0));
@@ -393,14 +439,15 @@ public class DecommissionManager {
     private void processPendingNodes() {
       while (!pendingNodes.isEmpty() &&
           (maxConcurrentTrackedNodes == 0 ||
-           decomNodeBlocks.size() < maxConcurrentTrackedNodes)) {
-        decomNodeBlocks.put(pendingNodes.poll(), null);
+          outOfServiceNodeBlocks.size() < maxConcurrentTrackedNodes)) {
+        outOfServiceNodeBlocks.put(pendingNodes.poll(), null);
       }
     }
 
     private void check() {
       final Iterator<Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>>
-          it = new CyclicIteration<>(decomNodeBlocks, iterkey).iterator();
+          it = new CyclicIteration<>(outOfServiceNodeBlocks,
+              iterkey).iterator();
       final LinkedList<DatanodeDescriptor> toRemove = new LinkedList<>();
 
       while (it.hasNext() && !exceededNumBlocksPerCheck()) {
@@ -410,6 +457,17 @@ public class DecommissionManager {
         final DatanodeDescriptor dn = entry.getKey();
         AbstractList<BlockInfo> blocks = entry.getValue();
         boolean fullScan = false;
+        if (dn.isMaintenance()) {
+          // TODO HDFS-9390 make sure blocks are minimally replicated
+          // before transitioning the node to IN_MAINTENANCE state.
+
+          // If maintenance expires, stop tracking it.
+          if (dn.maintenanceExpired()) {
+            stopMaintenance(dn);
+            toRemove.add(dn);
+          }
+          continue;
+        }
         if (blocks == null) {
           // This is a newly added datanode, run through its list to schedule 
           // under-replicated blocks for replication and collect the blocks 
@@ -417,7 +475,7 @@ public class DecommissionManager {
           LOG.debug("Newly-added node {}, doing full scan to find " +
               "insufficiently-replicated blocks.", dn);
           blocks = handleInsufficientlyStored(dn);
-          decomNodeBlocks.put(dn, blocks);
+          outOfServiceNodeBlocks.put(dn, blocks);
           fullScan = true;
         } else {
           // This is a known datanode, check if its # of insufficiently 
@@ -436,7 +494,7 @@ public class DecommissionManager {
             LOG.debug("Node {} has finished replicating current set of "
                 + "blocks, checking with the full block map.", dn);
             blocks = handleInsufficientlyStored(dn);
-            decomNodeBlocks.put(dn, blocks);
+            outOfServiceNodeBlocks.put(dn, blocks);
           }
           // If the full scan is clean AND the node liveness is okay, 
           // we can finally mark as decommissioned.
@@ -460,11 +518,12 @@ public class DecommissionManager {
         }
         iterkey = dn;
       }
-      // Remove the datanodes that are decommissioned
+      // Remove the datanodes that are decommissioned or in service after
+      // maintenance expiration.
       for (DatanodeDescriptor dn : toRemove) {
-        Preconditions.checkState(dn.isDecommissioned(),
-            "Removing a node that is not yet decommissioned!");
-        decomNodeBlocks.remove(dn);
+        Preconditions.checkState(dn.isDecommissioned() || dn.isInService(),
+            "Removing a node that is not yet decommissioned or in service!");
+        outOfServiceNodeBlocks.remove(dn);
       }
     }
 

+ 27 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java

@@ -265,6 +265,33 @@ class HeartbeatManager implements DatanodeStatistics {
     }
   }
 
+  synchronized void startMaintenance(final DatanodeDescriptor node) {
+    if (!node.isAlive()) {
+      LOG.info("Dead node {} is put in maintenance state immediately.", node);
+      node.setInMaintenance();
+    } else if (node.isDecommissioned()) {
+      LOG.info("Decommissioned node " + node + " is put in maintenance state"
+          + " immediately.");
+      node.setInMaintenance();
+    } else {
+      stats.subtract(node);
+      node.startMaintenance();
+      stats.add(node);
+    }
+  }
+
+  synchronized void stopMaintenance(final DatanodeDescriptor node) {
+    LOG.info("Stopping maintenance of {} node {}",
+        node.isAlive() ? "live" : "dead", node);
+    if (!node.isAlive()) {
+      node.stopMaintenance();
+    } else {
+      stats.subtract(node);
+      node.stopMaintenance();
+      stats.add(node);
+    }
+  }
+
   synchronized void stopDecommission(final DatanodeDescriptor node) {
     LOG.info("Stopping decommissioning of {} node {}",
         node.isAlive() ? "live" : "dead", node);

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostConfigManager.java

@@ -77,4 +77,11 @@ public abstract class HostConfigManager implements Configurable {
    * @return the upgrade domain of dn.
    */
   public abstract String getUpgradeDomain(DatanodeID dn);
+
+  /**
+   * Get the maintenance expiration time in milli seconds.
+   * @param dn the DatanodeID of the datanode
+   * @return the maintenance expiration time of dn.
+   */
+  public abstract long getMaintenanceExpirationTimeInMS(DatanodeID dn);
 }

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java

@@ -138,6 +138,12 @@ public class HostFileManager extends HostConfigManager {
     return null;
   }
 
+  @Override
+  public long getMaintenanceExpirationTimeInMS(DatanodeID dn) {
+    // The include/exclude files based config doesn't support maintenance mode.
+    return 0;
+  }
+
   /**
    * Read the includes and excludes lists from the named files.  Any previous
    * includes and excludes lists are discarded.

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

@@ -1306,6 +1306,7 @@ class BlockReceiver implements Closeable {
           long ackRecvNanoTime = 0;
           try {
             if (type != PacketResponderType.LAST_IN_PIPELINE && !mirrorError) {
+              DataNodeFaultInjector.get().failPipeline(replicaInfo, mirrorAddr);
               // read an ack from downstream datanode
               ack.readFields(downstreamIn);
               ackRecvNanoTime = System.nanoTime();

+ 9 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java

@@ -156,6 +156,9 @@ class BlockSender implements java.io.Closeable {
   /** The reference to the volume where the block is located */
   private FsVolumeReference volumeRef;
 
+  /** The replica of the block that is being read. */
+  private final Replica replica;
+
   // Cache-management related fields
   private final long readaheadLength;
 
@@ -238,7 +241,6 @@ class BlockSender implements java.io.Closeable {
             "If verifying checksum, currently must also send it.");
       }
       
-      final Replica replica;
       final long replicaVisibleLength;
       try(AutoCloseableLock lock = datanode.data.acquireDatasetLock()) {
         replica = getReplica(block, datanode);
@@ -688,8 +690,12 @@ class BlockSender implements java.io.Closeable {
       checksum.update(buf, dOff, dLen);
       if (!checksum.compare(buf, cOff)) {
         long failedPos = offset + datalen - dLeft;
-        throw new ChecksumException("Checksum failed at " + failedPos,
-            failedPos);
+        StringBuilder replicaInfoString = new StringBuilder();
+        if (replica != null) {
+          replicaInfoString.append(" for replica: " + replica.toString());
+        }
+        throw new ChecksumException("Checksum failed at " + failedPos
+            + replicaInfoString, failedPos);
       }
       dLeft -= dLen;
       dOff += dLen;

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

@@ -2451,7 +2451,8 @@ public class DataNode extends ReconfigurableBase
         blockSender.sendBlock(out, unbufOut, null);
 
         // no response necessary
-        LOG.info(getClass().getSimpleName() + ": Transmitted " + b
+        LOG.info(getClass().getSimpleName() + ", at "
+            + DataNode.this.getDisplayName() + ": Transmitted " + b
             + " (numBytes=" + b.getNumBytes() + ") to " + curTarget);
 
         // read ack

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeFaultInjector.java

@@ -55,4 +55,7 @@ public class DataNodeFaultInjector {
   public void noRegistration() throws IOException { }
 
   public void failMirrorConnection() throws IOException { }
+
+  public void failPipeline(ReplicaInPipelineInterface replicaInfo,
+      String mirrorAddr) throws IOException { }
 }

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -312,6 +312,12 @@ class DataXceiver extends Receiver implements Runnable {
         } else {
           LOG.info(s1 + "; " + t);          
         }
+      } else if (t instanceof InvalidToken) {
+        // The InvalidToken exception has already been logged in
+        // checkAccess() method and this is not a server error.
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(s, t);
+        }
       } else {
         LOG.error(s, t);
       }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java

@@ -281,12 +281,13 @@ public class VolumeScanner extends Thread {
             volume.getBasePath(), block);
         return;
       }
-      LOG.warn("Reporting bad {} on {}", block, volume.getBasePath());
+      LOG.warn("Reporting bad " + block + " with volume "
+          + volume.getBasePath(), e);
       try {
         scanner.datanode.reportBadBlocks(block, volume);
       } catch (IOException ie) {
         // This is bad, but not bad enough to shut down the scanner.
-        LOG.warn("Cannot report bad " + block.getBlockId(), e);
+        LOG.warn("Cannot report bad block " + block, ie);
       }
     }
   }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -1505,7 +1505,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           if (!rbw.attemptToSetWriter(null, Thread.currentThread())) {
             throw new MustStopExistingWriter(rbw);
           }
-          LOG.info("Recovering " + rbw);
+          LOG.info("At " + datanode.getDisplayName() + ", Recovering " + rbw);
           return recoverRbwImpl(rbw, b, newGS, minBytesRcvd, maxBytesRcvd);
         }
       } catch (MustStopExistingWriter e) {

+ 43 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java

@@ -18,7 +18,10 @@
 
 package org.apache.hadoop.hdfs.server.diskbalancer.command;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.lang.StringUtils;
@@ -221,7 +224,7 @@ public abstract class Command extends Configured {
    * @return Set of node names
    * @throws IOException
    */
-  private Set<String> getNodeList(String listArg) throws IOException {
+  protected Set<String> getNodeList(String listArg) throws IOException {
     URL listURL;
     String nodeData;
     Set<String> resultSet = new TreeSet<>();
@@ -242,6 +245,37 @@ public abstract class Command extends Configured {
     return resultSet;
   }
 
+  /**
+   * Returns a DiskBalancer Node list from the Cluster or null if not found.
+   *
+   * @param listArg String File URL or a comma separated list of node names.
+   * @return List of DiskBalancer Node
+   * @throws IOException
+   */
+  protected List<DiskBalancerDataNode> getNodes(String listArg)
+      throws IOException {
+    Set<String> nodeNames = null;
+    List<DiskBalancerDataNode> nodeList = Lists.newArrayList();
+
+    if ((listArg == null) || listArg.isEmpty()) {
+      return nodeList;
+    }
+    nodeNames = getNodeList(listArg);
+
+    DiskBalancerDataNode node = null;
+    if (!nodeNames.isEmpty()) {
+      for (String name : nodeNames) {
+        node = getNode(name);
+
+        if (node != null) {
+          nodeList.add(node);
+        }
+      }
+    }
+
+    return nodeList;
+  }
+
   /**
    * Verifies if the command line options are sane.
    *
@@ -471,4 +505,12 @@ public abstract class Command extends Configured {
   public int getTopNodes() {
     return topNodes;
   }
+
+  /**
+   * Set DiskBalancer cluster
+   */
+  @VisibleForTesting
+  public void setCluster(DiskBalancerCluster newCluster) {
+    this.cluster = newCluster;
+  }
 }

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

@@ -47,7 +47,10 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
@@ -57,6 +60,7 @@ import java.io.BufferedReader;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.net.InetSocketAddress;
 import java.net.URI;
 import java.text.DateFormat;
 import java.util.*;
@@ -579,6 +583,22 @@ public class Mover {
     }
   }
 
+  private static void checkKeytabAndInit(Configuration conf)
+      throws IOException {
+    if (conf.getBoolean(DFSConfigKeys.DFS_MOVER_KEYTAB_ENABLED_KEY,
+        DFSConfigKeys.DFS_MOVER_KEYTAB_ENABLED_DEFAULT)) {
+      LOG.info("Keytab is configured, will login using keytab.");
+      UserGroupInformation.setConfiguration(conf);
+      String addr = conf.get(DFSConfigKeys.DFS_MOVER_ADDRESS_KEY,
+          DFSConfigKeys.DFS_MOVER_ADDRESS_DEFAULT);
+      InetSocketAddress socAddr = NetUtils.createSocketAddr(addr, 0,
+          DFSConfigKeys.DFS_MOVER_ADDRESS_KEY);
+      SecurityUtil.login(conf, DFSConfigKeys.DFS_MOVER_KEYTAB_FILE_KEY,
+          DFSConfigKeys.DFS_MOVER_KERBEROS_PRINCIPAL_KEY,
+          socAddr.getHostName());
+    }
+  }
+
   static int run(Map<URI, List<Path>> namenodes, Configuration conf)
       throws IOException, InterruptedException {
     final long sleeptime =
@@ -588,7 +608,8 @@ public class Mover {
             DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000;
     AtomicInteger retryCount = new AtomicInteger(0);
     LOG.info("namenodes = " + namenodes);
-    
+
+    checkKeytabAndInit(conf);
     List<NameNodeConnector> connectors = Collections.emptyList();
     try {
       connectors = NameNodeConnector.newNameNodeConnectors(namenodes,

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

@@ -29,6 +29,7 @@ public class ContentSummaryComputationContext {
   private FSNamesystem fsn = null;
   private BlockStoragePolicySuite bsps = null;
   private ContentCounts counts = null;
+  private ContentCounts snapshotCounts = null;
   private long nextCountLimit = 0;
   private long limitPerRun = 0;
   private long yieldCount = 0;
@@ -51,6 +52,7 @@ public class ContentSummaryComputationContext {
     this.limitPerRun = limitPerRun;
     this.nextCountLimit = limitPerRun;
     this.counts = new ContentCounts.Builder().build();
+    this.snapshotCounts = new ContentCounts.Builder().build();
     this.sleepMilliSec = sleepMicroSec/1000;
     this.sleepNanoSec = (int)((sleepMicroSec%1000)*1000);
   }
@@ -125,6 +127,10 @@ public class ContentSummaryComputationContext {
     return counts;
   }
 
+  public ContentCounts getSnapshotCounts() {
+    return snapshotCounts;
+  }
+
   public BlockStoragePolicySuite getBlockStoragePolicySuite() {
     Preconditions.checkState((bsps != null || fsn != null),
         "BlockStoragePolicySuite must be either initialized or available via" +

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

@@ -85,9 +85,10 @@ final class FSDirAppendOp {
     final LocatedBlock lb;
     final FSDirectory fsd = fsn.getFSDirectory();
     final String src;
+    final INodesInPath iip;
     fsd.writeLock();
     try {
-      final INodesInPath iip = fsd.resolvePathForWrite(pc, srcArg);
+      iip = fsd.resolvePathForWrite(pc, srcArg);
       src = iip.getPath();
       // Verify that the destination does not exist as a directory already
       final INode inode = iip.getLastINode();
@@ -148,8 +149,7 @@ final class FSDirAppendOp {
       fsd.writeUnlock();
     }
 
-    HdfsFileStatus stat = FSDirStatAndListingOp.getFileInfo(fsd, src, false,
-        FSDirectory.isReservedRawName(srcArg));
+    HdfsFileStatus stat = FSDirStatAndListingOp.getFileInfo(fsd, iip);
     if (lb != null) {
       NameNode.stateChangeLog.debug(
           "DIR* NameSystem.appendFile: file {} for {} at {} block {} block"

+ 39 - 52
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java

@@ -32,10 +32,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import java.io.IOException;
-import java.util.AbstractMap;
 import java.util.List;
-import java.util.Map;
-
 import static org.apache.hadoop.util.Time.now;
 
 class FSDirMkdirOp {
@@ -63,7 +60,6 @@ class FSDirMkdirOp {
         throw new FileAlreadyExistsException("Path is not a directory: " + src);
       }
 
-      INodesInPath existing = lastINode != null ? iip : iip.getExistingINodes();
       if (lastINode == null) {
         if (fsd.isPermissionEnabled()) {
           fsd.checkAncestorAccess(pc, iip, FsAction.WRITE);
@@ -78,26 +74,20 @@ class FSDirMkdirOp {
         // create multiple inodes.
         fsn.checkFsObjectLimit();
 
-        List<String> nonExisting = iip.getPath(existing.length(),
-            iip.length() - existing.length());
-        int length = nonExisting.size();
-        if (length > 1) {
-          List<String> ancestors = nonExisting.subList(0, length - 1);
-          // Ensure that the user can traversal the path by adding implicit
-          // u+wx permission to all ancestor directories
-          existing = createChildrenDirectories(fsd, existing, ancestors,
-              addImplicitUwx(permissions, permissions));
-          if (existing == null) {
-            throw new IOException("Failed to create directory: " + src);
-          }
+        // Ensure that the user can traversal the path by adding implicit
+        // u+wx permission to all ancestor directories.
+        INodesInPath existing =
+            createParentDirectories(fsd, iip, permissions, false);
+        if (existing != null) {
+          existing = createSingleDirectory(
+              fsd, existing, iip.getLastLocalName(), permissions);
         }
-
-        if ((existing = createChildrenDirectories(fsd, existing,
-            nonExisting.subList(length - 1, length), permissions)) == null) {
+        if (existing == null) {
           throw new IOException("Failed to create directory: " + src);
         }
+        iip = existing;
       }
-      return fsd.getAuditFileInfo(existing);
+      return fsd.getAuditFileInfo(iip);
     } finally {
       fsd.writeUnlock();
     }
@@ -112,35 +102,18 @@ class FSDirMkdirOp {
    * For example, path="/foo/bar/spam", "/foo" is an existing directory,
    * "/foo/bar" is not existing yet, the function will create directory bar.
    *
-   * @return a tuple which contains both the new INodesInPath (with all the
-   * existing and newly created directories) and the last component in the
-   * relative path. Or return null if there are errors.
+   * @return a INodesInPath with all the existing and newly created
+   *         ancestor directories created.
+   *         Or return null if there are errors.
    */
-  static Map.Entry<INodesInPath, String> createAncestorDirectories(
+  static INodesInPath createAncestorDirectories(
       FSDirectory fsd, INodesInPath iip, PermissionStatus permission)
       throws IOException {
-    final String last = DFSUtil.bytes2String(iip.getLastLocalName());
-    INodesInPath existing = iip.getExistingINodes();
-    List<String> children = iip.getPath(existing.length(),
-        iip.length() - existing.length());
-    int size = children.size();
-    if (size > 1) { // otherwise all ancestors have been created
-      List<String> directories = children.subList(0, size - 1);
-      INode parentINode = existing.getLastINode();
-      // Ensure that the user can traversal the path by adding implicit
-      // u+wx permission to all ancestor directories
-      existing = createChildrenDirectories(fsd, existing, directories,
-          addImplicitUwx(parentINode.getPermissionStatus(), permission));
-      if (existing == null) {
-        return null;
-      }
-    }
-    return new AbstractMap.SimpleImmutableEntry<>(existing, last);
+    return createParentDirectories(fsd, iip, permission, true);
   }
 
   /**
-   * Create the directory {@code parent} / {@code children} and all ancestors
-   * along the path.
+   * Create all ancestor directories and return the parent inodes.
    *
    * @param fsd FSDirectory
    * @param existing The INodesInPath instance containing all the existing
@@ -149,21 +122,35 @@ class FSDirMkdirOp {
    *                 starting with "/"
    * @param perm the permission of the directory. Note that all ancestors
    *             created along the path has implicit {@code u+wx} permissions.
+   * @param inheritPerms if the ancestor directories should inherit permissions
+   *                 or use the specified permissions.
    *
    * @return {@link INodesInPath} which contains all inodes to the
    * target directory, After the execution parentPath points to the path of
    * the returned INodesInPath. The function return null if the operation has
    * failed.
    */
-  private static INodesInPath createChildrenDirectories(FSDirectory fsd,
-      INodesInPath existing, List<String> children, PermissionStatus perm)
+  private static INodesInPath createParentDirectories(FSDirectory fsd,
+      INodesInPath iip, PermissionStatus perm, boolean inheritPerms)
       throws IOException {
     assert fsd.hasWriteLock();
-
-    for (String component : children) {
-      existing = createSingleDirectory(fsd, existing, component, perm);
-      if (existing == null) {
-        return null;
+    // this is the desired parent iip if the subsequent delta is 1.
+    INodesInPath existing = iip.getExistingINodes();
+    int missing = iip.length() - existing.length();
+    if (missing == 0) {  // full path exists, return parents.
+      existing = iip.getParentINodesInPath();
+    } else if (missing > 1) { // need to create at least one ancestor dir.
+      // Ensure that the user can traversal the path by adding implicit
+      // u+wx permission to all ancestor directories.
+      PermissionStatus basePerm = inheritPerms
+          ? existing.getLastINode().getPermissionStatus()
+          : perm;
+      perm = addImplicitUwx(basePerm, perm);
+      // create all the missing directories.
+      final int last = iip.length() - 2;
+      for (int i = existing.length(); existing != null && i <= last; i++) {
+        byte[] component = iip.getPathComponent(i);
+        existing = createSingleDirectory(fsd, existing, component, perm);
       }
     }
     return existing;
@@ -183,11 +170,11 @@ class FSDirMkdirOp {
   }
 
   private static INodesInPath createSingleDirectory(FSDirectory fsd,
-      INodesInPath existing, String localName, PermissionStatus perm)
+      INodesInPath existing, byte[] localName, PermissionStatus perm)
       throws IOException {
     assert fsd.hasWriteLock();
     existing = unprotectedMkdir(fsd, fsd.allocateNewInodeId(), existing,
-        DFSUtil.string2Bytes(localName), perm, null, now());
+        localName, perm, null, now());
     if (existing == null) {
       return null;
     }

+ 23 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java

@@ -261,11 +261,29 @@ class FSDirRenameOp {
     src = srcIIP.getPath();
     dst = dstIIP.getPath();
     if (fsd.isPermissionEnabled()) {
-      // Rename does not operate on link targets
-      // Do not resolveLink when checking permissions of src and dst
-      // Check write access to parent of src
-      fsd.checkPermission(pc, srcIIP, false, null, FsAction.WRITE, null, null,
-          false);
+      boolean renameToTrash = false;
+      if (null != options &&
+          Arrays.asList(options).
+          contains(Options.Rename.TO_TRASH)) {
+        renameToTrash = true;
+      }
+
+      if(renameToTrash) {
+        // if destination is the trash directory,
+        // besides the permission check on "rename"
+        // we need to enforce the check for "delete"
+        // otherwise, it would expose a
+        // security hole that stuff moved to trash
+        // will be deleted by superuser
+        fsd.checkPermission(pc, srcIIP, false, null, FsAction.WRITE, null,
+            FsAction.ALL, true);
+      } else {
+        // Rename does not operate on link targets
+        // Do not resolveLink when checking permissions of src and dst
+        // Check write access to parent of src
+        fsd.checkPermission(pc, srcIIP, false, null, FsAction.WRITE, null,
+            null, false);
+      }
       // Check write access to ancestor of dst
       fsd.checkPermission(pc, dstIIP, false, FsAction.WRITE, null, null, null,
           false);

+ 39 - 43
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.QuotaUsage;
@@ -52,7 +53,6 @@ import static org.apache.hadoop.util.Time.now;
 class FSDirStatAndListingOp {
   static DirectoryListing getListingInt(FSDirectory fsd, final String srcArg,
       byte[] startAfter, boolean needLocation) throws IOException {
-    final String startAfterString = DFSUtil.bytes2String(startAfter);
     String src = null;
 
     final INodesInPath iip;
@@ -65,16 +65,20 @@ class FSDirStatAndListingOp {
       iip = fsd.getINodesInPath(src, true);
     }
 
-    // Get file name when startAfter is an INodePath
-    if (FSDirectory.isReservedName(startAfterString)) {
-      try {
-        String tmp = FSDirectory.resolvePath(startAfterString, fsd);
-        byte[][] regularPath = INode.getPathComponents(tmp);
-        startAfter = regularPath[regularPath.length - 1];
-      } catch (IOException e) {
-        // Possibly the inode is deleted
-        throw new DirectoryListingStartAfterNotFoundException(
-            "Can't find startAfter " + startAfterString);
+    // Get file name when startAfter is an INodePath.  This is not the
+    // common case so avoid any unnecessary processing unless required.
+    if (startAfter.length > 0 && startAfter[0] == Path.SEPARATOR_CHAR) {
+      final String startAfterString = DFSUtil.bytes2String(startAfter);
+      if (FSDirectory.isReservedName(startAfterString)) {
+        try {
+          byte[][] components = INode.getPathComponents(startAfterString);
+          components = FSDirectory.resolveComponents(components, fsd);
+          startAfter = components[components.length - 1];
+        } catch (IOException e) {
+          // Possibly the inode is deleted
+          throw new DirectoryListingStartAfterNotFoundException(
+              "Can't find startAfter " + startAfterString);
+        }
       }
     }
 
@@ -108,16 +112,16 @@ class FSDirStatAndListingOp {
     if (!DFSUtil.isValidName(src)) {
       throw new InvalidPathException("Invalid file name: " + src);
     }
+    final INodesInPath iip;
     if (fsd.isPermissionEnabled()) {
       FSPermissionChecker pc = fsd.getPermissionChecker();
-      final INodesInPath iip = fsd.resolvePath(pc, srcArg, resolveLink);
-      src = iip.getPath();
+      iip = fsd.resolvePath(pc, srcArg, resolveLink);
       fsd.checkPermission(pc, iip, false, null, null, null, null, false);
     } else {
       src = FSDirectory.resolvePath(srcArg, fsd);
+      iip = fsd.getINodesInPath(src, resolveLink);
     }
-    return getFileInfo(fsd, src, FSDirectory.isReservedRawName(srcArg),
-                       resolveLink);
+    return getFileInfo(fsd, iip);
   }
 
   /**
@@ -230,7 +234,6 @@ class FSDirStatAndListingOp {
       String src, byte[] startAfter, boolean needLocation, boolean isSuperUser)
       throws IOException {
     String srcs = FSDirectory.normalizePath(src);
-    final boolean isRawPath = FSDirectory.isReservedRawName(src);
     if (FSDirectory.isExactReservedName(srcs)) {
       return getReservedListing(fsd);
     }
@@ -257,7 +260,7 @@ class FSDirStatAndListingOp {
         return new DirectoryListing(
             new HdfsFileStatus[]{ createFileStatus(
                 fsd, HdfsFileStatus.EMPTY_NAME, nodeAttrs,
-                needLocation, parentStoragePolicy, snapshot, isRawPath, iip)
+                needLocation, parentStoragePolicy, iip)
             }, 0);
       }
 
@@ -282,7 +285,7 @@ class FSDirStatAndListingOp {
             cur.getLocalNameBytes());
         listing[i] = createFileStatus(fsd, cur.getLocalNameBytes(), nodeAttrs,
             needLocation, getStoragePolicyID(curPolicy, parentStoragePolicy),
-            snapshot, isRawPath, iipWithChild);
+            iipWithChild);
         listingCnt++;
         if (needLocation) {
             // Once we  hit lsLimit locations, stop.
@@ -339,7 +342,6 @@ class FSDirStatAndListingOp {
       listing[i] = createFileStatus(
           fsd, sRoot.getLocalNameBytes(), nodeAttrs,
           HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
-          Snapshot.CURRENT_STATE_ID, false,
           INodesInPath.fromINode(sRoot));
     }
     return new DirectoryListing(
@@ -363,10 +365,8 @@ class FSDirStatAndListingOp {
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  static HdfsFileStatus getFileInfo(
-      FSDirectory fsd, String path, INodesInPath iip, boolean isRawPath,
-      boolean includeStoragePolicy)
-      throws IOException {
+  static HdfsFileStatus getFileInfo(FSDirectory fsd,
+      INodesInPath iip, boolean includeStoragePolicy) throws IOException {
     fsd.readLock();
     try {
       final INode node = iip.getLastINode();
@@ -377,23 +377,21 @@ class FSDirStatAndListingOp {
       byte policyId = includeStoragePolicy && !node.isSymlink() ?
           node.getStoragePolicyID() :
           HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-      INodeAttributes nodeAttrs = getINodeAttributes(fsd, path,
+      INodeAttributes nodeAttrs = getINodeAttributes(fsd, iip.getPath(),
                                                      HdfsFileStatus.EMPTY_NAME,
                                                      node, iip.getPathSnapshotId());
       return createFileStatus(fsd, HdfsFileStatus.EMPTY_NAME, nodeAttrs,
-                              policyId, iip.getPathSnapshotId(), isRawPath, iip);
+                              policyId, iip);
     } finally {
       fsd.readUnlock();
     }
   }
 
-  static HdfsFileStatus getFileInfo(
-      FSDirectory fsd, String src, boolean resolveLink, boolean isRawPath)
+  static HdfsFileStatus getFileInfo(FSDirectory fsd, INodesInPath iip)
     throws IOException {
     fsd.readLock();
     try {
       HdfsFileStatus status = null;
-      final INodesInPath iip = fsd.getINodesInPath(src, resolveLink);
       if (FSDirectory.isExactReservedName(iip.getPathComponents())) {
         status = FSDirectory.DOT_RESERVED_STATUS;
       } else if (iip.isDotSnapshotDir()) {
@@ -401,7 +399,7 @@ class FSDirStatAndListingOp {
           status = FSDirectory.DOT_SNAPSHOT_DIR_STATUS;
         }
       } else {
-        status = getFileInfo(fsd, src, iip, isRawPath, true);
+        status = getFileInfo(fsd, iip, true);
       }
       return status;
     } finally {
@@ -423,15 +421,12 @@ class FSDirStatAndListingOp {
    */
   private static HdfsFileStatus createFileStatus(
       FSDirectory fsd, byte[] path, INodeAttributes nodeAttrs,
-      boolean needLocation, byte storagePolicy, int snapshot, boolean isRawPath,
-      INodesInPath iip)
+      boolean needLocation, byte storagePolicy, INodesInPath iip)
       throws IOException {
     if (needLocation) {
-      return createLocatedFileStatus(fsd, path, nodeAttrs, storagePolicy,
-                                     snapshot, isRawPath, iip);
+      return createLocatedFileStatus(fsd, path, nodeAttrs, storagePolicy, iip);
     } else {
-      return createFileStatus(fsd, path, nodeAttrs, storagePolicy,
-                              snapshot, isRawPath, iip);
+      return createFileStatus(fsd, path, nodeAttrs, storagePolicy, iip);
     }
   }
 
@@ -445,8 +440,7 @@ class FSDirStatAndListingOp {
       INodesInPath iip) throws IOException {
     INodeAttributes nodeAttrs = getINodeAttributes(
         fsd, fullPath, path, iip.getLastINode(), snapshot);
-    return createFileStatus(fsd, path, nodeAttrs, storagePolicy,
-                            snapshot, isRawPath, iip);
+    return createFileStatus(fsd, path, nodeAttrs, storagePolicy, iip);
   }
 
   /**
@@ -454,14 +448,15 @@ class FSDirStatAndListingOp {
    * @param iip the INodesInPath containing the target INode and its ancestors
    */
   static HdfsFileStatus createFileStatus(
-      FSDirectory fsd, byte[] path,
-      INodeAttributes nodeAttrs, byte storagePolicy, int snapshot,
-      boolean isRawPath, INodesInPath iip) throws IOException {
+      FSDirectory fsd, byte[] path, INodeAttributes nodeAttrs,
+      byte storagePolicy, INodesInPath iip) throws IOException {
     long size = 0;     // length is zero for directories
     short replication = 0;
     long blocksize = 0;
     final boolean isEncrypted;
     final INode node = iip.getLastINode();
+    final int snapshot = iip.getPathSnapshotId();
+    final boolean isRawPath = iip.isRaw();
 
     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
         .getFileEncryptionInfo(fsd, node, snapshot, iip);
@@ -511,10 +506,9 @@ class FSDirStatAndListingOp {
    * Create FileStatus with location info by file INode
    * @param iip the INodesInPath containing the target INode and its ancestors
    */
-  private static HdfsLocatedFileStatus createLocatedFileStatus(
+  private static HdfsFileStatus createLocatedFileStatus(
       FSDirectory fsd, byte[] path, INodeAttributes nodeAttrs,
-      byte storagePolicy, int snapshot,
-      boolean isRawPath, INodesInPath iip) throws IOException {
+      byte storagePolicy, INodesInPath iip) throws IOException {
     assert fsd.hasReadLock();
     long size = 0; // length is zero for directories
     short replication = 0;
@@ -522,6 +516,8 @@ class FSDirStatAndListingOp {
     LocatedBlocks loc = null;
     final boolean isEncrypted;
     final INode node = iip.getLastINode();
+    final int snapshot = iip.getPathSnapshotId();
+    final boolean isRawPath = iip.isRaw();
 
     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
         .getFileEncryptionInfo(fsd, node, snapshot, iip);

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

@@ -27,7 +27,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 
 import java.io.IOException;
-import java.util.Map;
 
 import static org.apache.hadoop.util.Time.now;
 
@@ -99,21 +98,21 @@ class FSDirSymlinkOp {
       INodesInPath iip, String target, PermissionStatus dirPerms,
       boolean createParent, boolean logRetryCache) throws IOException {
     final long mtime = now();
-    final byte[] localName = iip.getLastLocalName();
+    final INodesInPath parent;
     if (createParent) {
-      Map.Entry<INodesInPath, String> e = FSDirMkdirOp
-          .createAncestorDirectories(fsd, iip, dirPerms);
-      if (e == null) {
+      parent = FSDirMkdirOp.createAncestorDirectories(fsd, iip, dirPerms);
+      if (parent == null) {
         return null;
       }
-      iip = INodesInPath.append(e.getKey(), null, localName);
+    } else {
+      parent = iip.getParentINodesInPath();
     }
     final String userName = dirPerms.getUserName();
     long id = fsd.allocateNewInodeId();
     PermissionStatus perm = new PermissionStatus(
         userName, null, FsPermission.getDefault());
-    INodeSymlink newNode = unprotectedAddSymlink(fsd, iip.getExistingINodes(),
-        localName, id, target, mtime, mtime, perm);
+    INodeSymlink newNode = unprotectedAddSymlink(fsd, parent,
+        iip.getLastLocalName(), id, target, mtime, mtime, perm);
     if (newNode == null) {
       NameNode.stateChangeLog.info("addSymlink: failed to add " + path);
       return null;

+ 17 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.FSLimitException;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@@ -65,7 +66,6 @@ import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
@@ -340,7 +340,6 @@ class FSDirWriteFileOp {
       version = ezInfo.protocolVersion;
     }
 
-    boolean isRawPath = FSDirectory.isReservedRawName(src);
     FSDirectory fsd = fsn.getFSDirectory();
     INodesInPath iip = fsd.resolvePathForWrite(pc, src);
     src = iip.getPath();
@@ -420,10 +419,10 @@ class FSDirWriteFileOp {
     }
     fsn.checkFsObjectLimit();
     INodeFile newNode = null;
-    Map.Entry<INodesInPath, String> parent = FSDirMkdirOp
-        .createAncestorDirectories(fsd, iip, permissions);
+    INodesInPath parent =
+        FSDirMkdirOp.createAncestorDirectories(fsd, iip, permissions);
     if (parent != null) {
-      iip = addFile(fsd, parent.getKey(), parent.getValue(), permissions,
+      iip = addFile(fsd, parent, iip.getLastLocalName(), permissions,
                     replication, blockSize, holder, clientMachine);
       newNode = iip != null ? iip.getLastINode().asFile() : null;
     }
@@ -444,7 +443,7 @@ class FSDirWriteFileOp {
       NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: added " +
           src + " inode " + newNode.getId() + " " + holder);
     }
-    return FSDirStatAndListingOp.getFileInfo(fsd, src, false, isRawPath);
+    return FSDirStatAndListingOp.getFileInfo(fsd, iip);
   }
 
   static EncryptionKeyInfo getEncryptionKeyInfo(FSNamesystem fsn,
@@ -508,10 +507,13 @@ class FSDirWriteFileOp {
         return newNode;
       }
     } catch (IOException e) {
-      if(NameNode.stateChangeLog.isDebugEnabled()) {
-        NameNode.stateChangeLog.debug(
-            "DIR* FSDirectory.unprotectedAddFile: exception when add "
-                + existing.getPath() + " to the file system", e);
+      NameNode.stateChangeLog.warn(
+          "DIR* FSDirectory.unprotectedAddFile: exception when add " + existing
+              .getPath() + " to the file system", e);
+      if (e instanceof FSLimitException.MaxDirectoryItemsExceededException) {
+        NameNode.stateChangeLog.warn("Please increase "
+            + "dfs.namenode.fs-limits.max-directory-items and make it "
+            + "consistent across all NameNodes.");
       }
     }
     return null;
@@ -573,7 +575,7 @@ class FSDirWriteFileOp {
    * @return the new INodesInPath instance that contains the new INode
    */
   private static INodesInPath addFile(
-      FSDirectory fsd, INodesInPath existing, String localName,
+      FSDirectory fsd, INodesInPath existing, byte[] localName,
       PermissionStatus permissions, short replication, long preferredBlockSize,
       String clientName, String clientMachine)
       throws IOException {
@@ -590,7 +592,7 @@ class FSDirWriteFileOp {
       }
       INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
           modTime, modTime, replication, preferredBlockSize, ecPolicy != null);
-      newNode.setLocalName(DFSUtil.string2Bytes(localName));
+      newNode.setLocalName(localName);
       newNode.toUnderConstruction(clientName, clientMachine);
       newiip = fsd.addINode(existing, newNode);
     } finally {
@@ -598,12 +600,13 @@ class FSDirWriteFileOp {
     }
     if (newiip == null) {
       NameNode.stateChangeLog.info("DIR* addFile: failed to add " +
-          existing.getPath() + "/" + localName);
+          existing.getPath() + "/" + DFSUtil.bytes2String(localName));
       return null;
     }
 
     if(NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("DIR* addFile: " + localName + " is added");
+      NameNode.stateChangeLog.debug("DIR* addFile: " +
+          DFSUtil.bytes2String(localName) + " is added");
     }
     return newiip;
   }

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

@@ -531,21 +531,24 @@ public class FSDirectory implements Closeable {
    * @throws FileNotFoundException
    * @throws AccessControlException
    */
-  INodesInPath resolvePath(FSPermissionChecker pc, String src)
+  @VisibleForTesting
+  public INodesInPath resolvePath(FSPermissionChecker pc, String src)
       throws UnresolvedLinkException, FileNotFoundException,
       AccessControlException {
     return resolvePath(pc, src, true);
   }
 
-  INodesInPath resolvePath(FSPermissionChecker pc, String src,
+  @VisibleForTesting
+  public INodesInPath resolvePath(FSPermissionChecker pc, String src,
       boolean resolveLink) throws UnresolvedLinkException,
   FileNotFoundException, AccessControlException {
     byte[][] components = INode.getPathComponents(src);
-    if (isPermissionEnabled && pc != null && isReservedRawName(components)) {
+    boolean isRaw = isReservedRawName(components);
+    if (isPermissionEnabled && pc != null && isRaw) {
       pc.checkSuperuserPrivilege();
     }
     components = resolveComponents(components, this);
-    return INodesInPath.resolve(rootDir, components, resolveLink);
+    return INodesInPath.resolve(rootDir, components, isRaw, resolveLink);
   }
 
   INodesInPath resolvePathForWrite(FSPermissionChecker pc, String src)
@@ -1662,8 +1665,7 @@ public class FSDirectory implements Closeable {
   HdfsFileStatus getAuditFileInfo(INodesInPath iip)
       throws IOException {
     return (namesystem.isAuditEnabled() && namesystem.isExternalInvocation())
-        ? FSDirStatAndListingOp.getFileInfo(this, iip.getPath(), iip, false,
-            false) : null;
+        ? FSDirStatAndListingOp.getFileInfo(this, iip, false) : null;
   }
 
   /**

+ 79 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -71,6 +71,10 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_EXPIRYTIME_MILLIS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_EXPIRYTIME_MILLIS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_DEFAULT;
@@ -819,6 +823,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_KEY,
           DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT);
 
+      this.writeLockReportingThreshold = conf.getLong(
+          DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY,
+          DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT);
+      this.readLockReportingThreshold = conf.getLong(
+          DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY,
+          DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_DEFAULT);
+
       // For testing purposes, allow the DT secret manager to be started regardless
       // of whether security is enabled.
       alwaysUseDelegationTokensForTests = conf.getBoolean(
@@ -1498,17 +1509,44 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /** Threshold (ms) for long holding write lock report. */
-  static final short WRITELOCK_REPORTING_THRESHOLD = 1000;
+  private long writeLockReportingThreshold;
   /** Last time stamp for write lock. Keep the longest one for multi-entrance.*/
   private long writeLockHeldTimeStamp;
+  /** Threshold (ms) for long holding read lock report. */
+  private long readLockReportingThreshold;
+  /**
+   * Last time stamp for read lock. Keep the longest one for
+   * multi-entrance. This is ThreadLocal since there could be
+   * many read locks held simultaneously.
+   */
+  private static ThreadLocal<Long> readLockHeldTimeStamp =
+      new ThreadLocal<Long>() {
+        @Override
+        public Long initialValue() {
+          return Long.MAX_VALUE;
+        }
+      };
 
   @Override
   public void readLock() {
     this.fsLock.readLock().lock();
+    if (this.fsLock.getReadHoldCount() == 1) {
+      readLockHeldTimeStamp.set(monotonicNow());
+    }
   }
   @Override
   public void readUnlock() {
+    final boolean needReport = this.fsLock.getReadHoldCount() == 1;
+    final long readLockInterval = monotonicNow() - readLockHeldTimeStamp.get();
     this.fsLock.readLock().unlock();
+
+    if (needReport) {
+      readLockHeldTimeStamp.remove();
+      if (readLockInterval > this.readLockReportingThreshold) {
+        LOG.info("FSNamesystem read lock held for " + readLockInterval +
+            " ms via\n" + StringUtils.getStackTrace(Thread.currentThread()));
+      }
+    }
   }
   @Override
   public void writeLock() {
@@ -1532,7 +1570,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     this.fsLock.writeLock().unlock();
 
-    if (needReport && writeLockInterval >= WRITELOCK_REPORTING_THRESHOLD) {
+    if (needReport && writeLockInterval >= this.writeLockReportingThreshold) {
       LOG.info("FSNamesystem write lock held for " + writeLockInterval +
           " ms via\n" + StringUtils.getStackTrace(Thread.currentThread()));
     }
@@ -7004,6 +7042,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         logAuditMessage(sb.toString());
       }
     }
+
+    @Override
+    public void logAuditEvent(boolean succeeded, String userName,
+        InetAddress addr, String cmd, String src, String dst,
+        FileStatus status, UserGroupInformation ugi,
+        DelegationTokenSecretManager dtSecretManager) {
+      this.logAuditEvent(succeeded, userName, addr, cmd, src, dst, status,
+              null /*CallerContext*/, ugi, dtSecretManager);
+    }
+
     public void logAuditMessage(String message) {
       auditLog.info(message);
     }
@@ -7063,5 +7111,34 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return blockManager.getBytesInFuture();
   }
 
+
+  @Override // FSNamesystemMBean
+  public int getNumInMaintenanceLiveDataNodes() {
+    final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
+    getBlockManager().getDatanodeManager().fetchDatanodes(live, null, true);
+    int liveInMaintenance = 0;
+    for (DatanodeDescriptor node : live) {
+      liveInMaintenance += node.isInMaintenance() ? 1 : 0;
+    }
+    return liveInMaintenance;
+  }
+
+  @Override // FSNamesystemMBean
+  public int getNumInMaintenanceDeadDataNodes() {
+    final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
+    getBlockManager().getDatanodeManager().fetchDatanodes(null, dead, true);
+    int deadInMaintenance = 0;
+    for (DatanodeDescriptor node : dead) {
+      deadInMaintenance += node.isInMaintenance() ? 1 : 0;
+    }
+    return deadInMaintenance;
+  }
+
+  @Override // FSNamesystemMBean
+  public int getNumEnteringMaintenanceDataNodes() {
+    return getBlockManager().getDatanodeManager().getEnteringMaintenanceNodes()
+        .size();
+  }
+
 }
 

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

@@ -43,9 +43,9 @@ public abstract class HdfsAuditLogger implements AuditLogger {
 
   /**
    * Same as
-   * {@link #logAuditEvent(boolean, String, InetAddress, String, String, String, FileStatus)}
-   * with additional parameters related to logging delegation token tracking
-   * IDs.
+   * {@link #logAuditEvent(boolean, String, InetAddress, String, String, String,
+   * FileStatus)} with additional parameters related to logging delegation token
+   * tracking IDs.
    * 
    * @param succeeded Whether authorization succeeded.
    * @param userName Name of the user executing the request.
@@ -55,13 +55,28 @@ public abstract class HdfsAuditLogger implements AuditLogger {
    * @param dst Path of affected destination file (if any).
    * @param stat File information for operations that change the file's metadata
    *          (permissions, owner, times, etc).
+   * @param callerContext Context information of the caller
    * @param ugi UserGroupInformation of the current user, or null if not logging
    *          token tracking information
    * @param dtSecretManager The token secret manager, or null if not logging
    *          token tracking information
    */
-  public abstract void logAuditEvent(boolean succeeded, String userName,
+  public void logAuditEvent(boolean succeeded, String userName,
       InetAddress addr, String cmd, String src, String dst,
       FileStatus stat, CallerContext callerContext, UserGroupInformation ugi,
+      DelegationTokenSecretManager dtSecretManager) {
+    logAuditEvent(succeeded, userName, addr, cmd, src, dst, stat,
+                  ugi, dtSecretManager);
+  }
+
+  /**
+   * Same as
+   * {@link #logAuditEvent(boolean, String, InetAddress, String, String,
+   * String, FileStatus, CallerContext, UserGroupInformation,
+   * DelegationTokenSecretManager)} without {@link CallerContext} information.
+   */
+  public abstract void logAuditEvent(boolean succeeded, String userName,
+      InetAddress addr, String cmd, String src, String dst,
+      FileStatus stat, UserGroupInformation ugi,
       DelegationTokenSecretManager dtSecretManager);
 }

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -428,8 +428,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    */
   public final ContentSummary computeAndConvertContentSummary(int snapshotId,
       ContentSummaryComputationContext summary) {
-    ContentCounts counts = computeContentSummary(snapshotId, summary)
-        .getCounts();
+    computeContentSummary(snapshotId, summary);
+    final ContentCounts counts = summary.getCounts();
+    final ContentCounts snapshotCounts = summary.getSnapshotCounts();
     final QuotaCounts q = getQuotaCounts();
     return new ContentSummary.Builder().
         length(counts.getLength()).
@@ -440,6 +441,10 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
         spaceQuota(q.getStorageSpace()).
         typeConsumed(counts.getTypeSpaces()).
         typeQuota(q.getTypeSpaces().asArray()).
+        snapshotLength(snapshotCounts.getLength()).
+        snapshotFileCount(snapshotCounts.getFileCount()).
+        snapshotDirectoryCount(snapshotCounts.getDirectoryCount()).
+        snapshotSpaceConsumed(snapshotCounts.getStoragespace()).
         build();
   }
 

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

@@ -634,6 +634,10 @@ public class INodeDirectory extends INodeWithAdditionalFields
       // computation should include all the deleted files/directories
       sf.computeContentSummary4Snapshot(summary.getBlockStoragePolicySuite(),
           summary.getCounts());
+      // Also compute ContentSummary for snapshotCounts (So we can extract it
+      // later from the ContentSummary of all).
+      sf.computeContentSummary4Snapshot(summary.getBlockStoragePolicySuite(),
+          summary.getSnapshotCounts());
     }
     final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
     if (q != null && snapshotId == Snapshot.CURRENT_STATE_ID) {

+ 11 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -200,8 +200,8 @@ public class INodeFile extends INodeWithAdditionalFields
   public INodeFile(INodeFile that) {
     super(that);
     this.header = that.header;
-    this.blocks = that.blocks;
     this.features = that.features;
+    setBlocks(that.blocks);
   }
   
   public INodeFile(INodeFile that, FileDiffList diffs) {
@@ -271,9 +271,6 @@ public class INodeFile extends INodeWithAdditionalFields
   /** Assert all blocks are complete. */
   private void assertAllBlocksComplete(int numCommittedAllowed,
       short minReplication) {
-    if (blocks == null) {
-      return;
-    }
     for (int i = 0; i < blocks.length; i++) {
       final String err = checkBlockComplete(blocks, i, numCommittedAllowed,
           minReplication);
@@ -342,7 +339,7 @@ public class INodeFile extends INodeWithAdditionalFields
   BlockInfo removeLastBlock(Block oldblock) {
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
-    if (blocks == null || blocks.length == 0) {
+    if (blocks.length == 0) {
       return null;
     }
     int size_1 = blocks.length - 1;
@@ -618,7 +615,7 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   void addBlock(BlockInfo newblock) {
     Preconditions.checkArgument(newblock.isStriped() == this.isStriped());
-    if (this.blocks == null) {
+    if (this.blocks.length == 0) {
       this.setBlocks(new BlockInfo[]{newblock});
     } else {
       int size = this.blocks.length;
@@ -631,12 +628,12 @@ public class INodeFile extends INodeWithAdditionalFields
 
   /** Set the blocks. */
   private void setBlocks(BlockInfo[] blocks) {
-    this.blocks = blocks;
+    this.blocks = (blocks != null ? blocks : BlockInfo.EMPTY_ARRAY);
   }
 
   /** Clear all blocks of the file. */
   public void clearBlocks() {
-    setBlocks(BlockInfo.EMPTY_ARRAY);
+    this.blocks = BlockInfo.EMPTY_ARRAY;
   }
 
   @Override
@@ -836,7 +833,7 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   public final long computeFileSize(boolean includesLastUcBlock,
       boolean usePreferredBlockSize4LastUcBlock) {
-    if (blocks == null || blocks.length == 0) {
+    if (blocks.length == 0) {
       return 0;
     }
     final int last = blocks.length - 1;
@@ -876,10 +873,6 @@ public class INodeFile extends INodeWithAdditionalFields
   // TODO: support EC with heterogeneous storage
   public final QuotaCounts storagespaceConsumedStriped() {
     QuotaCounts counts = new QuotaCounts.Builder().build();
-    if (blocks == null || blocks.length == 0) {
-      return counts;
-    }
-
     for (BlockInfo b : blocks) {
       Preconditions.checkState(b.isStriped());
       long blockSize = b.isComplete() ?
@@ -931,7 +924,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * Return the penultimate allocated block for this file.
    */
   BlockInfo getPenultimateBlock() {
-    if (blocks == null || blocks.length <= 1) {
+    if (blocks.length <= 1) {
       return null;
     }
     return blocks[blocks.length - 2];
@@ -939,12 +932,12 @@ public class INodeFile extends INodeWithAdditionalFields
 
   @Override
   public BlockInfo getLastBlock() {
-    return blocks == null || blocks.length == 0? null: blocks[blocks.length-1];
+    return blocks.length == 0 ? null: blocks[blocks.length-1];
   }
 
   @Override
   public int numBlocks() {
-    return blocks == null ? 0 : blocks.length;
+    return blocks.length;
   }
 
   @VisibleForTesting
@@ -955,7 +948,7 @@ public class INodeFile extends INodeWithAdditionalFields
     out.print(", fileSize=" + computeFileSize(snapshotId));
     // only compare the first block
     out.print(", blocks=");
-    out.print(blocks == null || blocks.length == 0? null: blocks[0]);
+    out.print(blocks.length == 0 ? null: blocks[0]);
     out.println();
   }
 
@@ -1004,7 +997,7 @@ public class INodeFile extends INodeWithAdditionalFields
       long newLength, BlockStoragePolicy bsps,
       QuotaCounts delta) {
     final BlockInfo[] blocks = getBlocks();
-    if (blocks == null || blocks.length == 0) {
+    if (blocks.length == 0) {
       return;
     }
 

+ 42 - 33
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java

@@ -22,7 +22,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.NoSuchElementException;
 
-import com.google.common.collect.ImmutableList;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
@@ -126,6 +125,12 @@ public class INodesInPath {
   static INodesInPath resolve(final INodeDirectory startingDir,
       final byte[][] components, final boolean resolveLink)
       throws UnresolvedLinkException {
+    return resolve(startingDir, components, false, resolveLink);
+  }
+
+  static INodesInPath resolve(final INodeDirectory startingDir,
+      final byte[][] components, final boolean isRaw,
+      final boolean resolveLink) throws UnresolvedLinkException {
     Preconditions.checkArgument(startingDir.compareTo(components[0]) == 0);
 
     INode curNode = startingDir;
@@ -225,7 +230,7 @@ public class INodesInPath {
       System.arraycopy(inodes, 0, newNodes, 0, newNodes.length);
       inodes = newNodes;
     }
-    return new INodesInPath(inodes, components, isSnapshot, snapshotId);
+    return new INodesInPath(inodes, components, isRaw, isSnapshot, snapshotId);
   }
 
   private static boolean shouldUpdateLatestId(int sid, int snapshotId) {
@@ -249,7 +254,8 @@ public class INodesInPath {
     INode[] inodes = new INode[iip.inodes.length];
     System.arraycopy(iip.inodes, 0, inodes, 0, inodes.length);
     inodes[pos] = inode;
-    return new INodesInPath(inodes, iip.path, iip.isSnapshot, iip.snapshotId);
+    return new INodesInPath(inodes, iip.path, iip.isRaw,
+        iip.isSnapshot, iip.snapshotId);
   }
 
   /**
@@ -267,7 +273,8 @@ public class INodesInPath {
     byte[][] path = new byte[iip.path.length + 1][];
     System.arraycopy(iip.path, 0, path, 0, path.length - 1);
     path[path.length - 1] = childName;
-    return new INodesInPath(inodes, path, iip.isSnapshot, iip.snapshotId);
+    return new INodesInPath(inodes, path, iip.isRaw,
+        iip.isSnapshot, iip.snapshotId);
   }
 
   private final byte[][] path;
@@ -279,6 +286,13 @@ public class INodesInPath {
    * true if this path corresponds to a snapshot
    */
   private final boolean isSnapshot;
+
+  /**
+   * true if this is a /.reserved/raw path.  path component resolution strips
+   * it from the path so need to track it separately.
+   */
+  private final boolean isRaw;
+
   /**
    * For snapshot paths, it is the id of the snapshot; or 
    * {@link Snapshot#CURRENT_STATE_ID} if the snapshot does not exist. For 
@@ -287,17 +301,18 @@ public class INodesInPath {
    */
   private final int snapshotId;
 
-  private INodesInPath(INode[] inodes, byte[][] path, boolean isSnapshot,
-      int snapshotId) {
+  private INodesInPath(INode[] inodes, byte[][] path, boolean isRaw,
+      boolean isSnapshot,int snapshotId) {
     Preconditions.checkArgument(inodes != null && path != null);
     this.inodes = inodes;
     this.path = path;
+    this.isRaw = isRaw;
     this.isSnapshot = isSnapshot;
     this.snapshotId = snapshotId;
   }
 
   private INodesInPath(INode[] inodes, byte[][] path) {
-    this(inodes, path, false, CURRENT_STATE_ID);
+    this(inodes, path, false, false, CURRENT_STATE_ID);
   }
 
   /**
@@ -345,6 +360,10 @@ public class INodesInPath {
     return path;
   }
 
+  public byte[] getPathComponent(int i) {
+    return path[i];
+  }
+
   /** @return the full path in string form */
   public String getPath() {
     return DFSUtil.byteArray2PathString(path);
@@ -358,21 +377,6 @@ public class INodesInPath {
     return DFSUtil.byteArray2PathString(path, 0, pos + 1); // it's a length...
   }
 
-  /**
-   * @param offset start endpoint (inclusive)
-   * @param length number of path components
-   * @return sub-list of the path
-   */
-  public List<String> getPath(int offset, int length) {
-    Preconditions.checkArgument(offset >= 0 && length >= 0 && offset + length
-        <= path.length);
-    ImmutableList.Builder<String> components = ImmutableList.builder();
-    for (int i = offset; i < offset + length; i++) {
-      components.add(DFSUtil.bytes2String(path[i]));
-    }
-    return components.build();
-  }
-
   public int length() {
     return inodes.length;
   }
@@ -400,7 +404,7 @@ public class INodesInPath {
     final byte[][] apath = new byte[length][];
     System.arraycopy(this.inodes, 0, anodes, 0, length);
     System.arraycopy(this.path, 0, apath, 0, length);
-    return new INodesInPath(anodes, apath, false, snapshotId);
+    return new INodesInPath(anodes, apath, isRaw, false, snapshotId);
   }
 
   /**
@@ -413,22 +417,17 @@ public class INodesInPath {
   }
 
   /**
-   * @return a new INodesInPath instance that only contains exisitng INodes.
+   * @return a new INodesInPath instance that only contains existing INodes.
    * Note that this method only handles non-snapshot paths.
    */
   public INodesInPath getExistingINodes() {
     Preconditions.checkState(!isSnapshot());
-    int i = 0;
-    for (; i < inodes.length; i++) {
-      if (inodes[i] == null) {
-        break;
+    for (int i = inodes.length; i > 0; i--) {
+      if (inodes[i - 1] != null) {
+        return (i == inodes.length) ? this : getAncestorINodesInPath(i);
       }
     }
-    INode[] existing = new INode[i];
-    byte[][] existingPath = new byte[i][];
-    System.arraycopy(inodes, 0, existing, 0, i);
-    System.arraycopy(path, 0, existingPath, 0, i);
-    return new INodesInPath(existing, existingPath, false, snapshotId);
+    return null;
   }
 
   /**
@@ -438,10 +437,20 @@ public class INodesInPath {
     return this.isSnapshot;
   }
 
+  /**
+   * @return if .snapshot is the last path component.
+   */
   boolean isDotSnapshotDir() {
     return isDotSnapshotDir(getLastLocalName());
   }
 
+  /**
+   * @return if this is a /.reserved/raw path.
+   */
+  public boolean isRaw() {
+    return isRaw;
+  }
+
   private static String toString(INode inode) {
     return inode == null? null: inode.getLocalName();
   }

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java

@@ -208,4 +208,19 @@ public interface FSNamesystemMBean {
    * Return total time spent doing sync operations on FSEditLog.
    */
   String getTotalSyncTimes();
+
+  /**
+   * @return Number of IN_MAINTENANCE live data nodes
+   */
+  int getNumInMaintenanceLiveDataNodes();
+
+  /**
+   * @return Number of IN_MAINTENANCE dead data nodes
+   */
+  int getNumInMaintenanceDeadDataNodes();
+
+  /**
+   * @return Number of ENTERING_MAINTENANCE data nodes
+   */
+  int getNumEnteringMaintenanceDataNodes();
 }

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

@@ -789,11 +789,13 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(TokenServiceParam.NAME) @DefaultValue(TokenServiceParam.DEFAULT)
           final TokenServiceParam tokenService,
       @QueryParam(NoRedirectParam.NAME) @DefaultValue(NoRedirectParam.DEFAULT)
-          final NoRedirectParam noredirect
+          final NoRedirectParam noredirect,
+      @QueryParam(StartAfterParam.NAME) @DefaultValue(StartAfterParam.DEFAULT)
+          final StartAfterParam startAfter
       ) throws IOException, InterruptedException {
     return get(ugi, delegation, username, doAsUser, ROOT, op, offset, length,
         renewer, bufferSize, xattrNames, xattrEncoding, excludeDatanodes, fsAction,
-        tokenKind, tokenService, noredirect);
+        tokenKind, tokenService, noredirect, startAfter);
   }
 
   /** Handle HTTP GET request. */
@@ -832,12 +834,14 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(TokenServiceParam.NAME) @DefaultValue(TokenServiceParam.DEFAULT)
           final TokenServiceParam tokenService,
       @QueryParam(NoRedirectParam.NAME) @DefaultValue(NoRedirectParam.DEFAULT)
-          final NoRedirectParam noredirect
+          final NoRedirectParam noredirect,
+      @QueryParam(StartAfterParam.NAME) @DefaultValue(StartAfterParam.DEFAULT)
+          final StartAfterParam startAfter
       ) throws IOException, InterruptedException {
 
     init(ugi, delegation, username, doAsUser, path, op, offset, length,
         renewer, bufferSize, xattrEncoding, excludeDatanodes, fsAction,
-        tokenKind, tokenService);
+        tokenKind, tokenService, startAfter);
 
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
@@ -846,7 +850,7 @@ public class NamenodeWebHdfsMethods {
           return get(ugi, delegation, username, doAsUser,
               path.getAbsolutePath(), op, offset, length, renewer, bufferSize,
               xattrNames, xattrEncoding, excludeDatanodes, fsAction, tokenKind,
-              tokenService, noredirect);
+              tokenService, noredirect, startAfter);
         } finally {
           reset();
         }
@@ -871,7 +875,8 @@ public class NamenodeWebHdfsMethods {
       final FsActionParam fsAction,
       final TokenKindParam tokenKind,
       final TokenServiceParam tokenService,
-      final NoRedirectParam noredirectParam
+      final NoRedirectParam noredirectParam,
+      final StartAfterParam startAfter
       ) throws IOException, URISyntaxException {
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
     final Configuration conf = (Configuration) context
@@ -994,6 +999,16 @@ public class NamenodeWebHdfsMethods {
       np.checkAccess(fullpath, FsAction.getFsAction(fsAction.getValue()));
       return Response.ok().build();
     }
+    case LISTSTATUS_BATCH:
+    {
+      byte[] start = HdfsFileStatus.EMPTY_NAME;
+      if (startAfter != null && startAfter.getValue() != null) {
+        start = startAfter.getValue().getBytes(Charsets.UTF_8);
+      }
+      final DirectoryListing listing = getDirectoryListing(np, fullpath, start);
+      final String js = JsonUtil.toJsonString(listing);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java

@@ -186,6 +186,8 @@ public class OfflineImageViewer {
     
     options.addOption("p", "processor", true, "");
     options.addOption("h", "help", false, "");
+    options.addOption("maxSize", true, "");
+    options.addOption("step", true, "");
     options.addOption("skipBlocks", false, "");
     options.addOption("printToScreen", false, "");
     options.addOption("delimiter", true, "");

+ 39 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -96,6 +96,16 @@ public class JsonUtil {
     if (status == null) {
       return null;
     }
+    final Map<String, Object> m = toJsonMap(status);
+    try {
+      return includeType ?
+          toJsonString(FileStatus.class, m) : MAPPER.writeValueAsString(m);
+    } catch (IOException ignored) {
+    }
+    return null;
+  }
+
+  private static Map<String, Object> toJsonMap(HdfsFileStatus status) {
     final Map<String, Object> m = new TreeMap<String, Object>();
     m.put("pathSuffix", status.getLocalName());
     m.put("type", WebHdfsConstants.PathType.valueOf(status));
@@ -121,12 +131,7 @@ public class JsonUtil {
     m.put("fileId", status.getFileId());
     m.put("childrenNum", status.getChildrenNum());
     m.put("storagePolicy", status.getStoragePolicy());
-    try {
-      return includeType ?
-          toJsonString(FileStatus.class, m) : MAPPER.writeValueAsString(m);
-    } catch (IOException ignored) {
-    }
-    return null;
+    return m;
   }
 
   /** Convert an ExtendedBlock to a Json map. */
@@ -227,6 +232,34 @@ public class JsonUtil {
     return m;
   }
 
+  public static String toJsonString(final DirectoryListing listing) throws
+      IOException {
+
+    if (listing == null) {
+      return null;
+    }
+
+    final Map<String, Object> m = new TreeMap<>();
+    m.put("partialListing", toJsonArray(listing.getPartialListing()));
+    m.put("remainingEntries", listing.getRemainingEntries());
+    return MAPPER.writeValueAsString(m);
+  }
+
+  private static Object[] toJsonArray(HdfsFileStatus[] statuses) throws
+      IOException {
+    if (statuses == null) {
+      return null;
+    }
+    if (statuses.length == 0) {
+      return EMPTY_OBJECT_ARRAY;
+    }
+    final Object[] a = new Object[statuses.length];
+    for (int i = 0; i < statuses.length; i++) {
+      a[i] = toJsonMap(statuses[i]);
+    }
+    return a;
+  }
+
   /** Convert a LocatedBlock[] to a Json array. */
   private static Object[] toJsonArray(final List<LocatedBlock> array
       ) throws IOException {

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

@@ -2614,6 +2614,24 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.write-lock-reporting-threshold-ms</name>
+  <value>1000</value>
+  <description>When a write lock is held on the namenode for a long time,
+    this will be logged as the lock is released. This sets how long the
+    lock must be held for logging to occur.
+  </description>
+</property>
+
+<property>
+  <name>dfs.namenode.read-lock-reporting-threshold-ms</name>
+  <value>5000</value>
+  <description>When a read lock is held on the namenode for a long time,
+    this will be logged as the lock is released. This sets how long the
+    lock must be held for logging to occur.
+  </description>
+</property>
+
 <property>
   <name>dfs.namenode.startup.delay.block.deletion.sec</name>
   <value>0</value>
@@ -3702,6 +3720,46 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.mover.keytab.enabled</name>
+  <value>false</value>
+  <description>
+    Set to true to enable login using a keytab for Kerberized Hadoop.
+  </description>
+</property>
+
+<property>
+  <name>dfs.mover.address</name>
+  <value>0.0.0.0:0</value>
+  <description>
+    The hostname used for a keytab based Kerberos login. Keytab based login
+    can be enabled with dfs.mover.keytab.enabled.
+  </description>
+</property>
+
+<property>
+  <name>dfs.mover.keytab.file</name>
+  <value></value>
+  <description>
+    The keytab file used by the Mover to login as its
+    service principal. The principal name is configured with
+    dfs.mover.kerberos.principal. Keytab based login can be
+    enabled with dfs.mover.keytab.enabled.
+  </description>
+</property>
+
+<property>
+  <name>dfs.mover.kerberos.principal</name>
+  <value></value>
+  <description>
+    The Mover principal. This is typically set to
+    mover/_HOST@REALM.TLD. The Mover will substitute _HOST with its
+    own fully qualified hostname at startup. The _HOST placeholder
+    allows using the same configuration setting on different servers.
+    Keytab based login can be enabled with dfs.mover.keytab.enabled.
+  </description>
+</property>
+
 <property>
   <name>dfs.namenode.audit.log.async</name>
   <value>false</value>

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

@@ -113,7 +113,7 @@ Upgrading to a version of HDFS with snapshots
 
 The HDFS snapshot feature introduces a new reserved path name used to
 interact with snapshots: `.snapshot`. When upgrading from an
-older version of HDFS, existing paths named `.snapshot` need
+older version of HDFS which does not support snapshots, existing paths named `.snapshot` need
 to first be renamed or deleted to avoid conflicting with the reserved path.
 See the upgrade section in
 [the HDFS user guide](HdfsUserGuide.html#Upgrade_and_Rollback)

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

@@ -37,6 +37,7 @@ WebHDFS REST API
         * [Truncate a File](#Truncate_a_File)
         * [Status of a File/Directory](#Status_of_a_FileDirectory)
         * [List a Directory](#List_a_Directory)
+        * [Iteratively List a Directory](#Iteratively_List_a_Directory)
     * [Other File System Operations](#Other_File_System_Operations)
         * [Get Content Summary of a Directory](#Get_Content_Summary_of_a_Directory)
         * [Get File Checksum](#Get_File_Checksum)
@@ -143,6 +144,7 @@ The HTTP REST API supports the complete [FileSystem](../../api/org/apache/hadoop
     * [`OPEN`](#Open_and_Read_a_File) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).open)
     * [`GETFILESTATUS`](#Status_of_a_FileDirectory) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getFileStatus)
     * [`LISTSTATUS`](#List_a_Directory) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).listStatus)
+    * [`LISTSTATUS_BATCH`](#Iteratively_List_a_Directory) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).listStatusIterator)
     * [`GETCONTENTSUMMARY`](#Get_Content_Summary_of_a_Directory) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getContentSummary)
     * [`GETFILECHECKSUM`](#Get_File_Checksum) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getFileChecksum)
     * [`GETHOMEDIRECTORY`](#Get_Home_Directory) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getHomeDirectory)
@@ -589,6 +591,109 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getFileSt
 
 See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).listStatus
 
+### Iteratively List a Directory
+
+* Submit a HTTP GET request.
+
+        curl -i  "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=LISTSTATUS_BATCH&startAfter=<CHILD>"
+
+    The client receives a response with a batch of [`FileStatuses` JSON object](#FileStatuses_JSON_Schema), as well as iteration information:
+
+        HTTP/1.1 200 OK
+        Cache-Control: no-cache
+        Expires: Tue, 30 Aug 2016 16:42:16 GMT
+        Date: Tue, 30 Aug 2016 16:42:16 GMT
+        Pragma: no-cache
+        Expires: Tue, 30 Aug 2016 16:42:16 GMT
+        Date: Tue, 30 Aug 2016 16:42:16 GMT
+        Pragma: no-cache
+        Content-Type: application/json
+        X-FRAME-OPTIONS: SAMEORIGIN
+        Transfer-Encoding: chunked
+        Server: Jetty(6.1.26)
+
+        {
+            "partialListing": [
+                {
+                    "accessTime": 0,
+                    "blockSize": 0,
+                    "childrenNum": 0,
+                    "fileId": 16389,
+                    "group": "supergroup",
+                    "length": 0,
+                    "modificationTime": 1472575493064,
+                    "owner": "andrew",
+                    "pathSuffix": "anotherdir",
+                    "permission": "755",
+                    "replication": 0,
+                    "storagePolicy": 0,
+                    "type": "DIRECTORY"
+                },
+                {
+                    "accessTime": 0,
+                    "blockSize": 0,
+                    "childrenNum": 0,
+                    "fileId": 16386,
+                    "group": "supergroup",
+                    "length": 0,
+                    "modificationTime": 1472575274776,
+                    "owner": "andrew",
+                    "pathSuffix": "somedir",
+                    "permission": "755",
+                    "replication": 0,
+                    "storagePolicy": 0,
+                    "type": "DIRECTORY"
+                }
+            ],
+            "remainingEntries": 1
+        }
+
+
+If `remainingEntries` is non-zero, there are additional entries in the directory.
+To query the next batch, set the `startAfter` parameter to the `pathSuffix` of the last item returned in the current batch. For example:
+
+        curl -i  "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=LISTSTATUS_BATCH&startAfter=somedir"
+
+Which will return the next batch of directory entries:
+
+        HTTP/1.1 200 OK
+        Cache-Control: no-cache
+        Expires: Tue, 30 Aug 2016 16:46:23 GMT
+        Date: Tue, 30 Aug 2016 16:46:23 GMT
+        Pragma: no-cache
+        Expires: Tue, 30 Aug 2016 16:46:23 GMT
+        Date: Tue, 30 Aug 2016 16:46:23 GMT
+        Pragma: no-cache
+        Content-Type: application/json
+        X-FRAME-OPTIONS: SAMEORIGIN
+        Transfer-Encoding: chunked
+        Server: Jetty(6.1.26)
+
+        {
+            "partialListing": [
+                {
+                    "accessTime": 1472575333568,
+                    "blockSize": 1024,
+                    "childrenNum": 0,
+                    "fileId": 16388,
+                    "group": "supergroup",
+                    "length": 224,
+                    "modificationTime": 1472575334222,
+                    "owner": "andrew",
+                    "pathSuffix": "somefile",
+                    "permission": "644",
+                    "replication": 3,
+                    "storagePolicy": 0,
+                    "type": "FILE"
+                }
+            ],
+            "remainingEntries": 0
+        }
+
+Batch size is controlled by the `dfs.ls.limit` option on the NameNode.
+
+See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).listStatusIterator
+
 Other File System Operations
 ----------------------------
 

Certains fichiers n'ont pas été affichés car il y a eu trop de fichiers modifiés dans ce diff