浏览代码

Merge branch 'trunk' into HDFS-7240

Anu Engineer 8 年之前
父节点
当前提交
1dc5c7ebfc
共有 100 个文件被更改,包括 2426 次插入713 次删除
  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
 .settings
 target
 target
 build
 build
+
+# External tool builders
+*/.externalToolBuilders
+*/maven-eclipse.xml
+
 hadoop-common-project/hadoop-kms/downloads/
 hadoop-common-project/hadoop-kms/downloads/
 hadoop-hdfs-project/hadoop-hdfs/downloads
 hadoop-hdfs-project/hadoop-hdfs/downloads
 hadoop-hdfs-project/hadoop-hdfs-httpfs/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();

文件差异内容过多而无法显示
+ 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>
     <kdc.resource.dir>src/test/resources/kdc</kdc.resource.dir>
     <hadoop.component>common</hadoop.component>
     <hadoop.component>common</hadoop.component>
     <is.hadoop.component>true</is.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.dir>../etc/hadoop</wsce.config.dir>
     <wsce.config.file>wsce-site.xml</wsce.config.file>
     <wsce.config.file>wsce-site.xml</wsce.config.file>
   </properties>
   </properties>
@@ -512,6 +513,7 @@
             <exclude>src/test/resources/test.har/_masterindex</exclude>
             <exclude>src/test/resources/test.har/_masterindex</exclude>
             <exclude>src/test/resources/test.har/part-0</exclude>
             <exclude>src/test/resources/test.har/part-0</exclude>
             <exclude>src/test/resources/javakeystoreprovider.password</exclude>
             <exclude>src/test/resources/javakeystoreprovider.password</exclude>
+            <exclude>dev-support/jdiff-workaround.patch</exclude>
           </excludes>
           </excludes>
         </configuration>
         </configuration>
       </plugin>
       </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_ROOT_LOGGER=${HADOOP_ROOT_LOGGER:-${HADOOP_LOGLEVEL},console}
   HADOOP_DAEMON_ROOT_LOGGER=${HADOOP_DAEMON_ROOT_LOGGER:-${HADOOP_LOGLEVEL},RFA}
   HADOOP_DAEMON_ROOT_LOGGER=${HADOOP_DAEMON_ROOT_LOGGER:-${HADOOP_LOGLEVEL},RFA}
   HADOOP_SECURITY_LOGGER=${HADOOP_SECURITY_LOGGER:-INFO,NullAppender}
   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_LOG_DIR=${HADOOP_SECURE_LOG_DIR:-${HADOOP_LOG_DIR}}
   HADOOP_SECURE_PID_DIR=${HADOOP_SECURE_PID_DIR:-${HADOOP_PID_DIR}}
   HADOOP_SECURE_PID_DIR=${HADOOP_SECURE_PID_DIR:-${HADOOP_PID_DIR}}
   HADOOP_SSH_PARALLEL=${HADOOP_SSH_PARALLEL:-10}
   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)
   # User can specify hostnames or a file where the hostnames are (not both)
   if [[ -n "${HADOOP_WORKERS}" && -n "${HADOOP_WORKER_NAMES}" ]] ; then
   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
     exit 1
   elif [[ -z "${HADOOP_WORKER_NAMES}" ]]; then
   elif [[ -z "${HADOOP_WORKER_NAMES}" ]]; then
     if [[ -n "${HADOOP_WORKERS}" ]]; 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)) {
                new ChecksumFSInputChecker(fs, file)) {
         checker.seek(position);
         checker.seek(position);
         nread = checker.read(b, off, len);
         nread = checker.read(b, off, len);
-        checker.close();
       }
       }
       return nread;
       return nread;
     }
     }
@@ -356,12 +355,14 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   @Override
   @Override
   public FSDataOutputStream append(Path f, int bufferSize,
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
       Progressable progress) throws IOException {
-    throw new IOException("Not supported");
+    throw new UnsupportedOperationException("Append is not supported "
+        + "by ChecksumFileSystem");
   }
   }
 
 
   @Override
   @Override
   public boolean truncate(Path f, long newLength) throws IOException {
   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)) {
                new ChecksumFSInputChecker(fs, file)) {
         checker.seek(position);
         checker.seek(position);
         nread = checker.read(b, off, len);
         nread = checker.read(b, off, len);
-        checker.close();
       }
       }
       return nread;
       return nread;
     }
     }
@@ -297,7 +296,8 @@ public abstract class ChecksumFs extends FilterFs {
 
 
   @Override
   @Override
   public boolean truncate(Path f, long newLength) throws IOException {
   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 length;
   private long fileCount;
   private long fileCount;
   private long directoryCount;
   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
   /** We don't use generics. Instead override spaceConsumed and other methods
       in order to keep backward compatibility. */
       in order to keep backward compatibility. */
@@ -56,6 +61,26 @@ public class ContentSummary extends QuotaUsage implements Writable{
       return this;
       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
     @Override
     public Builder quota(long quota){
     public Builder quota(long quota){
       super.quota(quota);
       super.quota(quota);
@@ -107,6 +132,10 @@ public class ContentSummary extends QuotaUsage implements Writable{
     private long length;
     private long length;
     private long fileCount;
     private long fileCount;
     private long directoryCount;
     private long directoryCount;
+    private long snapshotLength;
+    private long snapshotFileCount;
+    private long snapshotDirectoryCount;
+    private long snapshotSpaceConsumed;
   }
   }
 
 
   /** Constructor deprecated by ContentSummary.Builder*/
   /** Constructor deprecated by ContentSummary.Builder*/
@@ -142,17 +171,37 @@ public class ContentSummary extends QuotaUsage implements Writable{
     this.length = builder.length;
     this.length = builder.length;
     this.fileCount = builder.fileCount;
     this.fileCount = builder.fileCount;
     this.directoryCount = builder.directoryCount;
     this.directoryCount = builder.directoryCount;
+    this.snapshotLength = builder.snapshotLength;
+    this.snapshotFileCount = builder.snapshotFileCount;
+    this.snapshotDirectoryCount = builder.snapshotDirectoryCount;
+    this.snapshotSpaceConsumed = builder.snapshotSpaceConsumed;
   }
   }
 
 
   /** @return the length */
   /** @return the length */
   public long getLength() {return length;}
   public long getLength() {return length;}
 
 
+  public long getSnapshotLength() {
+    return snapshotLength;
+  }
+
   /** @return the directory count */
   /** @return the directory count */
   public long getDirectoryCount() {return directoryCount;}
   public long getDirectoryCount() {return directoryCount;}
 
 
+  public long getSnapshotDirectoryCount() {
+    return snapshotDirectoryCount;
+  }
+
   /** @return the file count */
   /** @return the file count */
   public long getFileCount() {return fileCount;}
   public long getFileCount() {return fileCount;}
 
 
+  public long getSnapshotFileCount() {
+    return snapshotFileCount;
+  }
+
+  public long getSnapshotSpaceConsumed() {
+    return snapshotSpaceConsumed;
+  }
+
   @Override
   @Override
   @InterfaceAudience.Private
   @InterfaceAudience.Private
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
@@ -180,9 +229,14 @@ public class ContentSummary extends QuotaUsage implements Writable{
     if (this == to) {
     if (this == to) {
       return true;
       return true;
     } else if (to instanceof ContentSummary) {
     } 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);
           super.equals(to);
     } else {
     } else {
       return super.equals(to);
       return super.equals(to);
@@ -191,7 +245,9 @@ public class ContentSummary extends QuotaUsage implements Writable{
 
 
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
-    long result = getLength() ^ getFileCount() ^ getDirectoryCount();
+    long result = getLength() ^ getFileCount() ^ getDirectoryCount()
+        ^ getSnapshotLength() ^ getSnapshotFileCount()
+        ^ getSnapshotDirectoryCount() ^ getSnapshotSpaceConsumed();
     return ((int) result) ^ super.hashCode();
     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
    * @param qOption a flag indicating if quota needs to be printed or not
    * @return the string representation of the object
    * @return the string representation of the object
   */
   */
+  @Override
   public String toString(boolean qOption) {
   public String toString(boolean qOption) {
     return toString(qOption, false);
     return toString(qOption, false);
   }
   }
 
 
   /** Return the string representation of the object in the output format.
   /** 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 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
    * @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 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.
    * 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 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
    * @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,
   public String toString(boolean qOption, boolean hOption,
                          boolean tOption, List<StorageType> types) {
                          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 = "";
     String prefix = "";
 
 
     if (tOption) {
     if (tOption) {
@@ -296,10 +388,17 @@ public class ContentSummary extends QuotaUsage implements Writable{
       prefix = getQuotaUsage(hOption);
       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 {
   public static enum Rename {
     NONE((byte) 0), // No options
     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;
     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 Path trash; // path to trash directory
   protected long deletionInterval; // deletion interval for Emptier
   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
    * Used to setup the trash policy. Must be implemented by all TrashPolicy
    * implementations. Different from initialize(conf, fs, home), this one does
    * 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;
   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
    * Get an instance of the configured TrashPolicy based on the value
    * of the configuration parameter fs.trash.classname.
    * 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);
     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
   @Override
   public void initialize(Configuration conf, FileSystem fs) {
   public void initialize(Configuration conf, FileSystem fs) {
     this.fs = fs;
     this.fs = fs;
@@ -106,6 +95,7 @@ public class TrashPolicyDefault extends TrashPolicy {
     return deletionInterval != 0;
     return deletionInterval != 0;
   }
   }
 
 
+  @SuppressWarnings("deprecation")
   @Override
   @Override
   public boolean moveToTrash(Path path) throws IOException {
   public boolean moveToTrash(Path path) throws IOException {
     if (!isEnabled())
     if (!isEnabled())
@@ -156,10 +146,11 @@ public class TrashPolicyDefault extends TrashPolicy {
           trashPath = new Path(orig + Time.now());
           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) {
       } catch (IOException e) {
         cause = 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
   @Override
   public FSDataOutputStream append(Path f, int bufferSize,
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
       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\".";
       "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_DIR_NOTEMPTY = "Directory: %s is not empty.";
   public static final String E_FILE_CHECK_FAILED = "File check failed";
   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_SPATH_NOTEXIST = "Source path %s does not exist";
   public static final String E_DPATH_EXIST =
   public static final String E_DPATH_EXIST =
       "Destination path %s already exist, cannot rename!";
       "Destination path %s already exist, cannot rename!";
@@ -578,7 +577,8 @@ public class SFTPFileSystem extends FileSystem {
   public FSDataOutputStream append(Path f, int bufferSize,
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress)
       Progressable progress)
       throws IOException {
       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_HUMAN = "h";
   private static final String OPTION_HEADER = "v";
   private static final String OPTION_HEADER = "v";
   private static final String OPTION_TYPE = "t";
   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.
   //return the quota, namespace count and disk space usage.
   private static final String OPTION_QUOTA_AND_USAGE = "u";
   private static final String OPTION_QUOTA_AND_USAGE = "u";
 
 
@@ -58,7 +60,8 @@ public class Count extends FsCommand {
   public static final String USAGE =
   public static final String USAGE =
       "[-" + OPTION_QUOTA + "] [-" + OPTION_HUMAN + "] [-" + OPTION_HEADER
       "[-" + OPTION_QUOTA + "] [-" + OPTION_HUMAN + "] [-" + OPTION_HEADER
           + "] [-" + OPTION_TYPE + " [<storage type>]] [-" +
           + "] [-" + OPTION_TYPE + " [<storage type>]] [-" +
-          OPTION_QUOTA_AND_USAGE + "] <path> ...";
+          OPTION_QUOTA_AND_USAGE + "] [-" + OPTION_EXCLUDE_SNAPSHOT
+          + "] <path> ...";
   public static final String DESCRIPTION =
   public static final String DESCRIPTION =
       "Count the number of directories, files and bytes under the paths\n" +
       "Count the number of directories, files and bytes under the paths\n" +
           "that match the specified file pattern.  The output columns are:\n" +
           "that match the specified file pattern.  The output columns are:\n" +
@@ -72,6 +75,8 @@ public class Count extends FsCommand {
           "The -" + OPTION_HUMAN +
           "The -" + OPTION_HUMAN +
           " option shows file sizes in human readable format.\n" +
           " option shows file sizes in human readable format.\n" +
           "The -" + OPTION_HEADER + " option displays a header line.\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" +
           "The -" + OPTION_TYPE + " option displays quota by storage types.\n" +
           "It must be used with -" + OPTION_QUOTA + " option.\n" +
           "It must be used with -" + OPTION_QUOTA + " option.\n" +
           "If a comma-separated list of storage types is given after the -" +
           "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 boolean showQuotabyType;
   private List<StorageType> storageTypes = null;
   private List<StorageType> storageTypes = null;
   private boolean showQuotasAndUsageOnly;
   private boolean showQuotasAndUsageOnly;
+  private boolean excludeSnapshots;
 
 
   /** Constructor */
   /** Constructor */
   public Count() {}
   public Count() {}
@@ -106,7 +112,8 @@ public class Count extends FsCommand {
   @Override
   @Override
   protected void processOptions(LinkedList<String> args) {
   protected void processOptions(LinkedList<String> args) {
     CommandFormat cf = new CommandFormat(1, Integer.MAX_VALUE,
     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.addOptionWithValue(OPTION_TYPE);
     cf.parse(args);
     cf.parse(args);
     if (args.isEmpty()) { // default path is the current working directory
     if (args.isEmpty()) { // default path is the current working directory
@@ -115,6 +122,7 @@ public class Count extends FsCommand {
     showQuotas = cf.getOpt(OPTION_QUOTA);
     showQuotas = cf.getOpt(OPTION_QUOTA);
     humanReadable = cf.getOpt(OPTION_HUMAN);
     humanReadable = cf.getOpt(OPTION_HUMAN);
     showQuotasAndUsageOnly = cf.getOpt(OPTION_QUOTA_AND_USAGE);
     showQuotasAndUsageOnly = cf.getOpt(OPTION_QUOTA_AND_USAGE);
+    excludeSnapshots = cf.getOpt(OPTION_EXCLUDE_SNAPSHOT);
 
 
     if (showQuotas || showQuotasAndUsageOnly) {
     if (showQuotas || showQuotasAndUsageOnly) {
       String types = cf.getOptValue(OPTION_TYPE);
       String types = cf.getOptValue(OPTION_TYPE);
@@ -125,6 +133,11 @@ public class Count extends FsCommand {
       } else {
       } else {
         showQuotabyType = false;
         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)) {
     if (cf.getOpt(OPTION_HEADER)) {
@@ -163,7 +176,8 @@ public class Count extends FsCommand {
           storageTypes) + src);
           storageTypes) + src);
     } else {
     } else {
       ContentSummary summary = src.fs.getContentSummary(src.path);
       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 */
   /** show disk usage */
   public static class Du extends FsUsage {
   public static class Du extends FsUsage {
     public static final String NAME = "du";
     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 =
     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;
     protected boolean summary = false;
+    private boolean excludeSnapshots = false;
     
     
     @Override
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
     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);
       cf.parse(args);
       humanReadable = cf.getOpt("h");
       humanReadable = cf.getOpt("h");
       summary = cf.getOpt("s");
       summary = cf.getOpt("s");
+      excludeSnapshots = cf.getOpt("x");
       if (args.isEmpty()) args.add(Path.CUR_DIR);
       if (args.isEmpty()) args.add(Path.CUR_DIR);
     }
     }
 
 
@@ -156,6 +159,10 @@ class FsUsage extends FsCommand {
       ContentSummary contentSummary = item.fs.getContentSummary(item.path);
       ContentSummary contentSummary = item.fs.getContentSummary(item.path);
       long length = contentSummary.getLength();
       long length = contentSummary.getLength();
       long spaceConsumed = contentSummary.getSpaceConsumed();
       long spaceConsumed = contentSummary.getSpaceConsumed();
+      if (excludeSnapshots) {
+        length -= contentSummary.getSnapshotLength();
+        spaceConsumed -= contentSummary.getSnapshotSpaceConsumed();
+      }
       usagesTable.addRow(formatSize(length), formatSize(spaceConsumed), item);
       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()) {
     if (retryInfo.isFail()) {
       // fail.
       // fail.
       if (retryInfo.action.reason != null) {
       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;
       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.
  * This class is immutable.
  */
  */
 @InterfaceAudience.LimitedPrivate({"HBase", "HDFS", "Hive", "MapReduce",
 @InterfaceAudience.LimitedPrivate({"HBase", "HDFS", "Hive", "MapReduce",
-    "Pig", "YARN"})
+    "Pig", "Spark", "YARN"})
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public final class CallerContext {
 public final class CallerContext {
   public static final Charset SIGNATURE_ENCODING = StandardCharsets.UTF_8;
   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 {
     try {
       fs.append(basePath);
       fs.append(basePath);
+    } catch (UnsupportedOperationException ex) {
+      canAppend = false;
     } catch (IOException ex) {
     } catch (IOException ex) {
-      if (ex.getMessage().equals("Not supported")) {
-        canAppend = false;
-      }
+      // Ignore. The operation is supported.
     }
     }
 
 
     return canAppend;
     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);
         }, listenerThreadPool);
+        loadFromZKCache(false);
       }
       }
     } catch (Exception e) {
     } catch (Exception e) {
       throw new IOException("Could not start PathChildrenCache for keys", e);
       throw new IOException("Could not start PathChildrenCache for keys", e);
@@ -389,6 +390,7 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
             }
             }
           }
           }
         }, listenerThreadPool);
         }, listenerThreadPool);
+        loadFromZKCache(true);
       }
       }
     } catch (Exception e) {
     } catch (Exception e) {
       throw new IOException("Could not start PathChildrenCache for tokens", e);
       throw new IOException("Could not start PathChildrenCache for tokens", e);
@@ -396,6 +398,43 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
     super.startThreads();
     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 {
   private void processKeyAddOrUpdate(byte[] data) throws IOException {
     ByteArrayInputStream bin = new ByteArrayInputStream(data);
     ByteArrayInputStream bin = new ByteArrayInputStream(data);
     DataInputStream din = new DataInputStream(bin);
     DataInputStream din = new DataInputStream(bin);
@@ -890,4 +929,9 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
   public ExecutorService getListenerThreadPool() {
   public ExecutorService getListenerThreadPool() {
     return listenerThreadPool;
     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.NameValuePair;
 import org.apache.http.client.utils.URLEncodedUtils;
 import org.apache.http.client.utils.URLEncodedUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 import javax.servlet.FilterChain;
 import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
 import javax.servlet.FilterConfig;
@@ -81,6 +83,9 @@ public class DelegationTokenAuthenticationFilter
   private static final String ERROR_EXCEPTION_JSON = "exception";
   private static final String ERROR_EXCEPTION_JSON = "exception";
   private static final String ERROR_MESSAGE_JSON = "message";
   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
    * Sets an external <code>DelegationTokenSecretManager</code> instance to
    * manage creation and verification of Delegation Tokens.
    * manage creation and verification of Delegation Tokens.
@@ -261,6 +266,11 @@ public class DelegationTokenAuthenticationFilter
             HttpExceptionUtils.createServletExceptionResponse(response,
             HttpExceptionUtils.createServletExceptionResponse(response,
                 HttpServletResponse.SC_FORBIDDEN, ex);
                 HttpServletResponse.SC_FORBIDDEN, ex);
             requestCompleted = true;
             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();
         shexec.execute();
       } catch (ExitCodeException e) {
       } catch (ExitCodeException e) {
         // ignore the exit code of the script
         // ignore the exit code of the script
+        exceptionStackTrace = StringUtils.stringifyException(e);
         status = HealthCheckerExitStatus.FAILED_WITH_EXIT_CODE;
         status = HealthCheckerExitStatus.FAILED_WITH_EXIT_CODE;
         // On Windows, we will not hit the Stream closed IOException
         // On Windows, we will not hit the Stream closed IOException
         // thrown by stdout buffered reader for timeout event.
         // thrown by stdout buffered reader for timeout event.
@@ -162,7 +163,7 @@ public class NodeHealthScriptRunner extends AbstractService {
         setHealthStatus(false, exceptionStackTrace);
         setHealthStatus(false, exceptionStackTrace);
         break;
         break;
       case FAILED_WITH_EXIT_CODE:
       case FAILED_WITH_EXIT_CODE:
-        setHealthStatus(true, "", now);
+        setHealthStatus(false, exceptionStackTrace);
         break;
         break;
       case FAILED:
       case FAILED:
         setHealthStatus(false, shexec.getOutput());
         setHealthStatus(false, shexec.getOutput());

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

@@ -877,22 +877,27 @@
 <property>
 <property>
   <name>fs.s3a.aws.credentials.provider</name>
   <name>fs.s3a.aws.credentials.provider</name>
   <description>
   <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
     Specifying org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider allows
     anonymous access to a publicly accessible S3 bucket without any credentials.
     anonymous access to a publicly accessible S3 bucket without any credentials.
     Please note that allowing anonymous access to an S3 bucket compromises
     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.
     for accessing public data sets without requiring AWS credentials.
   </description>
   </description>
 </property>
 </property>
 
 
 <property>
 <property>
   <name>fs.s3a.session.token</name>
   <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>
 
 
 <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.
 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.
 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
 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
 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 -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
 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 -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:
 Example:
 
 
 * `hadoop fs -count hdfs://nn1.example.com/file1 hdfs://nn2.example.com/file2`
 * `hadoop fs -count hdfs://nn1.example.com/file1 hdfs://nn2.example.com/file2`
@@ -211,14 +215,15 @@ Example:
 du
 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.
 Displays sizes of files and directories contained in the given directory or the length of a file in case its just a file.
 
 
 Options:
 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 -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:
 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)`
 ### `FSDataOutputStream append(Path p, int bufferSize, Progressable progress)`
 
 
-Implementations MAY throw `UnsupportedOperationException`.
+Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
 
 #### Preconditions
 #### Preconditions
 
 
@@ -630,7 +630,7 @@ by appending data to the existing list.
 
 
 ### `FSDataInputStream open(Path f, int bufferSize)`
 ### `FSDataInputStream open(Path f, int bufferSize)`
 
 
-Implementations MAY throw `UnsupportedOperationException`.
+Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
 
 #### Preconditions
 #### 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
 Joins multiple blocks together to create a single file. This
 is a little-used operation currently implemented only by HDFS.
 is a little-used operation currently implemented only by HDFS.
 
 
-Implementations MAY throw `UnsupportedOperationException`
+Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
 
 #### Preconditions
 #### Preconditions
 
 
@@ -989,7 +989,7 @@ from specification.
 
 
 Truncate file `p` to the specified `newLength`.
 Truncate file `p` to the specified `newLength`.
 
 
-Implementations MAY throw `UnsupportedOperationException`.
+Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
 
 #### Preconditions
 #### 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
   @After
   public void tearDown() throws Exception {
   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
   @After
   public void tearDown() throws Exception {
   public void tearDown() throws Exception {
-    fc.delete(fileContextTestHelper.getTestRootPath(fc), true);
+    if (fc != null) {
+      fc.delete(fileContextTestHelper.getTestRootPath(fc), true);
+    }
   }
   }
   
   
   @Test
   @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 {
   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 {
     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() 
   public void testGetFileStatusThrowsExceptionForNonExistentFile() 
@@ -480,7 +497,8 @@ public abstract class FileSystemContractBaseTest extends TestCase {
   }
   }
   
   
   protected Path path(String pathString) {
   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 {
   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 static class TestTrashPolicy extends TrashPolicy {
     public TestTrashPolicy() { }
     public TestTrashPolicy() { }
 
 
-    @Override
-    public void initialize(Configuration conf, FileSystem fs, Path home) {
-    }
-
     @Override
     @Override
     public void initialize(Configuration conf, FileSystem fs) {
     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();
     Count count = new Count();
     String actual = count.getUsage();
     String actual = count.getUsage();
     String expected =
     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);
     assertEquals("Count.getUsage", expected, actual);
   }
   }
 
 
@@ -465,6 +465,7 @@ public class TestCount {
         + "      DIR_COUNT FILE_COUNT CONTENT_SIZE PATHNAME\n"
         + "      DIR_COUNT FILE_COUNT CONTENT_SIZE PATHNAME\n"
         + "The -h option shows file sizes in human readable format.\n"
         + "The -h option shows file sizes in human readable format.\n"
         + "The -v option displays a header line.\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"
         + "The -t option displays quota by storage types.\n"
         + "It must be used with -q option.\n"
         + "It must be used with -q option.\n"
         + "If a comma-separated list of storage types is given after the -t 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
     @Override
-    public String toString(boolean qOption, boolean hOption) {
+    public String toString(boolean qOption, boolean hOption, boolean xOption) {
       if (qOption) {
       if (qOption) {
         if (hOption) {
         if (hOption) {
           return (HUMAN + WITH_QUOTAS);
           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.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ExecutorService;
 
 
+import com.google.common.base.Supplier;
 import org.apache.curator.RetryPolicy;
 import org.apache.curator.RetryPolicy;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 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.Token;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Id;
@@ -44,12 +46,18 @@ import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Before;
 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 static org.junit.Assert.fail;
 
 
-import org.junit.Test;
 
 
 public class TestZKDelegationTokenSecretManager {
 public class TestZKDelegationTokenSecretManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestZKDelegationTokenSecretManager.class);
 
 
   private static final int TEST_RETRIES = 2;
   private static final int TEST_RETRIES = 2;
 
 
@@ -61,6 +69,9 @@ public class TestZKDelegationTokenSecretManager {
 
 
   private TestingServer zkServer;
   private TestingServer zkServer;
 
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   @Before
   public void setup() throws Exception {
   public void setup() throws Exception {
     zkServer = new TestingServer();
     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 {
   public void testNodeHealthScript() throws Exception {
     String errorScript = "echo ERROR\n echo \"Tracker not healthy\"";
     String errorScript = "echo ERROR\n echo \"Tracker not healthy\"";
     String normalScript = "echo \"I am all fine\"";
     String normalScript = "echo \"I am all fine\"";
+    String failWithExitCodeScript = "echo \"Not healthy\"; exit -1";
     String timeOutScript =
     String timeOutScript =
       Shell.WINDOWS ? "@echo off\nping -n 4 127.0.0.1 >nul\necho \"I am fine\""
       Shell.WINDOWS ? "@echo off\nping -n 4 127.0.0.1 >nul\necho \"I am fine\""
       : "sleep 4\necho \"I am fine\"";
       : "sleep 4\necho \"I am fine\"";
@@ -124,6 +125,12 @@ public class TestNodeHealthScriptRunner {
         nodeHealthScriptRunner.isHealthy());
         nodeHealthScriptRunner.isHealthy());
     Assert.assertEquals("", nodeHealthScriptRunner.getHealthReport());
     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.
     // Timeout script.
     writeNodeHealthScriptFile(timeOutScript, true);
     writeNodeHealthScriptFile(timeOutScript, true);
     timerTask.run();
     timerTask.run();

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

@@ -200,7 +200,7 @@
       <comparators>
       <comparators>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <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>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <type>RegexpComparator</type>
@@ -226,6 +226,10 @@
           <type>RegexpComparator</type>
           <type>RegexpComparator</type>
           <expected-output>\s*of bytes.\s*</expected-output>
           <expected-output>\s*of bytes.\s*</expected-output>
         </comparator>
         </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^\s*-x\s*Excludes snapshots from being counted.\s*</expected-output>
+        </comparator>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <type>RegexpComparator</type>
           <expected-output>^\s*Note that, even without the -s option, this only shows size summaries one level\s*</expected-output>
           <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>
       <comparators>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <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>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <type>RegexpComparator</type>
@@ -308,6 +312,10 @@
           <type>RegexpComparator</type>
           <type>RegexpComparator</type>
           <expected-output>^( |\t)*The -v option displays a header line.( )*</expected-output>
           <expected-output>^( |\t)*The -v option displays a header line.( )*</expected-output>
         </comparator>
         </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*The -x option excludes snapshots from being calculated.( )*</expected-output>
+        </comparator>
       </comparators>
       </comparators>
     </test>
     </test>
 
 

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

@@ -170,4 +170,14 @@
     </description>
     </description>
   </property>
   </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>
 </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;
 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.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 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.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
-import com.google.common.base.Joiner;
 import com.google.common.base.Strings;
 import com.google.common.base.Strings;
 import com.google.common.cache.Cache;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 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.collect.Sets;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 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.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 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.
  * types of events.
  */
  */
 public class KMSAudit {
 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.GET_KEY_VERSION, KMS.KMSOp.GET_CURRENT_KEY,
     KMS.KMSOp.DECRYPT_EEK, KMS.KMSOp.GENERATE_EEK
     KMS.KMSOp.DECRYPT_EEK, KMS.KMSOp.GENERATE_EEK
   );
   );
@@ -102,17 +65,21 @@ public class KMSAudit {
 
 
   public static final String KMS_LOGGER_NAME = "kms-audit";
   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.
    * 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()
     cache = CacheBuilder.newBuilder()
         .expireAfterWrite(windowMs, TimeUnit.MILLISECONDS)
         .expireAfterWrite(windowMs, TimeUnit.MILLISECONDS)
         .removalListener(
         .removalListener(
@@ -122,7 +89,7 @@ public class KMSAudit {
                   RemovalNotification<String, AuditEvent> entry) {
                   RemovalNotification<String, AuditEvent> entry) {
                 AuditEvent event = entry.getValue();
                 AuditEvent event = entry.getValue();
                 if (event.getAccessCount().get() > 0) {
                 if (event.getAccessCount().get() > 0) {
-                  KMSAudit.this.logEvent(event);
+                  KMSAudit.this.logEvent(OpStatus.OK, event);
                   event.getAccessCount().set(0);
                   event.getAccessCount().set(0);
                   KMSAudit.this.cache.put(entry.getKey(), event);
                   KMSAudit.this.cache.put(entry.getKey(), event);
                 }
                 }
@@ -136,87 +103,127 @@ public class KMSAudit {
         cache.cleanUp();
         cache.cleanUp();
       }
       }
     }, windowMs / 10, windowMs / 10, TimeUnit.MILLISECONDS);
     }, 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)
     if (!Strings.isNullOrEmpty(user) && !Strings.isNullOrEmpty(key)
         && (op != null)
         && (op != null)
         && AGGREGATE_OPS_WHITELIST.contains(op)) {
         && AGGREGATE_OPS_WHITELIST.contains(op)) {
       String cacheKey = createCacheKey(user, key, op);
       String cacheKey = createCacheKey(user, key, op);
       if (opStatus == OpStatus.UNAUTHORIZED) {
       if (opStatus == OpStatus.UNAUTHORIZED) {
         cache.invalidate(cacheKey);
         cache.invalidate(cacheKey);
-        AUDIT_LOG.info("UNAUTHORIZED[op={}, key={}, user={}] {}", op, key, user,
-            extraMsg);
+        logEvent(opStatus, new AuditEvent(op, ugi, key, remoteHost, extraMsg));
       } else {
       } else {
         try {
         try {
           AuditEvent event = cache.get(cacheKey, new Callable<AuditEvent>() {
           AuditEvent event = cache.get(cacheKey, new Callable<AuditEvent>() {
             @Override
             @Override
             public AuditEvent call() throws Exception {
             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
           // Log first access (initialized as -1 so
           // incrementAndGet() == 0 implies first access)
           // incrementAndGet() == 0 implies first access)
           if (event.getAccessCount().incrementAndGet() == 0) {
           if (event.getAccessCount().incrementAndGet() == 0) {
             event.getAccessCount().incrementAndGet();
             event.getAccessCount().incrementAndGet();
-            logEvent(event);
+            logEvent(opStatus, event);
           }
           }
         } catch (ExecutionException ex) {
         } catch (ExecutionException ex) {
           throw new RuntimeException(ex);
           throw new RuntimeException(ex);
         }
         }
       }
       }
     } else {
     } 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,
   public void ok(UserGroupInformation user, KMS.KMSOp op, String key,
       String extraMsg) {
       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) {
   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) {
   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,
   public void error(UserGroupInformation user, String method, String url,
       String extraMsg) {
       String extraMsg) {
-    op(OpStatus.ERROR, null, user.getShortUserName(), null, "Method:'" + method
+    op(OpStatus.ERROR, null, user, null, "Unknown", "Method:'" + method
         + "' Exception:'" + extraMsg + "'");
         + "' Exception:'" + extraMsg + "'");
   }
   }
 
 
   public void unauthenticated(String remoteHost, String method,
   public void unauthenticated(String remoteHost, String method,
       String url, String extraMsg) {
       String url, String extraMsg) {
-    op(OpStatus.UNAUTHENTICATED, null, null, null, "RemoteHost:"
+    op(OpStatus.UNAUTHENTICATED, null, null, null, remoteHost, "RemoteHost:"
         + remoteHost + " Method:" + method
         + remoteHost + " Method:" + method
         + " URL:" + url + " ErrorMsg:'" + extraMsg + "'");
         + " URL:" + url + " ErrorMsg:'" + extraMsg + "'");
   }
   }
@@ -227,6 +234,13 @@ public class KMSAudit {
 
 
   public void shutdown() {
   public void shutdown() {
     executor.shutdownNow();
     executor.shutdownNow();
+    for (KMSAuditLogger logger : auditLoggers) {
+      try {
+        logger.cleanup();
+      } catch (Exception ex) {
+        LOG.error("Failed to cleanup logger {}", logger.getClass(), ex);
+      }
+    }
   }
   }
 
 
   @VisibleForTesting
   @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
   // Delay for Audit logs that need aggregation
   public static final String KMS_AUDIT_AGGREGATION_WINDOW = CONFIG_PREFIX +
   public static final String KMS_AUDIT_AGGREGATION_WINDOW = CONFIG_PREFIX +
       "audit.aggregation.window.ms";
       "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;
   public static final boolean KEY_CACHE_ENABLE_DEFAULT = true;
   // 10 mins
   // 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(
       unauthenticatedCallsMeter = metricRegistry.register(
           UNAUTHENTICATED_CALLS_METER, new Meter());
           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.
       // this is required for the the JMXJsonServlet to work properly.
       // the JMXJsonServlet is behind the authentication filter,
       // 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.IOException;
 import java.io.OutputStream;
 import java.io.OutputStream;
 import java.io.PrintStream;
 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.crypto.key.kms.server.KMS.KMSOp;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.ThreadUtil;
 import org.apache.hadoop.util.ThreadUtil;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.PropertyConfigurator;
 import org.apache.log4j.PropertyConfigurator;
@@ -37,6 +40,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.Test;
 import org.junit.rules.Timeout;
 import org.junit.rules.Timeout;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
+import org.mockito.internal.util.reflection.Whitebox;
 
 
 public class TestKMSAudit {
 public class TestKMSAudit {
 
 
@@ -45,7 +49,7 @@ public class TestKMSAudit {
   private FilterOut filterOut;
   private FilterOut filterOut;
   private PrintStream capturedOut;
   private PrintStream capturedOut;
   
   
-  private KMSAudit kmsAudit; 
+  private KMSAudit kmsAudit;
 
 
   private static class FilterOut extends FilterOutputStream {
   private static class FilterOut extends FilterOutputStream {
     public FilterOut(OutputStream out) {
     public FilterOut(OutputStream out) {
@@ -71,9 +75,8 @@ public class TestKMSAudit {
         ThreadUtil.getResourceAsStream("log4j-kmsaudit.properties");
         ThreadUtil.getResourceAsStream("log4j-kmsaudit.properties");
     PropertyConfigurator.configure(is);
     PropertyConfigurator.configure(is);
     IOUtils.closeStream(is);
     IOUtils.closeStream(is);
-
-    this.kmsAudit =
-        new KMSAudit(KMSConfiguration.KMS_AUDIT_AGGREGATION_WINDOW_DEFAULT);
+    Configuration conf = new Configuration();
+    this.kmsAudit = new KMSAudit(conf);
   }
   }
 
 
   @After
   @After
@@ -175,4 +178,36 @@ public class TestKMSAudit {
             + "ERROR\\[user=luser\\] Method:'method' Exception:'testmsg'"
             + "ERROR\\[user=luser\\] Method:'method' Exception:'testmsg'"
             + "UNAUTHENTICATED RemoteHost:remotehost Method:method URL:url ErrorMsg:'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.
 # limitations under the License.
 #
 #
 
 
+log4j.rootLogger=INFO, kms-audit
+
 # LOG Appender
 # LOG Appender
 log4j.appender.kms-audit=org.apache.log4j.ConsoleAppender
 log4j.appender.kms-audit=org.apache.log4j.ConsoleAppender
 log4j.appender.kms-audit.Target=System.err
 log4j.appender.kms-audit.Target=System.err
 log4j.appender.kms-audit.layout=org.apache.log4j.PatternLayout
 log4j.appender.kms-audit.layout=org.apache.log4j.PatternLayout
 log4j.appender.kms-audit.layout.ConversionPattern=%m
 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.CachingStrategy;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
 import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
 import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
+import org.apache.hadoop.hdfs.util.IOUtilsClient;
 import org.apache.hadoop.io.ByteBufferPool;
 import org.apache.hadoop.io.ByteBufferPool;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
@@ -145,94 +146,6 @@ public class DFSInputStream extends FSInputStream
     return extendedReadBuffers;
     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
    * 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
    * 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);
     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.
   /* This is a used by regular read() and handles ChecksumExceptions.
    * name readBuffer() is chosen to imply similarity to readBuffer() in
    * name readBuffer() is chosen to imply similarity to readBuffer() in
    * ChecksumFileSystem
    * ChecksumFileSystem
    */
    */
-  private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
+  private synchronized int readBuffer(ReaderStrategy reader, int len,
                                       CorruptedBlocks corruptedBlocks)
                                       CorruptedBlocks corruptedBlocks)
       throws IOException {
       throws IOException {
     IOException ioe;
     IOException ioe;
@@ -892,7 +700,7 @@ public class DFSInputStream extends FSInputStream
     while (true) {
     while (true) {
       // retry as many times as seekToNewSource allows.
       // retry as many times as seekToNewSource allows.
       try {
       try {
-        return reader.doRead(blockReader, off, len);
+        return reader.readFromBlock(blockReader, len);
       } catch ( ChecksumException ce ) {
       } catch ( ChecksumException ce ) {
         DFSClient.LOG.warn("Found Checksum error for "
         DFSClient.LOG.warn("Found Checksum error for "
             + getCurrentBlock() + " from " + currentNode
             + 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();
     dfsClient.checkOpen();
     if (closed.get()) {
     if (closed.get()) {
       throw new IOException("Stream closed");
       throw new IOException("Stream closed");
     }
     }
 
 
+    int len = strategy.getTargetLength();
     CorruptedBlocks corruptedBlocks = new CorruptedBlocks();
     CorruptedBlocks corruptedBlocks = new CorruptedBlocks();
     failures = 0;
     failures = 0;
     if (pos < getFileLength()) {
     if (pos < getFileLength()) {
@@ -952,7 +761,7 @@ public class DFSInputStream extends FSInputStream
                   locatedBlocks.getFileLength() - pos);
                   locatedBlocks.getFileLength() - pos);
             }
             }
           }
           }
-          int result = readBuffer(strategy, off, realLen, corruptedBlocks);
+          int result = readBuffer(strategy, realLen, corruptedBlocks);
 
 
           if (result >= 0) {
           if (result >= 0) {
             pos += result;
             pos += result;
@@ -994,11 +803,12 @@ public class DFSInputStream extends FSInputStream
     if (len == 0) {
     if (len == 0) {
       return 0;
       return 0;
     }
     }
-    ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
+    ReaderStrategy byteArrayReader =
+        new ByteArrayStrategy(buf, off, len, readStatistics, dfsClient);
     try (TraceScope scope =
     try (TraceScope scope =
              dfsClient.newReaderTraceScope("DFSInputStream#byteArrayRead",
              dfsClient.newReaderTraceScope("DFSInputStream#byteArrayRead",
                  src, getPos(), len)) {
                  src, getPos(), len)) {
-      int retLen = readWithStrategy(byteArrayReader, off, len);
+      int retLen = readWithStrategy(byteArrayReader);
       if (retLen < len) {
       if (retLen < len) {
         dfsClient.addRetLenToReaderScope(scope, retLen);
         dfsClient.addRetLenToReaderScope(scope, retLen);
       }
       }
@@ -1008,12 +818,13 @@ public class DFSInputStream extends FSInputStream
 
 
   @Override
   @Override
   public synchronized int read(final ByteBuffer buf) throws IOException {
   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();
     int reqLen = buf.remaining();
     try (TraceScope scope =
     try (TraceScope scope =
              dfsClient.newReaderTraceScope("DFSInputStream#byteBufferRead",
              dfsClient.newReaderTraceScope("DFSInputStream#byteBufferRead",
                  src, getPos(), reqLen)){
                  src, getPos(), reqLen)){
-      int retLen = readWithStrategy(byteBufferReader, 0, reqLen);
+      int retLen = readWithStrategy(byteBufferReader);
       if (retLen < reqLen) {
       if (retLen < reqLen) {
         dfsClient.addRetLenToReaderScope(scope, retLen);
         dfsClient.addRetLenToReaderScope(scope, retLen);
       }
       }
@@ -1221,7 +1032,7 @@ public class DFSInputStream extends FSInputStream
         reader = getBlockReader(block, startInBlk, len, datanode.addr,
         reader = getBlockReader(block, startInBlk, len, datanode.addr,
             datanode.storageType, datanode.info);
             datanode.storageType, datanode.info);
         int nread = reader.readAll(buf, offset, len);
         int nread = reader.readAll(buf, offset, len);
-        updateReadStatistics(readStatistics, nread, reader);
+        IOUtilsClient.updateReadStatistics(readStatistics, nread, reader);
         dfsClient.updateFileSystemReadStats(
         dfsClient.updateFileSystemReadStats(
             reader.getNetworkDistance(), nread);
             reader.getNetworkDistance(), nread);
         if (nread != len) {
         if (nread != len) {
@@ -1721,18 +1532,14 @@ public class DFSInputStream extends FSInputStream
    * Get statistics about the reads which this DFSInputStream has done.
    * Get statistics about the reads which this DFSInputStream has done.
    */
    */
   public ReadStatistics getReadStatistics() {
   public ReadStatistics getReadStatistics() {
-    synchronized(infoLock) {
-      return new ReadStatistics(readStatistics);
-    }
+    return new ReadStatistics(readStatistics);
   }
   }
 
 
   /**
   /**
    * Clear statistics about the reads which this DFSInputStream has done.
    * Clear statistics about the reads which this DFSInputStream has done.
    */
    */
   public void clearReadStatistics() {
   public void clearReadStatistics() {
-    synchronized(infoLock) {
-      readStatistics.clear();
-    }
+    readStatistics.clear();
   }
   }
 
 
   public FileEncryptionInfo getFileEncryptionInfo() {
   public FileEncryptionInfo getFileEncryptionInfo() {
@@ -1759,7 +1566,8 @@ public class DFSInputStream extends FSInputStream
       throws IOException {
       throws IOException {
     synchronized (infoLock) {
     synchronized (infoLock) {
       this.cachingStrategy =
       this.cachingStrategy =
-          new CachingStrategy.Builder(this.cachingStrategy).setReadahead(readahead).build();
+          new CachingStrategy.Builder(this.cachingStrategy).
+              setReadahead(readahead).build();
     }
     }
     closeCurrentBlockReaders();
     closeCurrentBlockReaders();
   }
   }
@@ -1769,7 +1577,8 @@ public class DFSInputStream extends FSInputStream
       throws IOException {
       throws IOException {
     synchronized (infoLock) {
     synchronized (infoLock) {
       this.cachingStrategy =
       this.cachingStrategy =
-          new CachingStrategy.Builder(this.cachingStrategy).setDropBehind(dropBehind).build();
+          new CachingStrategy.Builder(this.cachingStrategy).
+              setDropBehind(dropBehind).build();
     }
     }
     closeCurrentBlockReaders();
     closeCurrentBlockReaders();
   }
   }
@@ -1883,9 +1692,7 @@ public class DFSInputStream extends FSInputStream
       buffer.position((int)blockPos);
       buffer.position((int)blockPos);
       buffer.limit((int)(blockPos + length));
       buffer.limit((int)(blockPos + length));
       getExtendedReadBuffers().put(buffer, clientMmap);
       getExtendedReadBuffers().put(buffer, clientMmap);
-      synchronized (infoLock) {
-        readStatistics.addZeroCopyBytes(length);
-      }
+      readStatistics.addZeroCopyBytes(length);
       DFSClient.LOG.debug("readZeroCopy read {} bytes from offset {} via the "
       DFSClient.LOG.debug("readZeroCopy read {} bytes from offset {} via the "
           + "zero-copy read path.  blockEnd = {}", length, curPos, blockEnd);
           + "zero-copy read path.  blockEnd = {}", length, curPos, blockEnd);
       success = true;
       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,
       ExtendedBlock currentBlock,
       CorruptedBlocks corruptedBlocks)
       CorruptedBlocks corruptedBlocks)
       throws IOException {
       throws IOException {
-    final int targetLength = strategy.buf.remaining();
+    final int targetLength = strategy.getTargetLength();
     int length = 0;
     int length = 0;
     try {
     try {
       while (length < targetLength) {
       while (length < targetLength) {
-        int ret = strategy.doRead(blockReader, 0, 0);
+        int ret = strategy.readFromBlock(blockReader);
         if (ret < 0) {
         if (ret < 0) {
           throw new IOException("Unexpected EOS from the reader");
           throw new IOException("Unexpected EOS from the reader");
         }
         }
@@ -425,13 +425,14 @@ public class DFSStripedInputStream extends DFSInputStream {
   }
   }
 
 
   @Override
   @Override
-  protected synchronized int readWithStrategy(ReaderStrategy strategy,
-      int off, int len) throws IOException {
+  protected synchronized int readWithStrategy(ReaderStrategy strategy)
+      throws IOException {
     dfsClient.checkOpen();
     dfsClient.checkOpen();
     if (closed.get()) {
     if (closed.get()) {
       throw new IOException("Stream closed");
       throw new IOException("Stream closed");
     }
     }
 
 
+    int len = strategy.getTargetLength();
     CorruptedBlocks corruptedBlocks = new CorruptedBlocks();
     CorruptedBlocks corruptedBlocks = new CorruptedBlocks();
     if (pos < getFileLength()) {
     if (pos < getFileLength()) {
       try {
       try {
@@ -452,7 +453,7 @@ public class DFSStripedInputStream extends DFSInputStream {
           if (!curStripeRange.include(getOffsetInBlockGroup())) {
           if (!curStripeRange.include(getOffsetInBlockGroup())) {
             readOneStripe(corruptedBlocks);
             readOneStripe(corruptedBlocks);
           }
           }
-          int ret = copyToTargetBuf(strategy, off + result, realLen - result);
+          int ret = copyToTargetBuf(strategy, realLen - result);
           result += ret;
           result += ret;
           pos += ret;
           pos += ret;
         }
         }
@@ -470,16 +471,14 @@ public class DFSStripedInputStream extends DFSInputStream {
   /**
   /**
    * Copy the data from {@link #curStripeBuf} into the given buffer
    * Copy the data from {@link #curStripeBuf} into the given buffer
    * @param strategy the ReaderStrategy containing 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
    * @param length target length
    * @return number of bytes copied
    * @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();
     final long offsetInBlk = getOffsetInBlockGroup();
     int bufOffset = getStripedBufOffset(offsetInBlk);
     int bufOffset = getStripedBufOffset(offsetInBlk);
     curStripeBuf.position(bufOffset);
     curStripeBuf.position(bufOffset);
-    return strategy.copyFrom(curStripeBuf, offset,
+    return strategy.readFromBuffer(curStripeBuf,
         Math.min(length, curStripeBuf.remaining()));
         Math.min(length, curStripeBuf.remaining()));
   }
   }
 
 
@@ -700,7 +699,8 @@ public class DFSStripedInputStream extends DFSInputStream {
 
 
     private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) {
     private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) {
       if (chunk.byteBuffer != null) {
       if (chunk.byteBuffer != null) {
-        ByteBufferStrategy strategy = new ByteBufferStrategy(chunk.byteBuffer);
+        ByteBufferStrategy strategy =
+            new ByteBufferStrategy(chunk.byteBuffer, readStatistics, dfsClient);
         return new ByteBufferStrategy[]{strategy};
         return new ByteBufferStrategy[]{strategy};
       } else {
       } else {
         ByteBufferStrategy[] strategies =
         ByteBufferStrategy[] strategies =
@@ -708,7 +708,8 @@ public class DFSStripedInputStream extends DFSInputStream {
         for (int i = 0; i < strategies.length; i++) {
         for (int i = 0; i < strategies.length; i++) {
           ByteBuffer buffer = ByteBuffer.wrap(chunk.byteArray.buf(),
           ByteBuffer buffer = ByteBuffer.wrap(chunk.byteArray.buf(),
               chunk.byteArray.getOffsets()[i], chunk.byteArray.getLengths()[i]);
               chunk.byteArray.getOffsets()[i], chunk.byteArray.getLengths()[i]);
-          strategies[i] = new ByteBufferStrategy(buffer);
+          strategies[i] =
+              new ByteBufferStrategy(buffer, readStatistics, dfsClient);
         }
         }
         return strategies;
         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.fs.FSDataInputStream;
 import org.apache.hadoop.crypto.CryptoInputStream;
 import org.apache.hadoop.crypto.CryptoInputStream;
 import org.apache.hadoop.hdfs.DFSInputStream;
 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.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 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
    * be higher than you would expect just by adding up the number of
    * bytes read through HdfsDataInputStream.
    * bytes read through HdfsDataInputStream.
    */
    */
-  public DFSInputStream.ReadStatistics getReadStatistics() {
+  public ReadStatistics getReadStatistics() {
     return getDFSInputStream().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 int port;
   private String upgradeDomain;
   private String upgradeDomain;
   private AdminStates adminState = AdminStates.NORMAL;
   private AdminStates adminState = AdminStates.NORMAL;
+  private long maintenanceExpireTimeInMS = Long.MAX_VALUE;
 
 
   /**
   /**
    * Return the host name of the datanode.
    * Return the host name of the datanode.
@@ -97,4 +98,22 @@ public class DatanodeAdminProperties {
   public void setAdminState(final AdminStates adminState) {
   public void setAdminState(final AdminStates adminState) {
     this.adminState = 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;
   protected AdminStates adminState;
+  private long maintenanceExpireTimeInMS;
 
 
   public DatanodeInfo(DatanodeInfo from) {
   public DatanodeInfo(DatanodeInfo from) {
     super(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() {
   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() {
   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;
     adminState = null;
   }
   }
 
 
+  public static boolean maintenanceNotExpired(long maintenanceExpireTimeInMS) {
+    return Time.monotonicNow() < maintenanceExpireTimeInMS;
+  }
   /**
   /**
    * Returns true if the node is is entering_maintenance
    * Returns true if the node is is entering_maintenance
    */
    */
@@ -541,6 +556,10 @@ public class DatanodeInfo extends DatanodeID implements Node {
         adminState == AdminStates.IN_MAINTENANCE);
         adminState == AdminStates.IN_MAINTENANCE);
   }
   }
 
 
+  public boolean maintenanceExpired() {
+    return !maintenanceNotExpired(this.maintenanceExpireTimeInMS);
+  }
+
   public boolean isInService() {
   public boolean isInService() {
     return getAdminState() == AdminStates.NORMAL;
     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
   // type of the datanode report
   public enum DatanodeReportType {
   public enum DatanodeReportType {
-    ALL, LIVE, DEAD, DECOMMISSIONING
+    ALL, LIVE, DEAD, DECOMMISSIONING, ENTERING_MAINTENANCE
   }
   }
 
 
   public static final byte RS_6_3_POLICY_ID = 0;
   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)
   public void rename2(String src, String dst, Rename... options)
       throws IOException {
       throws IOException {
     boolean overwrite = false;
     boolean overwrite = false;
+    boolean toTrash = false;
     if (options != null) {
     if (options != null) {
       for (Rename option : options) {
       for (Rename option : options) {
         if (option == Rename.OVERWRITE) {
         if (option == Rename.OVERWRITE) {
           overwrite = true;
           overwrite = true;
+        } else if (option == Rename.TO_TRASH) {
+          toTrash = true;
         }
         }
       }
       }
     }
     }
     Rename2RequestProto req = Rename2RequestProto.newBuilder().
     Rename2RequestProto req = Rename2RequestProto.newBuilder().
         setSrc(src).
         setSrc(src).
-        setDst(dst).setOverwriteDest(overwrite).
+        setDst(dst).
+        setOverwriteDest(overwrite).
+        setMoveToTrash(toTrash).
         build();
         build();
     try {
     try {
       if (Client.isAsynchronousMode()) {
       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()).
     builder.length(cs.getLength()).
         fileCount(cs.getFileCount()).
         fileCount(cs.getFileCount()).
         directoryCount(cs.getDirectoryCount()).
         directoryCount(cs.getDirectoryCount()).
+        snapshotLength(cs.getSnapshotLength()).
+        snapshotFileCount(cs.getSnapshotFileCount()).
+        snapshotDirectoryCount(cs.getSnapshotDirectoryCount()).
+        snapshotSpaceConsumed(cs.getSnapshotSpaceConsumed()).
         quota(cs.getQuota()).
         quota(cs.getQuota()).
         spaceConsumed(cs.getSpaceConsumed()).
         spaceConsumed(cs.getSpaceConsumed()).
         spaceQuota(cs.getSpaceQuota());
         spaceQuota(cs.getSpaceQuota());
@@ -2069,6 +2073,10 @@ public class PBHelperClient {
     builder.setLength(cs.getLength()).
     builder.setLength(cs.getLength()).
         setFileCount(cs.getFileCount()).
         setFileCount(cs.getFileCount()).
         setDirectoryCount(cs.getDirectoryCount()).
         setDirectoryCount(cs.getDirectoryCount()).
+        setSnapshotLength(cs.getSnapshotLength()).
+        setSnapshotFileCount(cs.getSnapshotFileCount()).
+        setSnapshotDirectoryCount(cs.getSnapshotDirectoryCount()).
+        setSnapshotSpaceConsumed(cs.getSnapshotSpaceConsumed()).
         setQuota(cs.getQuota()).
         setQuota(cs.getQuota()).
         setSpaceConsumed(cs.getSpaceConsumed()).
         setSpaceConsumed(cs.getSpaceConsumed()).
         setSpaceQuota(cs.getSpaceQuota());
         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;
 package org.apache.hadoop.hdfs.util;
 
 
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.ReadStatistics;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 
 
 import java.io.IOException;
 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;
 package org.apache.hadoop.hdfs.web;
 
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Maps;
 import org.apache.hadoop.fs.ContentSummary;
 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.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 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.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -141,6 +143,25 @@ class JsonUtilClient {
         storagePolicy, null);
         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. */
   /** Convert a Json map to an ExtendedBlock object. */
   static ExtendedBlock toExtendedBlock(final Map<?, ?> m) {
   static ExtendedBlock toExtendedBlock(final Map<?, ?> m) {
     if (m == null) {
     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.FileSystem;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.hdfs.DFSOpsCountStatistics;
 import org.apache.hadoop.hdfs.DFSOpsCountStatistics;
 import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
 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.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtilClient;
 import org.apache.hadoop.hdfs.HAUtilClient;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 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.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -106,6 +108,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
 
 
@@ -1499,6 +1502,58 @@ public class WebHdfsFileSystem extends FileSystem
     }.run();
     }.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
   @Override
   public Token<DelegationTokenIdentifier> getDelegationToken(
   public Token<DelegationTokenIdentifier> getDelegationToken(
       final String renewer) throws IOException {
       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),
     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 boolean redirect;
     final int expectedHttpResponseCode;
     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 src = 1;
   required string dst = 2;
   required string dst = 2;
   required bool overwriteDest = 3;
   required bool overwriteDest = 3;
+  optional bool moveToTrash = 4;
 }
 }
 
 
 message Rename2ResponseProto { // void response
 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 spaceConsumed = 5;
   required uint64 spaceQuota = 6;
   required uint64 spaceQuota = 6;
   optional StorageTypeQuotaInfosProto typeQuotaInfos = 7;
   optional StorageTypeQuotaInfosProto typeQuotaInfos = 7;
+  optional uint64 snapshotLength = 8;
+  optional uint64 snapshotFileCount = 9;
+  optional uint64 snapshotDirectoryCount = 10;
+  optional uint64 snapshotSpaceConsumed = 11;
 }
 }
 
 
 /**
 /**

文件差异内容过多而无法显示
+ 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
   public static final long
       DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT = 25;
       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 String  DFS_UPGRADE_DOMAIN_FACTOR = "dfs.namenode.upgrade.domain.factor";
   public static final int DFS_UPGRADE_DOMAIN_FACTOR_DEFAULT = DFS_REPLICATION_DEFAULT;
   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 int     DFS_MOVER_MOVERTHREADS_DEFAULT = 1000;
   public static final String  DFS_MOVER_RETRY_MAX_ATTEMPTS_KEY = "dfs.mover.retry.max.attempts";
   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 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 String  DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
   public static final int     DFS_DATANODE_DEFAULT_PORT = 9866;
   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;
 package org.apache.hadoop.hdfs.protocolPB;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.List;
 
 
@@ -602,10 +603,21 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   @Override
   @Override
   public Rename2ResponseProto rename2(RpcController controller,
   public Rename2ResponseProto rename2(RpcController controller,
       Rename2RequestProto req) throws ServiceException {
       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 {
     try {
       server.rename2(req.getSrc(), req.getDst(), 
       server.rename2(req.getSrc(), req.getDst(), 
-          req.getOverwriteDest() ? Rename.OVERWRITE : Rename.NONE);
+          optionList.toArray(new Rename[optionList.size()]));
     } catch (IOException e) {
     } catch (IOException e) {
       throw new ServiceException(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.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URL;
 import java.net.URL;
+import java.net.UnknownHostException;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.List;
 import java.util.List;
@@ -387,8 +388,12 @@ public class QuorumJournalManager implements JournalManager {
     
     
     List<InetSocketAddress> addrs = Lists.newArrayList();
     List<InetSocketAddress> addrs = Lists.newArrayList();
     for (String addr : parts) {
     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;
     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> {
     static class HostIterator extends UnmodifiableIterator<InetSocketAddress> {
       private final Iterator<Map.Entry<InetAddress,
       private final Iterator<Map.Entry<InetAddress,
           DatanodeAdminProperties>> it;
           DatanodeAdminProperties>> it;
@@ -236,6 +254,11 @@ public class CombinedHostFileManager extends HostConfigManager {
     return hostProperties.getUpgradeDomain(dn.getResolvedAddress());
     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
    * Set the properties lists by the new instances. The
    * old instance is discarded.
    * 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 */
   /** Get a datanode descriptor given corresponding DatanodeUUID */
-  DatanodeDescriptor getDatanode(final String datanodeUuid) {
+  public DatanodeDescriptor getDatanode(final String datanodeUuid) {
     if (datanodeUuid == null) {
     if (datanodeUuid == null) {
       return null;
       return null;
     }
     }
@@ -902,10 +902,14 @@ public class DatanodeManager {
    *
    *
    * @param nodeReg datanode
    * @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 the registered node is in exclude list, then decommission it
     if (getHostConfigManager().isExcluded(nodeReg)) {
     if (getHostConfigManager().isExcluded(nodeReg)) {
       decomManager.startDecommission(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
           // also treat the registration message as a heartbeat
           heartbeatManager.register(nodeS);
           heartbeatManager.register(nodeS);
           incrementVersionCount(nodeS.getSoftwareVersion());
           incrementVersionCount(nodeS.getSoftwareVersion());
-          startDecommissioningIfExcluded(nodeS);
+          startAdminOperationIfNecessary(nodeS);
           success = true;
           success = true;
         } finally {
         } finally {
           if (!success) {
           if (!success) {
@@ -1056,7 +1060,7 @@ public class DatanodeManager {
         heartbeatManager.addDatanode(nodeDescr);
         heartbeatManager.addDatanode(nodeDescr);
         heartbeatManager.updateDnStat(nodeDescr);
         heartbeatManager.updateDnStat(nodeDescr);
         incrementVersionCount(nodeReg.getSoftwareVersion());
         incrementVersionCount(nodeReg.getSoftwareVersion());
-        startDecommissioningIfExcluded(nodeDescr);
+        startAdminOperationIfNecessary(nodeDescr);
         success = true;
         success = true;
       } finally {
       } finally {
         if (!success) {
         if (!success) {
@@ -1122,9 +1126,14 @@ public class DatanodeManager {
       if (!hostConfigManager.isIncluded(node)) {
       if (!hostConfigManager.isIncluded(node)) {
         node.setDisallowed(true); // case 2.
         node.setDisallowed(true); // case 2.
       } else {
       } 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.
           decomManager.startDecommission(node); // case 3.
         } else {
         } else {
+          decomManager.stopMaintenance(node);
           decomManager.stopDecommission(node); // case 4.
           decomManager.stopDecommission(node); // case 4.
         }
         }
       }
       }
@@ -1157,7 +1166,12 @@ public class DatanodeManager {
     // A decommissioning DN may be "alive" or "dead".
     // A decommissioning DN may be "alive" or "dead".
     return getDatanodeListForReport(DatanodeReportType.DECOMMISSIONING);
     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 */
   /* Getter and Setter for stale DataNodes related attributes */
 
 
   /**
   /**
@@ -1342,6 +1356,9 @@ public class DatanodeManager {
     final boolean listDecommissioningNodes =
     final boolean listDecommissioningNodes =
         type == DatanodeReportType.ALL ||
         type == DatanodeReportType.ALL ||
         type == DatanodeReportType.DECOMMISSIONING;
         type == DatanodeReportType.DECOMMISSIONING;
+    final boolean listEnteringMaintenanceNodes =
+        type == DatanodeReportType.ALL ||
+        type == DatanodeReportType.ENTERING_MAINTENANCE;
 
 
     ArrayList<DatanodeDescriptor> nodes;
     ArrayList<DatanodeDescriptor> nodes;
     final HostSet foundNodes = new HostSet();
     final HostSet foundNodes = new HostSet();
@@ -1353,10 +1370,12 @@ public class DatanodeManager {
       for (DatanodeDescriptor dn : datanodeMap.values()) {
       for (DatanodeDescriptor dn : datanodeMap.values()) {
         final boolean isDead = isDatanodeDead(dn);
         final boolean isDead = isDatanodeDead(dn);
         final boolean isDecommissioning = dn.isDecommissionInProgress();
         final boolean isDecommissioning = dn.isDecommissionInProgress();
+        final boolean isEnteringMaintenance = dn.isEnteringMaintenance();
 
 
         if (((listLiveNodes && !isDead) ||
         if (((listLiveNodes && !isDead) ||
             (listDeadNodes && isDead) ||
             (listDeadNodes && isDead) ||
-            (listDecommissioningNodes && isDecommissioning)) &&
+            (listDecommissioningNodes && isDecommissioning) ||
+            (listEnteringMaintenanceNodes && isEnteringMaintenance)) &&
             hostConfigManager.isIncluded(dn)) {
             hostConfigManager.isIncluded(dn)) {
           nodes.add(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) {
   synchronized void add(final DatanodeDescriptor node) {
     xceiverCount += node.getXceiverCount();
     xceiverCount += node.getXceiverCount();
-    if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
+    if (node.isInService()) {
       capacityUsed += node.getDfsUsed();
       capacityUsed += node.getDfsUsed();
       blockPoolUsed += node.getBlockPoolUsed();
       blockPoolUsed += node.getBlockPoolUsed();
       nodesInService++;
       nodesInService++;
@@ -56,7 +56,8 @@ class DatanodeStats {
       capacityRemaining += node.getRemaining();
       capacityRemaining += node.getRemaining();
       cacheCapacity += node.getCacheCapacity();
       cacheCapacity += node.getCacheCapacity();
       cacheUsed += node.getCacheUsed();
       cacheUsed += node.getCacheUsed();
-    } else if (!node.isDecommissioned()) {
+    } else if (node.isDecommissionInProgress() ||
+        node.isEnteringMaintenance()) {
       cacheCapacity += node.getCacheCapacity();
       cacheCapacity += node.getCacheCapacity();
       cacheUsed += node.getCacheUsed();
       cacheUsed += node.getCacheUsed();
     }
     }
@@ -74,7 +75,7 @@ class DatanodeStats {
 
 
   synchronized void subtract(final DatanodeDescriptor node) {
   synchronized void subtract(final DatanodeDescriptor node) {
     xceiverCount -= node.getXceiverCount();
     xceiverCount -= node.getXceiverCount();
-    if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
+    if (node.isInService()) {
       capacityUsed -= node.getDfsUsed();
       capacityUsed -= node.getDfsUsed();
       blockPoolUsed -= node.getBlockPoolUsed();
       blockPoolUsed -= node.getBlockPoolUsed();
       nodesInService--;
       nodesInService--;
@@ -83,7 +84,8 @@ class DatanodeStats {
       capacityRemaining -= node.getRemaining();
       capacityRemaining -= node.getRemaining();
       cacheCapacity -= node.getCacheCapacity();
       cacheCapacity -= node.getCacheCapacity();
       cacheUsed -= node.getCacheUsed();
       cacheUsed -= node.getCacheUsed();
-    } else if (!node.isDecommissioned()) {
+    } else if (node.isDecommissionInProgress() ||
+        node.isEnteringMaintenance()) {
       cacheCapacity -= node.getCacheCapacity();
       cacheCapacity -= node.getCacheCapacity();
       cacheUsed -= node.getCacheUsed();
       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;
   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/>
    * <p/>
    * This holds a set of references to the under-replicated blocks on the DN at
    * 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
    * 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.
    * another check is done with the actual block map.
    */
    */
   private final TreeMap<DatanodeDescriptor, AbstractList<BlockInfo>>
   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;
   private final Queue<DatanodeDescriptor> pendingNodes;
 
 
@@ -122,7 +125,7 @@ public class DecommissionManager {
     executor = Executors.newScheduledThreadPool(1,
     executor = Executors.newScheduledThreadPool(1,
         new ThreadFactoryBuilder().setNameFormat("DecommissionMonitor-%d")
         new ThreadFactoryBuilder().setNameFormat("DecommissionMonitor-%d")
             .setDaemon(true).build());
             .setDaemon(true).build());
-    decomNodeBlocks = new TreeMap<>();
+    outOfServiceNodeBlocks = new TreeMap<>();
     pendingNodes = new LinkedList<>();
     pendingNodes = new LinkedList<>();
   }
   }
 
 
@@ -222,13 +225,56 @@ public class DecommissionManager {
       }
       }
       // Remove from tracking in DecommissionManager
       // Remove from tracking in DecommissionManager
       pendingNodes.remove(node);
       pendingNodes.remove(node);
-      decomNodeBlocks.remove(node);
+      outOfServiceNodeBlocks.remove(node);
     } else {
     } else {
       LOG.trace("stopDecommission: Node {} in {}, nothing to do." +
       LOG.trace("stopDecommission: Node {} in {}, nothing to do." +
           node, node.getAdminState());
           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) {
   private void setDecommissioned(DatanodeDescriptor dn) {
     dn.setDecommissioned();
     dn.setDecommissioned();
     LOG.info("Decommissioning complete for node {}", dn);
     LOG.info("Decommissioning complete for node {}", dn);
@@ -313,7 +359,7 @@ public class DecommissionManager {
 
 
   @VisibleForTesting
   @VisibleForTesting
   public int getNumTrackedNodes() {
   public int getNumTrackedNodes() {
-    return decomNodeBlocks.size();
+    return outOfServiceNodeBlocks.size();
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
@@ -333,8 +379,8 @@ public class DecommissionManager {
      */
      */
     private final int numBlocksPerCheck;
     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;
     private final int maxConcurrentTrackedNodes;
     /**
     /**
@@ -347,7 +393,7 @@ public class DecommissionManager {
      */
      */
     private int numNodesChecked = 0;
     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 
     private DatanodeDescriptor iterkey = new DatanodeDescriptor(new 
         DatanodeID("", "", "", 0, 0, 0, 0));
         DatanodeID("", "", "", 0, 0, 0, 0));
@@ -393,14 +439,15 @@ public class DecommissionManager {
     private void processPendingNodes() {
     private void processPendingNodes() {
       while (!pendingNodes.isEmpty() &&
       while (!pendingNodes.isEmpty() &&
           (maxConcurrentTrackedNodes == 0 ||
           (maxConcurrentTrackedNodes == 0 ||
-           decomNodeBlocks.size() < maxConcurrentTrackedNodes)) {
-        decomNodeBlocks.put(pendingNodes.poll(), null);
+          outOfServiceNodeBlocks.size() < maxConcurrentTrackedNodes)) {
+        outOfServiceNodeBlocks.put(pendingNodes.poll(), null);
       }
       }
     }
     }
 
 
     private void check() {
     private void check() {
       final Iterator<Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>>
       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<>();
       final LinkedList<DatanodeDescriptor> toRemove = new LinkedList<>();
 
 
       while (it.hasNext() && !exceededNumBlocksPerCheck()) {
       while (it.hasNext() && !exceededNumBlocksPerCheck()) {
@@ -410,6 +457,17 @@ public class DecommissionManager {
         final DatanodeDescriptor dn = entry.getKey();
         final DatanodeDescriptor dn = entry.getKey();
         AbstractList<BlockInfo> blocks = entry.getValue();
         AbstractList<BlockInfo> blocks = entry.getValue();
         boolean fullScan = false;
         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) {
         if (blocks == null) {
           // This is a newly added datanode, run through its list to schedule 
           // This is a newly added datanode, run through its list to schedule 
           // under-replicated blocks for replication and collect the blocks 
           // 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 " +
           LOG.debug("Newly-added node {}, doing full scan to find " +
               "insufficiently-replicated blocks.", dn);
               "insufficiently-replicated blocks.", dn);
           blocks = handleInsufficientlyStored(dn);
           blocks = handleInsufficientlyStored(dn);
-          decomNodeBlocks.put(dn, blocks);
+          outOfServiceNodeBlocks.put(dn, blocks);
           fullScan = true;
           fullScan = true;
         } else {
         } else {
           // This is a known datanode, check if its # of insufficiently 
           // 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 "
             LOG.debug("Node {} has finished replicating current set of "
                 + "blocks, checking with the full block map.", dn);
                 + "blocks, checking with the full block map.", dn);
             blocks = handleInsufficientlyStored(dn);
             blocks = handleInsufficientlyStored(dn);
-            decomNodeBlocks.put(dn, blocks);
+            outOfServiceNodeBlocks.put(dn, blocks);
           }
           }
           // If the full scan is clean AND the node liveness is okay, 
           // If the full scan is clean AND the node liveness is okay, 
           // we can finally mark as decommissioned.
           // we can finally mark as decommissioned.
@@ -460,11 +518,12 @@ public class DecommissionManager {
         }
         }
         iterkey = dn;
         iterkey = dn;
       }
       }
-      // Remove the datanodes that are decommissioned
+      // Remove the datanodes that are decommissioned or in service after
+      // maintenance expiration.
       for (DatanodeDescriptor dn : toRemove) {
       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) {
   synchronized void stopDecommission(final DatanodeDescriptor node) {
     LOG.info("Stopping decommissioning of {} node {}",
     LOG.info("Stopping decommissioning of {} node {}",
         node.isAlive() ? "live" : "dead", 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.
    * @return the upgrade domain of dn.
    */
    */
   public abstract String getUpgradeDomain(DatanodeID 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;
     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
    * Read the includes and excludes lists from the named files.  Any previous
    * includes and excludes lists are discarded.
    * 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;
           long ackRecvNanoTime = 0;
           try {
           try {
             if (type != PacketResponderType.LAST_IN_PIPELINE && !mirrorError) {
             if (type != PacketResponderType.LAST_IN_PIPELINE && !mirrorError) {
+              DataNodeFaultInjector.get().failPipeline(replicaInfo, mirrorAddr);
               // read an ack from downstream datanode
               // read an ack from downstream datanode
               ack.readFields(downstreamIn);
               ack.readFields(downstreamIn);
               ackRecvNanoTime = System.nanoTime();
               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 */
   /** The reference to the volume where the block is located */
   private FsVolumeReference volumeRef;
   private FsVolumeReference volumeRef;
 
 
+  /** The replica of the block that is being read. */
+  private final Replica replica;
+
   // Cache-management related fields
   // Cache-management related fields
   private final long readaheadLength;
   private final long readaheadLength;
 
 
@@ -238,7 +241,6 @@ class BlockSender implements java.io.Closeable {
             "If verifying checksum, currently must also send it.");
             "If verifying checksum, currently must also send it.");
       }
       }
       
       
-      final Replica replica;
       final long replicaVisibleLength;
       final long replicaVisibleLength;
       try(AutoCloseableLock lock = datanode.data.acquireDatasetLock()) {
       try(AutoCloseableLock lock = datanode.data.acquireDatasetLock()) {
         replica = getReplica(block, datanode);
         replica = getReplica(block, datanode);
@@ -688,8 +690,12 @@ class BlockSender implements java.io.Closeable {
       checksum.update(buf, dOff, dLen);
       checksum.update(buf, dOff, dLen);
       if (!checksum.compare(buf, cOff)) {
       if (!checksum.compare(buf, cOff)) {
         long failedPos = offset + datalen - dLeft;
         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;
       dLeft -= dLen;
       dOff += 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);
         blockSender.sendBlock(out, unbufOut, null);
 
 
         // no response necessary
         // no response necessary
-        LOG.info(getClass().getSimpleName() + ": Transmitted " + b
+        LOG.info(getClass().getSimpleName() + ", at "
+            + DataNode.this.getDisplayName() + ": Transmitted " + b
             + " (numBytes=" + b.getNumBytes() + ") to " + curTarget);
             + " (numBytes=" + b.getNumBytes() + ") to " + curTarget);
 
 
         // read ack
         // 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 noRegistration() throws IOException { }
 
 
   public void failMirrorConnection() 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 {
         } else {
           LOG.info(s1 + "; " + t);          
           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 {
       } else {
         LOG.error(s, t);
         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);
             volume.getBasePath(), block);
         return;
         return;
       }
       }
-      LOG.warn("Reporting bad {} on {}", block, volume.getBasePath());
+      LOG.warn("Reporting bad " + block + " with volume "
+          + volume.getBasePath(), e);
       try {
       try {
         scanner.datanode.reportBadBlocks(block, volume);
         scanner.datanode.reportBadBlocks(block, volume);
       } catch (IOException ie) {
       } catch (IOException ie) {
         // This is bad, but not bad enough to shut down the scanner.
         // 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())) {
           if (!rbw.attemptToSetWriter(null, Thread.currentThread())) {
             throw new MustStopExistingWriter(rbw);
             throw new MustStopExistingWriter(rbw);
           }
           }
-          LOG.info("Recovering " + rbw);
+          LOG.info("At " + datanode.getDisplayName() + ", Recovering " + rbw);
           return recoverRbwImpl(rbw, b, newGS, minBytesRcvd, maxBytesRcvd);
           return recoverRbwImpl(rbw, b, newGS, minBytesRcvd, maxBytesRcvd);
         }
         }
       } catch (MustStopExistingWriter e) {
       } 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;
 package org.apache.hadoop.hdfs.server.diskbalancer.command;
 
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Option;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.StringUtils;
@@ -221,7 +224,7 @@ public abstract class Command extends Configured {
    * @return Set of node names
    * @return Set of node names
    * @throws IOException
    * @throws IOException
    */
    */
-  private Set<String> getNodeList(String listArg) throws IOException {
+  protected Set<String> getNodeList(String listArg) throws IOException {
     URL listURL;
     URL listURL;
     String nodeData;
     String nodeData;
     Set<String> resultSet = new TreeSet<>();
     Set<String> resultSet = new TreeSet<>();
@@ -242,6 +245,37 @@ public abstract class Command extends Configured {
     return resultSet;
     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.
    * Verifies if the command line options are sane.
    *
    *
@@ -471,4 +505,12 @@ public abstract class Command extends Configured {
   public int getTopNodes() {
   public int getTopNodes() {
     return topNodes;
     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.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 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.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.Tool;
@@ -57,6 +60,7 @@ import java.io.BufferedReader;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
+import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.text.DateFormat;
 import java.text.DateFormat;
 import java.util.*;
 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)
   static int run(Map<URI, List<Path>> namenodes, Configuration conf)
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
     final long sleeptime =
     final long sleeptime =
@@ -588,7 +608,8 @@ public class Mover {
             DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000;
             DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000;
     AtomicInteger retryCount = new AtomicInteger(0);
     AtomicInteger retryCount = new AtomicInteger(0);
     LOG.info("namenodes = " + namenodes);
     LOG.info("namenodes = " + namenodes);
-    
+
+    checkKeytabAndInit(conf);
     List<NameNodeConnector> connectors = Collections.emptyList();
     List<NameNodeConnector> connectors = Collections.emptyList();
     try {
     try {
       connectors = NameNodeConnector.newNameNodeConnectors(namenodes,
       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 FSNamesystem fsn = null;
   private BlockStoragePolicySuite bsps = null;
   private BlockStoragePolicySuite bsps = null;
   private ContentCounts counts = null;
   private ContentCounts counts = null;
+  private ContentCounts snapshotCounts = null;
   private long nextCountLimit = 0;
   private long nextCountLimit = 0;
   private long limitPerRun = 0;
   private long limitPerRun = 0;
   private long yieldCount = 0;
   private long yieldCount = 0;
@@ -51,6 +52,7 @@ public class ContentSummaryComputationContext {
     this.limitPerRun = limitPerRun;
     this.limitPerRun = limitPerRun;
     this.nextCountLimit = limitPerRun;
     this.nextCountLimit = limitPerRun;
     this.counts = new ContentCounts.Builder().build();
     this.counts = new ContentCounts.Builder().build();
+    this.snapshotCounts = new ContentCounts.Builder().build();
     this.sleepMilliSec = sleepMicroSec/1000;
     this.sleepMilliSec = sleepMicroSec/1000;
     this.sleepNanoSec = (int)((sleepMicroSec%1000)*1000);
     this.sleepNanoSec = (int)((sleepMicroSec%1000)*1000);
   }
   }
@@ -125,6 +127,10 @@ public class ContentSummaryComputationContext {
     return counts;
     return counts;
   }
   }
 
 
+  public ContentCounts getSnapshotCounts() {
+    return snapshotCounts;
+  }
+
   public BlockStoragePolicySuite getBlockStoragePolicySuite() {
   public BlockStoragePolicySuite getBlockStoragePolicySuite() {
     Preconditions.checkState((bsps != null || fsn != null),
     Preconditions.checkState((bsps != null || fsn != null),
         "BlockStoragePolicySuite must be either initialized or available via" +
         "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 LocatedBlock lb;
     final FSDirectory fsd = fsn.getFSDirectory();
     final FSDirectory fsd = fsn.getFSDirectory();
     final String src;
     final String src;
+    final INodesInPath iip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
-      final INodesInPath iip = fsd.resolvePathForWrite(pc, srcArg);
+      iip = fsd.resolvePathForWrite(pc, srcArg);
       src = iip.getPath();
       src = iip.getPath();
       // Verify that the destination does not exist as a directory already
       // Verify that the destination does not exist as a directory already
       final INode inode = iip.getLastINode();
       final INode inode = iip.getLastINode();
@@ -148,8 +149,7 @@ final class FSDirAppendOp {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
 
 
-    HdfsFileStatus stat = FSDirStatAndListingOp.getFileInfo(fsd, src, false,
-        FSDirectory.isReservedRawName(srcArg));
+    HdfsFileStatus stat = FSDirStatAndListingOp.getFileInfo(fsd, iip);
     if (lb != null) {
     if (lb != null) {
       NameNode.stateChangeLog.debug(
       NameNode.stateChangeLog.debug(
           "DIR* NameSystem.appendFile: file {} for {} at {} block {} block"
           "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 org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.util.AbstractMap;
 import java.util.List;
 import java.util.List;
-import java.util.Map;
-
 import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.util.Time.now;
 
 
 class FSDirMkdirOp {
 class FSDirMkdirOp {
@@ -63,7 +60,6 @@ class FSDirMkdirOp {
         throw new FileAlreadyExistsException("Path is not a directory: " + src);
         throw new FileAlreadyExistsException("Path is not a directory: " + src);
       }
       }
 
 
-      INodesInPath existing = lastINode != null ? iip : iip.getExistingINodes();
       if (lastINode == null) {
       if (lastINode == null) {
         if (fsd.isPermissionEnabled()) {
         if (fsd.isPermissionEnabled()) {
           fsd.checkAncestorAccess(pc, iip, FsAction.WRITE);
           fsd.checkAncestorAccess(pc, iip, FsAction.WRITE);
@@ -78,26 +74,20 @@ class FSDirMkdirOp {
         // create multiple inodes.
         // create multiple inodes.
         fsn.checkFsObjectLimit();
         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);
           throw new IOException("Failed to create directory: " + src);
         }
         }
+        iip = existing;
       }
       }
-      return fsd.getAuditFileInfo(existing);
+      return fsd.getAuditFileInfo(iip);
     } finally {
     } finally {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
@@ -112,35 +102,18 @@ class FSDirMkdirOp {
    * For example, path="/foo/bar/spam", "/foo" is an existing directory,
    * For example, path="/foo/bar/spam", "/foo" is an existing directory,
    * "/foo/bar" is not existing yet, the function will create directory bar.
    * "/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)
       FSDirectory fsd, INodesInPath iip, PermissionStatus permission)
       throws IOException {
       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 fsd FSDirectory
    * @param existing The INodesInPath instance containing all the existing
    * @param existing The INodesInPath instance containing all the existing
@@ -149,21 +122,35 @@ class FSDirMkdirOp {
    *                 starting with "/"
    *                 starting with "/"
    * @param perm the permission of the directory. Note that all ancestors
    * @param perm the permission of the directory. Note that all ancestors
    *             created along the path has implicit {@code u+wx} permissions.
    *             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
    * @return {@link INodesInPath} which contains all inodes to the
    * target directory, After the execution parentPath points to the path of
    * target directory, After the execution parentPath points to the path of
    * the returned INodesInPath. The function return null if the operation has
    * the returned INodesInPath. The function return null if the operation has
    * failed.
    * 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 {
       throws IOException {
     assert fsd.hasWriteLock();
     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;
     return existing;
@@ -183,11 +170,11 @@ class FSDirMkdirOp {
   }
   }
 
 
   private static INodesInPath createSingleDirectory(FSDirectory fsd,
   private static INodesInPath createSingleDirectory(FSDirectory fsd,
-      INodesInPath existing, String localName, PermissionStatus perm)
+      INodesInPath existing, byte[] localName, PermissionStatus perm)
       throws IOException {
       throws IOException {
     assert fsd.hasWriteLock();
     assert fsd.hasWriteLock();
     existing = unprotectedMkdir(fsd, fsd.allocateNewInodeId(), existing,
     existing = unprotectedMkdir(fsd, fsd.allocateNewInodeId(), existing,
-        DFSUtil.string2Bytes(localName), perm, null, now());
+        localName, perm, null, now());
     if (existing == null) {
     if (existing == null) {
       return 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();
     src = srcIIP.getPath();
     dst = dstIIP.getPath();
     dst = dstIIP.getPath();
     if (fsd.isPermissionEnabled()) {
     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
       // Check write access to ancestor of dst
       fsd.checkPermission(pc, dstIIP, false, FsAction.WRITE, null, null, null,
       fsd.checkPermission(pc, dstIIP, false, FsAction.WRITE, null, null, null,
           false);
           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.DirectoryListingStartAfterNotFoundException;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.InvalidPathException;
 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.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.QuotaUsage;
@@ -52,7 +53,6 @@ import static org.apache.hadoop.util.Time.now;
 class FSDirStatAndListingOp {
 class FSDirStatAndListingOp {
   static DirectoryListing getListingInt(FSDirectory fsd, final String srcArg,
   static DirectoryListing getListingInt(FSDirectory fsd, final String srcArg,
       byte[] startAfter, boolean needLocation) throws IOException {
       byte[] startAfter, boolean needLocation) throws IOException {
-    final String startAfterString = DFSUtil.bytes2String(startAfter);
     String src = null;
     String src = null;
 
 
     final INodesInPath iip;
     final INodesInPath iip;
@@ -65,16 +65,20 @@ class FSDirStatAndListingOp {
       iip = fsd.getINodesInPath(src, true);
       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)) {
     if (!DFSUtil.isValidName(src)) {
       throw new InvalidPathException("Invalid file name: " + src);
       throw new InvalidPathException("Invalid file name: " + src);
     }
     }
+    final INodesInPath iip;
     if (fsd.isPermissionEnabled()) {
     if (fsd.isPermissionEnabled()) {
       FSPermissionChecker pc = fsd.getPermissionChecker();
       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);
       fsd.checkPermission(pc, iip, false, null, null, null, null, false);
     } else {
     } else {
       src = FSDirectory.resolvePath(srcArg, fsd);
       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)
       String src, byte[] startAfter, boolean needLocation, boolean isSuperUser)
       throws IOException {
       throws IOException {
     String srcs = FSDirectory.normalizePath(src);
     String srcs = FSDirectory.normalizePath(src);
-    final boolean isRawPath = FSDirectory.isReservedRawName(src);
     if (FSDirectory.isExactReservedName(srcs)) {
     if (FSDirectory.isExactReservedName(srcs)) {
       return getReservedListing(fsd);
       return getReservedListing(fsd);
     }
     }
@@ -257,7 +260,7 @@ class FSDirStatAndListingOp {
         return new DirectoryListing(
         return new DirectoryListing(
             new HdfsFileStatus[]{ createFileStatus(
             new HdfsFileStatus[]{ createFileStatus(
                 fsd, HdfsFileStatus.EMPTY_NAME, nodeAttrs,
                 fsd, HdfsFileStatus.EMPTY_NAME, nodeAttrs,
-                needLocation, parentStoragePolicy, snapshot, isRawPath, iip)
+                needLocation, parentStoragePolicy, iip)
             }, 0);
             }, 0);
       }
       }
 
 
@@ -282,7 +285,7 @@ class FSDirStatAndListingOp {
             cur.getLocalNameBytes());
             cur.getLocalNameBytes());
         listing[i] = createFileStatus(fsd, cur.getLocalNameBytes(), nodeAttrs,
         listing[i] = createFileStatus(fsd, cur.getLocalNameBytes(), nodeAttrs,
             needLocation, getStoragePolicyID(curPolicy, parentStoragePolicy),
             needLocation, getStoragePolicyID(curPolicy, parentStoragePolicy),
-            snapshot, isRawPath, iipWithChild);
+            iipWithChild);
         listingCnt++;
         listingCnt++;
         if (needLocation) {
         if (needLocation) {
             // Once we  hit lsLimit locations, stop.
             // Once we  hit lsLimit locations, stop.
@@ -339,7 +342,6 @@ class FSDirStatAndListingOp {
       listing[i] = createFileStatus(
       listing[i] = createFileStatus(
           fsd, sRoot.getLocalNameBytes(), nodeAttrs,
           fsd, sRoot.getLocalNameBytes(), nodeAttrs,
           HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
           HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
-          Snapshot.CURRENT_STATE_ID, false,
           INodesInPath.fromINode(sRoot));
           INodesInPath.fromINode(sRoot));
     }
     }
     return new DirectoryListing(
     return new DirectoryListing(
@@ -363,10 +365,8 @@ class FSDirStatAndListingOp {
    * @return object containing information regarding the file
    * @return object containing information regarding the file
    *         or null if file not found
    *         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();
     fsd.readLock();
     try {
     try {
       final INode node = iip.getLastINode();
       final INode node = iip.getLastINode();
@@ -377,23 +377,21 @@ class FSDirStatAndListingOp {
       byte policyId = includeStoragePolicy && !node.isSymlink() ?
       byte policyId = includeStoragePolicy && !node.isSymlink() ?
           node.getStoragePolicyID() :
           node.getStoragePolicyID() :
           HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
           HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-      INodeAttributes nodeAttrs = getINodeAttributes(fsd, path,
+      INodeAttributes nodeAttrs = getINodeAttributes(fsd, iip.getPath(),
                                                      HdfsFileStatus.EMPTY_NAME,
                                                      HdfsFileStatus.EMPTY_NAME,
                                                      node, iip.getPathSnapshotId());
                                                      node, iip.getPathSnapshotId());
       return createFileStatus(fsd, HdfsFileStatus.EMPTY_NAME, nodeAttrs,
       return createFileStatus(fsd, HdfsFileStatus.EMPTY_NAME, nodeAttrs,
-                              policyId, iip.getPathSnapshotId(), isRawPath, iip);
+                              policyId, iip);
     } finally {
     } finally {
       fsd.readUnlock();
       fsd.readUnlock();
     }
     }
   }
   }
 
 
-  static HdfsFileStatus getFileInfo(
-      FSDirectory fsd, String src, boolean resolveLink, boolean isRawPath)
+  static HdfsFileStatus getFileInfo(FSDirectory fsd, INodesInPath iip)
     throws IOException {
     throws IOException {
     fsd.readLock();
     fsd.readLock();
     try {
     try {
       HdfsFileStatus status = null;
       HdfsFileStatus status = null;
-      final INodesInPath iip = fsd.getINodesInPath(src, resolveLink);
       if (FSDirectory.isExactReservedName(iip.getPathComponents())) {
       if (FSDirectory.isExactReservedName(iip.getPathComponents())) {
         status = FSDirectory.DOT_RESERVED_STATUS;
         status = FSDirectory.DOT_RESERVED_STATUS;
       } else if (iip.isDotSnapshotDir()) {
       } else if (iip.isDotSnapshotDir()) {
@@ -401,7 +399,7 @@ class FSDirStatAndListingOp {
           status = FSDirectory.DOT_SNAPSHOT_DIR_STATUS;
           status = FSDirectory.DOT_SNAPSHOT_DIR_STATUS;
         }
         }
       } else {
       } else {
-        status = getFileInfo(fsd, src, iip, isRawPath, true);
+        status = getFileInfo(fsd, iip, true);
       }
       }
       return status;
       return status;
     } finally {
     } finally {
@@ -423,15 +421,12 @@ class FSDirStatAndListingOp {
    */
    */
   private static HdfsFileStatus createFileStatus(
   private static HdfsFileStatus createFileStatus(
       FSDirectory fsd, byte[] path, INodeAttributes nodeAttrs,
       FSDirectory fsd, byte[] path, INodeAttributes nodeAttrs,
-      boolean needLocation, byte storagePolicy, int snapshot, boolean isRawPath,
-      INodesInPath iip)
+      boolean needLocation, byte storagePolicy, INodesInPath iip)
       throws IOException {
       throws IOException {
     if (needLocation) {
     if (needLocation) {
-      return createLocatedFileStatus(fsd, path, nodeAttrs, storagePolicy,
-                                     snapshot, isRawPath, iip);
+      return createLocatedFileStatus(fsd, path, nodeAttrs, storagePolicy, iip);
     } else {
     } 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 {
       INodesInPath iip) throws IOException {
     INodeAttributes nodeAttrs = getINodeAttributes(
     INodeAttributes nodeAttrs = getINodeAttributes(
         fsd, fullPath, path, iip.getLastINode(), snapshot);
         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
    * @param iip the INodesInPath containing the target INode and its ancestors
    */
    */
   static HdfsFileStatus createFileStatus(
   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
     long size = 0;     // length is zero for directories
     short replication = 0;
     short replication = 0;
     long blocksize = 0;
     long blocksize = 0;
     final boolean isEncrypted;
     final boolean isEncrypted;
     final INode node = iip.getLastINode();
     final INode node = iip.getLastINode();
+    final int snapshot = iip.getPathSnapshotId();
+    final boolean isRawPath = iip.isRaw();
 
 
     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
         .getFileEncryptionInfo(fsd, node, snapshot, iip);
         .getFileEncryptionInfo(fsd, node, snapshot, iip);
@@ -511,10 +506,9 @@ class FSDirStatAndListingOp {
    * Create FileStatus with location info by file INode
    * Create FileStatus with location info by file INode
    * @param iip the INodesInPath containing the target INode and its ancestors
    * @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,
       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();
     assert fsd.hasReadLock();
     long size = 0; // length is zero for directories
     long size = 0; // length is zero for directories
     short replication = 0;
     short replication = 0;
@@ -522,6 +516,8 @@ class FSDirStatAndListingOp {
     LocatedBlocks loc = null;
     LocatedBlocks loc = null;
     final boolean isEncrypted;
     final boolean isEncrypted;
     final INode node = iip.getLastINode();
     final INode node = iip.getLastINode();
+    final int snapshot = iip.getPathSnapshotId();
+    final boolean isRawPath = iip.isRaw();
 
 
     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
         .getFileEncryptionInfo(fsd, node, snapshot, iip);
         .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 org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.util.Map;
 
 
 import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.util.Time.now;
 
 
@@ -99,21 +98,21 @@ class FSDirSymlinkOp {
       INodesInPath iip, String target, PermissionStatus dirPerms,
       INodesInPath iip, String target, PermissionStatus dirPerms,
       boolean createParent, boolean logRetryCache) throws IOException {
       boolean createParent, boolean logRetryCache) throws IOException {
     final long mtime = now();
     final long mtime = now();
-    final byte[] localName = iip.getLastLocalName();
+    final INodesInPath parent;
     if (createParent) {
     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;
         return null;
       }
       }
-      iip = INodesInPath.append(e.getKey(), null, localName);
+    } else {
+      parent = iip.getParentINodesInPath();
     }
     }
     final String userName = dirPerms.getUserName();
     final String userName = dirPerms.getUserName();
     long id = fsd.allocateNewInodeId();
     long id = fsd.allocateNewInodeId();
     PermissionStatus perm = new PermissionStatus(
     PermissionStatus perm = new PermissionStatus(
         userName, null, FsPermission.getDefault());
         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) {
     if (newNode == null) {
       NameNode.stateChangeLog.info("addSymlink: failed to add " + path);
       NameNode.stateChangeLog.info("addSymlink: failed to add " + path);
       return null;
       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.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 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.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
@@ -65,7 +66,6 @@ import java.util.Collections;
 import java.util.EnumSet;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 import java.util.Set;
 
 
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
@@ -340,7 +340,6 @@ class FSDirWriteFileOp {
       version = ezInfo.protocolVersion;
       version = ezInfo.protocolVersion;
     }
     }
 
 
-    boolean isRawPath = FSDirectory.isReservedRawName(src);
     FSDirectory fsd = fsn.getFSDirectory();
     FSDirectory fsd = fsn.getFSDirectory();
     INodesInPath iip = fsd.resolvePathForWrite(pc, src);
     INodesInPath iip = fsd.resolvePathForWrite(pc, src);
     src = iip.getPath();
     src = iip.getPath();
@@ -420,10 +419,10 @@ class FSDirWriteFileOp {
     }
     }
     fsn.checkFsObjectLimit();
     fsn.checkFsObjectLimit();
     INodeFile newNode = null;
     INodeFile newNode = null;
-    Map.Entry<INodesInPath, String> parent = FSDirMkdirOp
-        .createAncestorDirectories(fsd, iip, permissions);
+    INodesInPath parent =
+        FSDirMkdirOp.createAncestorDirectories(fsd, iip, permissions);
     if (parent != null) {
     if (parent != null) {
-      iip = addFile(fsd, parent.getKey(), parent.getValue(), permissions,
+      iip = addFile(fsd, parent, iip.getLastLocalName(), permissions,
                     replication, blockSize, holder, clientMachine);
                     replication, blockSize, holder, clientMachine);
       newNode = iip != null ? iip.getLastINode().asFile() : null;
       newNode = iip != null ? iip.getLastINode().asFile() : null;
     }
     }
@@ -444,7 +443,7 @@ class FSDirWriteFileOp {
       NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: added " +
       NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: added " +
           src + " inode " + newNode.getId() + " " + holder);
           src + " inode " + newNode.getId() + " " + holder);
     }
     }
-    return FSDirStatAndListingOp.getFileInfo(fsd, src, false, isRawPath);
+    return FSDirStatAndListingOp.getFileInfo(fsd, iip);
   }
   }
 
 
   static EncryptionKeyInfo getEncryptionKeyInfo(FSNamesystem fsn,
   static EncryptionKeyInfo getEncryptionKeyInfo(FSNamesystem fsn,
@@ -508,10 +507,13 @@ class FSDirWriteFileOp {
         return newNode;
         return newNode;
       }
       }
     } catch (IOException e) {
     } 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;
     return null;
@@ -573,7 +575,7 @@ class FSDirWriteFileOp {
    * @return the new INodesInPath instance that contains the new INode
    * @return the new INodesInPath instance that contains the new INode
    */
    */
   private static INodesInPath addFile(
   private static INodesInPath addFile(
-      FSDirectory fsd, INodesInPath existing, String localName,
+      FSDirectory fsd, INodesInPath existing, byte[] localName,
       PermissionStatus permissions, short replication, long preferredBlockSize,
       PermissionStatus permissions, short replication, long preferredBlockSize,
       String clientName, String clientMachine)
       String clientName, String clientMachine)
       throws IOException {
       throws IOException {
@@ -590,7 +592,7 @@ class FSDirWriteFileOp {
       }
       }
       INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
       INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
           modTime, modTime, replication, preferredBlockSize, ecPolicy != null);
           modTime, modTime, replication, preferredBlockSize, ecPolicy != null);
-      newNode.setLocalName(DFSUtil.string2Bytes(localName));
+      newNode.setLocalName(localName);
       newNode.toUnderConstruction(clientName, clientMachine);
       newNode.toUnderConstruction(clientName, clientMachine);
       newiip = fsd.addINode(existing, newNode);
       newiip = fsd.addINode(existing, newNode);
     } finally {
     } finally {
@@ -598,12 +600,13 @@ class FSDirWriteFileOp {
     }
     }
     if (newiip == null) {
     if (newiip == null) {
       NameNode.stateChangeLog.info("DIR* addFile: failed to add " +
       NameNode.stateChangeLog.info("DIR* addFile: failed to add " +
-          existing.getPath() + "/" + localName);
+          existing.getPath() + "/" + DFSUtil.bytes2String(localName));
       return null;
       return null;
     }
     }
 
 
     if(NameNode.stateChangeLog.isDebugEnabled()) {
     if(NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("DIR* addFile: " + localName + " is added");
+      NameNode.stateChangeLog.debug("DIR* addFile: " +
+          DFSUtil.bytes2String(localName) + " is added");
     }
     }
     return newiip;
     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 FileNotFoundException
    * @throws AccessControlException
    * @throws AccessControlException
    */
    */
-  INodesInPath resolvePath(FSPermissionChecker pc, String src)
+  @VisibleForTesting
+  public INodesInPath resolvePath(FSPermissionChecker pc, String src)
       throws UnresolvedLinkException, FileNotFoundException,
       throws UnresolvedLinkException, FileNotFoundException,
       AccessControlException {
       AccessControlException {
     return resolvePath(pc, src, true);
     return resolvePath(pc, src, true);
   }
   }
 
 
-  INodesInPath resolvePath(FSPermissionChecker pc, String src,
+  @VisibleForTesting
+  public INodesInPath resolvePath(FSPermissionChecker pc, String src,
       boolean resolveLink) throws UnresolvedLinkException,
       boolean resolveLink) throws UnresolvedLinkException,
   FileNotFoundException, AccessControlException {
   FileNotFoundException, AccessControlException {
     byte[][] components = INode.getPathComponents(src);
     byte[][] components = INode.getPathComponents(src);
-    if (isPermissionEnabled && pc != null && isReservedRawName(components)) {
+    boolean isRaw = isReservedRawName(components);
+    if (isPermissionEnabled && pc != null && isRaw) {
       pc.checkSuperuserPrivilege();
       pc.checkSuperuserPrivilege();
     }
     }
     components = resolveComponents(components, this);
     components = resolveComponents(components, this);
-    return INodesInPath.resolve(rootDir, components, resolveLink);
+    return INodesInPath.resolve(rootDir, components, isRaw, resolveLink);
   }
   }
 
 
   INodesInPath resolvePathForWrite(FSPermissionChecker pc, String src)
   INodesInPath resolvePathForWrite(FSPermissionChecker pc, String src)
@@ -1662,8 +1665,7 @@ public class FSDirectory implements Closeable {
   HdfsFileStatus getAuditFileInfo(INodesInPath iip)
   HdfsFileStatus getAuditFileInfo(INodesInPath iip)
       throws IOException {
       throws IOException {
     return (namesystem.isAuditEnabled() && namesystem.isExternalInvocation())
     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_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_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY;
 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_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_EXPIRYTIME_MILLIS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_DEFAULT;
 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_KEY,
           DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT);
           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
       // For testing purposes, allow the DT secret manager to be started regardless
       // of whether security is enabled.
       // of whether security is enabled.
       alwaysUseDelegationTokensForTests = conf.getBoolean(
       alwaysUseDelegationTokensForTests = conf.getBoolean(
@@ -1498,17 +1509,44 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
   }
 
 
   /** Threshold (ms) for long holding write lock report. */
   /** 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.*/
   /** Last time stamp for write lock. Keep the longest one for multi-entrance.*/
   private long writeLockHeldTimeStamp;
   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
   @Override
   public void readLock() {
   public void readLock() {
     this.fsLock.readLock().lock();
     this.fsLock.readLock().lock();
+    if (this.fsLock.getReadHoldCount() == 1) {
+      readLockHeldTimeStamp.set(monotonicNow());
+    }
   }
   }
   @Override
   @Override
   public void readUnlock() {
   public void readUnlock() {
+    final boolean needReport = this.fsLock.getReadHoldCount() == 1;
+    final long readLockInterval = monotonicNow() - readLockHeldTimeStamp.get();
     this.fsLock.readLock().unlock();
     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
   @Override
   public void writeLock() {
   public void writeLock() {
@@ -1532,7 +1570,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
 
     this.fsLock.writeLock().unlock();
     this.fsLock.writeLock().unlock();
 
 
-    if (needReport && writeLockInterval >= WRITELOCK_REPORTING_THRESHOLD) {
+    if (needReport && writeLockInterval >= this.writeLockReportingThreshold) {
       LOG.info("FSNamesystem write lock held for " + writeLockInterval +
       LOG.info("FSNamesystem write lock held for " + writeLockInterval +
           " ms via\n" + StringUtils.getStackTrace(Thread.currentThread()));
           " ms via\n" + StringUtils.getStackTrace(Thread.currentThread()));
     }
     }
@@ -7004,6 +7042,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         logAuditMessage(sb.toString());
         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) {
     public void logAuditMessage(String message) {
       auditLog.info(message);
       auditLog.info(message);
     }
     }
@@ -7063,5 +7111,34 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return blockManager.getBytesInFuture();
     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
    * 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 succeeded Whether authorization succeeded.
    * @param userName Name of the user executing the request.
    * @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 dst Path of affected destination file (if any).
    * @param stat File information for operations that change the file's metadata
    * @param stat File information for operations that change the file's metadata
    *          (permissions, owner, times, etc).
    *          (permissions, owner, times, etc).
+   * @param callerContext Context information of the caller
    * @param ugi UserGroupInformation of the current user, or null if not logging
    * @param ugi UserGroupInformation of the current user, or null if not logging
    *          token tracking information
    *          token tracking information
    * @param dtSecretManager The token secret manager, or null if not logging
    * @param dtSecretManager The token secret manager, or null if not logging
    *          token tracking information
    *          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,
       InetAddress addr, String cmd, String src, String dst,
       FileStatus stat, CallerContext callerContext, UserGroupInformation ugi,
       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);
       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,
   public final ContentSummary computeAndConvertContentSummary(int snapshotId,
       ContentSummaryComputationContext summary) {
       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();
     final QuotaCounts q = getQuotaCounts();
     return new ContentSummary.Builder().
     return new ContentSummary.Builder().
         length(counts.getLength()).
         length(counts.getLength()).
@@ -440,6 +441,10 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
         spaceQuota(q.getStorageSpace()).
         spaceQuota(q.getStorageSpace()).
         typeConsumed(counts.getTypeSpaces()).
         typeConsumed(counts.getTypeSpaces()).
         typeQuota(q.getTypeSpaces().asArray()).
         typeQuota(q.getTypeSpaces().asArray()).
+        snapshotLength(snapshotCounts.getLength()).
+        snapshotFileCount(snapshotCounts.getFileCount()).
+        snapshotDirectoryCount(snapshotCounts.getDirectoryCount()).
+        snapshotSpaceConsumed(snapshotCounts.getStoragespace()).
         build();
         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
       // computation should include all the deleted files/directories
       sf.computeContentSummary4Snapshot(summary.getBlockStoragePolicySuite(),
       sf.computeContentSummary4Snapshot(summary.getBlockStoragePolicySuite(),
           summary.getCounts());
           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();
     final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
     if (q != null && snapshotId == Snapshot.CURRENT_STATE_ID) {
     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) {
   public INodeFile(INodeFile that) {
     super(that);
     super(that);
     this.header = that.header;
     this.header = that.header;
-    this.blocks = that.blocks;
     this.features = that.features;
     this.features = that.features;
+    setBlocks(that.blocks);
   }
   }
   
   
   public INodeFile(INodeFile that, FileDiffList diffs) {
   public INodeFile(INodeFile that, FileDiffList diffs) {
@@ -271,9 +271,6 @@ public class INodeFile extends INodeWithAdditionalFields
   /** Assert all blocks are complete. */
   /** Assert all blocks are complete. */
   private void assertAllBlocksComplete(int numCommittedAllowed,
   private void assertAllBlocksComplete(int numCommittedAllowed,
       short minReplication) {
       short minReplication) {
-    if (blocks == null) {
-      return;
-    }
     for (int i = 0; i < blocks.length; i++) {
     for (int i = 0; i < blocks.length; i++) {
       final String err = checkBlockComplete(blocks, i, numCommittedAllowed,
       final String err = checkBlockComplete(blocks, i, numCommittedAllowed,
           minReplication);
           minReplication);
@@ -342,7 +339,7 @@ public class INodeFile extends INodeWithAdditionalFields
   BlockInfo removeLastBlock(Block oldblock) {
   BlockInfo removeLastBlock(Block oldblock) {
     Preconditions.checkState(isUnderConstruction(),
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
         "file is no longer under construction");
-    if (blocks == null || blocks.length == 0) {
+    if (blocks.length == 0) {
       return null;
       return null;
     }
     }
     int size_1 = blocks.length - 1;
     int size_1 = blocks.length - 1;
@@ -618,7 +615,7 @@ public class INodeFile extends INodeWithAdditionalFields
    */
    */
   void addBlock(BlockInfo newblock) {
   void addBlock(BlockInfo newblock) {
     Preconditions.checkArgument(newblock.isStriped() == this.isStriped());
     Preconditions.checkArgument(newblock.isStriped() == this.isStriped());
-    if (this.blocks == null) {
+    if (this.blocks.length == 0) {
       this.setBlocks(new BlockInfo[]{newblock});
       this.setBlocks(new BlockInfo[]{newblock});
     } else {
     } else {
       int size = this.blocks.length;
       int size = this.blocks.length;
@@ -631,12 +628,12 @@ public class INodeFile extends INodeWithAdditionalFields
 
 
   /** Set the blocks. */
   /** Set the blocks. */
   private void setBlocks(BlockInfo[] blocks) {
   private void setBlocks(BlockInfo[] blocks) {
-    this.blocks = blocks;
+    this.blocks = (blocks != null ? blocks : BlockInfo.EMPTY_ARRAY);
   }
   }
 
 
   /** Clear all blocks of the file. */
   /** Clear all blocks of the file. */
   public void clearBlocks() {
   public void clearBlocks() {
-    setBlocks(BlockInfo.EMPTY_ARRAY);
+    this.blocks = BlockInfo.EMPTY_ARRAY;
   }
   }
 
 
   @Override
   @Override
@@ -836,7 +833,7 @@ public class INodeFile extends INodeWithAdditionalFields
    */
    */
   public final long computeFileSize(boolean includesLastUcBlock,
   public final long computeFileSize(boolean includesLastUcBlock,
       boolean usePreferredBlockSize4LastUcBlock) {
       boolean usePreferredBlockSize4LastUcBlock) {
-    if (blocks == null || blocks.length == 0) {
+    if (blocks.length == 0) {
       return 0;
       return 0;
     }
     }
     final int last = blocks.length - 1;
     final int last = blocks.length - 1;
@@ -876,10 +873,6 @@ public class INodeFile extends INodeWithAdditionalFields
   // TODO: support EC with heterogeneous storage
   // TODO: support EC with heterogeneous storage
   public final QuotaCounts storagespaceConsumedStriped() {
   public final QuotaCounts storagespaceConsumedStriped() {
     QuotaCounts counts = new QuotaCounts.Builder().build();
     QuotaCounts counts = new QuotaCounts.Builder().build();
-    if (blocks == null || blocks.length == 0) {
-      return counts;
-    }
-
     for (BlockInfo b : blocks) {
     for (BlockInfo b : blocks) {
       Preconditions.checkState(b.isStriped());
       Preconditions.checkState(b.isStriped());
       long blockSize = b.isComplete() ?
       long blockSize = b.isComplete() ?
@@ -931,7 +924,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * Return the penultimate allocated block for this file.
    * Return the penultimate allocated block for this file.
    */
    */
   BlockInfo getPenultimateBlock() {
   BlockInfo getPenultimateBlock() {
-    if (blocks == null || blocks.length <= 1) {
+    if (blocks.length <= 1) {
       return null;
       return null;
     }
     }
     return blocks[blocks.length - 2];
     return blocks[blocks.length - 2];
@@ -939,12 +932,12 @@ public class INodeFile extends INodeWithAdditionalFields
 
 
   @Override
   @Override
   public BlockInfo getLastBlock() {
   public BlockInfo getLastBlock() {
-    return blocks == null || blocks.length == 0? null: blocks[blocks.length-1];
+    return blocks.length == 0 ? null: blocks[blocks.length-1];
   }
   }
 
 
   @Override
   @Override
   public int numBlocks() {
   public int numBlocks() {
-    return blocks == null ? 0 : blocks.length;
+    return blocks.length;
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
@@ -955,7 +948,7 @@ public class INodeFile extends INodeWithAdditionalFields
     out.print(", fileSize=" + computeFileSize(snapshotId));
     out.print(", fileSize=" + computeFileSize(snapshotId));
     // only compare the first block
     // only compare the first block
     out.print(", blocks=");
     out.print(", blocks=");
-    out.print(blocks == null || blocks.length == 0? null: blocks[0]);
+    out.print(blocks.length == 0 ? null: blocks[0]);
     out.println();
     out.println();
   }
   }
 
 
@@ -1004,7 +997,7 @@ public class INodeFile extends INodeWithAdditionalFields
       long newLength, BlockStoragePolicy bsps,
       long newLength, BlockStoragePolicy bsps,
       QuotaCounts delta) {
       QuotaCounts delta) {
     final BlockInfo[] blocks = getBlocks();
     final BlockInfo[] blocks = getBlocks();
-    if (blocks == null || blocks.length == 0) {
+    if (blocks.length == 0) {
       return;
       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.List;
 import java.util.NoSuchElementException;
 import java.util.NoSuchElementException;
 
 
-import com.google.common.collect.ImmutableList;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -126,6 +125,12 @@ public class INodesInPath {
   static INodesInPath resolve(final INodeDirectory startingDir,
   static INodesInPath resolve(final INodeDirectory startingDir,
       final byte[][] components, final boolean resolveLink)
       final byte[][] components, final boolean resolveLink)
       throws UnresolvedLinkException {
       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);
     Preconditions.checkArgument(startingDir.compareTo(components[0]) == 0);
 
 
     INode curNode = startingDir;
     INode curNode = startingDir;
@@ -225,7 +230,7 @@ public class INodesInPath {
       System.arraycopy(inodes, 0, newNodes, 0, newNodes.length);
       System.arraycopy(inodes, 0, newNodes, 0, newNodes.length);
       inodes = newNodes;
       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) {
   private static boolean shouldUpdateLatestId(int sid, int snapshotId) {
@@ -249,7 +254,8 @@ public class INodesInPath {
     INode[] inodes = new INode[iip.inodes.length];
     INode[] inodes = new INode[iip.inodes.length];
     System.arraycopy(iip.inodes, 0, inodes, 0, inodes.length);
     System.arraycopy(iip.inodes, 0, inodes, 0, inodes.length);
     inodes[pos] = inode;
     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][];
     byte[][] path = new byte[iip.path.length + 1][];
     System.arraycopy(iip.path, 0, path, 0, path.length - 1);
     System.arraycopy(iip.path, 0, path, 0, path.length - 1);
     path[path.length - 1] = childName;
     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;
   private final byte[][] path;
@@ -279,6 +286,13 @@ public class INodesInPath {
    * true if this path corresponds to a snapshot
    * true if this path corresponds to a snapshot
    */
    */
   private final boolean isSnapshot;
   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 
    * For snapshot paths, it is the id of the snapshot; or 
    * {@link Snapshot#CURRENT_STATE_ID} if the snapshot does not exist. For 
    * {@link Snapshot#CURRENT_STATE_ID} if the snapshot does not exist. For 
@@ -287,17 +301,18 @@ public class INodesInPath {
    */
    */
   private final int snapshotId;
   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);
     Preconditions.checkArgument(inodes != null && path != null);
     this.inodes = inodes;
     this.inodes = inodes;
     this.path = path;
     this.path = path;
+    this.isRaw = isRaw;
     this.isSnapshot = isSnapshot;
     this.isSnapshot = isSnapshot;
     this.snapshotId = snapshotId;
     this.snapshotId = snapshotId;
   }
   }
 
 
   private INodesInPath(INode[] inodes, byte[][] path) {
   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;
     return path;
   }
   }
 
 
+  public byte[] getPathComponent(int i) {
+    return path[i];
+  }
+
   /** @return the full path in string form */
   /** @return the full path in string form */
   public String getPath() {
   public String getPath() {
     return DFSUtil.byteArray2PathString(path);
     return DFSUtil.byteArray2PathString(path);
@@ -358,21 +377,6 @@ public class INodesInPath {
     return DFSUtil.byteArray2PathString(path, 0, pos + 1); // it's a length...
     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() {
   public int length() {
     return inodes.length;
     return inodes.length;
   }
   }
@@ -400,7 +404,7 @@ public class INodesInPath {
     final byte[][] apath = new byte[length][];
     final byte[][] apath = new byte[length][];
     System.arraycopy(this.inodes, 0, anodes, 0, length);
     System.arraycopy(this.inodes, 0, anodes, 0, length);
     System.arraycopy(this.path, 0, apath, 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.
    * Note that this method only handles non-snapshot paths.
    */
    */
   public INodesInPath getExistingINodes() {
   public INodesInPath getExistingINodes() {
     Preconditions.checkState(!isSnapshot());
     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 this.isSnapshot;
   }
   }
 
 
+  /**
+   * @return if .snapshot is the last path component.
+   */
   boolean isDotSnapshotDir() {
   boolean isDotSnapshotDir() {
     return isDotSnapshotDir(getLastLocalName());
     return isDotSnapshotDir(getLastLocalName());
   }
   }
 
 
+  /**
+   * @return if this is a /.reserved/raw path.
+   */
+  public boolean isRaw() {
+    return isRaw;
+  }
+
   private static String toString(INode inode) {
   private static String toString(INode inode) {
     return inode == null? null: inode.getLocalName();
     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.
    * Return total time spent doing sync operations on FSEditLog.
    */
    */
   String getTotalSyncTimes();
   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)
       @QueryParam(TokenServiceParam.NAME) @DefaultValue(TokenServiceParam.DEFAULT)
           final TokenServiceParam tokenService,
           final TokenServiceParam tokenService,
       @QueryParam(NoRedirectParam.NAME) @DefaultValue(NoRedirectParam.DEFAULT)
       @QueryParam(NoRedirectParam.NAME) @DefaultValue(NoRedirectParam.DEFAULT)
-          final NoRedirectParam noredirect
+          final NoRedirectParam noredirect,
+      @QueryParam(StartAfterParam.NAME) @DefaultValue(StartAfterParam.DEFAULT)
+          final StartAfterParam startAfter
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
     return get(ugi, delegation, username, doAsUser, ROOT, op, offset, length,
     return get(ugi, delegation, username, doAsUser, ROOT, op, offset, length,
         renewer, bufferSize, xattrNames, xattrEncoding, excludeDatanodes, fsAction,
         renewer, bufferSize, xattrNames, xattrEncoding, excludeDatanodes, fsAction,
-        tokenKind, tokenService, noredirect);
+        tokenKind, tokenService, noredirect, startAfter);
   }
   }
 
 
   /** Handle HTTP GET request. */
   /** Handle HTTP GET request. */
@@ -832,12 +834,14 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(TokenServiceParam.NAME) @DefaultValue(TokenServiceParam.DEFAULT)
       @QueryParam(TokenServiceParam.NAME) @DefaultValue(TokenServiceParam.DEFAULT)
           final TokenServiceParam tokenService,
           final TokenServiceParam tokenService,
       @QueryParam(NoRedirectParam.NAME) @DefaultValue(NoRedirectParam.DEFAULT)
       @QueryParam(NoRedirectParam.NAME) @DefaultValue(NoRedirectParam.DEFAULT)
-          final NoRedirectParam noredirect
+          final NoRedirectParam noredirect,
+      @QueryParam(StartAfterParam.NAME) @DefaultValue(StartAfterParam.DEFAULT)
+          final StartAfterParam startAfter
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
 
 
     init(ugi, delegation, username, doAsUser, path, op, offset, length,
     init(ugi, delegation, username, doAsUser, path, op, offset, length,
         renewer, bufferSize, xattrEncoding, excludeDatanodes, fsAction,
         renewer, bufferSize, xattrEncoding, excludeDatanodes, fsAction,
-        tokenKind, tokenService);
+        tokenKind, tokenService, startAfter);
 
 
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
     return ugi.doAs(new PrivilegedExceptionAction<Response>() {
       @Override
       @Override
@@ -846,7 +850,7 @@ public class NamenodeWebHdfsMethods {
           return get(ugi, delegation, username, doAsUser,
           return get(ugi, delegation, username, doAsUser,
               path.getAbsolutePath(), op, offset, length, renewer, bufferSize,
               path.getAbsolutePath(), op, offset, length, renewer, bufferSize,
               xattrNames, xattrEncoding, excludeDatanodes, fsAction, tokenKind,
               xattrNames, xattrEncoding, excludeDatanodes, fsAction, tokenKind,
-              tokenService, noredirect);
+              tokenService, noredirect, startAfter);
         } finally {
         } finally {
           reset();
           reset();
         }
         }
@@ -871,7 +875,8 @@ public class NamenodeWebHdfsMethods {
       final FsActionParam fsAction,
       final FsActionParam fsAction,
       final TokenKindParam tokenKind,
       final TokenKindParam tokenKind,
       final TokenServiceParam tokenService,
       final TokenServiceParam tokenService,
-      final NoRedirectParam noredirectParam
+      final NoRedirectParam noredirectParam,
+      final StartAfterParam startAfter
       ) throws IOException, URISyntaxException {
       ) throws IOException, URISyntaxException {
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
     final Configuration conf = (Configuration) context
     final Configuration conf = (Configuration) context
@@ -994,6 +999,16 @@ public class NamenodeWebHdfsMethods {
       np.checkAccess(fullpath, FsAction.getFsAction(fsAction.getValue()));
       np.checkAccess(fullpath, FsAction.getFsAction(fsAction.getValue()));
       return Response.ok().build();
       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:
     default:
       throw new UnsupportedOperationException(op + " is not supported");
       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("p", "processor", true, "");
     options.addOption("h", "help", false, "");
     options.addOption("h", "help", false, "");
+    options.addOption("maxSize", true, "");
+    options.addOption("step", true, "");
     options.addOption("skipBlocks", false, "");
     options.addOption("skipBlocks", false, "");
     options.addOption("printToScreen", false, "");
     options.addOption("printToScreen", false, "");
     options.addOption("delimiter", true, "");
     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) {
     if (status == null) {
       return 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>();
     final Map<String, Object> m = new TreeMap<String, Object>();
     m.put("pathSuffix", status.getLocalName());
     m.put("pathSuffix", status.getLocalName());
     m.put("type", WebHdfsConstants.PathType.valueOf(status));
     m.put("type", WebHdfsConstants.PathType.valueOf(status));
@@ -121,12 +131,7 @@ public class JsonUtil {
     m.put("fileId", status.getFileId());
     m.put("fileId", status.getFileId());
     m.put("childrenNum", status.getChildrenNum());
     m.put("childrenNum", status.getChildrenNum());
     m.put("storagePolicy", status.getStoragePolicy());
     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. */
   /** Convert an ExtendedBlock to a Json map. */
@@ -227,6 +232,34 @@ public class JsonUtil {
     return m;
     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. */
   /** Convert a LocatedBlock[] to a Json array. */
   private static Object[] toJsonArray(final List<LocatedBlock> array
   private static Object[] toJsonArray(final List<LocatedBlock> array
       ) throws IOException {
       ) throws IOException {

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

@@ -2614,6 +2614,24 @@
   </description>
   </description>
 </property>
 </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>
 <property>
   <name>dfs.namenode.startup.delay.block.deletion.sec</name>
   <name>dfs.namenode.startup.delay.block.deletion.sec</name>
   <value>0</value>
   <value>0</value>
@@ -3702,6 +3720,46 @@
   </description>
   </description>
 </property>
 </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>
 <property>
   <name>dfs.namenode.audit.log.async</name>
   <name>dfs.namenode.audit.log.async</name>
   <value>false</value>
   <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
 The HDFS snapshot feature introduces a new reserved path name used to
 interact with snapshots: `.snapshot`. When upgrading from an
 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.
 to first be renamed or deleted to avoid conflicting with the reserved path.
 See the upgrade section in
 See the upgrade section in
 [the HDFS user guide](HdfsUserGuide.html#Upgrade_and_Rollback)
 [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)
         * [Truncate a File](#Truncate_a_File)
         * [Status of a File/Directory](#Status_of_a_FileDirectory)
         * [Status of a File/Directory](#Status_of_a_FileDirectory)
         * [List a Directory](#List_a_Directory)
         * [List a Directory](#List_a_Directory)
+        * [Iteratively List a Directory](#Iteratively_List_a_Directory)
     * [Other File System Operations](#Other_File_System_Operations)
     * [Other File System Operations](#Other_File_System_Operations)
         * [Get Content Summary of a Directory](#Get_Content_Summary_of_a_Directory)
         * [Get Content Summary of a Directory](#Get_Content_Summary_of_a_Directory)
         * [Get File Checksum](#Get_File_Checksum)
         * [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)
     * [`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)
     * [`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`](#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)
     * [`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)
     * [`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)
     * [`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
 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
 Other File System Operations
 ----------------------------
 ----------------------------
 
 

部分文件因为文件数量过多而无法显示