浏览代码

merge trunk to branch HDFS-4949

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4949@1532952 13f79535-47bb-0310-9956-ffa450edef68
Andrew Wang 11 年之前
父节点
当前提交
34f08944b7
共有 100 个文件被更改,包括 3643 次插入1631 次删除
  1. 205 6
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
  3. 15 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  4. 19 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  5. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  6. 32 26
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  7. 43 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  8. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  9. 20 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
  10. 33 27
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
  11. 11 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java
  12. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
  13. 20 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
  14. 13 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  15. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  16. 19 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java
  17. 27 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  18. 25 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  19. 15 59
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  20. 150 207
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  21. 27 51
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java
  22. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
  23. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  24. 43 49
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  25. 40 28
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
  26. 11 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
  27. 45 124
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  28. 97 123
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java
  29. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/RollingLogs.java
  30. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RollingLogsImpl.java
  31. 11 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
  32. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  33. 255 182
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  34. 8 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
  35. 8 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
  36. 13 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
  37. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  38. 89 53
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  39. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
  40. 1 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
  41. 6 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StartupProgressServlet.java
  42. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java
  43. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DisallowedDatanodeException.java
  44. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightHashSet.java
  45. 9 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  46. 136 37
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  47. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c
  48. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  49. 16 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  50. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/corrupt_files.jsp
  51. 6 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp
  52. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsnodelist.jsp
  53. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm
  54. 二进制
      hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/federation-background.gif
  55. 二进制
      hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/federation.gif
  56. 116 27
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java
  57. 34 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java
  58. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/shell/TestHdfsTextCommand.java
  59. 24 49
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  60. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  61. 74 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
  62. 50 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
  63. 62 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
  64. 39 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
  65. 13 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
  66. 9 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
  67. 40 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
  68. 8 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileInputStreamCache.java
  69. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpDelegationToken.java
  70. 10 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpFileSystem.java
  71. 39 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestNameNodeHttpServer.java
  72. 13 16
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPeerCache.java
  73. 60 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java
  74. 213 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java
  75. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithNodeGroup.java
  76. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCachedBlocksList.java
  77. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
  78. 88 110
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
  79. 77 84
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java
  80. 152 19
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestJspHelper.java
  81. 13 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
  82. 11 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java
  83. 13 33
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
  84. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java
  85. 59 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterJspHelper.java
  86. 8 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
  87. 18 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
  88. 72 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java
  89. 16 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java
  90. 54 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  91. 268 22
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java
  92. 43 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
  93. 16 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupProgressServlet.java
  94. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java
  95. 104 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java
  96. 51 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java
  97. 57 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestLossyRetryInvocationHandler.java
  98. 9 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
  99. 0 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
  100. 113 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java

+ 205 - 6
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -120,8 +120,8 @@ Trunk (Unreleased)
 
 
     HDFS-4904. Remove JournalService. (Arpit Agarwal via cnauroth)
     HDFS-4904. Remove JournalService. (Arpit Agarwal via cnauroth)
 
 
-    HDFS-4953. Enable HDFS local reads via mmap.
-    (Colin Patrick McCabe via wang).
+    HDFS-5041. Add the time of last heartbeat to dead server Web UI (Shinichi
+    Yamashita via brandonli)
 
 
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
@@ -247,8 +247,19 @@ Release 2.3.0 - UNRELEASED
 
 
   NEW FEATURES
   NEW FEATURES
 
 
+    HDFS-5122. Support failover and retry in WebHdfsFileSystem for NN HA.
+    (Haohui Mai via jing9)
+
+    HDFS-4953. Enable HDFS local reads via mmap.
+    (Colin Patrick McCabe via wang).
+
+    HDFS-5342. Provide more information in the FSNamesystem JMX interfaces.
+    (Haohui Mai via jing9)
+ 
   IMPROVEMENTS
   IMPROVEMENTS
 
 
+    HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
+
     HDFS-4657.  Limit the number of blocks logged by the NN after a block
     HDFS-4657.  Limit the number of blocks logged by the NN after a block
     report to a configurable value.  (Aaron T. Myers via Colin Patrick
     report to a configurable value.  (Aaron T. Myers via Colin Patrick
     McCabe)
     McCabe)
@@ -256,9 +267,6 @@ Release 2.3.0 - UNRELEASED
     HDFS-4278. Log an ERROR when DFS_BLOCK_ACCESS_TOKEN_ENABLE config is
     HDFS-4278. Log an ERROR when DFS_BLOCK_ACCESS_TOKEN_ENABLE config is
     disabled but security is turned on. (Kousuke Saruta via harsh)
     disabled but security is turned on. (Kousuke Saruta via harsh)
 
 
-    HDFS-4817.  Make HDFS advisory caching configurable on a per-file basis.
-    (Colin Patrick McCabe)
-
     HDFS-5004. Add additional JMX bean for NameNode status data
     HDFS-5004. Add additional JMX bean for NameNode status data
     (Trevor Lorimer via cos)
     (Trevor Lorimer via cos)
 
 
@@ -275,8 +283,46 @@ Release 2.3.0 - UNRELEASED
     HDFS-4879. Add "blocked ArrayList" collection to avoid CMS full GCs
     HDFS-4879. Add "blocked ArrayList" collection to avoid CMS full GCs
     (Todd Lipcon via Colin Patrick McCabe)
     (Todd Lipcon via Colin Patrick McCabe)
 
 
+    HDFS-4096. Add snapshot information to namenode WebUI. (Haohui Mai via 
+    jing9)
+
+    HDFS-5188. In BlockPlacementPolicy, reduce the number of chooseTarget(..)
+    methods; replace HashMap with Map in parameter declarations and cleanup
+    some related code.  (szetszwo)
+
+    HDFS-5207. In BlockPlacementPolicy.chooseTarget(..), change the writer
+    and the excludedNodes parameter types respectively to Node and Set.
+    (Junping Du via szetszwo)
+
+    HDFS-5240. Separate formatting from logging in the audit logger API (daryn)
+
+    HDFS-5191. Revisit zero-copy API in FSDataInputStream to make it more
+    intuitive.  (Contributed by Colin Patrick McCabe)
+
+    HDFS-5260. Merge zero-copy memory-mapped HDFS client reads to trunk and
+    branch-2. (cnauroth)
+
+    HDFS-4517. Cover class RemoteBlockReader with unit tests. (Vadim Bondarev
+    and Dennis Y via kihwal)
+
+    HDFS-4512. Cover package org.apache.hadoop.hdfs.server.common with tests.
+    (Vadim Bondarev via kihwal)
+
+    HDFS-4510. Cover classes ClusterJspHelper/NamenodeJspHelper with unit
+    tests. (Andrey Klochkov via kihwal)
+
+    HDFS-5323.  Remove some deadcode in BlockManager (Colin Patrick McCabe)
+
+    HDFS-5338. Add a conf to disable hostname check in datanode registration.
+    (szetszwo)
+
+    HDFS-5130. Add test for snapshot related FsShell and DFSAdmin commands.
+    (Binglin Chang via jing9)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
+    HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
+
   BUG FIXES
   BUG FIXES
     HDFS-5034.  Remove debug prints from GetFileLinkInfo (Andrew Wang via Colin
     HDFS-5034.  Remove debug prints from GetFileLinkInfo (Andrew Wang via Colin
     Patrick McCabe)
     Patrick McCabe)
@@ -295,7 +341,130 @@ Release 2.3.0 - UNRELEASED
     HDFS-5170. BlockPlacementPolicyDefault uses the wrong classname when
     HDFS-5170. BlockPlacementPolicyDefault uses the wrong classname when
     alerting to enable debug logging. (Andrew Wang)
     alerting to enable debug logging. (Andrew Wang)
 
 
-Release 2.1.1-beta - UNRELEASED
+    HDFS-5031. BlockScanner scans the block multiple times. (Vinay via Arpit
+    Agarwal)
+
+    HDFS-5266. ElasticByteBufferPool#Key does not implement equals. (cnauroth)
+
+    HDFS-5352. Server#initLog() doesn't close InputStream in httpfs. (Ted Yu via
+    jing9)
+
+    HDFS-5283. Under construction blocks only inside snapshots should not be
+    counted in safemode threshhold.  (Vinay via szetszwo)
+
+    HDFS-4376. Fix race conditions in Balancer.  (Junping Du via szetszwo)
+
+Release 2.2.1 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+    HDFS-5360. Improvement of usage message of renameSnapshot and
+    deleteSnapshot. (Shinichi Yamashita via wang)
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    HDFS-5307. Support both HTTP and HTTPS in jsp pages (Haohui Mai via
+    brandonli)
+
+    HDFS-5291. Standby namenode after transition to active goes into safemode.
+    (jing9)
+
+    HDFS-5317. Go back to DFS Home link does not work on datanode webUI
+    (Haohui Mai via brandonli)
+
+    HDFS-5316. Namenode ignores the default https port (Haohui Mai via
+    brandonli)
+
+    HDFS-5281. COMMIT request should not block. (brandonli)
+
+    HDFS-5337. should do hsync for a commit request even there is no pending
+    writes (brandonli)
+
+    HDFS-5335. Hive query failed with possible race in dfs output stream.
+    (Haohui Mai via suresh)
+
+    HDFS-5322. HDFS delegation token not found in cache errors seen on secure HA 
+    clusters. (jing9)
+
+    HDFS-5329. Update FSNamesystem#getListing() to handle inode path in startAfter
+    token. (brandonli)
+
+    HDFS-5330. fix readdir and readdirplus for large directories (brandonli)
+
+    HDFS-5370. Typo in Error Message: different between range in condition
+    and range in error message. (Kousuke Saruta via suresh)
+
+Release 2.2.0 - 2013-10-13
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+    HDFS-4817.  Make HDFS advisory caching configurable on a per-file basis.
+    (Colin Patrick McCabe)
+
+    HDFS-5230. Introduce RpcInfo to decouple XDR classes from the RPC API.
+    (Haohui Mai via brandonli)
+
+  IMPROVEMENTS
+
+    HDFS-5246. Make Hadoop nfs server port and mount daemon port
+    configurable. (Jinghui Wang via brandonli)
+
+    HDFS-5256. Use guava LoadingCache to implement DFSClientCache. (Haohui Mai
+    via brandonli)
+
+    HDFS-5308. Replace HttpConfig#getSchemePrefix with implicit schemes in HDFS 
+    JSP. (Haohui Mai via jing9)
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    HDFS-5139. Remove redundant -R option from setrep.
+
+    HDFS-5251. Race between the initialization of NameNode and the http
+    server. (Haohui Mai via suresh)
+
+    HDFS-5258. Skip tests in TestHDFSCLI that are not applicable on Windows.
+    (Chuan Liu via cnauroth)
+
+    HDFS-5186. TestFileJournalManager fails on Windows due to file handle leaks.
+    (Chuan Liu via cnauroth)
+
+    HDFS-5268. NFS write commit verifier is not set in a few places (brandonli)
+
+    HDFS-5265. Namenode fails to start when dfs.https.port is unspecified.
+    (Haohui Mai via jing9)
+
+    HDFS-5255. Distcp job fails with hsftp when https is enabled in insecure
+    cluster. (Arpit Agarwal)
+
+    HDFS-5279. Guard against NullPointerException in NameNode JSP pages before
+    initialization of FSNamesystem. (cnauroth)
+
+    HDFS-5289. Race condition in TestRetryCacheWithHA#testCreateSymlink causes
+    spurious test failure. (atm)
+
+    HDFS-5300. FSNameSystem#deleteSnapshot() should not check owner in case of 
+    permissions disabled. (Vinay via jing9)
+
+    HDFS-5306. Datanode https port is not available at the namenode. (Suresh
+    Srinivas via brandonli)
+
+    HDFS-5299. DFS client hangs in updatePipeline RPC when failover happened.
+    (Vinay via jing9)
+
+    HDFS-5259. Support client which combines appended data with old data
+    before sends it to NFS server. (brandonli)
+
+Release 2.1.1-beta - 2013-09-23
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
 
 
@@ -336,6 +505,13 @@ Release 2.1.1-beta - UNRELEASED
     HDFS-5085. Refactor o.a.h.nfs to support different types of 
     HDFS-5085. Refactor o.a.h.nfs to support different types of 
     authentications. (jing9)
     authentications. (jing9)
 
 
+    HDFS-5067 Support symlink operations in NFS gateway. (brandonli)
+
+    HDFS-5199 Add more debug trace for NFS READ and WRITE. (brandonli)
+
+    HDFS-5234 Move RpcFrameDecoder out of the public API.
+    (Haohui Mai via brandonli)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-4513. Clarify in the WebHDFS REST API that all JSON respsonses may
     HDFS-4513. Clarify in the WebHDFS REST API that all JSON respsonses may
@@ -372,6 +548,12 @@ Release 2.1.1-beta - UNRELEASED
 
 
     HDFS-4680. Audit logging of delegation tokens for MR tracing. (Andrew Wang)
     HDFS-4680. Audit logging of delegation tokens for MR tracing. (Andrew Wang)
 
 
+    HDFS-5212. Refactor RpcMessage and NFS3Response to support different 
+    types of authentication information. (jing9)
+
+    HDFS-4971. Move IO operations out of locking in OpenFileCtx. (brandonli and
+    jing9)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -440,6 +622,17 @@ Release 2.1.1-beta - UNRELEASED
     HDFS-5159. Secondary NameNode fails to checkpoint if error occurs
     HDFS-5159. Secondary NameNode fails to checkpoint if error occurs
     downloading edits on first checkpoint. (atm)
     downloading edits on first checkpoint. (atm)
 
 
+    HDFS-5192. NameNode may fail to start when 
+    dfs.client.test.drop.namenode.response.number is set. (jing9)
+
+    HDFS-5219. Add configuration keys for retry policy in WebHDFSFileSystem.
+    (Haohui Mai via jing9)
+
+    HDFS-5231. Fix broken links in the document of HDFS Federation. (Haohui Mai
+    via jing9)
+
+    HDFS-5249. Fix dumper thread which may die silently. (brandonli)
+
 Release 2.1.0-beta - 2013-08-22
 Release 2.1.0-beta - 2013-08-22
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -893,6 +1086,9 @@ Release 2.1.0-beta - 2013-08-22
     HDFS-5016. Deadlock in pipeline recovery causes Datanode to be marked dead.
     HDFS-5016. Deadlock in pipeline recovery causes Datanode to be marked dead.
     (suresh)
     (suresh)
 
 
+    HDFS-5228. The RemoteIterator returned by DistributedFileSystem.listFiles
+    may throw NullPointerException.  (szetszwo and cnauroth via szetszwo)
+
   BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
   BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
 
 
     HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.
     HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.
@@ -3483,6 +3679,9 @@ Release 0.23.10 - UNRELEASED
     HDFS-5010. Reduce the frequency of getCurrentUser() calls from namenode
     HDFS-5010. Reduce the frequency of getCurrentUser() calls from namenode
     (kihwal)
     (kihwal)
 
 
+    HDFS-5346. Avoid unnecessary call to getNumLiveDataNodes() for each block 
+    during IBR processing (Ravi Prakash via kihwal)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml

@@ -36,7 +36,25 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <hadoop.common.build.dir>${basedir}/../../../../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
     <hadoop.common.build.dir>${basedir}/../../../../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
   </properties>
   </properties>
 
 
+  <dependencyManagement>
+    <dependencies>
+      <!-- This is a really old version of netty, that gets privatized
+           via shading and hence it is not managed via a parent pom -->
+      <dependency>
+        <groupId>org.jboss.netty</groupId>
+        <artifactId>netty</artifactId>
+        <version>3.2.4.Final</version>
+      </dependency>
+    </dependencies>
+  </dependencyManagement>
+
   <dependencies>
   <dependencies>
+    <dependency>
+      <groupId>org.jboss.netty</groupId>
+      <artifactId>netty</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
     <dependency>
     <dependency>
       <groupId>commons-logging</groupId>
       <groupId>commons-logging</groupId>
       <artifactId>commons-logging</artifactId>
       <artifactId>commons-logging</artifactId>

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

@@ -525,14 +525,17 @@ public class DFSClient implements java.io.Closeable {
     int numResponseToDrop = conf.getInt(
     int numResponseToDrop = conf.getInt(
         DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
         DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
         DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
         DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
+    NameNodeProxies.ProxyAndInfo<ClientProtocol> proxyInfo = null;
     if (numResponseToDrop > 0) {
     if (numResponseToDrop > 0) {
       // This case is used for testing.
       // This case is used for testing.
       LOG.warn(DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
       LOG.warn(DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
           + " is set to " + numResponseToDrop
           + " is set to " + numResponseToDrop
           + ", this hacked client will proactively drop responses");
           + ", this hacked client will proactively drop responses");
-      NameNodeProxies.ProxyAndInfo<ClientProtocol> proxyInfo = NameNodeProxies
-          .createProxyWithLossyRetryHandler(conf, nameNodeUri,
-              ClientProtocol.class, numResponseToDrop);
+      proxyInfo = NameNodeProxies.createProxyWithLossyRetryHandler(conf,
+          nameNodeUri, ClientProtocol.class, numResponseToDrop);
+    }
+    
+    if (proxyInfo != null) {
       this.dtService = proxyInfo.getDelegationTokenService();
       this.dtService = proxyInfo.getDelegationTokenService();
       this.namenode = proxyInfo.getProxy();
       this.namenode = proxyInfo.getProxy();
     } else if (rpcNamenode != null) {
     } else if (rpcNamenode != null) {
@@ -543,9 +546,8 @@ public class DFSClient implements java.io.Closeable {
     } else {
     } else {
       Preconditions.checkArgument(nameNodeUri != null,
       Preconditions.checkArgument(nameNodeUri != null,
           "null URI");
           "null URI");
-      NameNodeProxies.ProxyAndInfo<ClientProtocol> proxyInfo =
-        NameNodeProxies.createProxy(conf, nameNodeUri, ClientProtocol.class);
-      
+      proxyInfo = NameNodeProxies.createProxy(conf, nameNodeUri,
+          ClientProtocol.class);
       this.dtService = proxyInfo.getDelegationTokenService();
       this.dtService = proxyInfo.getDelegationTokenService();
       this.namenode = proxyInfo.getProxy();
       this.namenode = proxyInfo.getProxy();
     }
     }
@@ -902,10 +904,15 @@ public class DFSClient implements java.io.Closeable {
     assert dtService != null;
     assert dtService != null;
     Token<DelegationTokenIdentifier> token =
     Token<DelegationTokenIdentifier> token =
       namenode.getDelegationToken(renewer);
       namenode.getDelegationToken(renewer);
-    token.setService(this.dtService);
 
 
-    LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(token));
+    if (token != null) {
+      token.setService(this.dtService);
+      LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(token));
+    } else {
+      LOG.info("Cannot get delegation token from " + renewer);
+    }
     return token;
     return token;
+
   }
   }
 
 
   /**
   /**

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 
 
 /** 
 /** 
  * This class contains constants for configuration keys used
  * This class contains constants for configuration keys used
@@ -198,7 +199,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_DATANODE_SYNCONCLOSE_DEFAULT = false;
   public static final boolean DFS_DATANODE_SYNCONCLOSE_DEFAULT = false;
   public static final String  DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY = "dfs.datanode.socket.reuse.keepalive";
   public static final String  DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY = "dfs.datanode.socket.reuse.keepalive";
   public static final int     DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT = 1000;
   public static final int     DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT = 1000;
-  
+
+  public static final String DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_KEY = "dfs.namenode.datanode.registration.ip-hostname-check";
+  public static final boolean DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_DEFAULT = true;
+
   public static final String  DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES =
   public static final String  DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES =
       "dfs.namenode.list.cache.pools.num.responses";
       "dfs.namenode.list.cache.pools.num.responses";
   public static final int     DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES_DEFAULT = 100;
   public static final int     DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES_DEFAULT = 100;
@@ -364,6 +368,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_BLOCK_ACCESS_TOKEN_LIFETIME_KEY = "dfs.block.access.token.lifetime";
   public static final String  DFS_BLOCK_ACCESS_TOKEN_LIFETIME_KEY = "dfs.block.access.token.lifetime";
   public static final long    DFS_BLOCK_ACCESS_TOKEN_LIFETIME_DEFAULT = 600L;
   public static final long    DFS_BLOCK_ACCESS_TOKEN_LIFETIME_DEFAULT = 600L;
 
 
+  public static final String DFS_BLOCK_REPLICATOR_CLASSNAME_KEY = "dfs.block.replicator.classname";
+  public static final Class<BlockPlacementPolicyDefault> DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT = BlockPlacementPolicyDefault.class;
   public static final String  DFS_REPLICATION_MAX_KEY = "dfs.replication.max";
   public static final String  DFS_REPLICATION_MAX_KEY = "dfs.replication.max";
   public static final int     DFS_REPLICATION_MAX_DEFAULT = 512;
   public static final int     DFS_REPLICATION_MAX_DEFAULT = 512;
   public static final String  DFS_DF_INTERVAL_KEY = "dfs.df.interval";
   public static final String  DFS_DF_INTERVAL_KEY = "dfs.df.interval";
@@ -534,4 +540,16 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   // Timeout to wait for block receiver and responder thread to stop
   // Timeout to wait for block receiver and responder thread to stop
   public static final String DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY = "dfs.datanode.xceiver.stop.timeout.millis";
   public static final String DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY = "dfs.datanode.xceiver.stop.timeout.millis";
   public static final long   DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT = 60000;
   public static final long   DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT = 60000;
+
+  // WebHDFS retry policy
+  public static final String  DFS_HTTP_CLIENT_RETRY_POLICY_ENABLED_KEY = "dfs.http.client.retry.policy.enabled";
+  public static final boolean DFS_HTTP_CLIENT_RETRY_POLICY_ENABLED_DEFAULT = false;
+  public static final String  DFS_HTTP_CLIENT_RETRY_POLICY_SPEC_KEY = "dfs.http.client.retry.policy.spec";
+  public static final String  DFS_HTTP_CLIENT_RETRY_POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,...
+  public static final String  DFS_HTTP_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY = "dfs.http.client.failover.max.attempts";
+  public static final int     DFS_HTTP_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT = 15;
+  public static final String  DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY = "dfs.http.client.failover.sleep.base.millis";
+  public static final int     DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT = 500;
+  public static final String  DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY = "dfs.http.client.failover.sleep.max.millis";
+  public static final int     DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT = 15000;
 }
 }

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

@@ -403,7 +403,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
 
     //check offset
     //check offset
     if (offset < 0 || offset >= getFileLength()) {
     if (offset < 0 || offset >= getFileLength()) {
-      throw new IOException("offset < 0 || offset > getFileLength(), offset="
+      throw new IOException("offset < 0 || offset >= getFileLength(), offset="
           + offset
           + offset
           + ", updatePosition=" + updatePosition
           + ", updatePosition=" + updatePosition
           + ", locatedBlocks=" + locatedBlocks);
           + ", locatedBlocks=" + locatedBlocks);

+ 32 - 26
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -38,6 +38,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CanSetDropBehind;
 import org.apache.hadoop.fs.CanSetDropBehind;
@@ -85,7 +86,6 @@ import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
-import org.mortbay.log.Log;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheBuilder;
@@ -141,7 +141,7 @@ public class DFSOutputStream extends FSOutputSummer
   private long bytesCurBlock = 0; // bytes writen in current block
   private long bytesCurBlock = 0; // bytes writen in current block
   private int packetSize = 0; // write packet size, not including the header.
   private int packetSize = 0; // write packet size, not including the header.
   private int chunksPerPacket = 0;
   private int chunksPerPacket = 0;
-  private volatile IOException lastException = null;
+  private final AtomicReference<IOException> lastException = new AtomicReference<IOException>();
   private long artificialSlowdown = 0;
   private long artificialSlowdown = 0;
   private long lastFlushOffset = 0; // offset when flush was invoked
   private long lastFlushOffset = 0; // offset when flush was invoked
   //persist blocks on namenode
   //persist blocks on namenode
@@ -809,8 +809,8 @@ public class DFSOutputStream extends FSOutputSummer
         if (++pipelineRecoveryCount > 5) {
         if (++pipelineRecoveryCount > 5) {
           DFSClient.LOG.warn("Error recovering pipeline for writing " +
           DFSClient.LOG.warn("Error recovering pipeline for writing " +
               block + ". Already retried 5 times for the same packet.");
               block + ". Already retried 5 times for the same packet.");
-          lastException = new IOException("Failing write. Tried pipeline " +
-              "recovery 5 times without success.");
+          lastException.set(new IOException("Failing write. Tried pipeline " +
+              "recovery 5 times without success."));
           streamerClosed = true;
           streamerClosed = true;
           return false;
           return false;
         }
         }
@@ -1000,8 +1000,8 @@ public class DFSOutputStream extends FSOutputSummer
             }
             }
           }
           }
           if (nodes.length <= 1) {
           if (nodes.length <= 1) {
-            lastException = new IOException("All datanodes " + pipelineMsg
-                + " are bad. Aborting...");
+            lastException.set(new IOException("All datanodes " + pipelineMsg
+                + " are bad. Aborting..."));
             streamerClosed = true;
             streamerClosed = true;
             return false;
             return false;
           }
           }
@@ -1016,7 +1016,7 @@ public class DFSOutputStream extends FSOutputSummer
               newnodes.length-errorIndex);
               newnodes.length-errorIndex);
           nodes = newnodes;
           nodes = newnodes;
           hasError = false;
           hasError = false;
-          lastException = null;
+          lastException.set(null);
           errorIndex = -1;
           errorIndex = -1;
         }
         }
 
 
@@ -1060,7 +1060,7 @@ public class DFSOutputStream extends FSOutputSummer
       ExtendedBlock oldBlock = block;
       ExtendedBlock oldBlock = block;
       do {
       do {
         hasError = false;
         hasError = false;
-        lastException = null;
+        lastException.set(null);
         errorIndex = -1;
         errorIndex = -1;
         success = false;
         success = false;
 
 
@@ -1275,9 +1275,7 @@ public class DFSOutputStream extends FSOutputSummer
     }
     }
 
 
     private void setLastException(IOException e) {
     private void setLastException(IOException e) {
-      if (lastException == null) {
-        lastException = e;
-      }
+      lastException.compareAndSet(null, e);
     }
     }
   }
   }
 
 
@@ -1309,7 +1307,7 @@ public class DFSOutputStream extends FSOutputSummer
 
 
   protected void checkClosed() throws IOException {
   protected void checkClosed() throws IOException {
     if (closed) {
     if (closed) {
-      IOException e = lastException;
+      IOException e = lastException.get();
       throw e != null ? e : new ClosedChannelException();
       throw e != null ? e : new ClosedChannelException();
     }
     }
   }
   }
@@ -1465,6 +1463,7 @@ public class DFSOutputStream extends FSOutputSummer
 
 
   private void waitAndQueueCurrentPacket() throws IOException {
   private void waitAndQueueCurrentPacket() throws IOException {
     synchronized (dataQueue) {
     synchronized (dataQueue) {
+      try {
       // If queue is full, then wait till we have enough space
       // If queue is full, then wait till we have enough space
       while (!closed && dataQueue.size() + ackQueue.size()  > MAX_PACKETS) {
       while (!closed && dataQueue.size() + ackQueue.size()  > MAX_PACKETS) {
         try {
         try {
@@ -1483,6 +1482,8 @@ public class DFSOutputStream extends FSOutputSummer
       }
       }
       checkClosed();
       checkClosed();
       queueCurrentPacket();
       queueCurrentPacket();
+      } catch (ClosedChannelException e) {
+      }
     }
     }
   }
   }
 
 
@@ -1726,7 +1727,7 @@ public class DFSOutputStream extends FSOutputSummer
       DFSClient.LOG.warn("Error while syncing", e);
       DFSClient.LOG.warn("Error while syncing", e);
       synchronized (this) {
       synchronized (this) {
         if (!closed) {
         if (!closed) {
-          lastException = new IOException("IOException flush:" + e);
+          lastException.set(new IOException("IOException flush:" + e));
           closeThreads(true);
           closeThreads(true);
         }
         }
       }
       }
@@ -1784,21 +1785,25 @@ public class DFSOutputStream extends FSOutputSummer
     if (DFSClient.LOG.isDebugEnabled()) {
     if (DFSClient.LOG.isDebugEnabled()) {
       DFSClient.LOG.debug("Waiting for ack for: " + seqno);
       DFSClient.LOG.debug("Waiting for ack for: " + seqno);
     }
     }
-    synchronized (dataQueue) {
-      while (!closed) {
-        checkClosed();
-        if (lastAckedSeqno >= seqno) {
-          break;
-        }
-        try {
-          dataQueue.wait(1000); // when we receive an ack, we notify on dataQueue
-        } catch (InterruptedException ie) {
-          throw new InterruptedIOException(
-            "Interrupted while waiting for data to be acknowledged by pipeline");
+    try {
+      synchronized (dataQueue) {
+        while (!closed) {
+          checkClosed();
+          if (lastAckedSeqno >= seqno) {
+            break;
+          }
+          try {
+            dataQueue.wait(1000); // when we receive an ack, we notify on
+                                  // dataQueue
+          } catch (InterruptedException ie) {
+            throw new InterruptedIOException(
+                "Interrupted while waiting for data to be acknowledged by pipeline");
+          }
         }
         }
       }
       }
+      checkClosed();
+    } catch (ClosedChannelException e) {
     }
     }
-    checkClosed();
   }
   }
 
 
   private synchronized void start() {
   private synchronized void start() {
@@ -1844,7 +1849,7 @@ public class DFSOutputStream extends FSOutputSummer
   @Override
   @Override
   public synchronized void close() throws IOException {
   public synchronized void close() throws IOException {
     if (closed) {
     if (closed) {
-      IOException e = lastException;
+      IOException e = lastException.getAndSet(null);
       if (e == null)
       if (e == null)
         return;
         return;
       else
       else
@@ -1872,6 +1877,7 @@ public class DFSOutputStream extends FSOutputSummer
       closeThreads(false);
       closeThreads(false);
       completeFile(lastBlock);
       completeFile(lastBlock);
       dfsClient.endFileLease(src);
       dfsClient.endFileLease(src);
+    } catch (ClosedChannelException e) {
     } finally {
     } finally {
       closed = true;
       closed = true;
     }
     }

+ 43 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -38,6 +38,7 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.security.SecureRandom;
 import java.security.SecureRandom;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Comparator;
@@ -593,6 +594,48 @@ public class DFSUtil {
       Configuration conf) {
       Configuration conf) {
     return getAddresses(conf, null, DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
     return getAddresses(conf, null, DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
   }
   }
+
+  /**
+   * Returns list of InetSocketAddress corresponding to HA NN HTTP addresses from
+   * the configuration.
+   *
+   * @param conf configuration
+   * @return list of InetSocketAddresses
+   */
+  public static Map<String, Map<String, InetSocketAddress>> getHaNnHttpAddresses(
+      Configuration conf) {
+    return getAddresses(conf, null, DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+  }
+
+  /**
+   * Resolve an HDFS URL into real INetSocketAddress. It works like a DNS resolver
+   * when the URL points to an non-HA cluster. When the URL points to an HA
+   * cluster, the resolver further resolves the logical name (i.e., the authority
+   * in the URL) into real namenode addresses.
+   */
+  public static InetSocketAddress[] resolve(URI uri, int schemeDefaultPort,
+      Configuration conf) throws IOException {
+    ArrayList<InetSocketAddress> ret = new ArrayList<InetSocketAddress>();
+
+    if (!HAUtil.isLogicalUri(conf, uri)) {
+      InetSocketAddress addr = NetUtils.createSocketAddr(uri.getAuthority(),
+          schemeDefaultPort);
+      ret.add(addr);
+
+    } else {
+      Map<String, Map<String, InetSocketAddress>> addresses = DFSUtil
+          .getHaNnHttpAddresses(conf);
+
+      for (Map<String, InetSocketAddress> addrs : addresses.values()) {
+        for (InetSocketAddress addr : addrs.values()) {
+          ret.add(addr);
+        }
+      }
+    }
+
+    InetSocketAddress[] r = new InetSocketAddress[ret.size()];
+    return ret.toArray(r);
+  }
   
   
   /**
   /**
    * Returns list of InetSocketAddress corresponding to  backup node rpc 
    * Returns list of InetSocketAddress corresponding to  backup node rpc 

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

@@ -716,6 +716,7 @@ public class DistributedFileSystem extends FileSystem {
   protected RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path p,
   protected RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path p,
       final PathFilter filter)
       final PathFilter filter)
   throws IOException {
   throws IOException {
+    final Path absF = fixRelativePart(p);
     return new RemoteIterator<LocatedFileStatus>() {
     return new RemoteIterator<LocatedFileStatus>() {
       private DirectoryListing thisListing;
       private DirectoryListing thisListing;
       private int i;
       private int i;
@@ -725,7 +726,7 @@ public class DistributedFileSystem extends FileSystem {
       { // initializer
       { // initializer
         // Fully resolve symlinks in path first to avoid additional resolution
         // Fully resolve symlinks in path first to avoid additional resolution
         // round-trips as we fetch more batches of listings
         // round-trips as we fetch more batches of listings
-        src = getPathName(resolvePath(p));
+        src = getPathName(resolvePath(absF));
         // fetch the first batch of entries in the directory
         // fetch the first batch of entries in the directory
         thisListing = dfs.listPaths(src, HdfsFileStatus.EMPTY_NAME, true);
         thisListing = dfs.listPaths(src, HdfsFileStatus.EMPTY_NAME, true);
         statistics.incrementReadOps(1);
         statistics.incrementReadOps(1);
@@ -739,7 +740,7 @@ public class DistributedFileSystem extends FileSystem {
         while (curStat == null && hasNextNoFilter()) {
         while (curStat == null && hasNextNoFilter()) {
           LocatedFileStatus next = 
           LocatedFileStatus next = 
               ((HdfsLocatedFileStatus)thisListing.getPartialListing()[i++])
               ((HdfsLocatedFileStatus)thisListing.getPartialListing()[i++])
-              .makeQualifiedLocated(getUri(), p);
+              .makeQualifiedLocated(getUri(), absF);
           if (filter.accept(next.getPath())) {
           if (filter.accept(next.getPath())) {
             curStat = next;
             curStat = next;
           }
           }

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

@@ -17,15 +17,9 @@
  */
  */
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Map;
-
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 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.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -41,11 +35,17 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HA_DT_SERVICE_PREFIX;
 
 
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HA_DT_SERVICE_PREFIX;
 
 
 public class HAUtil {
 public class HAUtil {
   
   
@@ -265,10 +265,15 @@ public class HAUtil {
         tokenSelector.selectToken(haService, ugi.getTokens());
         tokenSelector.selectToken(haService, ugi.getTokens());
     if (haToken != null) {
     if (haToken != null) {
       for (InetSocketAddress singleNNAddr : nnAddrs) {
       for (InetSocketAddress singleNNAddr : nnAddrs) {
+        // this is a minor hack to prevent physical HA tokens from being
+        // exposed to the user via UGI.getCredentials(), otherwise these
+        // cloned tokens may be inadvertently propagated to jobs
         Token<DelegationTokenIdentifier> specificToken =
         Token<DelegationTokenIdentifier> specificToken =
-            new Token<DelegationTokenIdentifier>(haToken);
+            new Token.PrivateToken<DelegationTokenIdentifier>(haToken);
         SecurityUtil.setTokenService(specificToken, singleNNAddr);
         SecurityUtil.setTokenService(specificToken, singleNNAddr);
-        ugi.addToken(specificToken);
+        Text alias =
+            new Text(HA_DT_SERVICE_PREFIX + "//" + specificToken.getService());
+        ugi.addToken(alias, specificToken);
         LOG.debug("Mapped HA service delegation token for logical URI " +
         LOG.debug("Mapped HA service delegation token for logical URI " +
             haUri + " to namenode " + singleNNAddr);
             haUri + " to namenode " + singleNNAddr);
       }
       }

+ 33 - 27
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -94,7 +94,6 @@ public class HftpFileSystem extends FileSystem
   private URI hftpURI;
   private URI hftpURI;
 
 
   protected URI nnUri;
   protected URI nnUri;
-  protected URI nnSecureUri;
 
 
   public static final String HFTP_TIMEZONE = "UTC";
   public static final String HFTP_TIMEZONE = "UTC";
   public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
   public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
@@ -134,34 +133,33 @@ public class HftpFileSystem extends FileSystem
         DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT);
         DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT);
   }
   }
 
 
-  protected int getDefaultSecurePort() {
-    return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY,
-        DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT);
-  }
-
+  /**
+   *  We generate the address with one of the following ports, in
+   *  order of preference.
+   *  1. Port from the hftp URI e.g. hftp://namenode:4000/ will return 4000.
+   *  2. Port configured via DFS_NAMENODE_HTTP_PORT_KEY
+   *  3. DFS_NAMENODE_HTTP_PORT_DEFAULT i.e. 50070.
+   *
+   * @param uri
+   * @return
+   */
   protected InetSocketAddress getNamenodeAddr(URI uri) {
   protected InetSocketAddress getNamenodeAddr(URI uri) {
     // use authority so user supplied uri can override port
     // use authority so user supplied uri can override port
     return NetUtils.createSocketAddr(uri.getAuthority(), getDefaultPort());
     return NetUtils.createSocketAddr(uri.getAuthority(), getDefaultPort());
   }
   }
 
 
-  protected InetSocketAddress getNamenodeSecureAddr(URI uri) {
-    // must only use the host and the configured https port
-    return NetUtils.createSocketAddrForHost(uri.getHost(), getDefaultSecurePort());
-  }
-
   protected URI getNamenodeUri(URI uri) {
   protected URI getNamenodeUri(URI uri) {
-    return DFSUtil.createUri("http", getNamenodeAddr(uri));
-  }
-
-  protected URI getNamenodeSecureUri(URI uri) {
-    return DFSUtil.createUri("http", getNamenodeSecureAddr(uri));
+    return DFSUtil.createUri(getUnderlyingProtocol(), getNamenodeAddr(uri));
   }
   }
 
 
+  /**
+   * See the documentation of {@Link #getNamenodeAddr(URI)} for the logic
+   * behind selecting the canonical service name.
+   * @return
+   */
   @Override
   @Override
   public String getCanonicalServiceName() {
   public String getCanonicalServiceName() {
-    // unlike other filesystems, hftp's service is the secure port, not the
-    // actual port in the uri
-    return SecurityUtil.buildTokenService(nnSecureUri).toString();
+    return SecurityUtil.buildTokenService(nnUri).toString();
   }
   }
   
   
   @Override
   @Override
@@ -187,7 +185,6 @@ public class HftpFileSystem extends FileSystem
     setConf(conf);
     setConf(conf);
     this.ugi = UserGroupInformation.getCurrentUser(); 
     this.ugi = UserGroupInformation.getCurrentUser(); 
     this.nnUri = getNamenodeUri(name);
     this.nnUri = getNamenodeUri(name);
-    this.nnSecureUri = getNamenodeSecureUri(name);
     try {
     try {
       this.hftpURI = new URI(name.getScheme(), name.getAuthority(),
       this.hftpURI = new URI(name.getScheme(), name.getAuthority(),
                              null, null, null);
                              null, null, null);
@@ -225,7 +222,7 @@ public class HftpFileSystem extends FileSystem
 
 
   protected Token<DelegationTokenIdentifier> selectDelegationToken(
   protected Token<DelegationTokenIdentifier> selectDelegationToken(
       UserGroupInformation ugi) {
       UserGroupInformation ugi) {
-  	return hftpTokenSelector.selectToken(nnSecureUri, ugi.getTokens(), getConf());
+    return hftpTokenSelector.selectToken(nnUri, ugi.getTokens(), getConf());
   }
   }
   
   
 
 
@@ -234,6 +231,13 @@ public class HftpFileSystem extends FileSystem
     return renewToken;
     return renewToken;
   }
   }
 
 
+  /**
+   * Return the underlying protocol that is used to talk to the namenode.
+   */
+  protected String getUnderlyingProtocol() {
+    return "http";
+  }
+
   @Override
   @Override
   public synchronized <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
   public synchronized <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
     renewToken = token;
     renewToken = token;
@@ -257,7 +261,7 @@ public class HftpFileSystem extends FileSystem
       return ugi.doAs(new PrivilegedExceptionAction<Token<?>>() {
       return ugi.doAs(new PrivilegedExceptionAction<Token<?>>() {
         @Override
         @Override
         public Token<?> run() throws IOException {
         public Token<?> run() throws IOException {
-          final String nnHttpUrl = nnSecureUri.toString();
+          final String nnHttpUrl = nnUri.toString();
           Credentials c;
           Credentials c;
           try {
           try {
             c = DelegationTokenFetcher.getDTfromRemote(nnHttpUrl, renewer);
             c = DelegationTokenFetcher.getDTfromRemote(nnHttpUrl, renewer);
@@ -301,7 +305,7 @@ public class HftpFileSystem extends FileSystem
    * @throws IOException on error constructing the URL
    * @throws IOException on error constructing the URL
    */
    */
   protected URL getNamenodeURL(String path, String query) throws IOException {
   protected URL getNamenodeURL(String path, String query) throws IOException {
-    final URL url = new URL("http", nnUri.getHost(),
+    final URL url = new URL(getUnderlyingProtocol(), nnUri.getHost(),
           nnUri.getPort(), path + '?' + query);
           nnUri.getPort(), path + '?' + query);
     if (LOG.isTraceEnabled()) {
     if (LOG.isTraceEnabled()) {
       LOG.trace("url=" + url);
       LOG.trace("url=" + url);
@@ -703,17 +707,20 @@ public class HftpFileSystem extends FileSystem
       return true;
       return true;
     }
     }
 
 
+    protected String getUnderlyingProtocol() {
+      return "http";
+    }
+
     @SuppressWarnings("unchecked")
     @SuppressWarnings("unchecked")
     @Override
     @Override
     public long renew(Token<?> token, 
     public long renew(Token<?> token, 
                       Configuration conf) throws IOException {
                       Configuration conf) throws IOException {
       // update the kerberos credentials, if they are coming from a keytab
       // update the kerberos credentials, if they are coming from a keytab
       UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
       UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
-      // use http to renew the token
       InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
       InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
       return 
       return 
         DelegationTokenFetcher.renewDelegationToken
         DelegationTokenFetcher.renewDelegationToken
-        (DFSUtil.createUri("http", serviceAddr).toString(),
+        (DFSUtil.createUri(getUnderlyingProtocol(), serviceAddr).toString(),
          (Token<DelegationTokenIdentifier>) token);
          (Token<DelegationTokenIdentifier>) token);
     }
     }
 
 
@@ -723,10 +730,9 @@ public class HftpFileSystem extends FileSystem
                        Configuration conf) throws IOException {
                        Configuration conf) throws IOException {
       // update the kerberos credentials, if they are coming from a keytab
       // update the kerberos credentials, if they are coming from a keytab
       UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
       UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
-      // use http to cancel the token
       InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
       InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
       DelegationTokenFetcher.cancelDelegationToken
       DelegationTokenFetcher.cancelDelegationToken
-        (DFSUtil.createUri("http", serviceAddr).toString(),
+        (DFSUtil.createUri(getUnderlyingProtocol(), serviceAddr).toString(),
          (Token<DelegationTokenIdentifier>) token);
          (Token<DelegationTokenIdentifier>) token);
     }    
     }    
   }
   }

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

@@ -68,6 +68,14 @@ public class HsftpFileSystem extends HftpFileSystem {
     return "hsftp";
     return "hsftp";
   }
   }
 
 
+  /**
+   * Return the underlying protocol that is used to talk to the namenode.
+   */
+  @Override
+  protected String getUnderlyingProtocol() {
+    return "https";
+  }
+
   @Override
   @Override
   public void initialize(URI name, Configuration conf) throws IOException {
   public void initialize(URI name, Configuration conf) throws IOException {
     super.initialize(name, conf);
     super.initialize(name, conf);
@@ -134,24 +142,15 @@ public class HsftpFileSystem extends HftpFileSystem {
 
 
   @Override
   @Override
   protected int getDefaultPort() {
   protected int getDefaultPort() {
-    return getDefaultSecurePort();
+    return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY,
+                            DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT);
   }
   }
 
 
-  @Override
-  protected InetSocketAddress getNamenodeSecureAddr(URI uri) {
-    return getNamenodeAddr(uri);
-  }
-
-  @Override
-  protected URI getNamenodeUri(URI uri) {
-    return getNamenodeSecureUri(uri);
-  }
-  
   @Override
   @Override
   protected HttpURLConnection openConnection(String path, String query)
   protected HttpURLConnection openConnection(String path, String query)
       throws IOException {
       throws IOException {
     query = addDelegationTokenParam(query);
     query = addDelegationTokenParam(query);
-    final URL url = new URL("https", nnUri.getHost(), 
+    final URL url = new URL(getUnderlyingProtocol(), nnUri.getHost(),
         nnUri.getPort(), path + '?' + query);
         nnUri.getPort(), path + '?' + query);
     HttpsURLConnection conn;
     HttpsURLConnection conn;
     conn = (HttpsURLConnection)connectionFactory.openConnection(url);
     conn = (HttpsURLConnection)connectionFactory.openConnection(url);

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java

@@ -158,8 +158,8 @@ public class NameNodeProxies {
    * Generate a dummy namenode proxy instance that utilizes our hacked
    * Generate a dummy namenode proxy instance that utilizes our hacked
    * {@link LossyRetryInvocationHandler}. Proxy instance generated using this
    * {@link LossyRetryInvocationHandler}. Proxy instance generated using this
    * method will proactively drop RPC responses. Currently this method only
    * method will proactively drop RPC responses. Currently this method only
-   * support HA setup. IllegalStateException will be thrown if the given
-   * configuration is not for HA.
+   * support HA setup. null will be returned if the given configuration is not 
+   * for HA.
    * 
    * 
    * @param config the configuration containing the required IPC
    * @param config the configuration containing the required IPC
    *        properties, client failover configurations, etc.
    *        properties, client failover configurations, etc.
@@ -168,7 +168,8 @@ public class NameNodeProxies {
    * @param xface the IPC interface which should be created
    * @param xface the IPC interface which should be created
    * @param numResponseToDrop The number of responses to drop for each RPC call
    * @param numResponseToDrop The number of responses to drop for each RPC call
    * @return an object containing both the proxy and the associated
    * @return an object containing both the proxy and the associated
-   *         delegation token service it corresponds to
+   *         delegation token service it corresponds to. Will return null of the
+   *         given configuration does not support HA.
    * @throws IOException if there is an error creating the proxy
    * @throws IOException if there is an error creating the proxy
    */
    */
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
@@ -204,8 +205,9 @@ public class NameNodeProxies {
       Text dtService = HAUtil.buildTokenServiceForLogicalUri(nameNodeUri);
       Text dtService = HAUtil.buildTokenServiceForLogicalUri(nameNodeUri);
       return new ProxyAndInfo<T>(proxy, dtService);
       return new ProxyAndInfo<T>(proxy, dtService);
     } else {
     } else {
-      throw new IllegalStateException("Currently creating proxy using " +
+      LOG.warn("Currently creating proxy using " +
       		"LossyRetryInvocationHandler requires NN HA setup");
       		"LossyRetryInvocationHandler requires NN HA setup");
+      return null;
     }
     }
   }
   }
 
 

+ 20 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java

@@ -43,6 +43,7 @@ public class DatanodeID implements Comparable<DatanodeID> {
   private String storageID;  // unique per cluster storageID
   private String storageID;  // unique per cluster storageID
   private int xferPort;      // data streaming port
   private int xferPort;      // data streaming port
   private int infoPort;      // info server port
   private int infoPort;      // info server port
+  private int infoSecurePort; // info server port
   private int ipcPort;       // IPC server port
   private int ipcPort;       // IPC server port
 
 
   public DatanodeID(DatanodeID from) {
   public DatanodeID(DatanodeID from) {
@@ -51,6 +52,7 @@ public class DatanodeID implements Comparable<DatanodeID> {
         from.getStorageID(),
         from.getStorageID(),
         from.getXferPort(),
         from.getXferPort(),
         from.getInfoPort(),
         from.getInfoPort(),
+        from.getInfoSecurePort(),
         from.getIpcPort());
         from.getIpcPort());
     this.peerHostName = from.getPeerHostName();
     this.peerHostName = from.getPeerHostName();
   }
   }
@@ -65,12 +67,13 @@ public class DatanodeID implements Comparable<DatanodeID> {
    * @param ipcPort ipc server port
    * @param ipcPort ipc server port
    */
    */
   public DatanodeID(String ipAddr, String hostName, String storageID,
   public DatanodeID(String ipAddr, String hostName, String storageID,
-      int xferPort, int infoPort, int ipcPort) {
+      int xferPort, int infoPort, int infoSecurePort, int ipcPort) {
     this.ipAddr = ipAddr;
     this.ipAddr = ipAddr;
     this.hostName = hostName;
     this.hostName = hostName;
     this.storageID = storageID;
     this.storageID = storageID;
     this.xferPort = xferPort;
     this.xferPort = xferPort;
     this.infoPort = infoPort;
     this.infoPort = infoPort;
+    this.infoSecurePort = infoSecurePort;
     this.ipcPort = ipcPort;
     this.ipcPort = ipcPort;
   }
   }
   
   
@@ -128,6 +131,13 @@ public class DatanodeID implements Comparable<DatanodeID> {
     return ipAddr + ":" + infoPort;
     return ipAddr + ":" + infoPort;
   }
   }
 
 
+  /**
+   * @return IP:infoPort string
+   */
+  public String getInfoSecureAddr() {
+    return ipAddr + ":" + infoSecurePort;
+  }
+
   /**
   /**
    * @return hostname:xferPort
    * @return hostname:xferPort
    */
    */
@@ -179,6 +189,13 @@ public class DatanodeID implements Comparable<DatanodeID> {
     return infoPort;
     return infoPort;
   }
   }
 
 
+  /**
+   * @return infoSecurePort (the port at which the HTTPS server bound to)
+   */
+  public int getInfoSecurePort() {
+    return infoSecurePort;
+  }
+
   /**
   /**
    * @return ipcPort (the port at which the IPC server bound to)
    * @return ipcPort (the port at which the IPC server bound to)
    */
    */
@@ -218,13 +235,14 @@ public class DatanodeID implements Comparable<DatanodeID> {
     peerHostName = nodeReg.getPeerHostName();
     peerHostName = nodeReg.getPeerHostName();
     xferPort = nodeReg.getXferPort();
     xferPort = nodeReg.getXferPort();
     infoPort = nodeReg.getInfoPort();
     infoPort = nodeReg.getInfoPort();
+    infoSecurePort = nodeReg.getInfoSecurePort();
     ipcPort = nodeReg.getIpcPort();
     ipcPort = nodeReg.getIpcPort();
   }
   }
     
     
   /**
   /**
    * Compare based on data transfer address.
    * Compare based on data transfer address.
    *
    *
-   * @param that
+   * @param that datanode to compare with
    * @return as specified by Comparable
    * @return as specified by Comparable
    */
    */
   @Override
   @Override

+ 13 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -17,10 +17,6 @@
  */
  */
 package org.apache.hadoop.hdfs.protocol;
 package org.apache.hadoop.hdfs.protocol;
 
 
-import static org.apache.hadoop.hdfs.DFSUtil.percent2String;
-
-import java.util.Date;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -32,6 +28,10 @@ import org.apache.hadoop.net.NodeBase;
 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 java.util.Date;
+
+import static org.apache.hadoop.hdfs.DFSUtil.percent2String;
+
 /** 
 /** 
  * This class extends the primary identifier of a Datanode with ephemeral
  * This class extends the primary identifier of a Datanode with ephemeral
  * state, eg usage information, current administrative state, and the
  * state, eg usage information, current administrative state, and the
@@ -115,20 +115,23 @@ public class DatanodeInfo extends DatanodeID implements Node {
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
       final long lastUpdate, final int xceiverCount,
       final long lastUpdate, final int xceiverCount,
       final AdminStates adminState) {
       final AdminStates adminState) {
-    this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getStorageID(), nodeID.getXferPort(),
-        nodeID.getInfoPort(), nodeID.getIpcPort(), capacity, dfsUsed, remaining,
-        blockPoolUsed, cacheCapacity, cacheUsed, lastUpdate, xceiverCount,
-        location, adminState);
+    this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getStorageID(),
+        nodeID.getXferPort(), nodeID.getInfoPort(), nodeID.getInfoSecurePort(),
+        nodeID.getIpcPort(), capacity, dfsUsed, remaining, blockPoolUsed,
+        cacheCapacity, cacheUsed, lastUpdate, xceiverCount, location,
+        adminState);
   }
   }
 
 
   /** Constructor */
   /** Constructor */
   public DatanodeInfo(final String ipAddr, final String hostName,
   public DatanodeInfo(final String ipAddr, final String hostName,
-      final String storageID, final int xferPort, final int infoPort, final int ipcPort,
+      final String storageID, final int xferPort, final int infoPort,
+      final int infoSecurePort, final int ipcPort,
       final long capacity, final long dfsUsed, final long remaining,
       final long capacity, final long dfsUsed, final long remaining,
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
       final long lastUpdate, final int xceiverCount,
       final long lastUpdate, final int xceiverCount,
       final String networkLocation, final AdminStates adminState) {
       final String networkLocation, final AdminStates adminState) {
-    super(ipAddr, hostName, storageID, xferPort, infoPort, ipcPort);
+    super(ipAddr, hostName, storageID, xferPort, infoPort,
+            infoSecurePort, ipcPort);
     this.capacity = capacity;
     this.capacity = capacity;
     this.dfsUsed = dfsUsed;
     this.dfsUsed = dfsUsed;
     this.remaining = remaining;
     this.remaining = remaining;

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -228,7 +228,8 @@ public class PBHelper {
   // DatanodeId
   // DatanodeId
   public static DatanodeID convert(DatanodeIDProto dn) {
   public static DatanodeID convert(DatanodeIDProto dn) {
     return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getStorageID(),
     return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getStorageID(),
-        dn.getXferPort(), dn.getInfoPort(), dn.getIpcPort());
+        dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
+        .getInfoSecurePort() : 0, dn.getIpcPort());
   }
   }
 
 
   public static DatanodeIDProto convert(DatanodeID dn) {
   public static DatanodeIDProto convert(DatanodeID dn) {
@@ -238,6 +239,7 @@ public class PBHelper {
         .setStorageID(dn.getStorageID())
         .setStorageID(dn.getStorageID())
         .setXferPort(dn.getXferPort())
         .setXferPort(dn.getXferPort())
         .setInfoPort(dn.getInfoPort())
         .setInfoPort(dn.getInfoPort())
+        .setInfoSecurePort(dn.getInfoSecurePort())
         .setIpcPort(dn.getIpcPort()).build();
         .setIpcPort(dn.getIpcPort()).build();
   }
   }
 
 

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Co
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.RetriableException;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
@@ -115,6 +116,24 @@ public class DelegationTokenSecretManager
     return super.retrievePassword(identifier);
     return super.retrievePassword(identifier);
   }
   }
   
   
+  @Override
+  public byte[] retriableRetrievePassword(DelegationTokenIdentifier identifier)
+      throws InvalidToken, StandbyException, RetriableException, IOException {
+    namesystem.checkOperation(OperationCategory.READ);
+    try {
+      return super.retrievePassword(identifier);
+    } catch (InvalidToken it) {
+      if (namesystem.inTransitionToActive()) {
+        // if the namesystem is currently in the middle of transition to 
+        // active state, let client retry since the corresponding editlog may 
+        // have not been applied yet
+        throw new RetriableException(it);
+      } else {
+        throw it;
+      }
+    }
+  }
+  
   /**
   /**
    * Returns expiry time of a token given its identifier.
    * Returns expiry time of a token given its identifier.
    * 
    * 

+ 27 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -506,7 +506,7 @@ public class Balancer {
     final DatanodeInfo datanode;
     final DatanodeInfo datanode;
     final double utilization;
     final double utilization;
     final long maxSize2Move;
     final long maxSize2Move;
-    protected long scheduledSize = 0L;
+    private long scheduledSize = 0L;
     //  blocks being moved but not confirmed yet
     //  blocks being moved but not confirmed yet
     private List<PendingBlockMove> pendingBlocks = 
     private List<PendingBlockMove> pendingBlocks = 
       new ArrayList<PendingBlockMove>(MAX_NUM_CONCURRENT_MOVES); 
       new ArrayList<PendingBlockMove>(MAX_NUM_CONCURRENT_MOVES); 
@@ -555,20 +555,35 @@ public class Balancer {
     }
     }
     
     
     /** Decide if still need to move more bytes */
     /** Decide if still need to move more bytes */
-    protected boolean hasSpaceForScheduling() {
+    protected synchronized boolean hasSpaceForScheduling() {
       return scheduledSize<maxSize2Move;
       return scheduledSize<maxSize2Move;
     }
     }
 
 
     /** Return the total number of bytes that need to be moved */
     /** Return the total number of bytes that need to be moved */
-    protected long availableSizeToMove() {
+    protected synchronized long availableSizeToMove() {
       return maxSize2Move-scheduledSize;
       return maxSize2Move-scheduledSize;
     }
     }
     
     
-    /* increment scheduled size */
-    protected void incScheduledSize(long size) {
+    /** increment scheduled size */
+    protected synchronized void incScheduledSize(long size) {
       scheduledSize += size;
       scheduledSize += size;
     }
     }
     
     
+    /** decrement scheduled size */
+    protected synchronized void decScheduledSize(long size) {
+      scheduledSize -= size;
+    }
+    
+    /** get scheduled size */
+    protected synchronized long getScheduledSize(){
+      return scheduledSize;
+    }
+    
+    /** get scheduled size */
+    protected synchronized void setScheduledSize(long size){
+      scheduledSize = size;
+    }
+    
     /* Check if the node can schedule more blocks to move */
     /* Check if the node can schedule more blocks to move */
     synchronized private boolean isPendingQNotFull() {
     synchronized private boolean isPendingQNotFull() {
       if ( pendingBlocks.size() < MAX_NUM_CONCURRENT_MOVES ) {
       if ( pendingBlocks.size() < MAX_NUM_CONCURRENT_MOVES ) {
@@ -702,8 +717,8 @@ public class Balancer {
           pendingBlock.source = this;
           pendingBlock.source = this;
           pendingBlock.target = target;
           pendingBlock.target = target;
           if ( pendingBlock.chooseBlockAndProxy() ) {
           if ( pendingBlock.chooseBlockAndProxy() ) {
-            long blockSize = pendingBlock.block.getNumBytes(); 
-            scheduledSize -= blockSize;
+            long blockSize = pendingBlock.block.getNumBytes();
+            decScheduledSize(blockSize);
             task.size -= blockSize;
             task.size -= blockSize;
             if (task.size == 0) {
             if (task.size == 0) {
               tasks.remove();
               tasks.remove();
@@ -747,10 +762,11 @@ public class Balancer {
     private static final long MAX_ITERATION_TIME = 20*60*1000L; //20 mins
     private static final long MAX_ITERATION_TIME = 20*60*1000L; //20 mins
     private void dispatchBlocks() {
     private void dispatchBlocks() {
       long startTime = Time.now();
       long startTime = Time.now();
+      long scheduledSize = getScheduledSize();
       this.blocksToReceive = 2*scheduledSize;
       this.blocksToReceive = 2*scheduledSize;
       boolean isTimeUp = false;
       boolean isTimeUp = false;
       int noPendingBlockIteration = 0;
       int noPendingBlockIteration = 0;
-      while(!isTimeUp && scheduledSize>0 &&
+      while(!isTimeUp && getScheduledSize()>0 &&
           (!srcBlockList.isEmpty() || blocksToReceive>0)) {
           (!srcBlockList.isEmpty() || blocksToReceive>0)) {
         PendingBlockMove pendingBlock = chooseNextBlockToMove();
         PendingBlockMove pendingBlock = chooseNextBlockToMove();
         if (pendingBlock != null) {
         if (pendingBlock != null) {
@@ -779,7 +795,7 @@ public class Balancer {
           // in case no blocks can be moved for source node's task,
           // in case no blocks can be moved for source node's task,
           // jump out of while-loop after 5 iterations.
           // jump out of while-loop after 5 iterations.
           if (noPendingBlockIteration >= MAX_NO_PENDING_BLOCK_ITERATIONS) {
           if (noPendingBlockIteration >= MAX_NO_PENDING_BLOCK_ITERATIONS) {
-            scheduledSize = 0;
+            setScheduledSize(0);
           }
           }
         }
         }
         
         
@@ -992,7 +1008,7 @@ public class Balancer {
 
 
     long bytesToMove = 0L;
     long bytesToMove = 0L;
     for (Source src : sources) {
     for (Source src : sources) {
-      bytesToMove += src.scheduledSize;
+      bytesToMove += src.getScheduledSize();
     }
     }
     return bytesToMove;
     return bytesToMove;
   }
   }
@@ -1093,7 +1109,7 @@ public class Balancer {
       bytesMoved += bytes;
       bytesMoved += bytes;
     }
     }
 
 
-    private long get() {
+    private synchronized long get() {
       return bytesMoved;
       return bytesMoved;
     }
     }
   };
   };

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

@@ -26,6 +26,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
@@ -74,6 +75,7 @@ import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.LightWeightGSet;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
@@ -89,9 +91,6 @@ public class BlockManager {
   static final Log LOG = LogFactory.getLog(BlockManager.class);
   static final Log LOG = LogFactory.getLog(BlockManager.class);
   public static final Log blockLog = NameNode.blockStateChangeLog;
   public static final Log blockLog = NameNode.blockStateChangeLog;
 
 
-  /** Default load factor of map */
-  public static final float DEFAULT_MAP_LOAD_FACTOR = 0.75f;
-
   private static final String QUEUE_REASON_CORRUPT_STATE =
   private static final String QUEUE_REASON_CORRUPT_STATE =
     "it has the wrong state or generation stamp";
     "it has the wrong state or generation stamp";
 
 
@@ -243,7 +242,8 @@ public class BlockManager {
     invalidateBlocks = new InvalidateBlocks(datanodeManager);
     invalidateBlocks = new InvalidateBlocks(datanodeManager);
 
 
     // Compute the map capacity by allocating 2% of total memory
     // Compute the map capacity by allocating 2% of total memory
-    blocksMap = new BlocksMap(DEFAULT_MAP_LOAD_FACTOR);
+    blocksMap = new BlocksMap(
+        LightWeightGSet.computeCapacity(2.0, "BlocksMap"));
     blockplacement = BlockPlacementPolicy.getInstance(
     blockplacement = BlockPlacementPolicy.getInstance(
         conf, stats, datanodeManager.getNetworkTopology());
         conf, stats, datanodeManager.getNetworkTopology());
     pendingReplications = new PendingReplicationBlocks(conf.getInt(
     pendingReplications = new PendingReplicationBlocks(conf.getInt(
@@ -1256,22 +1256,19 @@ public class BlockManager {
       namesystem.writeUnlock();
       namesystem.writeUnlock();
     }
     }
 
 
-    HashMap<Node, Node> excludedNodes
-        = new HashMap<Node, Node>();
+    final Set<Node> excludedNodes = new HashSet<Node>();
     for(ReplicationWork rw : work){
     for(ReplicationWork rw : work){
       // Exclude all of the containing nodes from being targets.
       // Exclude all of the containing nodes from being targets.
       // This list includes decommissioning or corrupt nodes.
       // This list includes decommissioning or corrupt nodes.
       excludedNodes.clear();
       excludedNodes.clear();
       for (DatanodeDescriptor dn : rw.containingNodes) {
       for (DatanodeDescriptor dn : rw.containingNodes) {
-        excludedNodes.put(dn, dn);
+        excludedNodes.add(dn);
       }
       }
 
 
       // choose replication targets: NOT HOLDING THE GLOBAL LOCK
       // choose replication targets: NOT HOLDING THE GLOBAL LOCK
       // It is costly to extract the filename for which chooseTargets is called,
       // It is costly to extract the filename for which chooseTargets is called,
       // so for now we pass in the block collection itself.
       // so for now we pass in the block collection itself.
-      rw.targets = blockplacement.chooseTarget(rw.bc,
-          rw.additionalReplRequired, rw.srcNode, rw.liveReplicaNodes,
-          excludedNodes, rw.block.getNumBytes());
+      rw.chooseTargets(blockplacement, excludedNodes);
     }
     }
 
 
     namesystem.writeLock();
     namesystem.writeLock();
@@ -1378,12 +1375,12 @@ public class BlockManager {
    * 
    * 
    * @throws IOException
    * @throws IOException
    *           if the number of targets < minimum replication.
    *           if the number of targets < minimum replication.
-   * @see BlockPlacementPolicy#chooseTarget(String, int, DatanodeDescriptor,
-   *      List, boolean, HashMap, long)
+   * @see BlockPlacementPolicy#chooseTarget(String, int, Node,
+   *      List, boolean, Set, long)
    */
    */
   public DatanodeDescriptor[] chooseTarget(final String src,
   public DatanodeDescriptor[] chooseTarget(final String src,
       final int numOfReplicas, final DatanodeDescriptor client,
       final int numOfReplicas, final DatanodeDescriptor client,
-      final HashMap<Node, Node> excludedNodes,
+      final Set<Node> excludedNodes,
       final long blocksize, List<String> favoredNodes) throws IOException {
       final long blocksize, List<String> favoredNodes) throws IOException {
     List<DatanodeDescriptor> favoredDatanodeDescriptors = 
     List<DatanodeDescriptor> favoredDatanodeDescriptors = 
         getDatanodeDescriptors(favoredNodes);
         getDatanodeDescriptors(favoredNodes);
@@ -1783,6 +1780,14 @@ public class BlockManager {
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
         ((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
         ((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
             node, iblk, reportedState);
             node, iblk, reportedState);
+        // OpenFileBlocks only inside snapshots also will be added to safemode
+        // threshold. So we need to update such blocks to safemode
+        // refer HDFS-5283
+        BlockInfoUnderConstruction blockUC = (BlockInfoUnderConstruction) storedBlock;
+        if (namesystem.isInSnapshot(blockUC)) {
+          int numOfReplicas = blockUC.getNumExpectedLocations();
+          namesystem.incrementSafeBlockCount(numOfReplicas);
+        }
         //and fall through to next clause
         //and fall through to next clause
       }      
       }      
       //add replica if appropriate
       //add replica if appropriate
@@ -3256,6 +3261,13 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       this.priority = priority;
       this.priority = priority;
       this.targets = null;
       this.targets = null;
     }
     }
+    
+    private void chooseTargets(BlockPlacementPolicy blockplacement,
+        Set<Node> excludedNodes) {
+      targets = blockplacement.chooseTarget(bc.getName(),
+          additionalReplRequired, srcNode, liveReplicaNodes, false,
+          excludedNodes, block.getNumBytes());
+    }
   }
   }
 
 
   /**
   /**

+ 15 - 59
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java

@@ -19,14 +19,15 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.Set;
 
 
 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.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -51,25 +52,6 @@ public abstract class BlockPlacementPolicy {
     }
     }
   }
   }
     
     
-  /**
-   * choose <i>numOfReplicas</i> data nodes for <i>writer</i> 
-   * to re-replicate a block with size <i>blocksize</i> 
-   * If not, return as many as we can.
-   * 
-   * @param srcPath the file to which this chooseTargets is being invoked. 
-   * @param numOfReplicas additional number of replicas wanted.
-   * @param writer the writer's machine, null if not in the cluster.
-   * @param chosenNodes datanodes that have been chosen as targets.
-   * @param blocksize size of the data to be written.
-   * @return array of DatanodeDescriptor instances chosen as target 
-   * and sorted as a pipeline.
-   */
-  abstract DatanodeDescriptor[] chooseTarget(String srcPath,
-                                             int numOfReplicas,
-                                             DatanodeDescriptor writer,
-                                             List<DatanodeDescriptor> chosenNodes,
-                                             long blocksize);
-
   /**
   /**
    * choose <i>numOfReplicas</i> data nodes for <i>writer</i> 
    * choose <i>numOfReplicas</i> data nodes for <i>writer</i> 
    * to re-replicate a block with size <i>blocksize</i> 
    * to re-replicate a block with size <i>blocksize</i> 
@@ -87,48 +69,22 @@ public abstract class BlockPlacementPolicy {
    */
    */
   public abstract DatanodeDescriptor[] chooseTarget(String srcPath,
   public abstract DatanodeDescriptor[] chooseTarget(String srcPath,
                                              int numOfReplicas,
                                              int numOfReplicas,
-                                             DatanodeDescriptor writer,
+                                             Node writer,
                                              List<DatanodeDescriptor> chosenNodes,
                                              List<DatanodeDescriptor> chosenNodes,
                                              boolean returnChosenNodes,
                                              boolean returnChosenNodes,
-                                             HashMap<Node, Node> excludedNodes,
+                                             Set<Node> excludedNodes,
                                              long blocksize);
                                              long blocksize);
-
-  /**
-   * choose <i>numOfReplicas</i> data nodes for <i>writer</i>
-   * If not, return as many as we can.
-   * The base implemenatation extracts the pathname of the file from the
-   * specified srcBC, but this could be a costly operation depending on the
-   * file system implementation. Concrete implementations of this class should
-   * override this method to avoid this overhead.
-   * 
-   * @param srcBC block collection of file for which chooseTarget is invoked.
-   * @param numOfReplicas additional number of replicas wanted.
-   * @param writer the writer's machine, null if not in the cluster.
-   * @param chosenNodes datanodes that have been chosen as targets.
-   * @param blocksize size of the data to be written.
-   * @return array of DatanodeDescriptor instances chosen as target 
-   * and sorted as a pipeline.
-   */
-  DatanodeDescriptor[] chooseTarget(BlockCollection srcBC,
-                                    int numOfReplicas,
-                                    DatanodeDescriptor writer,
-                                    List<DatanodeDescriptor> chosenNodes,
-                                    HashMap<Node, Node> excludedNodes,
-                                    long blocksize) {
-    return chooseTarget(srcBC.getName(), numOfReplicas, writer,
-                        chosenNodes, false, excludedNodes, blocksize);
-  }
   
   
   /**
   /**
-   * Same as {@link #chooseTarget(String, int, DatanodeDescriptor, List, boolean, 
-   * HashMap, long)} with added parameter {@code favoredDatanodes}
+   * Same as {@link #chooseTarget(String, int, Node, List, boolean, 
+   * Set, long)} with added parameter {@code favoredDatanodes}
    * @param favoredNodes datanodes that should be favored as targets. This
    * @param favoredNodes datanodes that should be favored as targets. This
    *          is only a hint and due to cluster state, namenode may not be 
    *          is only a hint and due to cluster state, namenode may not be 
    *          able to place the blocks on these datanodes.
    *          able to place the blocks on these datanodes.
    */
    */
   DatanodeDescriptor[] chooseTarget(String src,
   DatanodeDescriptor[] chooseTarget(String src,
-      int numOfReplicas, DatanodeDescriptor writer,
-      HashMap<Node, Node> excludedNodes,
+      int numOfReplicas, Node writer,
+      Set<Node> excludedNodes,
       long blocksize, List<DatanodeDescriptor> favoredNodes) {
       long blocksize, List<DatanodeDescriptor> favoredNodes) {
     // This class does not provide the functionality of placing
     // This class does not provide the functionality of placing
     // a block in favored datanodes. The implementations of this class
     // a block in favored datanodes. The implementations of this class
@@ -183,7 +139,7 @@ public abstract class BlockPlacementPolicy {
     
     
   /**
   /**
    * Get an instance of the configured Block Placement Policy based on the
    * Get an instance of the configured Block Placement Policy based on the
-   * value of the configuration paramater dfs.block.replicator.classname.
+   * the configuration property {@link DFS_BLOCK_REPLICATOR_CLASSNAME_KEY}.
    * 
    * 
    * @param conf the configuration to be used
    * @param conf the configuration to be used
    * @param stats an object that is used to retrieve the load on the cluster
    * @param stats an object that is used to retrieve the load on the cluster
@@ -193,12 +149,12 @@ public abstract class BlockPlacementPolicy {
   public static BlockPlacementPolicy getInstance(Configuration conf, 
   public static BlockPlacementPolicy getInstance(Configuration conf, 
                                                  FSClusterStats stats,
                                                  FSClusterStats stats,
                                                  NetworkTopology clusterMap) {
                                                  NetworkTopology clusterMap) {
-    Class<? extends BlockPlacementPolicy> replicatorClass =
-                      conf.getClass("dfs.block.replicator.classname",
-                                    BlockPlacementPolicyDefault.class,
-                                    BlockPlacementPolicy.class);
-    BlockPlacementPolicy replicator = (BlockPlacementPolicy) ReflectionUtils.newInstance(
-                                                             replicatorClass, conf);
+    final Class<? extends BlockPlacementPolicy> replicatorClass = conf.getClass(
+        DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
+        DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT,
+        BlockPlacementPolicy.class);
+    final BlockPlacementPolicy replicator = ReflectionUtils.newInstance(
+        replicatorClass, conf);
     replicator.initialize(conf, stats, clusterMap);
     replicator.initialize(conf, stats, clusterMap);
     return replicator;
     return replicator;
   }
   }

+ 150 - 207
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -21,8 +21,7 @@ import static org.apache.hadoop.util.Time.now;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
+import java.util.HashSet;
 import java.util.List;
 import java.util.List;
 import java.util.Set;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.TreeSet;
@@ -57,6 +56,14 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     "For more information, please enable DEBUG log level on "
     "For more information, please enable DEBUG log level on "
     + BlockPlacementPolicy.class.getName();
     + BlockPlacementPolicy.class.getName();
 
 
+  private static final ThreadLocal<StringBuilder> debugLoggingBuilder
+      = new ThreadLocal<StringBuilder>() {
+        @Override
+        protected StringBuilder initialValue() {
+          return new StringBuilder();
+        }
+      };
+
   protected boolean considerLoad; 
   protected boolean considerLoad; 
   private boolean preferLocalNode = true;
   private boolean preferLocalNode = true;
   protected NetworkTopology clusterMap;
   protected NetworkTopology clusterMap;
@@ -95,40 +102,25 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT);
         DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT);
   }
   }
 
 
-  protected ThreadLocal<StringBuilder> threadLocalBuilder =
-    new ThreadLocal<StringBuilder>() {
-    @Override
-    protected StringBuilder initialValue() {
-      return new StringBuilder();
-    }
-  };
-
-  @Override
-  public DatanodeDescriptor[] chooseTarget(String srcPath,
-                                    int numOfReplicas,
-                                    DatanodeDescriptor writer,
-                                    List<DatanodeDescriptor> chosenNodes,
-                                    long blocksize) {
-    return chooseTarget(numOfReplicas, writer, chosenNodes, false,
-        null, blocksize);
-  }
-
   @Override
   @Override
   public DatanodeDescriptor[] chooseTarget(String srcPath,
   public DatanodeDescriptor[] chooseTarget(String srcPath,
                                     int numOfReplicas,
                                     int numOfReplicas,
-                                    DatanodeDescriptor writer,
+                                    Node writer,
                                     List<DatanodeDescriptor> chosenNodes,
                                     List<DatanodeDescriptor> chosenNodes,
                                     boolean returnChosenNodes,
                                     boolean returnChosenNodes,
-                                    HashMap<Node, Node> excludedNodes,
+                                    Set<Node> excludedNodes,
                                     long blocksize) {
                                     long blocksize) {
     return chooseTarget(numOfReplicas, writer, chosenNodes, returnChosenNodes,
     return chooseTarget(numOfReplicas, writer, chosenNodes, returnChosenNodes,
         excludedNodes, blocksize);
         excludedNodes, blocksize);
   }
   }
 
 
   @Override
   @Override
-  DatanodeDescriptor[] chooseTarget(String src, int numOfReplicas,
-      DatanodeDescriptor writer, HashMap<Node, Node> excludedNodes,
-      long blocksize, List<DatanodeDescriptor> favoredNodes) {
+  DatanodeDescriptor[] chooseTarget(String src,
+      int numOfReplicas,
+      Node writer,
+      Set<Node> excludedNodes,
+      long blocksize,
+      List<DatanodeDescriptor> favoredNodes) {
     try {
     try {
       if (favoredNodes == null || favoredNodes.size() == 0) {
       if (favoredNodes == null || favoredNodes.size() == 0) {
         // Favored nodes not specified, fall back to regular block placement.
         // Favored nodes not specified, fall back to regular block placement.
@@ -137,8 +129,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
             excludedNodes, blocksize);
             excludedNodes, blocksize);
       }
       }
 
 
-      HashMap<Node, Node> favoriteAndExcludedNodes = excludedNodes == null ?
-          new HashMap<Node, Node>() : new HashMap<Node, Node>(excludedNodes);
+      Set<Node> favoriteAndExcludedNodes = excludedNodes == null ?
+          new HashSet<Node>() : new HashSet<Node>(excludedNodes);
 
 
       // Choose favored nodes
       // Choose favored nodes
       List<DatanodeDescriptor> results = new ArrayList<DatanodeDescriptor>();
       List<DatanodeDescriptor> results = new ArrayList<DatanodeDescriptor>();
@@ -157,10 +149,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
               + " with favored node " + favoredNode); 
               + " with favored node " + favoredNode); 
           continue;
           continue;
         }
         }
-        favoriteAndExcludedNodes.put(target, target);
+        favoriteAndExcludedNodes.add(target);
       }
       }
 
 
-      if (results.size() < numOfReplicas) {        
+      if (results.size() < numOfReplicas) {
         // Not enough favored nodes, choose other nodes.
         // Not enough favored nodes, choose other nodes.
         numOfReplicas -= results.size();
         numOfReplicas -= results.size();
         DatanodeDescriptor[] remainingTargets = 
         DatanodeDescriptor[] remainingTargets = 
@@ -181,18 +173,18 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   }
   }
 
 
   /** This is the implementation. */
   /** This is the implementation. */
-  DatanodeDescriptor[] chooseTarget(int numOfReplicas,
-                                    DatanodeDescriptor writer,
+  private DatanodeDescriptor[] chooseTarget(int numOfReplicas,
+                                    Node writer,
                                     List<DatanodeDescriptor> chosenNodes,
                                     List<DatanodeDescriptor> chosenNodes,
                                     boolean returnChosenNodes,
                                     boolean returnChosenNodes,
-                                    HashMap<Node, Node> excludedNodes,
+                                    Set<Node> excludedNodes,
                                     long blocksize) {
                                     long blocksize) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
-      return new DatanodeDescriptor[0];
+      return DatanodeDescriptor.EMPTY_ARRAY;
     }
     }
       
       
     if (excludedNodes == null) {
     if (excludedNodes == null) {
-      excludedNodes = new HashMap<Node, Node>();
+      excludedNodes = new HashSet<Node>();
     }
     }
      
      
     int[] result = getMaxNodesPerRack(chosenNodes, numOfReplicas);
     int[] result = getMaxNodesPerRack(chosenNodes, numOfReplicas);
@@ -204,16 +196,15 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     for (DatanodeDescriptor node:chosenNodes) {
     for (DatanodeDescriptor node:chosenNodes) {
       // add localMachine and related nodes to excludedNodes
       // add localMachine and related nodes to excludedNodes
       addToExcludedNodes(node, excludedNodes);
       addToExcludedNodes(node, excludedNodes);
-      adjustExcludedNodes(excludedNodes, node);
     }
     }
       
       
     if (!clusterMap.contains(writer)) {
     if (!clusterMap.contains(writer)) {
-      writer=null;
+      writer = null;
     }
     }
       
       
     boolean avoidStaleNodes = (stats != null
     boolean avoidStaleNodes = (stats != null
         && stats.isAvoidingStaleDataNodesForWrite());
         && stats.isAvoidingStaleDataNodesForWrite());
-    DatanodeDescriptor localNode = chooseTarget(numOfReplicas, writer,
+    Node localNode = chooseTarget(numOfReplicas, writer,
         excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes);
         excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes);
     if (!returnChosenNodes) {  
     if (!returnChosenNodes) {  
       results.removeAll(chosenNodes);
       results.removeAll(chosenNodes);
@@ -236,10 +227,20 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     return new int[] {numOfReplicas, maxNodesPerRack};
     return new int[] {numOfReplicas, maxNodesPerRack};
   }
   }
     
     
-  /* choose <i>numOfReplicas</i> from all data nodes */
-  private DatanodeDescriptor chooseTarget(int numOfReplicas,
-                                          DatanodeDescriptor writer,
-                                          HashMap<Node, Node> excludedNodes,
+  /**
+   * choose <i>numOfReplicas</i> from all data nodes
+   * @param numOfReplicas additional number of replicas wanted
+   * @param writer the writer's machine, could be a non-DatanodeDescriptor node
+   * @param excludedNodes datanodes that should not be considered as targets
+   * @param blocksize size of the data to be written
+   * @param maxNodesPerRack max nodes allowed per rack
+   * @param results the target nodes already chosen
+   * @param avoidStaleNodes avoid stale nodes in replica choosing
+   * @return local node of writer (not chosen node)
+   */
+  private Node chooseTarget(int numOfReplicas,
+                                          Node writer,
+                                          Set<Node> excludedNodes,
                                           long blocksize,
                                           long blocksize,
                                           int maxNodesPerRack,
                                           int maxNodesPerRack,
                                           List<DatanodeDescriptor> results,
                                           List<DatanodeDescriptor> results,
@@ -251,13 +252,13 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       
       
     int numOfResults = results.size();
     int numOfResults = results.size();
     boolean newBlock = (numOfResults==0);
     boolean newBlock = (numOfResults==0);
-    if (writer == null && !newBlock) {
+    if ((writer == null || !(writer instanceof DatanodeDescriptor)) && !newBlock) {
       writer = results.get(0);
       writer = results.get(0);
     }
     }
 
 
     // Keep a copy of original excludedNodes
     // Keep a copy of original excludedNodes
-    final HashMap<Node, Node> oldExcludedNodes = avoidStaleNodes ? 
-        new HashMap<Node, Node>(excludedNodes) : null;
+    final Set<Node> oldExcludedNodes = avoidStaleNodes ? 
+        new HashSet<Node>(excludedNodes) : null;
     try {
     try {
       if (numOfResults == 0) {
       if (numOfResults == 0) {
         writer = chooseLocalNode(writer, excludedNodes, blocksize,
         writer = chooseLocalNode(writer, excludedNodes, blocksize,
@@ -304,7 +305,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         // We need to additionally exclude the nodes that were added to the 
         // We need to additionally exclude the nodes that were added to the 
         // result list in the successful calls to choose*() above.
         // result list in the successful calls to choose*() above.
         for (Node node : results) {
         for (Node node : results) {
-          oldExcludedNodes.put(node, node);
+          oldExcludedNodes.add(node);
         }
         }
         // Set numOfReplicas, since it can get out of sync with the result list
         // Set numOfReplicas, since it can get out of sync with the result list
         // if the NotEnoughReplicasException was thrown in chooseRandom().
         // if the NotEnoughReplicasException was thrown in chooseRandom().
@@ -316,33 +317,30 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     return writer;
     return writer;
   }
   }
     
     
-  /* choose <i>localMachine</i> as the target.
+  /**
+   * Choose <i>localMachine</i> as the target.
    * if <i>localMachine</i> is not available, 
    * if <i>localMachine</i> is not available, 
    * choose a node on the same rack
    * choose a node on the same rack
    * @return the chosen node
    * @return the chosen node
    */
    */
-  protected DatanodeDescriptor chooseLocalNode(
-                                             DatanodeDescriptor localMachine,
-                                             HashMap<Node, Node> excludedNodes,
+  protected DatanodeDescriptor chooseLocalNode(Node localMachine,
+                                             Set<Node> excludedNodes,
                                              long blocksize,
                                              long blocksize,
                                              int maxNodesPerRack,
                                              int maxNodesPerRack,
                                              List<DatanodeDescriptor> results,
                                              List<DatanodeDescriptor> results,
                                              boolean avoidStaleNodes)
                                              boolean avoidStaleNodes)
-    throws NotEnoughReplicasException {
+      throws NotEnoughReplicasException {
     // if no local machine, randomly choose one node
     // if no local machine, randomly choose one node
     if (localMachine == null)
     if (localMachine == null)
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
           maxNodesPerRack, results, avoidStaleNodes);
           maxNodesPerRack, results, avoidStaleNodes);
-    if (preferLocalNode) {
+    if (preferLocalNode && localMachine instanceof DatanodeDescriptor) {
+      DatanodeDescriptor localDatanode = (DatanodeDescriptor) localMachine;
       // otherwise try local machine first
       // otherwise try local machine first
-      Node oldNode = excludedNodes.put(localMachine, localMachine);
-      if (oldNode == null) { // was not in the excluded list
-        if (isGoodTarget(localMachine, blocksize, maxNodesPerRack, false,
-            results, avoidStaleNodes)) {
-          results.add(localMachine);
-          // add localMachine and related nodes to excludedNode
-          addToExcludedNodes(localMachine, excludedNodes);
-          return localMachine;
+      if (excludedNodes.add(localMachine)) { // was not in the excluded list
+        if (addIfIsGoodTarget(localDatanode, excludedNodes, blocksize,
+            maxNodesPerRack, false, results, avoidStaleNodes) >= 0) {
+          return localDatanode;
         }
         }
       } 
       } 
     }      
     }      
@@ -358,26 +356,25 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    * @return number of new excluded nodes
    * @return number of new excluded nodes
    */
    */
   protected int addToExcludedNodes(DatanodeDescriptor localMachine,
   protected int addToExcludedNodes(DatanodeDescriptor localMachine,
-      HashMap<Node, Node> excludedNodes) {
-    Node node = excludedNodes.put(localMachine, localMachine);
-    return node == null?1:0;
+      Set<Node> excludedNodes) {
+    return excludedNodes.add(localMachine) ? 1 : 0;
   }
   }
 
 
-  /* choose one node from the rack that <i>localMachine</i> is on.
+  /**
+   * Choose one node from the rack that <i>localMachine</i> is on.
    * if no such node is available, choose one node from the rack where
    * if no such node is available, choose one node from the rack where
    * a second replica is on.
    * a second replica is on.
    * if still no such node is available, choose a random node 
    * if still no such node is available, choose a random node 
    * in the cluster.
    * in the cluster.
    * @return the chosen node
    * @return the chosen node
    */
    */
-  protected DatanodeDescriptor chooseLocalRack(
-                                             DatanodeDescriptor localMachine,
-                                             HashMap<Node, Node> excludedNodes,
+  protected DatanodeDescriptor chooseLocalRack(Node localMachine,
+                                             Set<Node> excludedNodes,
                                              long blocksize,
                                              long blocksize,
                                              int maxNodesPerRack,
                                              int maxNodesPerRack,
                                              List<DatanodeDescriptor> results,
                                              List<DatanodeDescriptor> results,
                                              boolean avoidStaleNodes)
                                              boolean avoidStaleNodes)
-    throws NotEnoughReplicasException {
+      throws NotEnoughReplicasException {
     // no local machine, so choose a random machine
     // no local machine, so choose a random machine
     if (localMachine == null) {
     if (localMachine == null) {
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
@@ -391,9 +388,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     } catch (NotEnoughReplicasException e1) {
     } catch (NotEnoughReplicasException e1) {
       // find the second replica
       // find the second replica
       DatanodeDescriptor newLocal=null;
       DatanodeDescriptor newLocal=null;
-      for(Iterator<DatanodeDescriptor> iter=results.iterator();
-          iter.hasNext();) {
-        DatanodeDescriptor nextNode = iter.next();
+      for(DatanodeDescriptor nextNode : results) {
         if (nextNode != localMachine) {
         if (nextNode != localMachine) {
           newLocal = nextNode;
           newLocal = nextNode;
           break;
           break;
@@ -416,7 +411,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     }
     }
   }
   }
     
     
-  /* choose <i>numOfReplicas</i> nodes from the racks 
+  /** 
+   * Choose <i>numOfReplicas</i> nodes from the racks 
    * that <i>localMachine</i> is NOT on.
    * that <i>localMachine</i> is NOT on.
    * if not enough nodes are available, choose the remaining ones 
    * if not enough nodes are available, choose the remaining ones 
    * from the local rack
    * from the local rack
@@ -424,12 +420,12 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     
     
   protected void chooseRemoteRack(int numOfReplicas,
   protected void chooseRemoteRack(int numOfReplicas,
                                 DatanodeDescriptor localMachine,
                                 DatanodeDescriptor localMachine,
-                                HashMap<Node, Node> excludedNodes,
+                                Set<Node> excludedNodes,
                                 long blocksize,
                                 long blocksize,
                                 int maxReplicasPerRack,
                                 int maxReplicasPerRack,
                                 List<DatanodeDescriptor> results,
                                 List<DatanodeDescriptor> results,
                                 boolean avoidStaleNodes)
                                 boolean avoidStaleNodes)
-    throws NotEnoughReplicasException {
+                                    throws NotEnoughReplicasException {
     int oldNumOfReplicas = results.size();
     int oldNumOfReplicas = results.size();
     // randomly choose one node from remote racks
     // randomly choose one node from remote racks
     try {
     try {
@@ -443,91 +439,58 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     }
     }
   }
   }
 
 
-  /* Randomly choose one target from <i>nodes</i>.
-   * @return the chosen node
+  /**
+   * Randomly choose one target from the given <i>scope</i>.
+   * @return the chosen node, if there is any.
    */
    */
-  protected DatanodeDescriptor chooseRandom(
-                                          String nodes,
-                                          HashMap<Node, Node> excludedNodes,
-                                          long blocksize,
-                                          int maxNodesPerRack,
-                                          List<DatanodeDescriptor> results,
-                                          boolean avoidStaleNodes) 
-    throws NotEnoughReplicasException {
-    int numOfAvailableNodes =
-      clusterMap.countNumOfAvailableNodes(nodes, excludedNodes.keySet());
-    StringBuilder builder = null;
-    if (LOG.isDebugEnabled()) {
-      builder = threadLocalBuilder.get();
-      builder.setLength(0);
-      builder.append("[");
-    }
-    boolean badTarget = false;
-    while(numOfAvailableNodes > 0) {
-      DatanodeDescriptor chosenNode = 
-        (DatanodeDescriptor)(clusterMap.chooseRandom(nodes));
-
-      Node oldNode = excludedNodes.put(chosenNode, chosenNode);
-      if (oldNode == null) { // chosenNode was not in the excluded list
-        numOfAvailableNodes--;
-        if (isGoodTarget(chosenNode, blocksize, 
-                maxNodesPerRack, results, avoidStaleNodes)) {
-          results.add(chosenNode);
-          // add chosenNode and related nodes to excludedNode
-          addToExcludedNodes(chosenNode, excludedNodes);
-          adjustExcludedNodes(excludedNodes, chosenNode);
-          return chosenNode;
-        } else {
-          badTarget = true;
-        }
-      }
-    }
-
-    String detail = enableDebugLogging;
-    if (LOG.isDebugEnabled()) {
-      if (badTarget && builder != null) {
-        detail = builder.append("]").toString();
-        builder.setLength(0);
-      } else detail = "";
-    }
-    throw new NotEnoughReplicasException(detail);
+  protected DatanodeDescriptor chooseRandom(String scope,
+      Set<Node> excludedNodes,
+      long blocksize,
+      int maxNodesPerRack,
+      List<DatanodeDescriptor> results,
+      boolean avoidStaleNodes)
+          throws NotEnoughReplicasException {
+    return chooseRandom(1, scope, excludedNodes, blocksize, maxNodesPerRack,
+        results, avoidStaleNodes);
   }
   }
-    
-  /* Randomly choose <i>numOfReplicas</i> targets from <i>nodes</i>.
+
+  /**
+   * Randomly choose <i>numOfReplicas</i> targets from the given <i>scope</i>.
+   * @return the first chosen node, if there is any.
    */
    */
-  protected void chooseRandom(int numOfReplicas,
-                            String nodes,
-                            HashMap<Node, Node> excludedNodes,
+  protected DatanodeDescriptor chooseRandom(int numOfReplicas,
+                            String scope,
+                            Set<Node> excludedNodes,
                             long blocksize,
                             long blocksize,
                             int maxNodesPerRack,
                             int maxNodesPerRack,
                             List<DatanodeDescriptor> results,
                             List<DatanodeDescriptor> results,
                             boolean avoidStaleNodes)
                             boolean avoidStaleNodes)
-    throws NotEnoughReplicasException {
+                                throws NotEnoughReplicasException {
       
       
-    int numOfAvailableNodes =
-      clusterMap.countNumOfAvailableNodes(nodes, excludedNodes.keySet());
+    int numOfAvailableNodes = clusterMap.countNumOfAvailableNodes(
+        scope, excludedNodes);
     StringBuilder builder = null;
     StringBuilder builder = null;
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
-      builder = threadLocalBuilder.get();
+      builder = debugLoggingBuilder.get();
       builder.setLength(0);
       builder.setLength(0);
       builder.append("[");
       builder.append("[");
     }
     }
     boolean badTarget = false;
     boolean badTarget = false;
+    DatanodeDescriptor firstChosen = null;
     while(numOfReplicas > 0 && numOfAvailableNodes > 0) {
     while(numOfReplicas > 0 && numOfAvailableNodes > 0) {
       DatanodeDescriptor chosenNode = 
       DatanodeDescriptor chosenNode = 
-        (DatanodeDescriptor)(clusterMap.chooseRandom(nodes));
-      Node oldNode = excludedNodes.put(chosenNode, chosenNode);
-      if (oldNode == null) {
+          (DatanodeDescriptor)clusterMap.chooseRandom(scope);
+      if (excludedNodes.add(chosenNode)) { //was not in the excluded list
         numOfAvailableNodes--;
         numOfAvailableNodes--;
 
 
-        if (isGoodTarget(chosenNode, blocksize, 
-              maxNodesPerRack, results, avoidStaleNodes)) {
+        int newExcludedNodes = addIfIsGoodTarget(chosenNode, excludedNodes,
+            blocksize, maxNodesPerRack, considerLoad, results, avoidStaleNodes);
+        if (newExcludedNodes >= 0) {
           numOfReplicas--;
           numOfReplicas--;
-          results.add(chosenNode);
-          // add chosenNode and related nodes to excludedNode
-          int newExcludedNodes = addToExcludedNodes(chosenNode, excludedNodes);
+          if (firstChosen == null) {
+            firstChosen = chosenNode;
+          }
           numOfAvailableNodes -= newExcludedNodes;
           numOfAvailableNodes -= newExcludedNodes;
-          adjustExcludedNodes(excludedNodes, chosenNode);
         } else {
         } else {
           badTarget = true;
           badTarget = true;
         }
         }
@@ -544,34 +507,44 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       }
       }
       throw new NotEnoughReplicasException(detail);
       throw new NotEnoughReplicasException(detail);
     }
     }
+    
+    return firstChosen;
   }
   }
-  
+
   /**
   /**
-   * After choosing a node to place replica, adjust excluded nodes accordingly.
-   * It should do nothing here as chosenNode is already put into exlcudeNodes, 
-   * but it can be overridden in subclass to put more related nodes into 
-   * excludedNodes.
-   * 
-   * @param excludedNodes
-   * @param chosenNode
+   * If the given node is a good target, add it to the result list and
+   * update the set of excluded nodes.
+   * @return -1 if the given is not a good target;
+   *         otherwise, return the number of nodes added to excludedNodes set.
    */
    */
-  protected void adjustExcludedNodes(HashMap<Node, Node> excludedNodes,
-      Node chosenNode) {
-    // do nothing here.
+  int addIfIsGoodTarget(DatanodeDescriptor node,
+      Set<Node> excludedNodes,
+      long blockSize,
+      int maxNodesPerRack,
+      boolean considerLoad,
+      List<DatanodeDescriptor> results,
+      boolean avoidStaleNodes) {
+    if (isGoodTarget(node, blockSize, maxNodesPerRack, considerLoad,
+        results, avoidStaleNodes)) {
+      results.add(node);
+      // add node and related nodes to excludedNode
+      return addToExcludedNodes(node, excludedNodes);
+    } else { 
+      return -1;
+    }
   }
   }
 
 
-  /* judge if a node is a good target.
-   * return true if <i>node</i> has enough space, 
-   * does not have too much load, and the rack does not have too many nodes
-   */
-  private boolean isGoodTarget(DatanodeDescriptor node,
-                               long blockSize, int maxTargetPerRack,
-                               List<DatanodeDescriptor> results, 
-                               boolean avoidStaleNodes) {
-    return isGoodTarget(node, blockSize, maxTargetPerRack, this.considerLoad,
-        results, avoidStaleNodes);
+  private static void logNodeIsNotChosen(DatanodeDescriptor node, String reason) {
+    if (LOG.isDebugEnabled()) {
+      // build the error message for later use.
+      debugLoggingBuilder.get()
+          .append(node).append(": ")
+          .append("Node ").append(NodeBase.getPath(node))
+          .append(" is not chosen because ")
+          .append(reason);
+    }
   }
   }
-  
+
   /**
   /**
    * Determine if a node is a good target. 
    * Determine if a node is a good target. 
    * 
    * 
@@ -588,28 +561,20 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    *         does not have too much load, 
    *         does not have too much load, 
    *         and the rack does not have too many nodes.
    *         and the rack does not have too many nodes.
    */
    */
-  protected boolean isGoodTarget(DatanodeDescriptor node,
+  private boolean isGoodTarget(DatanodeDescriptor node,
                                long blockSize, int maxTargetPerRack,
                                long blockSize, int maxTargetPerRack,
                                boolean considerLoad,
                                boolean considerLoad,
                                List<DatanodeDescriptor> results,                           
                                List<DatanodeDescriptor> results,                           
                                boolean avoidStaleNodes) {
                                boolean avoidStaleNodes) {
     // check if the node is (being) decommissed
     // check if the node is (being) decommissed
     if (node.isDecommissionInProgress() || node.isDecommissioned()) {
     if (node.isDecommissionInProgress() || node.isDecommissioned()) {
-      if(LOG.isDebugEnabled()) {
-        threadLocalBuilder.get().append(node.toString()).append(": ")
-          .append("Node ").append(NodeBase.getPath(node))
-          .append(" is not chosen because the node is (being) decommissioned ");
-      }
+      logNodeIsNotChosen(node, "the node is (being) decommissioned ");
       return false;
       return false;
     }
     }
 
 
     if (avoidStaleNodes) {
     if (avoidStaleNodes) {
       if (node.isStale(this.staleInterval)) {
       if (node.isStale(this.staleInterval)) {
-        if (LOG.isDebugEnabled()) {
-          threadLocalBuilder.get().append(node.toString()).append(": ")
-              .append("Node ").append(NodeBase.getPath(node))
-              .append(" is not chosen because the node is stale ");
-        }
+        logNodeIsNotChosen(node, "the node is stale ");
         return false;
         return false;
       }
       }
     }
     }
@@ -618,11 +583,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                      (node.getBlocksScheduled() * blockSize); 
                      (node.getBlocksScheduled() * blockSize); 
     // check the remaining capacity of the target machine
     // check the remaining capacity of the target machine
     if (blockSize* HdfsConstants.MIN_BLOCKS_FOR_WRITE>remaining) {
     if (blockSize* HdfsConstants.MIN_BLOCKS_FOR_WRITE>remaining) {
-      if(LOG.isDebugEnabled()) {
-        threadLocalBuilder.get().append(node.toString()).append(": ")
-          .append("Node ").append(NodeBase.getPath(node))
-          .append(" is not chosen because the node does not have enough space ");
-      }
+      logNodeIsNotChosen(node, "the node does not have enough space ");
       return false;
       return false;
     }
     }
       
       
@@ -634,11 +595,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         avgLoad = (double)stats.getTotalLoad()/size;
         avgLoad = (double)stats.getTotalLoad()/size;
       }
       }
       if (node.getXceiverCount() > (2.0 * avgLoad)) {
       if (node.getXceiverCount() > (2.0 * avgLoad)) {
-        if(LOG.isDebugEnabled()) {
-          threadLocalBuilder.get().append(node.toString()).append(": ")
-            .append("Node ").append(NodeBase.getPath(node))
-            .append(" is not chosen because the node is too busy ");
-        }
+        logNodeIsNotChosen(node, "the node is too busy ");
         return false;
         return false;
       }
       }
     }
     }
@@ -646,31 +603,25 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     // check if the target rack has chosen too many nodes
     // check if the target rack has chosen too many nodes
     String rackname = node.getNetworkLocation();
     String rackname = node.getNetworkLocation();
     int counter=1;
     int counter=1;
-    for(Iterator<DatanodeDescriptor> iter = results.iterator();
-        iter.hasNext();) {
-      Node result = iter.next();
+    for(Node result : results) {
       if (rackname.equals(result.getNetworkLocation())) {
       if (rackname.equals(result.getNetworkLocation())) {
         counter++;
         counter++;
       }
       }
     }
     }
     if (counter>maxTargetPerRack) {
     if (counter>maxTargetPerRack) {
-      if(LOG.isDebugEnabled()) {
-        threadLocalBuilder.get().append(node.toString()).append(": ")
-          .append("Node ").append(NodeBase.getPath(node))
-          .append(" is not chosen because the rack has too many chosen nodes ");
-      }
+      logNodeIsNotChosen(node, "the rack has too many chosen nodes ");
       return false;
       return false;
     }
     }
     return true;
     return true;
   }
   }
     
     
-  /* Return a pipeline of nodes.
+  /**
+   * Return a pipeline of nodes.
    * The pipeline is formed finding a shortest path that 
    * The pipeline is formed finding a shortest path that 
    * starts from the writer and traverses all <i>nodes</i>
    * starts from the writer and traverses all <i>nodes</i>
    * This is basically a traveling salesman problem.
    * This is basically a traveling salesman problem.
    */
    */
-  private DatanodeDescriptor[] getPipeline(
-                                           DatanodeDescriptor writer,
+  private DatanodeDescriptor[] getPipeline(Node writer,
                                            DatanodeDescriptor[] nodes) {
                                            DatanodeDescriptor[] nodes) {
     if (nodes.length==0) return nodes;
     if (nodes.length==0) return nodes;
       
       
@@ -709,7 +660,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                                   int minRacks) {
                                   int minRacks) {
     DatanodeInfo[] locs = lBlk.getLocations();
     DatanodeInfo[] locs = lBlk.getLocations();
     if (locs == null)
     if (locs == null)
-      locs = new DatanodeInfo[0];
+      locs = DatanodeDescriptor.EMPTY_ARRAY;
     int numRacks = clusterMap.getNumOfRacks();
     int numRacks = clusterMap.getNumOfRacks();
     if(numRacks <= 1) // only one rack
     if(numRacks <= 1) // only one rack
       return 0;
       return 0;
@@ -724,24 +675,18 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
 
 
   @Override
   @Override
   public DatanodeDescriptor chooseReplicaToDelete(BlockCollection bc,
   public DatanodeDescriptor chooseReplicaToDelete(BlockCollection bc,
-                                                 Block block,
-                                                 short replicationFactor,
-                                                 Collection<DatanodeDescriptor> first, 
-                                                 Collection<DatanodeDescriptor> second) {
+      Block block, short replicationFactor,
+      Collection<DatanodeDescriptor> first,
+      Collection<DatanodeDescriptor> second) {
     long oldestHeartbeat =
     long oldestHeartbeat =
       now() - heartbeatInterval * tolerateHeartbeatMultiplier;
       now() - heartbeatInterval * tolerateHeartbeatMultiplier;
     DatanodeDescriptor oldestHeartbeatNode = null;
     DatanodeDescriptor oldestHeartbeatNode = null;
     long minSpace = Long.MAX_VALUE;
     long minSpace = Long.MAX_VALUE;
     DatanodeDescriptor minSpaceNode = null;
     DatanodeDescriptor minSpaceNode = null;
 
 
-    // pick replica from the first Set. If first is empty, then pick replicas
-    // from second set.
-    Iterator<DatanodeDescriptor> iter = pickupReplicaSet(first, second);
-
     // Pick the node with the oldest heartbeat or with the least free space,
     // Pick the node with the oldest heartbeat or with the least free space,
     // if all hearbeats are within the tolerable heartbeat interval
     // if all hearbeats are within the tolerable heartbeat interval
-    while (iter.hasNext() ) {
-      DatanodeDescriptor node = iter.next();
+    for(DatanodeDescriptor node : pickupReplicaSet(first, second)) {
       long free = node.getRemaining();
       long free = node.getRemaining();
       long lastHeartbeat = node.getLastUpdate();
       long lastHeartbeat = node.getLastUpdate();
       if(lastHeartbeat < oldestHeartbeat) {
       if(lastHeartbeat < oldestHeartbeat) {
@@ -762,12 +707,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    * replica while second set contains remaining replica nodes.
    * replica while second set contains remaining replica nodes.
    * So pick up first set if not empty. If first is empty, then pick second.
    * So pick up first set if not empty. If first is empty, then pick second.
    */
    */
-  protected Iterator<DatanodeDescriptor> pickupReplicaSet(
+  protected Collection<DatanodeDescriptor> pickupReplicaSet(
       Collection<DatanodeDescriptor> first,
       Collection<DatanodeDescriptor> first,
       Collection<DatanodeDescriptor> second) {
       Collection<DatanodeDescriptor> second) {
-    Iterator<DatanodeDescriptor> iter =
-        first.isEmpty() ? second.iterator() : first.iterator();
-    return iter;
+    return first.isEmpty() ? second : first;
   }
   }
   
   
   @VisibleForTesting
   @VisibleForTesting

+ 27 - 51
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java

@@ -20,9 +20,9 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.Set;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -64,8 +64,8 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
    * @return the chosen node
    * @return the chosen node
    */
    */
   @Override
   @Override
-  protected DatanodeDescriptor chooseLocalNode(DatanodeDescriptor localMachine,
-      HashMap<Node, Node> excludedNodes, long blocksize, int maxNodesPerRack,
+  protected DatanodeDescriptor chooseLocalNode(Node localMachine,
+      Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
       List<DatanodeDescriptor> results, boolean avoidStaleNodes)
       List<DatanodeDescriptor> results, boolean avoidStaleNodes)
         throws NotEnoughReplicasException {
         throws NotEnoughReplicasException {
     // if no local machine, randomly choose one node
     // if no local machine, randomly choose one node
@@ -73,18 +73,16 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
       return chooseRandom(NodeBase.ROOT, excludedNodes, 
       return chooseRandom(NodeBase.ROOT, excludedNodes, 
           blocksize, maxNodesPerRack, results, avoidStaleNodes);
           blocksize, maxNodesPerRack, results, avoidStaleNodes);
 
 
-    // otherwise try local machine first
-    Node oldNode = excludedNodes.put(localMachine, localMachine);
-    if (oldNode == null) { // was not in the excluded list
-      if (isGoodTarget(localMachine, blocksize,
-          maxNodesPerRack, false, results, avoidStaleNodes)) {
-        results.add(localMachine);
-        // Nodes under same nodegroup should be excluded.
-        addNodeGroupToExcludedNodes(excludedNodes,
-            localMachine.getNetworkLocation());
-        return localMachine;
+    if (localMachine instanceof DatanodeDescriptor) {
+      DatanodeDescriptor localDataNode = (DatanodeDescriptor)localMachine;
+      // otherwise try local machine first
+      if (excludedNodes.add(localMachine)) { // was not in the excluded list
+        if (addIfIsGoodTarget(localDataNode, excludedNodes, blocksize,
+            maxNodesPerRack, false, results, avoidStaleNodes) >= 0) {
+          return localDataNode;
+        }
       }
       }
-    } 
+    }
 
 
     // try a node on local node group
     // try a node on local node group
     DatanodeDescriptor chosenNode = chooseLocalNodeGroup(
     DatanodeDescriptor chosenNode = chooseLocalNodeGroup(
@@ -98,26 +96,10 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
         blocksize, maxNodesPerRack, results, avoidStaleNodes);
         blocksize, maxNodesPerRack, results, avoidStaleNodes);
   }
   }
 
 
-  @Override
-  protected void adjustExcludedNodes(HashMap<Node, Node> excludedNodes,
-      Node chosenNode) {
-    // as node-group aware implementation, it should make sure no two replica
-    // are placing on the same node group.
-    addNodeGroupToExcludedNodes(excludedNodes, chosenNode.getNetworkLocation());
-  }
   
   
-  // add all nodes under specific nodegroup to excludedNodes.
-  private void addNodeGroupToExcludedNodes(HashMap<Node, Node> excludedNodes,
-      String nodeGroup) {
-    List<Node> leafNodes = clusterMap.getLeaves(nodeGroup);
-    for (Node node : leafNodes) {
-      excludedNodes.put(node, node);
-    }
-  }
-
   @Override
   @Override
-  protected DatanodeDescriptor chooseLocalRack(DatanodeDescriptor localMachine,
-      HashMap<Node, Node> excludedNodes, long blocksize, int maxNodesPerRack,
+  protected DatanodeDescriptor chooseLocalRack(Node localMachine,
+      Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
       List<DatanodeDescriptor> results, boolean avoidStaleNodes)
       List<DatanodeDescriptor> results, boolean avoidStaleNodes)
       throws NotEnoughReplicasException {
       throws NotEnoughReplicasException {
     // no local machine, so choose a random machine
     // no local machine, so choose a random machine
@@ -137,9 +119,7 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
     } catch (NotEnoughReplicasException e1) {
     } catch (NotEnoughReplicasException e1) {
       // find the second replica
       // find the second replica
       DatanodeDescriptor newLocal=null;
       DatanodeDescriptor newLocal=null;
-      for(Iterator<DatanodeDescriptor> iter=results.iterator();
-          iter.hasNext();) {
-        DatanodeDescriptor nextNode = iter.next();
+      for(DatanodeDescriptor nextNode : results) {
         if (nextNode != localMachine) {
         if (nextNode != localMachine) {
           newLocal = nextNode;
           newLocal = nextNode;
           break;
           break;
@@ -165,7 +145,7 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
 
 
   @Override
   @Override
   protected void chooseRemoteRack(int numOfReplicas,
   protected void chooseRemoteRack(int numOfReplicas,
-      DatanodeDescriptor localMachine, HashMap<Node, Node> excludedNodes,
+      DatanodeDescriptor localMachine, Set<Node> excludedNodes,
       long blocksize, int maxReplicasPerRack, List<DatanodeDescriptor> results,
       long blocksize, int maxReplicasPerRack, List<DatanodeDescriptor> results,
       boolean avoidStaleNodes) throws NotEnoughReplicasException {
       boolean avoidStaleNodes) throws NotEnoughReplicasException {
     int oldNumOfReplicas = results.size();
     int oldNumOfReplicas = results.size();
@@ -191,8 +171,8 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
    * @return the chosen node
    * @return the chosen node
    */
    */
   private DatanodeDescriptor chooseLocalNodeGroup(
   private DatanodeDescriptor chooseLocalNodeGroup(
-      NetworkTopologyWithNodeGroup clusterMap, DatanodeDescriptor localMachine,
-      HashMap<Node, Node> excludedNodes, long blocksize, int maxNodesPerRack,
+      NetworkTopologyWithNodeGroup clusterMap, Node localMachine,
+      Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
       List<DatanodeDescriptor> results, boolean avoidStaleNodes)
       List<DatanodeDescriptor> results, boolean avoidStaleNodes)
       throws NotEnoughReplicasException {
       throws NotEnoughReplicasException {
     // no local machine, so choose a random machine
     // no local machine, so choose a random machine
@@ -209,9 +189,7 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
     } catch (NotEnoughReplicasException e1) {
     } catch (NotEnoughReplicasException e1) {
       // find the second replica
       // find the second replica
       DatanodeDescriptor newLocal=null;
       DatanodeDescriptor newLocal=null;
-      for(Iterator<DatanodeDescriptor> iter=results.iterator();
-        iter.hasNext();) {
-        DatanodeDescriptor nextNode = iter.next();
+      for(DatanodeDescriptor nextNode : results) {
         if (nextNode != localMachine) {
         if (nextNode != localMachine) {
           newLocal = nextNode;
           newLocal = nextNode;
           break;
           break;
@@ -248,14 +226,14 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
    * within the same nodegroup
    * within the same nodegroup
    * @return number of new excluded nodes
    * @return number of new excluded nodes
    */
    */
-  protected int addToExcludedNodes(DatanodeDescriptor localMachine,
-      HashMap<Node, Node> excludedNodes) {
+  @Override
+  protected int addToExcludedNodes(DatanodeDescriptor chosenNode,
+      Set<Node> excludedNodes) {
     int countOfExcludedNodes = 0;
     int countOfExcludedNodes = 0;
-    String nodeGroupScope = localMachine.getNetworkLocation();
+    String nodeGroupScope = chosenNode.getNetworkLocation();
     List<Node> leafNodes = clusterMap.getLeaves(nodeGroupScope);
     List<Node> leafNodes = clusterMap.getLeaves(nodeGroupScope);
     for (Node leafNode : leafNodes) {
     for (Node leafNode : leafNodes) {
-      Node node = excludedNodes.put(leafNode, leafNode);
-      if (node == null) {
+      if (excludedNodes.add(leafNode)) {
         // not a existing node in excludedNodes
         // not a existing node in excludedNodes
         countOfExcludedNodes++;
         countOfExcludedNodes++;
       }
       }
@@ -274,12 +252,12 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
    * If first is empty, then pick second.
    * If first is empty, then pick second.
    */
    */
   @Override
   @Override
-  public Iterator<DatanodeDescriptor> pickupReplicaSet(
+  public Collection<DatanodeDescriptor> pickupReplicaSet(
       Collection<DatanodeDescriptor> first,
       Collection<DatanodeDescriptor> first,
       Collection<DatanodeDescriptor> second) {
       Collection<DatanodeDescriptor> second) {
     // If no replica within same rack, return directly.
     // If no replica within same rack, return directly.
     if (first.isEmpty()) {
     if (first.isEmpty()) {
-      return second.iterator();
+      return second;
     }
     }
     // Split data nodes in the first set into two sets, 
     // Split data nodes in the first set into two sets, 
     // moreThanOne contains nodes on nodegroup with more than one replica
     // moreThanOne contains nodes on nodegroup with more than one replica
@@ -312,9 +290,7 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
       }
       }
     }
     }
     
     
-    Iterator<DatanodeDescriptor> iter =
-        moreThanOne.isEmpty() ? exactlyOne.iterator() : moreThanOne.iterator();
-    return iter;
+    return moreThanOne.isEmpty()? exactlyOne : moreThanOne;
   }
   }
   
   
 }
 }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java

@@ -57,11 +57,11 @@ class BlocksMap {
   /** Constant {@link LightWeightGSet} capacity. */
   /** Constant {@link LightWeightGSet} capacity. */
   private final int capacity;
   private final int capacity;
   
   
-  private volatile GSet<Block, BlockInfo> blocks;
+  private GSet<Block, BlockInfo> blocks;
 
 
-  BlocksMap(final float loadFactor) {
+  BlocksMap(int capacity) {
     // Use 2% of total memory to size the GSet capacity
     // Use 2% of total memory to size the GSet capacity
-    this.capacity = LightWeightGSet.computeCapacity(2.0, "BlocksMap");
+    this.capacity = capacity;
     this.blocks = new LightWeightGSet<Block, BlockInfo>(capacity);
     this.blocks = new LightWeightGSet<Block, BlockInfo>(capacity);
   }
   }
 
 

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

@@ -44,7 +44,8 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public class DatanodeDescriptor extends DatanodeInfo {
 public class DatanodeDescriptor extends DatanodeInfo {
-  
+  public static final DatanodeDescriptor[] EMPTY_ARRAY = {};
+
   // Stores status of decommissioning.
   // Stores status of decommissioning.
   // If node is not decommissioning, do not use this object for anything.
   // If node is not decommissioning, do not use this object for anything.
   public DecommissioningStatus decommissioningStatus = new DecommissioningStatus();
   public DecommissioningStatus decommissioningStatus = new DecommissioningStatus();

+ 43 - 49
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -17,21 +17,9 @@
  */
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
-import static org.apache.hadoop.util.Time.now;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.net.InetAddresses;
 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.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -41,13 +29,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
@@ -57,34 +40,24 @@ import org.apache.hadoop.hdfs.server.namenode.HostFileManager.EntrySet;
 import org.apache.hadoop.hdfs.server.namenode.HostFileManager.MutableEntrySet;
 import org.apache.hadoop.hdfs.server.namenode.HostFileManager.MutableEntrySet;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
-import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
-import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
-import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
+import org.apache.hadoop.hdfs.server.protocol.*;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
-import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
 import org.apache.hadoop.hdfs.util.CyclicIteration;
 import org.apache.hadoop.hdfs.util.CyclicIteration;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.net.CachedDNSToSwitchMapping;
-import org.apache.hadoop.net.DNSToSwitchMapping;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.net.*;
 import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;
 import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;
-import org.apache.hadoop.net.Node;
-import org.apache.hadoop.net.NodeBase;
-import org.apache.hadoop.net.ScriptBasedMapping;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.IntrusiveCollection;
 import org.apache.hadoop.util.IntrusiveCollection;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.net.InetAddresses;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.*;
+
+import static org.apache.hadoop.util.Time.now;
 
 
 /**
 /**
  * Manage datanodes, include decommission and other activities.
  * Manage datanodes, include decommission and other activities.
@@ -131,6 +104,8 @@ public class DatanodeManager {
   
   
   private final int defaultInfoPort;
   private final int defaultInfoPort;
 
 
+  private final int defaultInfoSecurePort;
+
   private final int defaultIpcPort;
   private final int defaultIpcPort;
 
 
   /** Read include/exclude files*/
   /** Read include/exclude files*/
@@ -170,6 +145,7 @@ public class DatanodeManager {
    */
    */
   private boolean hasClusterEverBeenMultiRack = false;
   private boolean hasClusterEverBeenMultiRack = false;
 
 
+  private final boolean checkIpHostnameInRegistration;
   /**
   /**
    * Whether we should tell datanodes what to cache in replies to
    * Whether we should tell datanodes what to cache in replies to
    * heartbeat messages.
    * heartbeat messages.
@@ -198,7 +174,10 @@ public class DatanodeManager {
               DFSConfigKeys.DFS_DATANODE_ADDRESS_DEFAULT)).getPort();
               DFSConfigKeys.DFS_DATANODE_ADDRESS_DEFAULT)).getPort();
     this.defaultInfoPort = NetUtils.createSocketAddr(
     this.defaultInfoPort = NetUtils.createSocketAddr(
           conf.get(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY,
           conf.get(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY,
-              DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_DEFAULT)).getPort();
+              DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_DEFAULT)).getPort();
+    this.defaultInfoSecurePort = NetUtils.createSocketAddr(
+        conf.get(DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY,
+            DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_DEFAULT)).getPort();
     this.defaultIpcPort = NetUtils.createSocketAddr(
     this.defaultIpcPort = NetUtils.createSocketAddr(
           conf.get(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY,
           conf.get(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY,
               DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_DEFAULT)).getPort();
               DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_DEFAULT)).getPort();
@@ -241,6 +220,12 @@ public class DatanodeManager {
     LOG.info(DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY
     LOG.info(DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY
         + "=" + this.blockInvalidateLimit);
         + "=" + this.blockInvalidateLimit);
 
 
+    this.checkIpHostnameInRegistration = conf.getBoolean(
+        DFSConfigKeys.DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_KEY,
+        DFSConfigKeys.DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_DEFAULT);
+    LOG.info(DFSConfigKeys.DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_KEY
+        + "=" + checkIpHostnameInRegistration);
+
     this.avoidStaleDataNodesForRead = conf.getBoolean(
     this.avoidStaleDataNodesForRead = conf.getBoolean(
         DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY,
         DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY,
         DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_DEFAULT);
         DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_DEFAULT);
@@ -764,11 +749,13 @@ public class DatanodeManager {
       // Mostly called inside an RPC, update ip and peer hostname
       // Mostly called inside an RPC, update ip and peer hostname
       String hostname = dnAddress.getHostName();
       String hostname = dnAddress.getHostName();
       String ip = dnAddress.getHostAddress();
       String ip = dnAddress.getHostAddress();
-      if (!isNameResolved(dnAddress)) {
+      if (checkIpHostnameInRegistration && !isNameResolved(dnAddress)) {
         // Reject registration of unresolved datanode to prevent performance
         // Reject registration of unresolved datanode to prevent performance
         // impact of repetitive DNS lookups later.
         // impact of repetitive DNS lookups later.
-        LOG.warn("Unresolved datanode registration from " + ip);
-        throw new DisallowedDatanodeException(nodeReg);
+        final String message = "hostname cannot be resolved (ip="
+            + ip + ", hostname=" + hostname + ")";
+        LOG.warn("Unresolved datanode registration: " + message);
+        throw new DisallowedDatanodeException(nodeReg, message);
       }
       }
       // update node registration with the ip and hostname from rpc request
       // update node registration with the ip and hostname from rpc request
       nodeReg.setIpAddr(ip);
       nodeReg.setIpAddr(ip);
@@ -897,7 +884,12 @@ public class DatanodeManager {
       // If the network location is invalid, clear the cached mappings
       // If the network location is invalid, clear the cached mappings
       // so that we have a chance to re-add this DataNode with the
       // so that we have a chance to re-add this DataNode with the
       // correct network location later.
       // correct network location later.
-      dnsToSwitchMapping.reloadCachedMappings();
+      List<String> invalidNodeNames = new ArrayList<String>(3);
+      // clear cache for nodes in IP or Hostname
+      invalidNodeNames.add(nodeReg.getIpAddr());
+      invalidNodeNames.add(nodeReg.getHostName());
+      invalidNodeNames.add(nodeReg.getPeerHostName());
+      dnsToSwitchMapping.reloadCachedMappings(invalidNodeNames);
       throw e;
       throw e;
     }
     }
   }
   }
@@ -1133,6 +1125,7 @@ public class DatanodeManager {
       // The IP:port is sufficient for listing in a report
       // The IP:port is sufficient for listing in a report
       dnId = new DatanodeID(hostStr, "", "", port,
       dnId = new DatanodeID(hostStr, "", "", port,
           DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
           DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
+          DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
           DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT);
           DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT);
     } else {
     } else {
       String ipAddr = "";
       String ipAddr = "";
@@ -1143,6 +1136,7 @@ public class DatanodeManager {
       }
       }
       dnId = new DatanodeID(ipAddr, hostStr, "", port,
       dnId = new DatanodeID(ipAddr, hostStr, "", port,
           DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
           DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
+          DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
           DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT);
           DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT);
     }
     }
     return dnId;
     return dnId;
@@ -1190,7 +1184,7 @@ public class DatanodeManager {
               new DatanodeDescriptor(new DatanodeID(entry.getIpAddress(),
               new DatanodeDescriptor(new DatanodeID(entry.getIpAddress(),
                   entry.getPrefix(), "",
                   entry.getPrefix(), "",
                   entry.getPort() == 0 ? defaultXferPort : entry.getPort(),
                   entry.getPort() == 0 ? defaultXferPort : entry.getPort(),
-                  defaultInfoPort, defaultIpcPort));
+                  defaultInfoPort, defaultInfoSecurePort, defaultIpcPort));
           dn.setLastUpdate(0); // Consider this node dead for reporting
           dn.setLastUpdate(0); // Consider this node dead for reporting
           nodes.add(dn);
           nodes.add(dn);
         }
         }
@@ -1209,17 +1203,17 @@ public class DatanodeManager {
   /**
   /**
    * Checks if name resolution was successful for the given address.  If IP
    * Checks if name resolution was successful for the given address.  If IP
    * address and host name are the same, then it means name resolution has
    * address and host name are the same, then it means name resolution has
-   * failed.  As a special case, the loopback address is also considered
+   * failed.  As a special case, local addresses are also considered
    * acceptable.  This is particularly important on Windows, where 127.0.0.1 does
    * acceptable.  This is particularly important on Windows, where 127.0.0.1 does
    * not resolve to "localhost".
    * not resolve to "localhost".
    *
    *
    * @param address InetAddress to check
    * @param address InetAddress to check
-   * @return boolean true if name resolution successful or address is loopback
+   * @return boolean true if name resolution successful or address is local
    */
    */
   private static boolean isNameResolved(InetAddress address) {
   private static boolean isNameResolved(InetAddress address) {
     String hostname = address.getHostName();
     String hostname = address.getHostName();
     String ip = address.getHostAddress();
     String ip = address.getHostAddress();
-    return !hostname.equals(ip) || address.isLoopbackAddress();
+    return !hostname.equals(ip) || NetUtils.isLocalAddress(address);
   }
   }
   
   
   private void setDatanodeDead(DatanodeDescriptor node) {
   private void setDatanodeDead(DatanodeDescriptor node) {

+ 40 - 28
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java

@@ -18,24 +18,7 @@
 
 
 package org.apache.hadoop.hdfs.server.common;
 package org.apache.hadoop.hdfs.server.common;
 
 
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.net.URL;
-import java.net.URLEncoder;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.TreeSet;
-
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.jsp.JspWriter;
+import com.google.common.base.Charsets;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -47,13 +30,9 @@ import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
@@ -74,10 +53,22 @@ import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
 
 
-import com.google.common.base.Charsets;
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.jsp.JspWriter;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.*;
 
 
-import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER;
+import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER;
 
 
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class JspHelper {
 public class JspHelper {
@@ -112,7 +103,7 @@ public class JspHelper {
       return super.hashCode();
       return super.hashCode();
     }
     }
   }
   }
- 
+
   // compare two records based on their frequency
   // compare two records based on their frequency
   private static class NodeRecordComparator implements Comparator<NodeRecord> {
   private static class NodeRecordComparator implements Comparator<NodeRecord> {
 
 
@@ -126,6 +117,27 @@ public class JspHelper {
       return 0;
       return 0;
     }
     }
   }
   }
+
+  /**
+   * A helper class that generates the correct URL for different schema.
+   *
+   */
+  public static final class Url {
+    public static String authority(String scheme, DatanodeID d) {
+      if (scheme.equals("http")) {
+        return d.getInfoAddr();
+      } else if (scheme.equals("https")) {
+        return d.getInfoSecureAddr();
+      } else {
+        throw new IllegalArgumentException("Unknown scheme:" + scheme);
+      }
+    }
+
+    public static String url(String scheme, DatanodeID d) {
+      return scheme + "://" + authority(scheme, d);
+    }
+  }
+
   public static DatanodeInfo bestNode(LocatedBlocks blks, Configuration conf)
   public static DatanodeInfo bestNode(LocatedBlocks blks, Configuration conf)
       throws IOException {
       throws IOException {
     HashMap<DatanodeInfo, NodeRecord> map =
     HashMap<DatanodeInfo, NodeRecord> map =
@@ -217,7 +229,7 @@ public class JspHelper {
         offsetIntoBlock, amtToRead,  true,
         offsetIntoBlock, amtToRead,  true,
         "JspHelper", TcpPeerServer.peerFromSocketAndKey(s, encryptionKey),
         "JspHelper", TcpPeerServer.peerFromSocketAndKey(s, encryptionKey),
         new DatanodeID(addr.getAddress().getHostAddress(),
         new DatanodeID(addr.getAddress().getHostAddress(),
-            addr.getHostName(), poolId, addr.getPort(), 0, 0), null,
+            addr.getHostName(), poolId, addr.getPort(), 0, 0, 0), null,
             null, null, false, CachingStrategy.newDefaultStrategy());
             null, null, false, CachingStrategy.newDefaultStrategy());
         
         
     final byte[] buf = new byte[amtToRead];
     final byte[] buf = new byte[amtToRead];

+ 11 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java

@@ -100,6 +100,7 @@ class BlockPoolSliceScanner {
   private long currentPeriodStart = Time.now();
   private long currentPeriodStart = Time.now();
   private long bytesLeft = 0; // Bytes to scan in this period
   private long bytesLeft = 0; // Bytes to scan in this period
   private long totalBytesToScan = 0;
   private long totalBytesToScan = 0;
+  private boolean isNewPeriod = true;
   
   
   private final LogFileHandler verificationLog;
   private final LogFileHandler verificationLog;
   
   
@@ -126,7 +127,10 @@ class BlockPoolSliceScanner {
       public int compare(BlockScanInfo left, BlockScanInfo right) {
       public int compare(BlockScanInfo left, BlockScanInfo right) {
         final long l = left.lastScanTime;
         final long l = left.lastScanTime;
         final long r = right.lastScanTime;
         final long r = right.lastScanTime;
-        return l < r? -1: l > r? 1: 0; 
+        // compare blocks itself if scantimes are same to avoid.
+        // because TreeMap uses comparator if available to check existence of
+        // the object. 
+        return l < r? -1: l > r? 1: left.compareTo(right); 
       }
       }
     };
     };
 
 
@@ -148,8 +152,6 @@ class BlockPoolSliceScanner {
     public boolean equals(Object that) {
     public boolean equals(Object that) {
       if (this == that) {
       if (this == that) {
         return true;
         return true;
-      } else if (that == null || !(that instanceof BlockScanInfo)) {
-        return false;
       }
       }
       return super.equals(that);
       return super.equals(that);
     }
     }
@@ -539,10 +541,12 @@ class BlockPoolSliceScanner {
                   entry.genStamp));
                   entry.genStamp));
               if (info != null) {
               if (info != null) {
                 if (processedBlocks.get(entry.blockId) == null) {
                 if (processedBlocks.get(entry.blockId) == null) {
-                  updateBytesLeft(-info.getNumBytes());
+                  if (isNewPeriod) {
+                    updateBytesLeft(-info.getNumBytes());
+                  }
                   processedBlocks.put(entry.blockId, 1);
                   processedBlocks.put(entry.blockId, 1);
                 }
                 }
-                if (logIterator.isPrevious()) {
+                if (logIterator.isLastReadFromPrevious()) {
                   // write the log entry to current file
                   // write the log entry to current file
                   // so that the entry is preserved for later runs.
                   // so that the entry is preserved for later runs.
                   verificationLog.append(entry.verificationTime, entry.genStamp,
                   verificationLog.append(entry.verificationTime, entry.genStamp,
@@ -557,6 +561,7 @@ class BlockPoolSliceScanner {
       } finally {
       } finally {
         IOUtils.closeStream(logIterator);
         IOUtils.closeStream(logIterator);
       }
       }
+      isNewPeriod = false;
     }
     }
     
     
     
     
@@ -597,6 +602,7 @@ class BlockPoolSliceScanner {
     // reset the byte counts :
     // reset the byte counts :
     bytesLeft = totalBytesToScan;
     bytesLeft = totalBytesToScan;
     currentPeriodStart = Time.now();
     currentPeriodStart = Time.now();
+    isNewPeriod = true;
   }
   }
   
   
   private synchronized boolean workRemainingInCurrentPeriod() {
   private synchronized boolean workRemainingInCurrentPeriod() {

+ 45 - 124
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -18,67 +18,10 @@
 package org.apache.hadoop.hdfs.server.datanode;
 package org.apache.hadoop.hdfs.server.datanode;
 
 
 
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PLUGINS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTPS_ENABLE_KEY;
-import static org.apache.hadoop.util.ExitUtil.terminate;
-
-import java.io.BufferedOutputStream;
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.net.SocketException;
-import java.net.SocketTimeoutException;
-import java.net.URI;
-import java.net.UnknownHostException;
-import java.nio.channels.ClosedByInterruptException;
-import java.nio.channels.SocketChannel;
-import java.security.PrivilegedExceptionAction;
-import java.util.AbstractList;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.BlockingService;
 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.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -95,37 +38,15 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.net.DomainPeerServer;
 import org.apache.hadoop.hdfs.net.DomainPeerServer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
-import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
-import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.hadoop.hdfs.protocol.datatransfer.*;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InterDatanodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InterDatanodeProtocolService;
-import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolPB;
-import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolServerSideTranslatorPB;
-import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
-import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolPB;
-import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolServerSideTranslatorPB;
-import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolTranslatorPB;
-import org.apache.hadoop.hdfs.protocolPB.PBHelper;
-import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
+import org.apache.hadoop.hdfs.protocolPB.*;
+import org.apache.hadoop.hdfs.security.token.block.*;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
-import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -140,11 +61,7 @@ import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMetho
 import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
 import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
 import org.apache.hadoop.hdfs.server.namenode.StreamFile;
 import org.apache.hadoop.hdfs.server.namenode.StreamFile;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.apache.hadoop.hdfs.server.protocol.*;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.Param;
 import org.apache.hadoop.hdfs.web.resources.Param;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.http.HttpServer;
@@ -166,22 +83,21 @@ import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.DiskChecker;
+import org.apache.hadoop.util.*;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
-import org.apache.hadoop.util.GenericOptionsParser;
-import org.apache.hadoop.util.JvmPauseMonitor;
-import org.apache.hadoop.util.ServicePlugin;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.Time;
-import org.apache.hadoop.util.VersionInfo;
 import org.mortbay.util.ajax.JSON;
 import org.mortbay.util.ajax.JSON;
 
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.protobuf.BlockingService;
+import java.io.*;
+import java.net.*;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.SocketChannel;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import static org.apache.hadoop.util.ExitUtil.terminate;
 
 
 /**********************************************************
 /**********************************************************
  * DataNode is a class (and program) that stores a set of
  * DataNode is a class (and program) that stores a set of
@@ -263,6 +179,7 @@ public class DataNode extends Configured
   private volatile boolean heartbeatsDisabledForTests = false;
   private volatile boolean heartbeatsDisabledForTests = false;
   private DataStorage storage = null;
   private DataStorage storage = null;
   private HttpServer infoServer = null;
   private HttpServer infoServer = null;
+  private int infoSecurePort;
   DataNodeMetrics metrics;
   DataNodeMetrics metrics;
   private InetSocketAddress streamingAddr;
   private InetSocketAddress streamingAddr;
   
   
@@ -386,16 +303,13 @@ public class DataNode extends Configured
     InetSocketAddress infoSocAddr = DataNode.getInfoAddr(conf);
     InetSocketAddress infoSocAddr = DataNode.getInfoAddr(conf);
     String infoHost = infoSocAddr.getHostName();
     String infoHost = infoSocAddr.getHostName();
     int tmpInfoPort = infoSocAddr.getPort();
     int tmpInfoPort = infoSocAddr.getPort();
-    this.infoServer = (secureResources == null) 
-        ? new HttpServer.Builder().setName("datanode")
-            .setBindAddress(infoHost).setPort(tmpInfoPort)
-            .setFindPort(tmpInfoPort == 0).setConf(conf)
-            .setACL(new AccessControlList(conf.get(DFS_ADMIN, " "))).build()
-        : new HttpServer.Builder().setName("datanode")
-            .setBindAddress(infoHost).setPort(tmpInfoPort)
-            .setFindPort(tmpInfoPort == 0).setConf(conf)
-            .setACL(new AccessControlList(conf.get(DFS_ADMIN, " ")))
-            .setConnector(secureResources.getListener()).build();
+    HttpServer.Builder builder = new HttpServer.Builder().setName("datanode")
+        .setBindAddress(infoHost).setPort(tmpInfoPort)
+        .setFindPort(tmpInfoPort == 0).setConf(conf)
+        .setACL(new AccessControlList(conf.get(DFS_ADMIN, " ")));
+    this.infoServer = (secureResources == null) ? builder.build() :
+        builder.setConnector(secureResources.getListener()).build();
+
     LOG.info("Opened info server at " + infoHost + ":" + tmpInfoPort);
     LOG.info("Opened info server at " + infoHost + ":" + tmpInfoPort);
     if (conf.getBoolean(DFS_HTTPS_ENABLE_KEY, false)) {
     if (conf.getBoolean(DFS_HTTPS_ENABLE_KEY, false)) {
       boolean needClientAuth = conf.getBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
       boolean needClientAuth = conf.getBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
@@ -409,6 +323,7 @@ public class DataNode extends Configured
       if(LOG.isDebugEnabled()) {
       if(LOG.isDebugEnabled()) {
         LOG.debug("Datanode listening for SSL on " + secInfoSocAddr);
         LOG.debug("Datanode listening for SSL on " + secInfoSocAddr);
       }
       }
+      infoSecurePort = secInfoSocAddr.getPort();
     }
     }
     this.infoServer.addInternalServlet(null, "/streamFile/*", StreamFile.class);
     this.infoServer.addInternalServlet(null, "/streamFile/*", StreamFile.class);
     this.infoServer.addInternalServlet(null, "/getFileChecksum/*",
     this.infoServer.addInternalServlet(null, "/getFileChecksum/*",
@@ -796,7 +711,8 @@ public class DataNode extends Configured
     }
     }
     DatanodeID dnId = new DatanodeID(
     DatanodeID dnId = new DatanodeID(
         streamingAddr.getAddress().getHostAddress(), hostName, 
         streamingAddr.getAddress().getHostAddress(), hostName, 
-        getStorageId(), getXferPort(), getInfoPort(), getIpcPort());
+        getStorageId(), getXferPort(), getInfoPort(),
+            infoSecurePort, getIpcPort());
     return new DatanodeRegistration(dnId, storageInfo, 
     return new DatanodeRegistration(dnId, storageInfo, 
         new ExportedBlockKeys(), VersionInfo.getVersion());
         new ExportedBlockKeys(), VersionInfo.getVersion());
   }
   }
@@ -894,7 +810,7 @@ public class DataNode extends Configured
    * If this is the first block pool to register, this also initializes
    * If this is the first block pool to register, this also initializes
    * the datanode-scoped storage.
    * the datanode-scoped storage.
    * 
    * 
-   * @param nsInfo the handshake response from the NN.
+   * @param bpos Block pool offer service
    * @throws IOException if the NN is inconsistent with the local storage.
    * @throws IOException if the NN is inconsistent with the local storage.
    */
    */
   void initBlockPool(BPOfferService bpos) throws IOException {
   void initBlockPool(BPOfferService bpos) throws IOException {
@@ -1419,15 +1335,13 @@ public class DataNode extends Configured
     
     
     int numTargets = xferTargets.length;
     int numTargets = xferTargets.length;
     if (numTargets > 0) {
     if (numTargets > 0) {
-      if (LOG.isInfoEnabled()) {
-        StringBuilder xfersBuilder = new StringBuilder();
-        for (int i = 0; i < numTargets; i++) {
-          xfersBuilder.append(xferTargets[i]);
-          xfersBuilder.append(" ");
-        }
-        LOG.info(bpReg + " Starting thread to transfer " + 
-                 block + " to " + xfersBuilder);                       
+      StringBuilder xfersBuilder = new StringBuilder();
+      for (int i = 0; i < numTargets; i++) {
+        xfersBuilder.append(xferTargets[i]);
+        xfersBuilder.append(" ");
       }
       }
+      LOG.info(bpReg + " Starting thread to transfer " + 
+               block + " to " + xfersBuilder);                       
 
 
       new Daemon(new DataTransfer(xferTargets, block,
       new Daemon(new DataTransfer(xferTargets, block,
           BlockConstructionStage.PIPELINE_SETUP_CREATE, "")).start();
           BlockConstructionStage.PIPELINE_SETUP_CREATE, "")).start();
@@ -2351,6 +2265,13 @@ public class DataNode extends Configured
     return infoServer.getPort();
     return infoServer.getPort();
   }
   }
 
 
+  /**
+   * @return the datanode's https port
+   */
+  public int getInfoSecurePort() {
+    return infoSecurePort;
+  }
+
   /**
   /**
    * Returned information is a JSON representation of a map with 
    * Returned information is a JSON representation of a map with 
    * name node host name as the key and block pool Id as the value.
    * name node host name as the key and block pool Id as the value.

+ 97 - 123
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java

@@ -18,10 +18,9 @@
 package org.apache.hadoop.hdfs.server.datanode;
 package org.apache.hadoop.hdfs.server.datanode;
 
 
 import java.io.File;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
-import java.net.URI;
 import java.net.URL;
 import java.net.URL;
 import java.net.URLEncoder;
 import java.net.URLEncoder;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
@@ -37,9 +36,9 @@ import javax.servlet.jsp.JspWriter;
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
 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.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -47,20 +46,23 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer;
 import org.apache.hadoop.http.HtmlQuoting;
 import org.apache.hadoop.http.HtmlQuoting;
-import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.ServletUtil;
 import org.apache.hadoop.util.ServletUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
 
 
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class DatanodeJspHelper {
 public class DatanodeJspHelper {
+  private static final int PREV_BLOCK = -1;
+  private static final int NEXT_BLOCK = 1;
+
   private static DFSClient getDFSClient(final UserGroupInformation user,
   private static DFSClient getDFSClient(final UserGroupInformation user,
                                         final String addr,
                                         final String addr,
                                         final Configuration conf
                                         final Configuration conf
@@ -143,10 +145,10 @@ public class DatanodeJspHelper {
           out.print("Empty file");
           out.print("Empty file");
         } else {
         } else {
           DatanodeInfo chosenNode = JspHelper.bestNode(firstBlock, conf);
           DatanodeInfo chosenNode = JspHelper.bestNode(firstBlock, conf);
-          String fqdn = canonicalize(chosenNode.getIpAddr());
           int datanodePort = chosenNode.getXferPort();
           int datanodePort = chosenNode.getXferPort();
-          String redirectLocation = HttpConfig.getSchemePrefix() + fqdn + ":"
-              + chosenNode.getInfoPort() + "/browseBlock.jsp?blockId="
+          String redirectLocation = JspHelper.Url.url(req.getScheme(),
+              chosenNode)
+              + "/browseBlock.jsp?blockId="
               + firstBlock.getBlock().getBlockId() + "&blockSize="
               + firstBlock.getBlock().getBlockId() + "&blockSize="
               + firstBlock.getBlock().getNumBytes() + "&genstamp="
               + firstBlock.getBlock().getNumBytes() + "&genstamp="
               + firstBlock.getBlock().getGenerationStamp() + "&filename="
               + firstBlock.getBlock().getGenerationStamp() + "&filename="
@@ -225,7 +227,7 @@ public class DatanodeJspHelper {
         JspHelper.addTableFooter(out);
         JspHelper.addTableFooter(out);
       }
       }
     }
     }
-    out.print("<br><a href=\"" + HttpConfig.getSchemePrefix()
+    out.print("<br><a href=\"///"
         + canonicalize(nnAddr) + ":"
         + canonicalize(nnAddr) + ":"
         + namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
         + namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
     dfs.close();
     dfs.close();
@@ -302,8 +304,7 @@ public class DatanodeJspHelper {
         Long.MAX_VALUE).getLocatedBlocks();
         Long.MAX_VALUE).getLocatedBlocks();
     // Add the various links for looking at the file contents
     // Add the various links for looking at the file contents
     // URL for downloading the full file
     // URL for downloading the full file
-    String downloadUrl = HttpConfig.getSchemePrefix() + req.getServerName() + ":"
-        + req.getServerPort() + "/streamFile" + ServletUtil.encodePath(filename)
+    String downloadUrl = "/streamFile" + ServletUtil.encodePath(filename)
         + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr, true)
         + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr, true)
         + JspHelper.getDelegationTokenUrlParam(tokenString);
         + JspHelper.getDelegationTokenUrlParam(tokenString);
     out.print("<a name=\"viewOptions\"></a>");
     out.print("<a name=\"viewOptions\"></a>");
@@ -319,8 +320,8 @@ public class DatanodeJspHelper {
       dfs.close();
       dfs.close();
       return;
       return;
     }
     }
-    String fqdn = canonicalize(chosenNode.getIpAddr());
-    String tailUrl = HttpConfig.getSchemePrefix() + fqdn + ":" + chosenNode.getInfoPort()
+
+    String tailUrl = "///" + JspHelper.Url.authority(req.getScheme(), chosenNode)
         + "/tail.jsp?filename=" + URLEncoder.encode(filename, "UTF-8")
         + "/tail.jsp?filename=" + URLEncoder.encode(filename, "UTF-8")
         + "&namenodeInfoPort=" + namenodeInfoPort
         + "&namenodeInfoPort=" + namenodeInfoPort
         + "&chunkSizeToView=" + chunkSizeToView
         + "&chunkSizeToView=" + chunkSizeToView
@@ -368,8 +369,7 @@ public class DatanodeJspHelper {
       for (int j = 0; j < locs.length; j++) {
       for (int j = 0; j < locs.length; j++) {
         String datanodeAddr = locs[j].getXferAddr();
         String datanodeAddr = locs[j].getXferAddr();
         datanodePort = locs[j].getXferPort();
         datanodePort = locs[j].getXferPort();
-        fqdn = canonicalize(locs[j].getIpAddr());
-        String blockUrl = HttpConfig.getSchemePrefix() + fqdn + ":" + locs[j].getInfoPort()
+        String blockUrl = "///" + JspHelper.Url.authority(req.getScheme(), locs[j])
             + "/browseBlock.jsp?blockId=" + blockidstring
             + "/browseBlock.jsp?blockId=" + blockidstring
             + "&blockSize=" + blockSize
             + "&blockSize=" + blockSize
             + "&filename=" + URLEncoder.encode(filename, "UTF-8")
             + "&filename=" + URLEncoder.encode(filename, "UTF-8")
@@ -380,7 +380,7 @@ public class DatanodeJspHelper {
             + JspHelper.getDelegationTokenUrlParam(tokenString)
             + JspHelper.getDelegationTokenUrlParam(tokenString)
             + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
             + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
 
 
-        String blockInfoUrl = HttpConfig.getSchemePrefix() + nnCanonicalName + ":"
+        String blockInfoUrl = "///" + nnCanonicalName + ":"
             + namenodeInfoPort
             + namenodeInfoPort
             + "/block_info_xml.jsp?blockId=" + blockidstring;
             + "/block_info_xml.jsp?blockId=" + blockidstring;
         out.print("<td>&nbsp</td><td><a href=\"" + blockUrl + "\">"
         out.print("<td>&nbsp</td><td><a href=\"" + blockUrl + "\">"
@@ -391,7 +391,7 @@ public class DatanodeJspHelper {
     }
     }
     out.println("</table>");
     out.println("</table>");
     out.print("<hr>");
     out.print("<hr>");
-    out.print("<br><a href=\"" + HttpConfig.getSchemePrefix()
+    out.print("<br><a href=\"///"
         + nnCanonicalName + ":"
         + nnCanonicalName + ":"
         + namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
         + namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
     dfs.close();
     dfs.close();
@@ -491,9 +491,7 @@ public class DatanodeJspHelper {
     String parent = new File(filename).getParent();
     String parent = new File(filename).getParent();
     JspHelper.printGotoForm(out, namenodeInfoPort, tokenString, parent, nnAddr);
     JspHelper.printGotoForm(out, namenodeInfoPort, tokenString, parent, nnAddr);
     out.print("<hr>");
     out.print("<hr>");
-    out.print("<a href=\"" + HttpConfig.getSchemePrefix()
-        + req.getServerName() + ":" + req.getServerPort()
-        + "/browseDirectory.jsp?dir=" + URLEncoder.encode(parent, "UTF-8")
+    out.print("<a href=\"/browseDirectory.jsp?dir=" + URLEncoder.encode(parent, "UTF-8")
         + "&namenodeInfoPort=" + namenodeInfoPort
         + "&namenodeInfoPort=" + namenodeInfoPort
         + JspHelper.getDelegationTokenUrlParam(tokenString)
         + JspHelper.getDelegationTokenUrlParam(tokenString)
         + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr)
         + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr)
@@ -501,112 +499,23 @@ public class DatanodeJspHelper {
     out.print("<a href=\"#viewOptions\">Advanced view/download options</a><br>");
     out.print("<a href=\"#viewOptions\">Advanced view/download options</a><br>");
     out.print("<hr>");
     out.print("<hr>");
 
 
-    // Determine the prev & next blocks
-    long nextStartOffset = 0;
-    long nextBlockSize = 0;
-    String nextBlockIdStr = null;
-    String nextGenStamp = null;
-    String nextHost = req.getServerName();
-    int nextPort = req.getServerPort();
-    int nextDatanodePort = datanodePort;
-    // determine data for the next link
-    if (startOffset + chunkSizeToView >= blockSize) {
-      // we have to go to the next block from this point onwards
-      List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
-          Long.MAX_VALUE).getLocatedBlocks();
-      for (int i = 0; i < blocks.size(); i++) {
-        if (blocks.get(i).getBlock().getBlockId() == blockId) {
-          if (i != blocks.size() - 1) {
-            LocatedBlock nextBlock = blocks.get(i + 1);
-            nextBlockIdStr = Long.toString(nextBlock.getBlock().getBlockId());
-            nextGenStamp = Long.toString(nextBlock.getBlock()
-                .getGenerationStamp());
-            nextStartOffset = 0;
-            nextBlockSize = nextBlock.getBlock().getNumBytes();
-            DatanodeInfo d = JspHelper.bestNode(nextBlock, conf);
-            nextDatanodePort = d.getXferPort();
-            nextHost = d.getIpAddr();
-            nextPort = d.getInfoPort();
-          }
-        }
-      }
-    } else {
-      // we are in the same block
-      nextBlockIdStr = blockId.toString();
-      nextStartOffset = startOffset + chunkSizeToView;
-      nextBlockSize = blockSize;
-      nextGenStamp = genStamp.toString();
-    }
-    String nextUrl = null;
-    if (nextBlockIdStr != null) {
-      nextUrl = HttpConfig.getSchemePrefix() + canonicalize(nextHost) + ":" + nextPort
-          + "/browseBlock.jsp?blockId=" + nextBlockIdStr
-          + "&blockSize=" + nextBlockSize
-          + "&startOffset=" + nextStartOffset
-          + "&genstamp=" + nextGenStamp
-          + "&filename=" + URLEncoder.encode(filename, "UTF-8")
-          + "&chunkSizeToView=" + chunkSizeToView
-          + "&datanodePort=" + nextDatanodePort
-          + "&namenodeInfoPort=" + namenodeInfoPort
-          + JspHelper.getDelegationTokenUrlParam(tokenString)
-          + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
+    String authority = req.getServerName() + ":" + req.getServerPort();
+    String nextUrl = generateLinksForAdjacentBlock(NEXT_BLOCK, authority,
+        datanodePort, startOffset, chunkSizeToView, blockSize, blockId,
+        genStamp, dfs, filename, conf, req.getScheme(), tokenString,
+        namenodeInfoPort, nnAddr);
+    if (nextUrl != null) {
       out.print("<a href=\"" + nextUrl + "\">View Next chunk</a>&nbsp;&nbsp;");
       out.print("<a href=\"" + nextUrl + "\">View Next chunk</a>&nbsp;&nbsp;");
     }
     }
-    // determine data for the prev link
-    String prevBlockIdStr = null;
-    String prevGenStamp = null;
-    long prevStartOffset = 0;
-    long prevBlockSize = 0;
-    String prevHost = req.getServerName();
-    int prevPort = req.getServerPort();
-    int prevDatanodePort = datanodePort;
-    if (startOffset == 0) {
-      List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
-          Long.MAX_VALUE).getLocatedBlocks();
-      for (int i = 0; i < blocks.size(); i++) {
-        if (blocks.get(i).getBlock().getBlockId() == blockId) {
-          if (i != 0) {
-            LocatedBlock prevBlock = blocks.get(i - 1);
-            prevBlockIdStr = Long.toString(prevBlock.getBlock().getBlockId());
-            prevGenStamp = Long.toString(prevBlock.getBlock()
-                .getGenerationStamp());
-            prevStartOffset = prevBlock.getBlock().getNumBytes()
-                - chunkSizeToView;
-            if (prevStartOffset < 0)
-              prevStartOffset = 0;
-            prevBlockSize = prevBlock.getBlock().getNumBytes();
-            DatanodeInfo d = JspHelper.bestNode(prevBlock, conf);
-            prevDatanodePort = d.getXferPort();
-            prevHost = d.getIpAddr();
-            prevPort = d.getInfoPort();
-          }
-        }
-      }
-    } else {
-      // we are in the same block
-      prevBlockIdStr = blockId.toString();
-      prevStartOffset = startOffset - chunkSizeToView;
-      if (prevStartOffset < 0)
-        prevStartOffset = 0;
-      prevBlockSize = blockSize;
-      prevGenStamp = genStamp.toString();
-    }
 
 
-    String prevUrl = null;
-    if (prevBlockIdStr != null) {
-      prevUrl = HttpConfig.getSchemePrefix() + canonicalize(prevHost) + ":" + prevPort
-          + "/browseBlock.jsp?blockId=" + prevBlockIdStr
-          + "&blockSize=" + prevBlockSize
-          + "&startOffset=" + prevStartOffset
-          + "&filename=" + URLEncoder.encode(filename, "UTF-8")
-          + "&chunkSizeToView=" + chunkSizeToView
-          + "&genstamp=" + prevGenStamp
-          + "&datanodePort=" + prevDatanodePort
-          + "&namenodeInfoPort=" + namenodeInfoPort
-          + JspHelper.getDelegationTokenUrlParam(tokenString)
-          + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
+    String prevUrl = generateLinksForAdjacentBlock(PREV_BLOCK, authority,
+        datanodePort, startOffset, chunkSizeToView, blockSize, blockId,
+        genStamp, dfs, filename, conf, req.getScheme(), tokenString,
+        namenodeInfoPort, nnAddr);
+    if (prevUrl != null) {
       out.print("<a href=\"" + prevUrl + "\">View Prev chunk</a>&nbsp;&nbsp;");
       out.print("<a href=\"" + prevUrl + "\">View Prev chunk</a>&nbsp;&nbsp;");
     }
     }
+
     out.print("<hr>");
     out.print("<hr>");
     out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
     out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
     try {
     try {
@@ -621,6 +530,71 @@ public class DatanodeJspHelper {
     dfs.close();
     dfs.close();
   }
   }
 
 
+  private static String generateLinksForAdjacentBlock(final int direction,
+      String authority, int datanodePort, long startOffset,
+      int chunkSizeToView, long blockSize, long blockId, Long genStamp,
+      final DFSClient dfs, final String filename, final Configuration conf,
+      final String scheme, final String tokenString,
+      final int namenodeInfoPort, final String nnAddr)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+
+    boolean found = false;
+    if ((direction == NEXT_BLOCK && startOffset + chunkSizeToView < blockSize)
+        || (direction == PREV_BLOCK && startOffset != 0)) {
+      // we are in the same block
+      found = true;
+
+      if (direction == NEXT_BLOCK) {
+        startOffset = startOffset + chunkSizeToView;
+      } else {
+        startOffset = Math.max(0, startOffset - chunkSizeToView);
+      }
+    } else {
+      List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
+          Long.MAX_VALUE).getLocatedBlocks();
+
+      final long curBlockId = blockId;
+      int curBlockIdx = Iterables.indexOf(blocks, new Predicate<LocatedBlock>() {
+        @Override
+        public boolean apply(LocatedBlock b) {
+          return b.getBlock().getBlockId() == curBlockId;
+        }
+      });
+      found = curBlockIdx != -1 &&
+          ((direction == NEXT_BLOCK && curBlockIdx < blocks.size() - 1)
+              || (direction == PREV_BLOCK && curBlockIdx > 0));
+
+      if (found) {
+        LocatedBlock nextBlock = blocks.get(curBlockIdx + direction);
+
+        blockId = nextBlock.getBlock().getBlockId();
+        genStamp = nextBlock.getBlock().getGenerationStamp();
+        startOffset = 0;
+        blockSize = nextBlock.getBlock().getNumBytes();
+        DatanodeInfo d = JspHelper.bestNode(nextBlock, conf);
+        datanodePort = d.getXferPort();
+        authority = JspHelper.Url.authority(scheme, d);
+      }
+    }
+
+    if (found) {
+      return "///" + authority
+          + "/browseBlock.jsp?blockId=" + blockId
+          + "&blockSize=" + blockSize
+          + "&startOffset=" + startOffset
+          + "&genstamp=" + genStamp
+          + "&filename=" + URLEncoder.encode(filename, "UTF-8")
+          + "&chunkSizeToView=" + chunkSizeToView
+          + "&datanodePort=" + datanodePort
+          + "&namenodeInfoPort=" + namenodeInfoPort
+          + JspHelper.getDelegationTokenUrlParam(tokenString)
+          + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
+    } else {
+      return null;
+    }
+  }
+
   static void generateFileChunksForTail(JspWriter out, HttpServletRequest req,
   static void generateFileChunksForTail(JspWriter out, HttpServletRequest req,
                                         Configuration conf
                                         Configuration conf
                                         ) throws IOException,
                                         ) throws IOException,

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

@@ -33,6 +33,12 @@ public interface RollingLogs {
   public interface LineIterator extends Iterator<String>, Closeable {
   public interface LineIterator extends Iterator<String>, Closeable {
     /** Is the iterator iterating the previous? */
     /** Is the iterator iterating the previous? */
     public boolean isPrevious();
     public boolean isPrevious();
+
+    /**
+     * Is the last read entry from previous? This should be called after
+     * reading.
+     */
+    public boolean isLastReadFromPrevious();
   }
   }
 
 
   /**
   /**

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RollingLogsImpl.java

@@ -134,6 +134,7 @@ class RollingLogsImpl implements RollingLogs {
    */
    */
   private class Reader implements RollingLogs.LineIterator {
   private class Reader implements RollingLogs.LineIterator {
     private File file;
     private File file;
+    private File lastReadFile;
     private BufferedReader reader;
     private BufferedReader reader;
     private String line;
     private String line;
     private boolean closed = false;
     private boolean closed = false;
@@ -149,6 +150,11 @@ class RollingLogsImpl implements RollingLogs {
       return file == prev;
       return file == prev;
     }
     }
 
 
+    @Override
+    public boolean isLastReadFromPrevious() {
+      return lastReadFile == prev;
+    }
+
     private boolean openFile() throws IOException {
     private boolean openFile() throws IOException {
 
 
       for(int i=0; i<2; i++) {
       for(int i=0; i<2; i++) {
@@ -203,6 +209,7 @@ class RollingLogsImpl implements RollingLogs {
     public String next() {
     public String next() {
       String curLine = line;
       String curLine = line;
       try {
       try {
+        lastReadFile = file;
         readNext();
         readNext();
       } catch (IOException e) {
       } catch (IOException e) {
         DataBlockScanner.LOG.warn("Failed to read next line.", e);
         DataBlockScanner.LOG.warn("Failed to read next line.", e);

+ 11 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java

@@ -827,7 +827,7 @@ class ClusterJspHelper {
     doc.startTag("item");
     doc.startTag("item");
     doc.attribute("label", label);
     doc.attribute("label", label);
     doc.attribute("value", value);
     doc.attribute("value", value);
-    doc.attribute("link", HttpConfig.getSchemePrefix() + url);
+    doc.attribute("link", "///" + url);
     doc.endTag(); // item
     doc.endTag(); // item
   }
   }
 
 
@@ -887,7 +887,16 @@ class ClusterJspHelper {
 
 
   private static String queryMbean(String httpAddress, Configuration conf) 
   private static String queryMbean(String httpAddress, Configuration conf) 
     throws IOException {
     throws IOException {
-    URL url = new URL(HttpConfig.getSchemePrefix() + httpAddress+JMX_QRY);
+    /**
+     * Although the other namenode might support HTTPS, it is fundamentally
+     * broken to get the JMX via an HTTPS connection inside the namenode,
+     * because in HTTPS set up the principal of the client and the one of
+     * the namenode differs. Therefore, there is no guarantees that the
+     * HTTPS connection can be set up.
+     *
+     * As a result, we just hard code the connection as an HTTP connection.
+     */
+    URL url = new URL("http://" + httpAddress + JMX_QRY);
     return readOutput(url);
     return readOutput(url);
   }
   }
   /**
   /**

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

@@ -2600,10 +2600,12 @@ public class FSDirectory implements Closeable {
           node.getUserName(snapshot), node.getGroupName(snapshot),
           node.getUserName(snapshot), node.getGroupName(snapshot),
           node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
           node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
           node.getId(), loc, childrenNum);
           node.getId(), loc, childrenNum);
-        // Set caching information for the located blocks.
-    CacheManager cacheManager = namesystem.getCacheManager();
-    for (LocatedBlock lb: loc.getLocatedBlocks()) {
-      cacheManager.setCachedLocations(lb);
+    // Set caching information for the located blocks.
+    if (loc != null) {
+      CacheManager cacheManager = namesystem.getCacheManager();
+      for (LocatedBlock lb: loc.getLocatedBlocks()) {
+        cacheManager.setCachedLocations(lb);
+      }
     }
     }
     return status;
     return status;
   }
   }

+ 255 - 182
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -124,6 +124,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
@@ -168,7 +169,14 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.*;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
+import org.apache.hadoop.hdfs.server.blockmanagement.OutOfV1GenerationStampsException;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
@@ -181,12 +189,6 @@ import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.Status;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
-import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
 import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
 import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
@@ -198,6 +200,12 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottab
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.Status;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
+import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -210,6 +218,7 @@ import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.RetriableException;
 import org.apache.hadoop.ipc.RetryCache;
 import org.apache.hadoop.ipc.RetryCache;
 import org.apache.hadoop.ipc.RetryCache.CacheEntry;
 import org.apache.hadoop.ipc.RetryCache.CacheEntry;
 import org.apache.hadoop.ipc.RetryCache.CacheEntryWithPayload;
 import org.apache.hadoop.ipc.RetryCache.CacheEntryWithPayload;
@@ -238,6 +247,7 @@ import org.mortbay.util.ajax.JSON;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
 
 
 /**
 /**
@@ -440,7 +450,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private final long accessTimePrecision;
   private final long accessTimePrecision;
 
 
   /** Lock to protect FSNamesystem. */
   /** Lock to protect FSNamesystem. */
-  private ReentrantReadWriteLock fsLock = new ReentrantReadWriteLock(true);
+  private ReentrantReadWriteLock fsLock;
 
 
   /**
   /**
    * Used when this NN is in standby state to read from the shared edit log.
    * Used when this NN is in standby state to read from the shared edit log.
@@ -459,6 +469,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private HAContext haContext;
   private HAContext haContext;
 
 
   private final boolean haEnabled;
   private final boolean haEnabled;
+  
+  /**
+   * Whether the namenode is in the middle of starting the active service
+   */
+  private volatile boolean startingActiveService = false;
     
     
   private INodeId inodeId;
   private INodeId inodeId;
   
   
@@ -615,6 +630,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
    */
   FSNamesystem(Configuration conf, FSImage fsImage, boolean ignoreRetryCache)
   FSNamesystem(Configuration conf, FSImage fsImage, boolean ignoreRetryCache)
       throws IOException {
       throws IOException {
+    boolean fair = conf.getBoolean("dfs.namenode.fslock.fair", true);
+    LOG.info("fsLock is fair:" + fair);
+    fsLock = new ReentrantReadWriteLock(fair);
     try {
     try {
       resourceRecheckInterval = conf.getLong(
       resourceRecheckInterval = conf.getLong(
           DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY,
           DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY,
@@ -911,6 +929,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws IOException
    * @throws IOException
    */
    */
   void startActiveServices() throws IOException {
   void startActiveServices() throws IOException {
+    startingActiveService = true;
     LOG.info("Starting services required for active state");
     LOG.info("Starting services required for active state");
     writeLock();
     writeLock();
     try {
     try {
@@ -967,8 +986,19 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       blockManager.getDatanodeManager().setSendCachingCommands(true);
       blockManager.getDatanodeManager().setSendCachingCommands(true);
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
+      startingActiveService = false;
     }
     }
   }
   }
+  
+  /**
+   * @return Whether the namenode is transitioning to active state and is in the
+   *         middle of the {@link #startActiveServices()}
+   */
+  public boolean inTransitionToActive() {
+    return haEnabled && haContext != null
+        && haContext.getState().getServiceState() == HAServiceState.ACTIVE
+        && startingActiveService;
+  }
 
 
   private boolean shouldUseDelegationTokens() {
   private boolean shouldUseDelegationTokens() {
     return UserGroupInformation.isSecurityEnabled() ||
     return UserGroupInformation.isSecurityEnabled() ||
@@ -1063,6 +1093,26 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     }
   }
   }
   
   
+  /**
+   * @throws RetriableException
+   *           If 1) The NameNode is in SafeMode, 2) HA is enabled, and 3)
+   *           NameNode is in active state
+   * @throws SafeModeException
+   *           Otherwise if NameNode is in SafeMode.
+   */
+  private void checkNameNodeSafeMode(String errorMsg)
+      throws RetriableException, SafeModeException {
+    if (isInSafeMode()) {
+      SafeModeException se = new SafeModeException(errorMsg, safeMode);
+      if (haEnabled && haContext != null
+          && haContext.getState().getServiceState() == HAServiceState.ACTIVE) {
+        throw new RetriableException(se);
+      } else {
+        throw se;
+      }
+    }
+  }
+  
   public static Collection<URI> getNamespaceDirs(Configuration conf) {
   public static Collection<URI> getNamespaceDirs(Configuration conf) {
     return getStorageDirs(conf, DFS_NAMENODE_NAME_DIR_KEY);
     return getStorageDirs(conf, DFS_NAMENODE_NAME_DIR_KEY);
   }
   }
@@ -1364,9 +1414,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot set permission for " + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot set permission for " + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       checkOwner(pc, src);
       checkOwner(pc, src);
       dir.setPermission(src, permission);
       dir.setPermission(src, permission);
@@ -1403,9 +1451,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot set owner for " + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot set owner for " + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       checkOwner(pc, src);
       checkOwner(pc, src);
       if (!pc.isSuperUser()) {
       if (!pc.isSuperUser()) {
@@ -1485,8 +1531,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       for (LocatedBlock b : ret.getLocatedBlocks()) {
       for (LocatedBlock b : ret.getLocatedBlocks()) {
         // if safemode & no block locations yet then throw safemodeException
         // if safemode & no block locations yet then throw safemodeException
         if ((b.getLocations() == null) || (b.getLocations().length == 0)) {
         if ((b.getLocations() == null) || (b.getLocations().length == 0)) {
-          throw new SafeModeException("Zero blocklocations for " + src,
-              safeMode);
+          SafeModeException se = new SafeModeException(
+              "Zero blocklocations for " + src, safeMode);
+          if (haEnabled && haContext != null && 
+              haContext.getState().getServiceState() == HAServiceState.ACTIVE) {
+            throw new RetriableException(se);
+          } else {
+            throw se;
+          }
         }
         }
       }
       }
     }
     }
@@ -1633,9 +1685,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot concat " + target, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot concat " + target);
       concatInternal(pc, target, srcs, logRetryCache);
       concatInternal(pc, target, srcs, logRetryCache);
       resultingStat = getAuditFileInfo(target, false);
       resultingStat = getAuditFileInfo(target, false);
     } finally {
     } finally {
@@ -1783,9 +1833,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot set times " + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot set times " + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
 
 
       // Write access is required to set access and modification times
       // Write access is required to set access and modification times
@@ -1812,16 +1860,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   void createSymlink(String target, String link,
   void createSymlink(String target, String link,
       PermissionStatus dirPerms, boolean createParent) 
       PermissionStatus dirPerms, boolean createParent) 
       throws IOException, UnresolvedLinkException {
       throws IOException, UnresolvedLinkException {
-    CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
-    if (cacheEntry != null && cacheEntry.isSuccess()) {
-      return; // Return previous response
-    }
     if (!DFSUtil.isValidName(link)) {
     if (!DFSUtil.isValidName(link)) {
       throw new InvalidPathException("Invalid link name: " + link);
       throw new InvalidPathException("Invalid link name: " + link);
     }
     }
     if (FSDirectory.isReservedName(target)) {
     if (FSDirectory.isReservedName(target)) {
       throw new InvalidPathException("Invalid target name: " + target);
       throw new InvalidPathException("Invalid target name: " + target);
     }
     }
+    CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return; // Return previous response
+    }
     boolean success = false;
     boolean success = false;
     try {
     try {
       createSymlinkInt(target, link, dirPerms, createParent, cacheEntry != null);
       createSymlinkInt(target, link, dirPerms, createParent, cacheEntry != null);
@@ -1848,9 +1896,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot create symlink " + link, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot create symlink " + link);
       link = FSDirectory.resolvePath(link, pathComponents, dir);
       link = FSDirectory.resolvePath(link, pathComponents, dir);
       if (!createParent) {
       if (!createParent) {
         verifyParentDir(link);
         verifyParentDir(link);
@@ -1908,9 +1954,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot set replication for " + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot set replication for " + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       if (isPermissionEnabled) {
       if (isPermissionEnabled) {
         checkPathAccess(pc, src, FsAction.WRITE);
         checkPathAccess(pc, src, FsAction.WRITE);
@@ -2040,9 +2084,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot create file" + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot create file" + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       startFileInternal(pc, src, permissions, holder, clientMachine, create,
       startFileInternal(pc, src, permissions, holder, clientMachine, create,
           overwrite, createParent, replication, blockSize, logRetryCache);
           overwrite, createParent, replication, blockSize, logRetryCache);
@@ -2261,10 +2303,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException(
-            "Cannot recover the lease of " + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot recover the lease of " + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       final INodeFile inode = INodeFile.valueOf(dir.getINode(src), src);
       final INodeFile inode = INodeFile.valueOf(dir.getINode(src), src);
       if (!inode.isUnderConstruction()) {
       if (!inode.isUnderConstruction()) {
@@ -2415,9 +2454,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot append to file" + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot append to file" + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       lb = appendFileInternal(pc, src, holder, clientMachine, logRetryCache);
       lb = appendFileInternal(pc, src, holder, clientMachine, logRetryCache);
     } catch (StandbyException se) {
     } catch (StandbyException se) {
@@ -2464,7 +2501,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * client to "try again later".
    * client to "try again later".
    */
    */
   LocatedBlock getAdditionalBlock(String src, long fileId, String clientName,
   LocatedBlock getAdditionalBlock(String src, long fileId, String clientName,
-      ExtendedBlock previous, HashMap<Node, Node> excludedNodes, 
+      ExtendedBlock previous, Set<Node> excludedNodes, 
       List<String> favoredNodes)
       List<String> favoredNodes)
       throws LeaseExpiredException, NotReplicatedYetException,
       throws LeaseExpiredException, NotReplicatedYetException,
       QuotaExceededException, SafeModeException, UnresolvedLinkException,
       QuotaExceededException, SafeModeException, UnresolvedLinkException,
@@ -2567,9 +2604,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     checkBlock(previous);
     checkBlock(previous);
     onRetryBlock[0] = null;
     onRetryBlock[0] = null;
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
-    if (isInSafeMode()) {
-      throw new SafeModeException("Cannot add block to " + src, safeMode);
-    }
+    checkNameNodeSafeMode("Cannot add block to " + src);
 
 
     // have we exceeded the configured limit of fs objects.
     // have we exceeded the configured limit of fs objects.
     checkFsObjectLimit();
     checkFsObjectLimit();
@@ -2663,7 +2698,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
 
   /** @see NameNode#getAdditionalDatanode(String, ExtendedBlock, DatanodeInfo[], DatanodeInfo[], int, String) */
   /** @see NameNode#getAdditionalDatanode(String, ExtendedBlock, DatanodeInfo[], DatanodeInfo[], int, String) */
   LocatedBlock getAdditionalDatanode(String src, final ExtendedBlock blk,
   LocatedBlock getAdditionalDatanode(String src, final ExtendedBlock blk,
-      final DatanodeInfo[] existings,  final HashMap<Node, Node> excludes,
+      final DatanodeInfo[] existings,  final Set<Node> excludes,
       final int numAdditionalNodes, final String clientName
       final int numAdditionalNodes, final String clientName
       ) throws IOException {
       ) throws IOException {
     //check if the feature is enabled
     //check if the feature is enabled
@@ -2678,10 +2713,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
     try {
       checkOperation(OperationCategory.READ);
       checkOperation(OperationCategory.READ);
       //check safe mode
       //check safe mode
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot add datanode; src=" + src
-            + ", blk=" + blk, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot add datanode; src=" + src + ", blk=" + blk);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
 
 
       //check lease
       //check lease
@@ -2726,10 +2758,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot abandon block " + b +
-                                    " for fle" + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot abandon block " + b + " for fle" + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
 
 
       //
       //
@@ -2812,9 +2841,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot complete file " + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot complete file " + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       success = completeFileInternal(src, holder,
       success = completeFileInternal(src, holder,
         ExtendedBlock.getLocalBlock(last), fileId);
         ExtendedBlock.getLocalBlock(last), fileId);
@@ -2990,9 +3017,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot rename " + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot rename " + src);
       src = FSDirectory.resolvePath(src, srcComponents, dir);
       src = FSDirectory.resolvePath(src, srcComponents, dir);
       dst = FSDirectory.resolvePath(dst, dstComponents, dir);
       dst = FSDirectory.resolvePath(dst, dstComponents, dir);
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
@@ -3042,10 +3067,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /** Rename src to dst */
   /** Rename src to dst */
   void renameTo(String src, String dst, Options.Rename... options)
   void renameTo(String src, String dst, Options.Rename... options)
       throws IOException, UnresolvedLinkException {
       throws IOException, UnresolvedLinkException {
-    CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
-    if (cacheEntry != null && cacheEntry.isSuccess()) {
-      return; // Return previous response
-    }
     if (NameNode.stateChangeLog.isDebugEnabled()) {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: with options - "
       NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: with options - "
           + src + " to " + dst);
           + src + " to " + dst);
@@ -3053,8 +3074,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     if (!DFSUtil.isValidName(dst)) {
     if (!DFSUtil.isValidName(dst)) {
       throw new InvalidPathException("Invalid name: " + dst);
       throw new InvalidPathException("Invalid name: " + dst);
     }
     }
-    FSPermissionChecker pc = getPermissionChecker();
+    final FSPermissionChecker pc = getPermissionChecker();
+    
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
+    CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return; // Return previous response
+    }
     byte[][] srcComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] srcComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] dstComponents = FSDirectory.getPathComponentsForReservedPath(dst);
     byte[][] dstComponents = FSDirectory.getPathComponentsForReservedPath(dst);
     HdfsFileStatus resultingStat = null;
     HdfsFileStatus resultingStat = null;
@@ -3062,9 +3088,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot rename " + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot rename " + src);
       src = FSDirectory.resolvePath(src, srcComponents, dir);
       src = FSDirectory.resolvePath(src, srcComponents, dir);
       dst = FSDirectory.resolvePath(dst, dstComponents, dir);
       dst = FSDirectory.resolvePath(dst, dstComponents, dir);
       renameToInternal(pc, src, dst, cacheEntry != null, options);
       renameToInternal(pc, src, dst, cacheEntry != null, options);
@@ -3170,9 +3194,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot delete " + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot delete " + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       if (!recursive && dir.isNonEmptyDirectory(src)) {
       if (!recursive && dir.isNonEmptyDirectory(src)) {
         throw new IOException(src + " is non empty");
         throw new IOException(src + " is non empty");
@@ -3391,9 +3413,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);   
       checkOperation(OperationCategory.WRITE);   
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot create directory " + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot create directory " + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       status = mkdirsInternal(pc, src, permissions, createParent);
       status = mkdirsInternal(pc, src, permissions, createParent);
       if (status) {
       if (status) {
@@ -3493,9 +3513,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot set quota on " + path, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot set quota on " + path);
       dir.setQuota(path, nsQuota, dsQuota);
       dir.setQuota(path, nsQuota, dsQuota);
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
@@ -3518,9 +3536,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot fsync file " + src, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot fsync file " + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       INodeFileUnderConstruction pendingFile  = checkLease(src, clientName);
       INodeFileUnderConstruction pendingFile  = checkLease(src, clientName);
       if (lastBlockLength > 0) {
       if (lastBlockLength > 0) {
@@ -3724,6 +3740,39 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   BlockInfo getStoredBlock(Block block) {
   BlockInfo getStoredBlock(Block block) {
     return blockManager.getStoredBlock(block);
     return blockManager.getStoredBlock(block);
   }
   }
+  
+  @Override
+  public boolean isInSnapshot(BlockInfoUnderConstruction blockUC) {
+    assert hasReadOrWriteLock();
+    final BlockCollection bc = blockUC.getBlockCollection();
+    if (bc == null || !(bc instanceof INodeFileUnderConstruction)) {
+      return false;
+    }
+
+    INodeFileUnderConstruction inodeUC = (INodeFileUnderConstruction) blockUC
+        .getBlockCollection();
+    String fullName = inodeUC.getName();
+    try {
+      if (fullName != null && fullName.startsWith(Path.SEPARATOR)
+          && dir.getINode(fullName) == inodeUC) {
+        // If file exists in normal path then no need to look in snapshot
+        return false;
+      }
+    } catch (UnresolvedLinkException e) {
+      LOG.error("Error while resolving the link : " + fullName, e);
+      return false;
+    }
+    /*
+     * 1. if bc is an instance of INodeFileUnderConstructionWithSnapshot, and
+     * bc is not in the current fsdirectory tree, bc must represent a snapshot
+     * file. 
+     * 2. if fullName is not an absolute path, bc cannot be existent in the 
+     * current fsdirectory tree. 
+     * 3. if bc is not the current node associated with fullName, bc must be a
+     * snapshot inode.
+     */
+    return true;
+  }
 
 
   void commitBlockSynchronization(ExtendedBlock lastblock,
   void commitBlockSynchronization(ExtendedBlock lastblock,
       long newgenerationstamp, long newlength,
       long newgenerationstamp, long newlength,
@@ -3745,11 +3794,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       // If a DN tries to commit to the standby, the recovery will
       // If a DN tries to commit to the standby, the recovery will
       // fail, and the next retry will succeed on the new NN.
       // fail, and the next retry will succeed on the new NN.
   
   
-      if (isInSafeMode()) {
-        throw new SafeModeException(
-          "Cannot commitBlockSynchronization while in safe mode",
-          safeMode);
-      }
+      checkNameNodeSafeMode(
+          "Cannot commitBlockSynchronization while in safe mode");
       final BlockInfo storedBlock = getStoredBlock(
       final BlockInfo storedBlock = getStoredBlock(
           ExtendedBlock.getLocalBlock(lastblock));
           ExtendedBlock.getLocalBlock(lastblock));
       if (storedBlock == null) {
       if (storedBlock == null) {
@@ -3895,9 +3941,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot renew lease for " + holder, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot renew lease for " + holder);
       leaseManager.renewLease(holder);
       leaseManager.renewLease(holder);
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
@@ -3934,11 +3978,27 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.READ);
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    String startAfterString = new String(startAfter);
     readLock();
     readLock();
     try {
     try {
       checkOperation(OperationCategory.READ);
       checkOperation(OperationCategory.READ);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
 
 
+      // Get file name when startAfter is an INodePath
+      if (FSDirectory.isReservedName(startAfterString)) {
+        byte[][] startAfterComponents = FSDirectory
+            .getPathComponentsForReservedPath(startAfterString);
+        try {
+          String tmp = FSDirectory.resolvePath(src, startAfterComponents, dir);
+          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);
+        }
+      }
+      
       if (isPermissionEnabled) {
       if (isPermissionEnabled) {
         if (dir.isDir(src)) {
         if (dir.isDir(src)) {
           checkPathAccess(pc, src, FsAction.READ_EXECUTE);
           checkPathAccess(pc, src, FsAction.READ_EXECUTE);
@@ -4220,6 +4280,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return this.snapshotManager.getNumSnapshots();
     return this.snapshotManager.getNumSnapshots();
   }
   }
 
 
+  @Override
+  public String getSnapshotStats() {
+    Map<String, Object> info = new HashMap<String, Object>();
+    info.put("SnapshottableDirectories", this.getNumSnapshottableDirs());
+    info.put("Snapshots", this.getNumSnapshots());
+    return JSON.toString(info);
+  }
+
   int getNumberOfDatanodes(DatanodeReportType type) {
   int getNumberOfDatanodes(DatanodeReportType type) {
     readLock();
     readLock();
     try {
     try {
@@ -4259,19 +4327,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws IOException if 
    * @throws IOException if 
    */
    */
   void saveNamespace() throws AccessControlException, IOException {
   void saveNamespace() throws AccessControlException, IOException {
+    checkOperation(OperationCategory.UNCHECKED);
+    checkSuperuserPrivilege();
+    
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return; // Return previous response
       return; // Return previous response
     }
     }
-    checkSuperuserPrivilege();
-    checkOperation(OperationCategory.UNCHECKED);
     boolean success = false;
     boolean success = false;
     readLock();
     readLock();
     try {
     try {
       checkOperation(OperationCategory.UNCHECKED);
       checkOperation(OperationCategory.UNCHECKED);
       if (!isInSafeMode()) {
       if (!isInSafeMode()) {
-        throw new IOException("Safe mode should be turned ON " +
-                              "in order to create namespace image.");
+        throw new IOException("Safe mode should be turned ON "
+            + "in order to create namespace image.");
       }
       }
       getFSImage().saveNamespace(this);
       getFSImage().saveNamespace(this);
       success = true;
       success = true;
@@ -4348,7 +4417,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * replicas, and calculates the ratio of safe blocks to the total number
    * replicas, and calculates the ratio of safe blocks to the total number
    * of blocks in the system, which is the size of blocks in
    * of blocks in the system, which is the size of blocks in
    * {@link FSNamesystem#blockManager}. When the ratio reaches the
    * {@link FSNamesystem#blockManager}. When the ratio reaches the
-   * {@link #threshold} it starts the {@link SafeModeMonitor} daemon in order
+   * {@link #threshold} it starts the SafeModeMonitor daemon in order
    * to monitor whether the safe mode {@link #extension} is passed.
    * to monitor whether the safe mode {@link #extension} is passed.
    * Then it leaves safe mode and destroys itself.
    * Then it leaves safe mode and destroys itself.
    * <p>
    * <p>
@@ -4356,10 +4425,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * not tracked because the name node is not intended to leave safe mode
    * not tracked because the name node is not intended to leave safe mode
    * automatically in the case.
    * automatically in the case.
    *
    *
-   * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction)
-   * @see SafeModeMonitor
+   * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction, boolean)
    */
    */
-  class SafeModeInfo {
+  public class SafeModeInfo {
     // configuration fields
     // configuration fields
     /** Safe mode threshold condition %.*/
     /** Safe mode threshold condition %.*/
     private double threshold;
     private double threshold;
@@ -4572,7 +4640,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
      */
      */
     private boolean needEnter() {
     private boolean needEnter() {
       return (threshold != 0 && blockSafe < blockThreshold) ||
       return (threshold != 0 && blockSafe < blockThreshold) ||
-        (getNumLiveDataNodes() < datanodeThreshold) ||
+        (datanodeThreshold != 0 && getNumLiveDataNodes() < datanodeThreshold) ||
         (!nameNodeHasResourcesAvailable());
         (!nameNodeHasResourcesAvailable());
     }
     }
       
       
@@ -5101,9 +5169,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.JOURNAL);
       checkOperation(OperationCategory.JOURNAL);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Log not rolled", safeMode);
-      }
+      checkNameNodeSafeMode("Log not rolled");
       LOG.info("Roll Edit Log from " + Server.getRemoteAddress());
       LOG.info("Roll Edit Log from " + Server.getRemoteAddress());
       return getFSImage().rollEditLog();
       return getFSImage().rollEditLog();
     } finally {
     } finally {
@@ -5124,9 +5190,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
     try {
       checkOperation(OperationCategory.CHECKPOINT);
       checkOperation(OperationCategory.CHECKPOINT);
 
 
-      if (isInSafeMode()) {
-        throw new SafeModeException("Checkpoint not started", safeMode);
-      }
+      checkNameNodeSafeMode("Checkpoint not started");
       LOG.info("Start checkpoint for " + backupNode.getAddress());
       LOG.info("Start checkpoint for " + backupNode.getAddress());
       cmd = getFSImage().startCheckpoint(backupNode, activeNamenode);
       cmd = getFSImage().startCheckpoint(backupNode, activeNamenode);
       getEditLog().logSync();
       getEditLog().logSync();
@@ -5150,19 +5214,17 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   
   
   void endCheckpoint(NamenodeRegistration registration,
   void endCheckpoint(NamenodeRegistration registration,
                             CheckpointSignature sig) throws IOException {
                             CheckpointSignature sig) throws IOException {
+    checkOperation(OperationCategory.CHECKPOINT);
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return; // Return previous response
       return; // Return previous response
     }
     }
-    checkOperation(OperationCategory.CHECKPOINT);
     boolean success = false;
     boolean success = false;
     readLock();
     readLock();
     try {
     try {
       checkOperation(OperationCategory.CHECKPOINT);
       checkOperation(OperationCategory.CHECKPOINT);
 
 
-      if (isInSafeMode()) {
-        throw new SafeModeException("Checkpoint not ended", safeMode);
-      }
+      checkNameNodeSafeMode("Checkpoint not ended");
       LOG.info("End checkpoint for " + registration.getAddress());
       LOG.info("End checkpoint for " + registration.getAddress());
       getFSImage().endCheckpoint(sig);
       getFSImage().endCheckpoint(sig);
       success = true;
       success = true;
@@ -5263,7 +5325,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
   /**
    * Get the total number of objects in the system. 
    * Get the total number of objects in the system. 
    */
    */
-  long getMaxObjects() {
+  @Override // FSNamesystemMBean
+  public long getMaxObjects() {
     return maxFsObjects;
     return maxFsObjects;
   }
   }
 
 
@@ -5408,7 +5471,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   @Override // FSNamesystemMBean
   @Override // FSNamesystemMBean
   public int getNumDecomDeadDataNodes() {
   public int getNumDecomDeadDataNodes() {
     final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
     final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
-    getBlockManager().getDatanodeManager().fetchDatanodes(dead, null, true);
+    getBlockManager().getDatanodeManager().fetchDatanodes(null, dead, true);
     int deadDecommissioned = 0;
     int deadDecommissioned = 0;
     for (DatanodeDescriptor node : dead) {
     for (DatanodeDescriptor node : dead) {
       deadDecommissioned += node.isDecommissioned() ? 1 : 0;
       deadDecommissioned += node.isDecommissioned() ? 1 : 0;
@@ -5416,6 +5479,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return deadDecommissioned;
     return deadDecommissioned;
   }
   }
 
 
+  @Override // FSNamesystemMBean
+  public int getNumDecommissioningDataNodes() {
+    return getBlockManager().getDatanodeManager().getDecommissioningNodes()
+        .size();
+  }
+
   @Override // FSNamesystemMBean
   @Override // FSNamesystemMBean
   @Metric({"StaleDataNodes", 
   @Metric({"StaleDataNodes", 
     "Number of datanodes marked stale due to delayed heartbeat"})
     "Number of datanodes marked stale due to delayed heartbeat"})
@@ -5514,10 +5583,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   long nextGenerationStamp(boolean legacyBlock)
   long nextGenerationStamp(boolean legacyBlock)
       throws IOException, SafeModeException {
       throws IOException, SafeModeException {
     assert hasWriteLock();
     assert hasWriteLock();
-    if (isInSafeMode()) {
-      throw new SafeModeException(
-          "Cannot get next generation stamp", safeMode);
-    }
+    checkNameNodeSafeMode("Cannot get next generation stamp");
 
 
     long gs;
     long gs;
     if (legacyBlock) {
     if (legacyBlock) {
@@ -5570,12 +5636,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
   /**
    * Increments, logs and then returns the block ID
    * Increments, logs and then returns the block ID
    */
    */
-  private long nextBlockId() throws SafeModeException {
+  private long nextBlockId() throws IOException {
     assert hasWriteLock();
     assert hasWriteLock();
-    if (isInSafeMode()) {
-      throw new SafeModeException(
-          "Cannot get next block ID", safeMode);
-    }
+    checkNameNodeSafeMode("Cannot get next block ID");
     final long blockId = blockIdGenerator.nextValue();
     final long blockId = blockIdGenerator.nextValue();
     getEditLog().logAllocateBlockId(blockId);
     getEditLog().logAllocateBlockId(blockId);
     // NB: callers sync the log
     // NB: callers sync the log
@@ -5585,10 +5648,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private INodeFileUnderConstruction checkUCBlock(ExtendedBlock block,
   private INodeFileUnderConstruction checkUCBlock(ExtendedBlock block,
       String clientName) throws IOException {
       String clientName) throws IOException {
     assert hasWriteLock();
     assert hasWriteLock();
-    if (isInSafeMode()) {
-      throw new SafeModeException("Cannot get a new generation stamp and an " +
-                                "access token for block " + block, safeMode);
-    }
+    checkNameNodeSafeMode("Cannot get a new generation stamp and an "
+        + "access token for block " + block);
     
     
     // check stored block state
     // check stored block state
     BlockInfo storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
     BlockInfo storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
@@ -5686,11 +5747,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   void updatePipeline(String clientName, ExtendedBlock oldBlock, 
   void updatePipeline(String clientName, ExtendedBlock oldBlock, 
       ExtendedBlock newBlock, DatanodeID[] newNodes)
       ExtendedBlock newBlock, DatanodeID[] newNodes)
       throws IOException {
       throws IOException {
+    checkOperation(OperationCategory.WRITE);
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return; // Return previous response
       return; // Return previous response
     }
     }
-    checkOperation(OperationCategory.WRITE);
     LOG.info("updatePipeline(block=" + oldBlock
     LOG.info("updatePipeline(block=" + oldBlock
              + ", newGenerationStamp=" + newBlock.getGenerationStamp()
              + ", newGenerationStamp=" + newBlock.getGenerationStamp()
              + ", newLength=" + newBlock.getNumBytes()
              + ", newLength=" + newBlock.getNumBytes()
@@ -5701,9 +5762,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     boolean success = false;
     boolean success = false;
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Pipeline not updated", safeMode);
-      }
+      checkNameNodeSafeMode("Pipeline not updated");
       assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and "
       assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and "
         + oldBlock + " has different block identifier";
         + oldBlock + " has different block identifier";
       updatePipelineInternal(clientName, oldBlock, newBlock, newNodes,
       updatePipelineInternal(clientName, oldBlock, newBlock, newNodes,
@@ -5963,9 +6022,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot issue delegation token", safeMode);
-      }
+      checkNameNodeSafeMode("Cannot issue delegation token");
       if (!isAllowedDelegationTokenOp()) {
       if (!isAllowedDelegationTokenOp()) {
         throw new IOException(
         throw new IOException(
           "Delegation Token can be issued only with kerberos or web authentication");
           "Delegation Token can be issued only with kerberos or web authentication");
@@ -6010,9 +6067,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
 
 
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot renew delegation token", safeMode);
-      }
+      checkNameNodeSafeMode("Cannot renew delegation token");
       if (!isAllowedDelegationTokenOp()) {
       if (!isAllowedDelegationTokenOp()) {
         throw new IOException(
         throw new IOException(
             "Delegation Token can be renewed only with kerberos or web authentication");
             "Delegation Token can be renewed only with kerberos or web authentication");
@@ -6043,9 +6098,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
 
 
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot cancel delegation token", safeMode);
-      }
+      checkNameNodeSafeMode("Cannot cancel delegation token");
       String canceller = getRemoteUser().getUserName();
       String canceller = getRemoteUser().getUserName();
       DelegationTokenIdentifier id = dtSecretManager
       DelegationTokenIdentifier id = dtSecretManager
         .cancelToken(token, canceller);
         .cancelToken(token, canceller);
@@ -6271,14 +6324,25 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
     final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
     blockManager.getDatanodeManager().fetchDatanodes(live, null, true);
     blockManager.getDatanodeManager().fetchDatanodes(live, null, true);
     for (DatanodeDescriptor node : live) {
     for (DatanodeDescriptor node : live) {
-      final Map<String, Object> innerinfo = new HashMap<String, Object>();
-      innerinfo.put("lastContact", getLastContact(node));
-      innerinfo.put("usedSpace", getDfsUsed(node));
-      innerinfo.put("adminState", node.getAdminState().toString());
-      innerinfo.put("nonDfsUsedSpace", node.getNonDfsUsed());
-      innerinfo.put("capacity", node.getCapacity());
-      innerinfo.put("numBlocks", node.numBlocks());
-      innerinfo.put("version", node.getSoftwareVersion());
+      Map<String, Object> innerinfo = ImmutableMap.<String, Object>builder()
+          .put("infoAddr", node.getInfoAddr())
+          .put("infoSecureAddr", node.getInfoSecureAddr())
+          .put("xferaddr", node.getXferAddr())
+          .put("lastContact", getLastContact(node))
+          .put("usedSpace", getDfsUsed(node))
+          .put("adminState", node.getAdminState().toString())
+          .put("nonDfsUsedSpace", node.getNonDfsUsed())
+          .put("capacity", node.getCapacity())
+          .put("numBlocks", node.numBlocks())
+          .put("version", node.getSoftwareVersion())
+          .put("used", node.getDfsUsed())
+          .put("remaining", node.getRemaining())
+          .put("blockScheduled", node.getBlocksScheduled())
+          .put("blockPoolUsed", node.getBlockPoolUsed())
+          .put("blockPoolUsedPercent", node.getBlockPoolUsedPercent())
+          .put("volfails", node.getVolumeFailures())
+          .build();
+
       info.put(node.getHostName(), innerinfo);
       info.put(node.getHostName(), innerinfo);
     }
     }
     return JSON.toString(info);
     return JSON.toString(info);
@@ -6295,9 +6359,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
     final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
     blockManager.getDatanodeManager().fetchDatanodes(null, dead, true);
     blockManager.getDatanodeManager().fetchDatanodes(null, dead, true);
     for (DatanodeDescriptor node : dead) {
     for (DatanodeDescriptor node : dead) {
-      final Map<String, Object> innerinfo = new HashMap<String, Object>();
-      innerinfo.put("lastContact", getLastContact(node));
-      innerinfo.put("decommissioned", node.isDecommissioned());
+      Map<String, Object> innerinfo = ImmutableMap.<String, Object>builder()
+          .put("lastContact", getLastContact(node))
+          .put("decommissioned", node.isDecommissioned())
+          .put("xferaddr", node.getXferAddr())
+          .build();
       info.put(node.getHostName(), innerinfo);
       info.put(node.getHostName(), innerinfo);
     }
     }
     return JSON.toString(info);
     return JSON.toString(info);
@@ -6314,13 +6380,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     final List<DatanodeDescriptor> decomNodeList = blockManager.getDatanodeManager(
     final List<DatanodeDescriptor> decomNodeList = blockManager.getDatanodeManager(
         ).getDecommissioningNodes();
         ).getDecommissioningNodes();
     for (DatanodeDescriptor node : decomNodeList) {
     for (DatanodeDescriptor node : decomNodeList) {
-      final Map<String, Object> innerinfo = new HashMap<String, Object>();
-      innerinfo.put("underReplicatedBlocks", node.decommissioningStatus
-          .getUnderReplicatedBlocks());
-      innerinfo.put("decommissionOnlyReplicas", node.decommissioningStatus
-          .getDecommissionOnlyReplicas());
-      innerinfo.put("underReplicateInOpenFiles", node.decommissioningStatus
-          .getUnderReplicatedInOpenFiles());
+      Map<String, Object> innerinfo = ImmutableMap
+          .<String, Object> builder()
+          .put("xferaddr", node.getXferAddr())
+          .put("underReplicatedBlocks",
+              node.decommissioningStatus.getUnderReplicatedBlocks())
+          .put("decommissionOnlyReplicas",
+              node.decommissioningStatus.getDecommissionOnlyReplicas())
+          .put("underReplicateInOpenFiles",
+              node.decommissioningStatus.getUnderReplicatedInOpenFiles())
+          .build();
       info.put(node.getHostName(), innerinfo);
       info.put(node.getHostName(), innerinfo);
     }
     }
     return JSON.toString(info);
     return JSON.toString(info);
@@ -6514,11 +6583,17 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * Verifies that the given identifier and password are valid and match.
    * Verifies that the given identifier and password are valid and match.
    * @param identifier Token identifier.
    * @param identifier Token identifier.
    * @param password Password in the token.
    * @param password Password in the token.
-   * @throws InvalidToken
    */
    */
   public synchronized void verifyToken(DelegationTokenIdentifier identifier,
   public synchronized void verifyToken(DelegationTokenIdentifier identifier,
-      byte[] password) throws InvalidToken {
-    getDelegationTokenSecretManager().verifyToken(identifier, password);
+      byte[] password) throws InvalidToken, RetriableException {
+    try {
+      getDelegationTokenSecretManager().verifyToken(identifier, password);
+    } catch (InvalidToken it) {
+      if (inTransitionToActive()) {
+        throw new RetriableException(it);
+      }
+      throw it;
+    }
   }
   }
   
   
   @Override
   @Override
@@ -6535,6 +6610,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return editLogTailer;
     return editLogTailer;
   }
   }
   
   
+  @VisibleForTesting
+  public void setEditLogTailerForTests(EditLogTailer tailer) {
+    this.editLogTailer = tailer;
+  }
+  
   @VisibleForTesting
   @VisibleForTesting
   void setFsLockForTests(ReentrantReadWriteLock lock) {
   void setFsLockForTests(ReentrantReadWriteLock lock) {
     this.fsLock = lock;
     this.fsLock = lock;
@@ -6570,10 +6650,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot allow snapshot for " + path,
-            safeMode);
-      }
+      checkNameNodeSafeMode("Cannot allow snapshot for " + path);
       checkSuperuserPrivilege();
       checkSuperuserPrivilege();
 
 
       dir.writeLock();
       dir.writeLock();
@@ -6598,10 +6675,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot disallow snapshot for " + path,
-            safeMode);
-      }
+      checkNameNodeSafeMode("Cannot disallow snapshot for " + path);
       checkSuperuserPrivilege();
       checkSuperuserPrivilege();
 
 
       dir.writeLock();
       dir.writeLock();
@@ -6628,20 +6702,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
    */
   String createSnapshot(String snapshotRoot, String snapshotName)
   String createSnapshot(String snapshotRoot, String snapshotName)
       throws SafeModeException, IOException {
       throws SafeModeException, IOException {
+    checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     CacheEntryWithPayload cacheEntry = RetryCache.waitForCompletion(retryCache,
     CacheEntryWithPayload cacheEntry = RetryCache.waitForCompletion(retryCache,
         null);
         null);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return (String) cacheEntry.getPayload();
       return (String) cacheEntry.getPayload();
     }
     }
-    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     writeLock();
     String snapshotPath = null;
     String snapshotPath = null;
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot create snapshot for "
-            + snapshotRoot, safeMode);
-      }
+      checkNameNodeSafeMode("Cannot create snapshot for " + snapshotRoot);
       if (isPermissionEnabled) {
       if (isPermissionEnabled) {
         checkOwner(pc, snapshotRoot);
         checkOwner(pc, snapshotRoot);
       }
       }
@@ -6680,19 +6752,17 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
    */
   void renameSnapshot(String path, String snapshotOldName,
   void renameSnapshot(String path, String snapshotOldName,
       String snapshotNewName) throws SafeModeException, IOException {
       String snapshotNewName) throws SafeModeException, IOException {
+    checkOperation(OperationCategory.WRITE);
+    final FSPermissionChecker pc = getPermissionChecker();
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return; // Return previous response
       return; // Return previous response
     }
     }
-    final FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     writeLock();
     boolean success = false;
     boolean success = false;
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException("Cannot rename snapshot for " + path,
-            safeMode);
-      }
+      checkNameNodeSafeMode("Cannot rename snapshot for " + path);
       if (isPermissionEnabled) {
       if (isPermissionEnabled) {
         checkOwner(pc, path);
         checkOwner(pc, path);
       }
       }
@@ -6725,10 +6795,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException {
       throws IOException {
     SnapshottableDirectoryStatus[] status = null;
     SnapshottableDirectoryStatus[] status = null;
+    final FSPermissionChecker checker = getPermissionChecker();
     readLock();
     readLock();
     try {
     try {
       checkOperation(OperationCategory.READ);
       checkOperation(OperationCategory.READ);
-      FSPermissionChecker checker = getPermissionChecker();
       final String user = checker.isSuperUser()? null : checker.getUser();
       final String user = checker.isSuperUser()? null : checker.getUser();
       status = snapshotManager.getSnapshottableDirListing(user);
       status = snapshotManager.getSnapshottableDirListing(user);
     } finally {
     } finally {
@@ -6796,21 +6866,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
    */
   void deleteSnapshot(String snapshotRoot, String snapshotName)
   void deleteSnapshot(String snapshotRoot, String snapshotName)
       throws SafeModeException, IOException {
       throws SafeModeException, IOException {
+    checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
     final FSPermissionChecker pc = getPermissionChecker();
+    
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
     if (cacheEntry != null && cacheEntry.isSuccess()) {
       return; // Return previous response
       return; // Return previous response
     }
     }
     boolean success = false;
     boolean success = false;
-    checkOperation(OperationCategory.WRITE);
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
-      if (isInSafeMode()) {
-        throw new SafeModeException(
-            "Cannot delete snapshot for " + snapshotRoot, safeMode);
+      checkNameNodeSafeMode("Cannot delete snapshot for " + snapshotRoot);
+      if (isPermissionEnabled) {
+        checkOwner(pc, snapshotRoot);
       }
       }
-      checkOwner(pc, snapshotRoot);
 
 
       BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
       BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
       List<INode> removedINodes = new ChunkedArrayList<INode>();
       List<INode> removedINodes = new ChunkedArrayList<INode>();
@@ -7113,8 +7183,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           }
           }
           sb.append(trackingId);
           sb.append(trackingId);
         }
         }
-        auditLog.info(sb);
+        logAuditMessage(sb.toString());
       }
       }
     }
     }
+    public void logAuditMessage(String message) {
+      auditLog.info(message);
+    }
   }
   }
 }
 }

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

@@ -57,9 +57,14 @@ public class FileChecksumServlets {
       final String hostname = host instanceof DatanodeInfo 
       final String hostname = host instanceof DatanodeInfo 
           ? ((DatanodeInfo)host).getHostName() : host.getIpAddr();
           ? ((DatanodeInfo)host).getHostName() : host.getIpAddr();
       final String scheme = request.getScheme();
       final String scheme = request.getScheme();
-      final int port = "https".equals(scheme)
-          ? (Integer)getServletContext().getAttribute(DFSConfigKeys.DFS_DATANODE_HTTPS_PORT_KEY)
-          : host.getInfoPort();
+      int port = host.getInfoPort();
+      if ("https".equals(scheme)) {
+        final Integer portObject = (Integer) getServletContext().getAttribute(
+            DFSConfigKeys.DFS_DATANODE_HTTPS_PORT_KEY);
+        if (portObject != null) {
+          port = portObject;
+        }
+      }
       final String encodedPath = ServletUtil.getRawPath(request, "/fileChecksum");
       final String encodedPath = ServletUtil.getRawPath(request, "/fileChecksum");
 
 
       String dtParam = "";
       String dtParam = "";

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

@@ -61,9 +61,14 @@ public class FileDataServlet extends DfsServlet {
     } else {
     } else {
       hostname = host.getIpAddr();
       hostname = host.getIpAddr();
     }
     }
-    final int port = "https".equals(scheme)
-      ? (Integer)getServletContext().getAttribute(DFSConfigKeys.DFS_DATANODE_HTTPS_PORT_KEY)
-      : host.getInfoPort();
+    int port = host.getInfoPort();
+    if ("https".equals(scheme)) {
+      final Integer portObject = (Integer) getServletContext().getAttribute(
+          DFSConfigKeys.DFS_DATANODE_HTTPS_PORT_KEY);
+      if (portObject != null) {
+        port = portObject;
+      }
+    }
 
 
     String dtParam = "";
     String dtParam = "";
     if (dt != null) {
     if (dt != null) {

+ 13 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java

@@ -52,6 +52,7 @@ public class NameNodeHttpServer {
   private final NameNode nn;
   private final NameNode nn;
   
   
   private InetSocketAddress httpAddress;
   private InetSocketAddress httpAddress;
+  private InetSocketAddress httpsAddress;
   private InetSocketAddress bindAddress;
   private InetSocketAddress bindAddress;
   
   
   public static final String NAMENODE_ADDRESS_ATTRIBUTE_KEY = "name.node.address";
   public static final String NAMENODE_ADDRESS_ATTRIBUTE_KEY = "name.node.address";
@@ -99,14 +100,15 @@ public class NameNodeHttpServer {
     boolean certSSL = conf.getBoolean(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY, false);
     boolean certSSL = conf.getBoolean(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY, false);
     if (certSSL) {
     if (certSSL) {
       boolean needClientAuth = conf.getBoolean("dfs.https.need.client.auth", false);
       boolean needClientAuth = conf.getBoolean("dfs.https.need.client.auth", false);
-      InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(infoHost + ":" + conf.get(
-        DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY, infoHost + ":" + 0));
+      httpsAddress = NetUtils.createSocketAddr(conf.get(
+          DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY,
+          DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT));
+
       Configuration sslConf = new Configuration(false);
       Configuration sslConf = new Configuration(false);
-      if (certSSL) {
-        sslConf.addResource(conf.get(DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
-                                     "ssl-server.xml"));
-      }
-      httpServer.addSslListener(secInfoSocAddr, sslConf, needClientAuth);
+      sslConf.addResource(conf.get(
+          DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
+          DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT));
+      httpServer.addSslListener(httpsAddress, sslConf, needClientAuth);
       // assume same ssl port for all datanodes
       // assume same ssl port for all datanodes
       InetSocketAddress datanodeSslPort = NetUtils.createSocketAddr(conf.get(
       InetSocketAddress datanodeSslPort = NetUtils.createSocketAddr(conf.get(
         DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY, infoHost + ":" + 50475));
         DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY, infoHost + ":" + 50475));
@@ -163,6 +165,10 @@ public class NameNodeHttpServer {
     return httpAddress;
     return httpAddress;
   }
   }
 
 
+  public InetSocketAddress getHttpsAddress() {
+    return httpsAddress;
+  }
+
   /**
   /**
    * Sets fsimage for use by servlets.
    * Sets fsimage for use by servlets.
    * 
    * 

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

@@ -29,8 +29,9 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
-import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.List;
+import java.util.Set;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -553,11 +554,11 @@ class NameNodeRpcServer implements NamenodeProtocols {
       stateChangeLog.debug("*BLOCK* NameNode.addBlock: file " + src
       stateChangeLog.debug("*BLOCK* NameNode.addBlock: file " + src
           + " fileId=" + fileId + " for " + clientName);
           + " fileId=" + fileId + " for " + clientName);
     }
     }
-    HashMap<Node, Node> excludedNodesSet = null;
+    Set<Node> excludedNodesSet = null;
     if (excludedNodes != null) {
     if (excludedNodes != null) {
-      excludedNodesSet = new HashMap<Node, Node>(excludedNodes.length);
+      excludedNodesSet = new HashSet<Node>(excludedNodes.length);
       for (Node node : excludedNodes) {
       for (Node node : excludedNodes) {
-        excludedNodesSet.put(node, node);
+        excludedNodesSet.add(node);
       }
       }
     }
     }
     List<String> favoredNodesList = (favoredNodes == null) ? null
     List<String> favoredNodesList = (favoredNodes == null) ? null
@@ -585,11 +586,11 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
 
     metrics.incrGetAdditionalDatanodeOps();
     metrics.incrGetAdditionalDatanodeOps();
 
 
-    HashMap<Node, Node> excludeSet = null;
+    Set<Node> excludeSet = null;
     if (excludes != null) {
     if (excludes != null) {
-      excludeSet = new HashMap<Node, Node>(excludes.length);
+      excludeSet = new HashSet<Node>(excludes.length);
       for (Node node : excludes) {
       for (Node node : excludes) {
-        excludeSet.put(node, node);
+        excludeSet.add(node);
       }
       }
     }
     }
     return namesystem.getAdditionalDatanode(src, blk,
     return namesystem.getAdditionalDatanode(src, blk,

+ 89 - 53
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java

@@ -30,6 +30,8 @@ import java.net.URLEncoder;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
@@ -101,6 +103,10 @@ class NamenodeJspHelper {
   }
   }
 
 
   static String getRollingUpgradeText(FSNamesystem fsn) {
   static String getRollingUpgradeText(FSNamesystem fsn) {
+    if (fsn == null) {
+      return "";
+    }
+
     DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
     DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
     Map<String, Integer> list = dm.getDatanodesSoftwareVersions();
     Map<String, Integer> list = dm.getDatanodesSoftwareVersions();
     if(list.size() > 1) {
     if(list.size() > 1) {
@@ -203,6 +209,20 @@ class NamenodeJspHelper {
     return "";
     return "";
   }
   }
 
 
+  static void generateSnapshotReport(JspWriter out, FSNamesystem fsn)
+      throws IOException {
+    if (fsn == null) {
+      return;
+    }
+    out.println("<div id=\"snapshotstats\"><div class=\"dfstable\">"
+        + "<table class=\"storage\" title=\"Snapshot Summary\">\n"
+        + "<thead><tr><td><b>Snapshottable directories</b></td>"
+        + "<td><b>Snapshotted directories</b></td></tr></thead>");
+
+    out.println(String.format("<td>%d</td><td>%d</td>", fsn.getNumSnapshottableDirs(), fsn.getNumSnapshots()));
+    out.println("</table></div></div>");
+  }
+
   static class HealthJsp {
   static class HealthJsp {
     private int rowNum = 0;
     private int rowNum = 0;
     private int colNum = 0;
     private int colNum = 0;
@@ -636,25 +656,22 @@ class NamenodeJspHelper {
         .getAttribute(JspHelper.CURRENT_CONF);
         .getAttribute(JspHelper.CURRENT_CONF);
     // We can't redirect if there isn't a DN to redirect to.
     // We can't redirect if there isn't a DN to redirect to.
     // Lets instead show a proper error message.
     // Lets instead show a proper error message.
-    if (nn.getNamesystem().getNumLiveDataNodes() < 1) {
+    FSNamesystem fsn = nn.getNamesystem();
+
+    DatanodeID datanode = null;
+    if (fsn != null && fsn.getNumLiveDataNodes() >= 1) {
+      datanode = getRandomDatanode(nn);
+    }
+
+    if (datanode == null) {
       throw new IOException("Can't browse the DFS since there are no " +
       throw new IOException("Can't browse the DFS since there are no " +
           "live nodes available to redirect to.");
           "live nodes available to redirect to.");
     }
     }
-    final DatanodeID datanode = getRandomDatanode(nn);;
+
     UserGroupInformation ugi = JspHelper.getUGI(context, request, conf);
     UserGroupInformation ugi = JspHelper.getUGI(context, request, conf);
+    // if the user is defined, get a delegation token and stringify it
     String tokenString = getDelegationToken(
     String tokenString = getDelegationToken(
         nn.getRpcServer(), request, conf, ugi);
         nn.getRpcServer(), request, conf, ugi);
-    // if the user is defined, get a delegation token and stringify it
-    final String redirectLocation;
-    final String nodeToRedirect;
-    int redirectPort;
-    if (datanode != null) {
-      nodeToRedirect = datanode.getIpAddr();
-      redirectPort = datanode.getInfoPort();
-    } else {
-      nodeToRedirect = nn.getHttpAddress().getHostName();
-      redirectPort = nn.getHttpAddress().getPort();
-    }
 
 
     InetSocketAddress rpcAddr = nn.getNameNodeAddress();
     InetSocketAddress rpcAddr = nn.getNameNodeAddress();
     String rpcHost = rpcAddr.getAddress().isAnyLocalAddress()
     String rpcHost = rpcAddr.getAddress().isAnyLocalAddress()
@@ -662,16 +679,31 @@ class NamenodeJspHelper {
       : rpcAddr.getAddress().getHostAddress();
       : rpcAddr.getAddress().getHostAddress();
     String addr = rpcHost + ":" + rpcAddr.getPort();
     String addr = rpcHost + ":" + rpcAddr.getPort();
 
 
-    String fqdn = InetAddress.getByName(nodeToRedirect).getCanonicalHostName();
-    redirectLocation = HttpConfig.getSchemePrefix() + fqdn + ":" + redirectPort
+    final String redirectLocation =
+        JspHelper.Url.url(request.getScheme(), datanode)
         + "/browseDirectory.jsp?namenodeInfoPort="
         + "/browseDirectory.jsp?namenodeInfoPort="
-        + nn.getHttpAddress().getPort() + "&dir=/"
+        + request.getServerPort() + "&dir=/"
         + (tokenString == null ? "" :
         + (tokenString == null ? "" :
            JspHelper.getDelegationTokenUrlParam(tokenString))
            JspHelper.getDelegationTokenUrlParam(tokenString))
         + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, addr);
         + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, addr);
+
     resp.sendRedirect(redirectLocation);
     resp.sendRedirect(redirectLocation);
   }
   }
 
 
+  /**
+   * Returns a descriptive label for the running NameNode.  If the NameNode has
+   * initialized to the point of running its RPC server, then this label consists
+   * of the host and port of the RPC server.  Otherwise, the label is a message
+   * stating that the NameNode is still initializing.
+   * 
+   * @param nn NameNode to describe
+   * @return String NameNode label
+   */
+  static String getNameNodeLabel(NameNode nn) {
+    return nn.getRpcServer() != null ? nn.getNameNodeAddressHostPortString() :
+      "initializing";
+  }
+
   static class NodeListJsp {
   static class NodeListJsp {
     private int rowNum = 0;
     private int rowNum = 0;
 
 
@@ -709,12 +741,11 @@ class NamenodeJspHelper {
     }
     }
 
 
     private void generateNodeDataHeader(JspWriter out, DatanodeDescriptor d,
     private void generateNodeDataHeader(JspWriter out, DatanodeDescriptor d,
-        String suffix, boolean alive, int nnHttpPort, String nnaddr)
+        String suffix, boolean alive, int nnInfoPort, String nnaddr, String scheme)
         throws IOException {
         throws IOException {
       // from nn_browsedfscontent.jsp:
       // from nn_browsedfscontent.jsp:
-      String url = HttpConfig.getSchemePrefix() + d.getHostName() + ":"
-          + d.getInfoPort()
-          + "/browseDirectory.jsp?namenodeInfoPort=" + nnHttpPort + "&dir="
+      String url = "///" + JspHelper.Url.authority(scheme, d)
+          + "/browseDirectory.jsp?namenodeInfoPort=" + nnInfoPort + "&dir="
           + URLEncoder.encode("/", "UTF-8")
           + URLEncoder.encode("/", "UTF-8")
           + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnaddr);
           + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnaddr);
 
 
@@ -731,9 +762,9 @@ class NamenodeJspHelper {
     }
     }
 
 
     void generateDecommissioningNodeData(JspWriter out, DatanodeDescriptor d,
     void generateDecommissioningNodeData(JspWriter out, DatanodeDescriptor d,
-        String suffix, boolean alive, int nnHttpPort, String nnaddr)
+        String suffix, boolean alive, int nnInfoPort, String nnaddr, String scheme)
         throws IOException {
         throws IOException {
-      generateNodeDataHeader(out, d, suffix, alive, nnHttpPort, nnaddr);
+      generateNodeDataHeader(out, d, suffix, alive, nnInfoPort, nnaddr, scheme);
       if (!alive) {
       if (!alive) {
         return;
         return;
       }
       }
@@ -757,7 +788,7 @@ class NamenodeJspHelper {
     }
     }
     
     
     void generateNodeData(JspWriter out, DatanodeDescriptor d, String suffix,
     void generateNodeData(JspWriter out, DatanodeDescriptor d, String suffix,
-        boolean alive, int nnHttpPort, String nnaddr) throws IOException {
+        boolean alive, int nnInfoPort, String nnaddr, String scheme) throws IOException {
       /*
       /*
        * Say the datanode is dn1.hadoop.apache.org with ip 192.168.0.5 we use:
        * Say the datanode is dn1.hadoop.apache.org with ip 192.168.0.5 we use:
        * 1) d.getHostName():d.getPort() to display. Domain and port are stripped
        * 1) d.getHostName():d.getPort() to display. Domain and port are stripped
@@ -769,10 +800,14 @@ class NamenodeJspHelper {
        * interact with datanodes.
        * interact with datanodes.
        */
        */
 
 
-      generateNodeDataHeader(out, d, suffix, alive, nnHttpPort, nnaddr);
+      generateNodeDataHeader(out, d, suffix, alive, nnInfoPort, nnaddr, scheme);
+      long currentTime = Time.now();
+      long timestamp = d.getLastUpdate();
       if (!alive) {
       if (!alive) {
-        out.print("<td class=\"decommissioned\"> " + 
-            d.isDecommissioned() + "\n");
+        out.print("<td class=\"lastcontact\"> "
+            + new Date(timestamp) 
+            + "<td class=\"decommissioned\"> "
+            + d.isDecommissioned() + "\n");
         return;
         return;
       }
       }
 
 
@@ -785,9 +820,6 @@ class NamenodeJspHelper {
       String percentRemaining = fraction2String(d.getRemainingPercent());
       String percentRemaining = fraction2String(d.getRemainingPercent());
 
 
       String adminState = d.getAdminState().toString();
       String adminState = d.getAdminState().toString();
-
-      long timestamp = d.getLastUpdate();
-      long currentTime = Time.now();
       
       
       long bpUsed = d.getBlockPoolUsed();
       long bpUsed = d.getBlockPoolUsed();
       String percentBpUsed = fraction2String(d.getBlockPoolUsedPercent());
       String percentBpUsed = fraction2String(d.getBlockPoolUsedPercent());
@@ -826,17 +858,17 @@ class NamenodeJspHelper {
         HttpServletRequest request) throws IOException {
         HttpServletRequest request) throws IOException {
       final NameNode nn = NameNodeHttpServer.getNameNodeFromContext(context);
       final NameNode nn = NameNodeHttpServer.getNameNodeFromContext(context);
       final FSNamesystem ns = nn.getNamesystem();
       final FSNamesystem ns = nn.getNamesystem();
+      if (ns == null) {
+        return;
+      }
       final DatanodeManager dm = ns.getBlockManager().getDatanodeManager();
       final DatanodeManager dm = ns.getBlockManager().getDatanodeManager();
 
 
       final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
       final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
       final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
       final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
       dm.fetchDatanodes(live, dead, true);
       dm.fetchDatanodes(live, dead, true);
 
 
-      InetSocketAddress nnSocketAddress =
-          (InetSocketAddress)context.getAttribute(
-              NameNodeHttpServer.NAMENODE_ADDRESS_ATTRIBUTE_KEY);
-      String nnaddr = nnSocketAddress.getAddress().getHostAddress() + ":"
-          + nnSocketAddress.getPort();
+      String nnaddr = nn.getServiceRpcAddress().getAddress().getHostName() + ":"
+          + nn.getServiceRpcAddress().getPort();
 
 
       whatNodes = request.getParameter("whatNodes"); // show only live or only
       whatNodes = request.getParameter("whatNodes"); // show only live or only
                                                      // dead nodes
                                                      // dead nodes
@@ -872,16 +904,11 @@ class NamenodeJspHelper {
 
 
       counterReset();
       counterReset();
 
 
-      try {
-        Thread.sleep(1000);
-      } catch (InterruptedException e) {
-      }
-
       if (live.isEmpty() && dead.isEmpty()) {
       if (live.isEmpty() && dead.isEmpty()) {
         out.print("There are no datanodes in the cluster");
         out.print("There are no datanodes in the cluster");
       } else {
       } else {
 
 
-        int nnHttpPort = nn.getHttpAddress().getPort();
+        int nnInfoPort = request.getServerPort();
         out.print("<div id=\"dfsnodetable\"> ");
         out.print("<div id=\"dfsnodetable\"> ");
         if (whatNodes.equals("LIVE")) {
         if (whatNodes.equals("LIVE")) {
           out.print("<a name=\"LiveNodes\" id=\"title\">" + "Live Datanodes : "
           out.print("<a name=\"LiveNodes\" id=\"title\">" + "Live Datanodes : "
@@ -923,8 +950,8 @@ class NamenodeJspHelper {
 
 
             JspHelper.sortNodeList(live, sorterField, sorterOrder);
             JspHelper.sortNodeList(live, sorterField, sorterOrder);
             for (int i = 0; i < live.size(); i++) {
             for (int i = 0; i < live.size(); i++) {
-              generateNodeData(out, live.get(i), port_suffix, true, nnHttpPort,
-                  nnaddr);
+              generateNodeData(out, live.get(i), port_suffix, true, nnInfoPort,
+                  nnaddr, request.getScheme());
             }
             }
           }
           }
           out.print("</table>\n");
           out.print("</table>\n");
@@ -938,13 +965,15 @@ class NamenodeJspHelper {
                 + "<th " + nodeHeaderStr("node")
                 + "<th " + nodeHeaderStr("node")
                 + "> Node <th " + nodeHeaderStr("address")
                 + "> Node <th " + nodeHeaderStr("address")
                 + "> Transferring<br>Address <th "
                 + "> Transferring<br>Address <th "
+                + nodeHeaderStr("lastcontact")
+                + "> Last <br>Contact <th "
                 + nodeHeaderStr("decommissioned")
                 + nodeHeaderStr("decommissioned")
                 + "> Decommissioned\n");
                 + "> Decommissioned\n");
 
 
             JspHelper.sortNodeList(dead, sorterField, sorterOrder);
             JspHelper.sortNodeList(dead, sorterField, sorterOrder);
             for (int i = 0; i < dead.size(); i++) {
             for (int i = 0; i < dead.size(); i++) {
               generateNodeData(out, dead.get(i), port_suffix, false,
               generateNodeData(out, dead.get(i), port_suffix, false,
-                  nnHttpPort, nnaddr);
+                  nnInfoPort, nnaddr, request.getScheme());
             }
             }
 
 
             out.print("</table>\n");
             out.print("</table>\n");
@@ -975,7 +1004,7 @@ class NamenodeJspHelper {
             JspHelper.sortNodeList(decommissioning, "name", "ASC");
             JspHelper.sortNodeList(decommissioning, "name", "ASC");
             for (int i = 0; i < decommissioning.size(); i++) {
             for (int i = 0; i < decommissioning.size(); i++) {
               generateDecommissioningNodeData(out, decommissioning.get(i),
               generateDecommissioningNodeData(out, decommissioning.get(i),
-                  port_suffix, true, nnHttpPort, nnaddr);
+                  port_suffix, true, nnInfoPort, nnaddr, request.getScheme());
             }
             }
             out.print("</table>\n");
             out.print("</table>\n");
           }
           }
@@ -1003,14 +1032,16 @@ class NamenodeJspHelper {
     final BlockManager blockManager;
     final BlockManager blockManager;
     
     
     XMLBlockInfo(FSNamesystem fsn, Long blockId) {
     XMLBlockInfo(FSNamesystem fsn, Long blockId) {
-      this.blockManager = fsn.getBlockManager();
+      this.blockManager = fsn != null ? fsn.getBlockManager() : null;
 
 
       if (blockId == null) {
       if (blockId == null) {
         this.block = null;
         this.block = null;
         this.inode = null;
         this.inode = null;
       } else {
       } else {
         this.block = new Block(blockId);
         this.block = new Block(blockId);
-        this.inode = ((INode)blockManager.getBlockCollection(block)).asFile();
+        this.inode = blockManager != null ?
+          ((INode)blockManager.getBlockCollection(block)).asFile() :
+          null;
       }
       }
     }
     }
 
 
@@ -1084,8 +1115,10 @@ class NamenodeJspHelper {
         } 
         } 
 
 
         doc.startTag("replicas");
         doc.startTag("replicas");
-        for(final Iterator<DatanodeDescriptor> it = blockManager.datanodeIterator(block);
-            it.hasNext(); ) {
+        for (final Iterator<DatanodeDescriptor> it = blockManager != null ?
+            blockManager.datanodeIterator(block) :
+            Collections.<DatanodeDescriptor>emptyList().iterator();
+            it.hasNext();) {
           doc.startTag("replica");
           doc.startTag("replica");
 
 
           DatanodeDescriptor dd = it.next();
           DatanodeDescriptor dd = it.next();
@@ -1121,7 +1154,7 @@ class NamenodeJspHelper {
     
     
     XMLCorruptBlockInfo(FSNamesystem fsn, Configuration conf,
     XMLCorruptBlockInfo(FSNamesystem fsn, Configuration conf,
                                int numCorruptBlocks, Long startingBlockId) {
                                int numCorruptBlocks, Long startingBlockId) {
-      this.blockManager = fsn.getBlockManager();
+      this.blockManager = fsn != null ? fsn.getBlockManager() : null;
       this.conf = conf;
       this.conf = conf;
       this.numCorruptBlocks = numCorruptBlocks;
       this.numCorruptBlocks = numCorruptBlocks;
       this.startingBlockId = startingBlockId;
       this.startingBlockId = startingBlockId;
@@ -1144,16 +1177,19 @@ class NamenodeJspHelper {
       doc.endTag();
       doc.endTag();
       
       
       doc.startTag("num_missing_blocks");
       doc.startTag("num_missing_blocks");
-      doc.pcdata(""+blockManager.getMissingBlocksCount());
+      doc.pcdata("" + (blockManager != null ?
+        blockManager.getMissingBlocksCount() : 0));
       doc.endTag();
       doc.endTag();
       
       
       doc.startTag("num_corrupt_replica_blocks");
       doc.startTag("num_corrupt_replica_blocks");
-      doc.pcdata(""+blockManager.getCorruptReplicaBlocksCount());
+      doc.pcdata("" + (blockManager != null ?
+        blockManager.getCorruptReplicaBlocksCount() : 0));
       doc.endTag();
       doc.endTag();
      
      
       doc.startTag("corrupt_replica_block_ids");
       doc.startTag("corrupt_replica_block_ids");
-      final long[] corruptBlockIds = blockManager.getCorruptReplicaBlockIds(
-          numCorruptBlocks, startingBlockId);
+      final long[] corruptBlockIds = blockManager != null ?
+        blockManager.getCorruptReplicaBlockIds(numCorruptBlocks,
+        startingBlockId) : null;
       if (corruptBlockIds != null) {
       if (corruptBlockIds != null) {
         for (Long blockId: corruptBlockIds) {
         for (Long blockId: corruptBlockIds) {
           doc.startTag("block_id");
           doc.startTag("block_id");

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

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.ipc.StandbyException;
@@ -43,4 +44,6 @@ public interface Namesystem extends RwLock, SafeMode {
   public void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
   public void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
 
 
   public void checkOperation(OperationCategory read) throws StandbyException;
   public void checkOperation(OperationCategory read) throws StandbyException;
+
+  public boolean isInSnapshot(BlockInfoUnderConstruction blockUC);
 }
 }

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

@@ -33,10 +33,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 public class SafeModeException extends IOException {
 public class SafeModeException extends IOException {
   private static final long serialVersionUID = 1L;
   private static final long serialVersionUID = 1L;
 
 
-  public SafeModeException() {}
-
   public SafeModeException(String text, FSNamesystem.SafeModeInfo mode ) {
   public SafeModeException(String text, FSNamesystem.SafeModeInfo mode ) {
     super(text + ". Name node is in safe mode.\n" + mode.getTurnOffTip());
     super(text + ". Name node is in safe mode.\n" + mode.getTurnOffTip());
   }
   }
-
-}
+}

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

@@ -44,6 +44,7 @@ public class StartupProgressServlet extends DfsServlet {
   private static final String ELAPSED_TIME = "elapsedTime";
   private static final String ELAPSED_TIME = "elapsedTime";
   private static final String FILE = "file";
   private static final String FILE = "file";
   private static final String NAME = "name";
   private static final String NAME = "name";
+  private static final String DESC = "desc";
   private static final String PERCENT_COMPLETE = "percentComplete";
   private static final String PERCENT_COMPLETE = "percentComplete";
   private static final String PHASES = "phases";
   private static final String PHASES = "phases";
   private static final String SIZE = "size";
   private static final String SIZE = "size";
@@ -70,6 +71,7 @@ public class StartupProgressServlet extends DfsServlet {
       for (Phase phase: view.getPhases()) {
       for (Phase phase: view.getPhases()) {
         json.writeStartObject();
         json.writeStartObject();
         json.writeStringField(NAME, phase.getName());
         json.writeStringField(NAME, phase.getName());
+        json.writeStringField(DESC, phase.getDescription());
         json.writeStringField(STATUS, view.getStatus(phase).toString());
         json.writeStringField(STATUS, view.getStatus(phase).toString());
         json.writeNumberField(PERCENT_COMPLETE, view.getPercentComplete(phase));
         json.writeNumberField(PERCENT_COMPLETE, view.getPercentComplete(phase));
         json.writeNumberField(ELAPSED_TIME, view.getElapsedTime(phase));
         json.writeNumberField(ELAPSED_TIME, view.getElapsedTime(phase));
@@ -80,8 +82,10 @@ public class StartupProgressServlet extends DfsServlet {
         for (Step step: view.getSteps(phase)) {
         for (Step step: view.getSteps(phase)) {
           json.writeStartObject();
           json.writeStartObject();
           StepType type = step.getType();
           StepType type = step.getType();
-          String name = type != null ? type.getName() : null;
-          writeStringFieldIfNotNull(json, NAME, name);
+          if (type != null) {
+            json.writeStringField(NAME, type.getName());
+            json.writeStringField(DESC, type.getDescription());
+          }
           json.writeNumberField(COUNT, view.getCount(phase, step));
           json.writeNumberField(COUNT, view.getCount(phase, step));
           writeStringFieldIfNotNull(json, FILE, step.getFile());
           writeStringFieldIfNotNull(json, FILE, step.getFile());
           writeNumberFieldIfDefined(json, SIZE, step.getSize());
           writeNumberFieldIfDefined(json, SIZE, step.getSize());

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

@@ -130,4 +130,19 @@ public interface FSNamesystemMBean {
    * @return number of decommissioned dead data nodes
    * @return number of decommissioned dead data nodes
    */
    */
   public int getNumDecomDeadDataNodes();
   public int getNumDecomDeadDataNodes();
+
+  /**
+   * Number of data nodes that are in the decommissioning state
+   */
+  public int getNumDecommissioningDataNodes();
+
+  /**
+   * The statistics of snapshots
+   */
+  public String getSnapshotStats();
+
+  /**
+   * Return the maximum number of inodes in the file system
+   */
+  public long getMaxObjects();
 }
 }

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DisallowedDatanodeException.java

@@ -37,7 +37,12 @@ public class DisallowedDatanodeException extends IOException {
   /** for java.io.Serializable */
   /** for java.io.Serializable */
   private static final long serialVersionUID = 1L;
   private static final long serialVersionUID = 1L;
 
 
+  public DisallowedDatanodeException(DatanodeID nodeID, String reason) {
+    super("Datanode denied communication with namenode because "
+        + reason + ": " + nodeID);
+  }
+
   public DisallowedDatanodeException(DatanodeID nodeID) {
   public DisallowedDatanodeException(DatanodeID nodeID) {
-    super("Datanode denied communication with namenode: " + nodeID);
+    this(nodeID, "the host is not in the include-list");
   }
   }
 }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightHashSet.java

@@ -87,7 +87,7 @@ public class LightWeightHashSet<T> implements Collection<T> {
    *
    *
    * @see ConcurrentModificationException
    * @see ConcurrentModificationException
    */
    */
-  protected volatile int modification = 0;
+  protected int modification = 0;
 
 
   private float maxLoadFactor;
   private float maxLoadFactor;
   private float minLoadFactor;
   private float minLoadFactor;

+ 9 - 20
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -17,29 +17,11 @@
  */
  */
 package org.apache.hadoop.hdfs.web;
 package org.apache.hadoop.hdfs.web;
 
 
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.FileChecksum;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
-import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
-import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
+import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
@@ -50,6 +32,11 @@ import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.mortbay.util.ajax.JSON;
 import org.mortbay.util.ajax.JSON;
 
 
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.*;
+
 /** JSON Utilities */
 /** JSON Utilities */
 public class JsonUtil {
 public class JsonUtil {
   private static final Object[] EMPTY_OBJECT_ARRAY = {};
   private static final Object[] EMPTY_OBJECT_ARRAY = {};
@@ -295,6 +282,7 @@ public class JsonUtil {
     m.put("storageID", datanodeinfo.getStorageID());
     m.put("storageID", datanodeinfo.getStorageID());
     m.put("xferPort", datanodeinfo.getXferPort());
     m.put("xferPort", datanodeinfo.getXferPort());
     m.put("infoPort", datanodeinfo.getInfoPort());
     m.put("infoPort", datanodeinfo.getInfoPort());
+    m.put("infoSecurePort", datanodeinfo.getInfoSecurePort());
     m.put("ipcPort", datanodeinfo.getIpcPort());
     m.put("ipcPort", datanodeinfo.getIpcPort());
 
 
     m.put("capacity", datanodeinfo.getCapacity());
     m.put("capacity", datanodeinfo.getCapacity());
@@ -322,6 +310,7 @@ public class JsonUtil {
         (String)m.get("storageID"),
         (String)m.get("storageID"),
         (int)(long)(Long)m.get("xferPort"),
         (int)(long)(Long)m.get("xferPort"),
         (int)(long)(Long)m.get("infoPort"),
         (int)(long)(Long)m.get("infoPort"),
+        (int)(long)(Long)m.get("infoSecurePort"),
         (int)(long)(Long)m.get("ipcPort"),
         (int)(long)(Long)m.get("ipcPort"),
 
 
         (Long)m.get("capacity"),
         (Long)m.get("capacity"),

+ 136 - 37
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -54,6 +54,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.ByteRangeInputStream;
 import org.apache.hadoop.hdfs.ByteRangeInputStream;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
 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;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
@@ -86,6 +87,7 @@ import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
 import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam;
 import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryUtils;
 import org.apache.hadoop.io.retry.RetryUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
@@ -119,7 +121,7 @@ public class WebHdfsFileSystem extends FileSystem
 
 
   /** SPNEGO authenticator */
   /** SPNEGO authenticator */
   private static final KerberosUgiAuthenticator AUTH = new KerberosUgiAuthenticator();
   private static final KerberosUgiAuthenticator AUTH = new KerberosUgiAuthenticator();
-  /** Default connection factory may be overriden in tests to use smaller timeout values */
+  /** Default connection factory may be overridden in tests to use smaller timeout values */
   URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
   URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
   /** Configures connections for AuthenticatedURL */
   /** Configures connections for AuthenticatedURL */
   private final ConnectionConfigurator CONN_CONFIGURATOR =
   private final ConnectionConfigurator CONN_CONFIGURATOR =
@@ -159,12 +161,13 @@ public class WebHdfsFileSystem extends FileSystem
   }
   }
 
 
   private UserGroupInformation ugi;
   private UserGroupInformation ugi;
-  private InetSocketAddress nnAddr;
   private URI uri;
   private URI uri;
   private boolean hasInitedToken;
   private boolean hasInitedToken;
   private Token<?> delegationToken;
   private Token<?> delegationToken;
   private RetryPolicy retryPolicy = null;
   private RetryPolicy retryPolicy = null;
   private Path workingDir;
   private Path workingDir;
+  private InetSocketAddress nnAddrs[];
+  private int currentNNAddrIndex;
 
 
   /**
   /**
    * Return the protocol scheme for the FileSystem.
    * Return the protocol scheme for the FileSystem.
@@ -174,7 +177,7 @@ public class WebHdfsFileSystem extends FileSystem
    */
    */
   @Override
   @Override
   public String getScheme() {
   public String getScheme() {
-    return "webhdfs";
+    return SCHEME;
   }
   }
 
 
   @Override
   @Override
@@ -183,20 +186,42 @@ public class WebHdfsFileSystem extends FileSystem
     super.initialize(uri, conf);
     super.initialize(uri, conf);
     setConf(conf);
     setConf(conf);
     ugi = UserGroupInformation.getCurrentUser();
     ugi = UserGroupInformation.getCurrentUser();
+
     try {
     try {
-      this.uri = new URI(uri.getScheme(), uri.getAuthority(), null, null, null);
+      this.uri = new URI(uri.getScheme(), uri.getAuthority(), null,
+          null, null);
+      this.nnAddrs = DFSUtil.resolve(this.uri, getDefaultPort(), conf);
     } catch (URISyntaxException e) {
     } catch (URISyntaxException e) {
       throw new IllegalArgumentException(e);
       throw new IllegalArgumentException(e);
     }
     }
-    this.nnAddr = NetUtils.createSocketAddr(uri.getAuthority(), getDefaultPort());
-    this.retryPolicy = 
-        RetryUtils.getDefaultRetryPolicy(
-            conf, 
-            DFSConfigKeys.DFS_CLIENT_RETRY_POLICY_ENABLED_KEY, 
-            DFSConfigKeys.DFS_CLIENT_RETRY_POLICY_ENABLED_DEFAULT, 
-            DFSConfigKeys.DFS_CLIENT_RETRY_POLICY_SPEC_KEY,
-            DFSConfigKeys.DFS_CLIENT_RETRY_POLICY_SPEC_DEFAULT,
-            SafeModeException.class);
+
+    if (!HAUtil.isLogicalUri(conf, this.uri)) {
+      this.retryPolicy =
+          RetryUtils.getDefaultRetryPolicy(
+              conf,
+              DFSConfigKeys.DFS_HTTP_CLIENT_RETRY_POLICY_ENABLED_KEY,
+              DFSConfigKeys.DFS_HTTP_CLIENT_RETRY_POLICY_ENABLED_DEFAULT,
+              DFSConfigKeys.DFS_HTTP_CLIENT_RETRY_POLICY_SPEC_KEY,
+              DFSConfigKeys.DFS_HTTP_CLIENT_RETRY_POLICY_SPEC_DEFAULT,
+              SafeModeException.class);
+    } else {
+
+      int maxFailoverAttempts = conf.getInt(
+          DFSConfigKeys.DFS_HTTP_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
+          DFSConfigKeys.DFS_HTTP_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
+      int failoverSleepBaseMillis = conf.getInt(
+          DFSConfigKeys.DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY,
+          DFSConfigKeys.DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT);
+      int failoverSleepMaxMillis = conf.getInt(
+          DFSConfigKeys.DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY,
+          DFSConfigKeys.DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT);
+
+      this.retryPolicy = RetryPolicies
+          .failoverOnNetworkException(RetryPolicies.TRY_ONCE_THEN_FAIL,
+              maxFailoverAttempts, failoverSleepBaseMillis,
+              failoverSleepMaxMillis);
+    }
+
     this.workingDir = getHomeDirectory();
     this.workingDir = getHomeDirectory();
 
 
     if (UserGroupInformation.isSecurityEnabled()) {
     if (UserGroupInformation.isSecurityEnabled()) {
@@ -348,6 +373,19 @@ public class WebHdfsFileSystem extends FileSystem
     return ((RemoteException)ioe).unwrapRemoteException();
     return ((RemoteException)ioe).unwrapRemoteException();
   }
   }
 
 
+  private synchronized InetSocketAddress getCurrentNNAddr() {
+    return nnAddrs[currentNNAddrIndex];
+  }
+
+  /**
+   * Reset the appropriate state to gracefully fail over to another name node
+   */
+  private synchronized void resetStateToFailOver() {
+    currentNNAddrIndex = (currentNNAddrIndex + 1) % nnAddrs.length;
+    delegationToken = null;
+    hasInitedToken = false;
+  }
+
   /**
   /**
    * Return a URL pointing to given path on the namenode.
    * Return a URL pointing to given path on the namenode.
    *
    *
@@ -357,6 +395,7 @@ public class WebHdfsFileSystem extends FileSystem
    * @throws IOException on error constructing the URL
    * @throws IOException on error constructing the URL
    */
    */
   private URL getNamenodeURL(String path, String query) throws IOException {
   private URL getNamenodeURL(String path, String query) throws IOException {
+    InetSocketAddress nnAddr = getCurrentNNAddr();
     final URL url = new URL("http", nnAddr.getHostName(),
     final URL url = new URL("http", nnAddr.getHostName(),
           nnAddr.getPort(), path + '?' + query);
           nnAddr.getPort(), path + '?' + query);
     if (LOG.isTraceEnabled()) {
     if (LOG.isTraceEnabled()) {
@@ -414,38 +453,28 @@ public class WebHdfsFileSystem extends FileSystem
    */
    */
   private Map<?, ?> run(final HttpOpParam.Op op, final Path fspath,
   private Map<?, ?> run(final HttpOpParam.Op op, final Path fspath,
       final Param<?,?>... parameters) throws IOException {
       final Param<?,?>... parameters) throws IOException {
-    return new Runner(op, fspath, parameters).run().json;
+    return new FsPathRunner(op, fspath, parameters).run().json;
   }
   }
 
 
   /**
   /**
    * This class is for initialing a HTTP connection, connecting to server,
    * This class is for initialing a HTTP connection, connecting to server,
    * obtaining a response, and also handling retry on failures.
    * obtaining a response, and also handling retry on failures.
    */
    */
-  class Runner {
-    private final HttpOpParam.Op op;
-    private final URL url;
+  abstract class AbstractRunner {
+    abstract protected URL getUrl() throws IOException;
+
+    protected final HttpOpParam.Op op;
     private final boolean redirected;
     private final boolean redirected;
 
 
     private boolean checkRetry;
     private boolean checkRetry;
-    private HttpURLConnection conn = null;
+    protected HttpURLConnection conn = null;
     private Map<?, ?> json = null;
     private Map<?, ?> json = null;
 
 
-    Runner(final HttpOpParam.Op op, final URL url, final boolean redirected) {
+    protected AbstractRunner(final HttpOpParam.Op op, boolean redirected) {
       this.op = op;
       this.op = op;
-      this.url = url;
       this.redirected = redirected;
       this.redirected = redirected;
     }
     }
 
 
-    Runner(final HttpOpParam.Op op, final Path fspath,
-        final Param<?,?>... parameters) throws IOException {
-      this(op, toUrl(op, fspath, parameters), false);
-    }
-
-    Runner(final HttpOpParam.Op op, final HttpURLConnection conn) {
-      this(op, null, false);
-      this.conn = conn;
-    }
-
     private HttpURLConnection getHttpUrlConnection(final URL url)
     private HttpURLConnection getHttpUrlConnection(final URL url)
         throws IOException, AuthenticationException {
         throws IOException, AuthenticationException {
       UserGroupInformation connectUgi = ugi.getRealUser();
       UserGroupInformation connectUgi = ugi.getRealUser();
@@ -493,6 +522,7 @@ public class WebHdfsFileSystem extends FileSystem
   
   
     private void init() throws IOException {
     private void init() throws IOException {
       checkRetry = !redirected;
       checkRetry = !redirected;
+      URL url = getUrl();
       try {
       try {
         conn = getHttpUrlConnection(url);
         conn = getHttpUrlConnection(url);
       } catch(AuthenticationException ae) {
       } catch(AuthenticationException ae) {
@@ -519,7 +549,23 @@ public class WebHdfsFileSystem extends FileSystem
       }
       }
     }
     }
 
 
-    Runner run() throws IOException {
+    AbstractRunner run() throws IOException {
+      /**
+       * Do the real work.
+       *
+       * There are three cases that the code inside the loop can throw an
+       * IOException:
+       *
+       * <ul>
+       * <li>The connection has failed (e.g., ConnectException,
+       * @see FailoverOnNetworkExceptionRetry for more details)</li>
+       * <li>The namenode enters the standby state (i.e., StandbyException).</li>
+       * <li>The server returns errors for the command (i.e., RemoteException)</li>
+       * </ul>
+       *
+       * The call to shouldRetry() will conduct the retry policy. The policy
+       * examines the exception and swallows it if it decides to rerun the work.
+       */
       for(int retry = 0; ; retry++) {
       for(int retry = 0; ; retry++) {
         try {
         try {
           init();
           init();
@@ -537,14 +583,25 @@ public class WebHdfsFileSystem extends FileSystem
 
 
     private void shouldRetry(final IOException ioe, final int retry
     private void shouldRetry(final IOException ioe, final int retry
         ) throws IOException {
         ) throws IOException {
+      InetSocketAddress nnAddr = getCurrentNNAddr();
       if (checkRetry) {
       if (checkRetry) {
         try {
         try {
           final RetryPolicy.RetryAction a = retryPolicy.shouldRetry(
           final RetryPolicy.RetryAction a = retryPolicy.shouldRetry(
               ioe, retry, 0, true);
               ioe, retry, 0, true);
-          if (a.action == RetryPolicy.RetryAction.RetryDecision.RETRY) {
+
+          boolean isRetry = a.action == RetryPolicy.RetryAction.RetryDecision.RETRY;
+          boolean isFailoverAndRetry =
+              a.action == RetryPolicy.RetryAction.RetryDecision.FAILOVER_AND_RETRY;
+
+          if (isRetry || isFailoverAndRetry) {
             LOG.info("Retrying connect to namenode: " + nnAddr
             LOG.info("Retrying connect to namenode: " + nnAddr
                 + ". Already tried " + retry + " time(s); retry policy is "
                 + ". Already tried " + retry + " time(s); retry policy is "
-                + retryPolicy + ", delay " + a.delayMillis + "ms.");      
+                + retryPolicy + ", delay " + a.delayMillis + "ms.");
+
+            if (isFailoverAndRetry) {
+              resetStateToFailOver();
+            }
+
             Thread.sleep(a.delayMillis);
             Thread.sleep(a.delayMillis);
             return;
             return;
           }
           }
@@ -617,6 +674,48 @@ public class WebHdfsFileSystem extends FileSystem
     }
     }
   }
   }
 
 
+  final class FsPathRunner extends AbstractRunner {
+    private final Path fspath;
+    private final Param<?, ?>[] parameters;
+
+    FsPathRunner(final HttpOpParam.Op op, final Path fspath, final Param<?,?>... parameters) {
+      super(op, false);
+      this.fspath = fspath;
+      this.parameters = parameters;
+    }
+
+    @Override
+    protected URL getUrl() throws IOException {
+      return toUrl(op, fspath, parameters);
+    }
+  }
+
+  final class URLRunner extends AbstractRunner {
+    private final URL url;
+    @Override
+    protected URL getUrl() {
+      return url;
+    }
+
+    protected URLRunner(final HttpOpParam.Op op, final URL url, boolean redirected) {
+      super(op, redirected);
+      this.url = url;
+    }
+  }
+
+  @VisibleForTesting
+  final class ConnRunner extends AbstractRunner {
+    protected ConnRunner(final HttpOpParam.Op op, HttpURLConnection conn) {
+      super(op, false);
+      this.conn = conn;
+    }
+
+    @Override
+    protected URL getUrl() {
+      return null;
+    }
+  }
+
   private FsPermission applyUMask(FsPermission permission) {
   private FsPermission applyUMask(FsPermission permission) {
     if (permission == null) {
     if (permission == null) {
       permission = FsPermission.getDefault();
       permission = FsPermission.getDefault();
@@ -772,7 +871,7 @@ public class WebHdfsFileSystem extends FileSystem
     statistics.incrementWriteOps(1);
     statistics.incrementWriteOps(1);
 
 
     final HttpOpParam.Op op = PutOpParam.Op.CREATE;
     final HttpOpParam.Op op = PutOpParam.Op.CREATE;
-    return new Runner(op, f, 
+    return new FsPathRunner(op, f,
         new PermissionParam(applyUMask(permission)),
         new PermissionParam(applyUMask(permission)),
         new OverwriteParam(overwrite),
         new OverwriteParam(overwrite),
         new BufferSizeParam(bufferSize),
         new BufferSizeParam(bufferSize),
@@ -788,7 +887,7 @@ public class WebHdfsFileSystem extends FileSystem
     statistics.incrementWriteOps(1);
     statistics.incrementWriteOps(1);
 
 
     final HttpOpParam.Op op = PostOpParam.Op.APPEND;
     final HttpOpParam.Op op = PostOpParam.Op.APPEND;
-    return new Runner(op, f, new BufferSizeParam(bufferSize))
+    return new FsPathRunner(op, f, new BufferSizeParam(bufferSize))
       .run()
       .run()
       .write(bufferSize);
       .write(bufferSize);
   }
   }
@@ -835,7 +934,7 @@ public class WebHdfsFileSystem extends FileSystem
         final boolean resolved) throws IOException {
         final boolean resolved) throws IOException {
       final URL offsetUrl = offset == 0L? url
       final URL offsetUrl = offset == 0L? url
           : new URL(url + "&" + new OffsetParam(offset));
           : new URL(url + "&" + new OffsetParam(offset));
-      return new Runner(GetOpParam.Op.OPEN, offsetUrl, resolved).run().conn;
+      return new URLRunner(GetOpParam.Op.OPEN, offsetUrl, resolved).run().conn;
     }  
     }  
   }
   }
 
 
@@ -909,7 +1008,7 @@ public class WebHdfsFileSystem extends FileSystem
     final HttpOpParam.Op op = GetOpParam.Op.GETDELEGATIONTOKEN;
     final HttpOpParam.Op op = GetOpParam.Op.GETDELEGATIONTOKEN;
     final Map<?, ?> m = run(op, null, new RenewerParam(renewer));
     final Map<?, ?> m = run(op, null, new RenewerParam(renewer));
     final Token<DelegationTokenIdentifier> token = JsonUtil.toDelegationToken(m); 
     final Token<DelegationTokenIdentifier> token = JsonUtil.toDelegationToken(m); 
-    SecurityUtil.setTokenService(token, nnAddr);
+    SecurityUtil.setTokenService(token, getCurrentNNAddr());
     return token;
     return token;
   }
   }
 
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/hdfs.c

@@ -2373,7 +2373,7 @@ static int translateZCRException(JNIEnv *env, jthrowable exc)
         ret = EPROTONOSUPPORT;
         ret = EPROTONOSUPPORT;
         goto done;
         goto done;
     }
     }
-    ret = printExceptionAndFree(env, exc, PRINT_EXC_ALL,
+    ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
             "hadoopZeroCopyRead: ZeroCopyCursor#read failed");
             "hadoopZeroCopyRead: ZeroCopyCursor#read failed");
 done:
 done:
     free(className);
     free(className);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -52,8 +52,9 @@ message DatanodeIDProto {
   required string hostName = 2;  // hostname
   required string hostName = 2;  // hostname
   required string storageID = 3; // unique storage id
   required string storageID = 3; // unique storage id
   required uint32 xferPort = 4;  // data streaming port
   required uint32 xferPort = 4;  // data streaming port
-  required uint32 infoPort = 5;  // info server port
+  required uint32 infoPort = 5;  // datanode http port
   required uint32 ipcPort = 6;   // ipc server port
   required uint32 ipcPort = 6;   // ipc server port
+  optional uint32 infoSecurePort = 7 [default = 0]; // datanode https port
 }
 }
 
 
 /**
 /**

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

@@ -569,6 +569,22 @@
   </description>
   </description>
 </property>
 </property>
 
 
+<property>
+  <name>dfs.namenode.datanode.registration.ip-hostname-check</name>
+  <value>true</value>
+  <description>
+    If true (the default), then the namenode requires that a connecting
+    datanode's address must be resolved to a hostname.  If necessary, a reverse
+    DNS lookup is performed.  All attempts to register a datanode from an
+    unresolvable address are rejected.
+
+    It is recommended that this setting be left on to prevent accidental
+    registration of datanodes listed by hostname in the excludes file during a
+    DNS outage.  Only set this to false in environments where there is no
+    infrastructure to support reverse DNS lookup.
+  </description>
+</property>
+
 <property>
 <property>
   <name>dfs.namenode.decommission.interval</name>
   <name>dfs.namenode.decommission.interval</name>
   <value>30</value>
   <value>30</value>

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/corrupt_files.jsp

@@ -25,6 +25,7 @@
 	import="org.apache.hadoop.fs.Path"
 	import="org.apache.hadoop.fs.Path"
 	import="org.apache.hadoop.ha.HAServiceProtocol.HAServiceState"
 	import="org.apache.hadoop.ha.HAServiceProtocol.HAServiceState"
 	import="java.util.Collection"
 	import="java.util.Collection"
+	import="java.util.Collections"
 	import="java.util.Arrays" %>
 	import="java.util.Arrays" %>
 <%!//for java.io.Serializable
 <%!//for java.io.Serializable
   private static final long serialVersionUID = 1L;%>
   private static final long serialVersionUID = 1L;%>
@@ -34,9 +35,10 @@
   HAServiceState nnHAState = nn.getServiceState();
   HAServiceState nnHAState = nn.getServiceState();
   boolean isActive = (nnHAState == HAServiceState.ACTIVE);
   boolean isActive = (nnHAState == HAServiceState.ACTIVE);
   String namenodeRole = nn.getRole().toString();
   String namenodeRole = nn.getRole().toString();
-  String namenodeLabel = nn.getNameNodeAddressHostPortString();
-  Collection<FSNamesystem.CorruptFileBlockInfo> corruptFileBlocks = 
-	fsn.listCorruptFileBlocks("/", null);
+  String namenodeLabel = NamenodeJspHelper.getNameNodeLabel(nn);
+  Collection<FSNamesystem.CorruptFileBlockInfo> corruptFileBlocks = fsn != null ?
+    fsn.listCorruptFileBlocks("/", null) :
+    Collections.<FSNamesystem.CorruptFileBlockInfo>emptyList();
   int corruptFileCount = corruptFileBlocks.size();
   int corruptFileCount = corruptFileBlocks.size();
 %>
 %>
 
 
@@ -48,7 +50,7 @@
 <h1><%=namenodeRole%> '<%=namenodeLabel%>'</h1>
 <h1><%=namenodeRole%> '<%=namenodeLabel%>'</h1>
 <%=NamenodeJspHelper.getVersionTable(fsn)%>
 <%=NamenodeJspHelper.getVersionTable(fsn)%>
 <br>
 <br>
-<% if (isActive) { %> 
+<% if (isActive && fsn != null) { %> 
   <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b>
   <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b>
   <br>
   <br>
 <% } %> 
 <% } %> 

+ 6 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp

@@ -34,29 +34,20 @@
   boolean isActive = (nnHAState == HAServiceState.ACTIVE);
   boolean isActive = (nnHAState == HAServiceState.ACTIVE);
   String namenodeRole = nn.getRole().toString();
   String namenodeRole = nn.getRole().toString();
   String namenodeState = nnHAState.toString();
   String namenodeState = nnHAState.toString();
-  String namenodeLabel = nn.getRpcServer() != null ?
-    nn.getNameNodeAddressHostPortString() : null;
+  String namenodeLabel = NamenodeJspHelper.getNameNodeLabel(nn);
 %>
 %>
 
 
 <!DOCTYPE html>
 <!DOCTYPE html>
 <html>
 <html>
 <head>
 <head>
 <link rel="stylesheet" type="text/css" href="/static/hadoop.css">
 <link rel="stylesheet" type="text/css" href="/static/hadoop.css">
-<% if (namenodeLabel != null) { %>
 <title>Hadoop <%=namenodeRole%>&nbsp;<%=namenodeLabel%></title>
 <title>Hadoop <%=namenodeRole%>&nbsp;<%=namenodeLabel%></title>
-<% } else { %>
-<title>Hadoop <%=namenodeRole%></title>
-<% } %>
 </head>    
 </head>    
 <body>
 <body>
-<% if (namenodeLabel != null) { %>
 <h1><%=namenodeRole%> '<%=namenodeLabel%>' (<%=namenodeState%>)</h1>
 <h1><%=namenodeRole%> '<%=namenodeLabel%>' (<%=namenodeState%>)</h1>
-<% } else { %>
-<h1><%=namenodeRole%> (<%=namenodeState%>)</h1>
-<% } %>
 <%= NamenodeJspHelper.getVersionTable(fsn) %>
 <%= NamenodeJspHelper.getVersionTable(fsn) %>
 <br />
 <br />
-<% if (isActive) { %> 
+<% if (isActive && fsn != null) { %> 
   <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
   <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
 <% } %> 
 <% } %> 
 <b><a href="/logs/"><%=namenodeRole%> Logs</a></b>
 <b><a href="/logs/"><%=namenodeRole%> Logs</a></b>
@@ -73,7 +64,10 @@
 <% healthjsp.generateJournalReport(out, nn, request); %>
 <% healthjsp.generateJournalReport(out, nn, request); %>
 <hr/>
 <hr/>
 <% healthjsp.generateConfReport(out, nn, request); %>
 <% healthjsp.generateConfReport(out, nn, request); %>
-<hr>
+<hr/>
+<h3>Snapshot Summary</h3>
+<% NamenodeJspHelper.generateSnapshotReport(out, fsn); %>
+<hr/>
 <h3>Startup Progress</h3>
 <h3>Startup Progress</h3>
 <% healthjsp.generateStartupProgress(out, nn.getStartupProgress()); %>
 <% healthjsp.generateStartupProgress(out, nn.getStartupProgress()); %>
 <%
 <%

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsnodelist.jsp

@@ -33,7 +33,7 @@ String namenodeRole = nn.getRole().toString();
 FSNamesystem fsn = nn.getNamesystem();
 FSNamesystem fsn = nn.getNamesystem();
 HAServiceState nnHAState = nn.getServiceState();
 HAServiceState nnHAState = nn.getServiceState();
 boolean isActive = (nnHAState == HAServiceState.ACTIVE);
 boolean isActive = (nnHAState == HAServiceState.ACTIVE);
-String namenodeLabel = nn.getNameNodeAddressHostPortString();
+String namenodeLabel = NamenodeJspHelper.getNameNodeLabel(nn);
 %>
 %>
 
 
 <!DOCTYPE html>
 <!DOCTYPE html>
@@ -46,7 +46,7 @@ String namenodeLabel = nn.getNameNodeAddressHostPortString();
 <h1><%=namenodeRole%> '<%=namenodeLabel%>'</h1>
 <h1><%=namenodeRole%> '<%=namenodeLabel%>'</h1>
 <%= NamenodeJspHelper.getVersionTable(fsn) %>
 <%= NamenodeJspHelper.getVersionTable(fsn) %>
 <br />
 <br />
-<% if (isActive) { %> 
+<% if (isActive && fsn != null) { %> 
   <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
   <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
 <% } %> 
 <% } %> 
 <b><a href="/logs/"><%=namenodeRole%> Logs</a></b><br>
 <b><a href="/logs/"><%=namenodeRole%> Logs</a></b><br>

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm

@@ -28,7 +28,7 @@ HDFS Federation
 
 
 * {Background}
 * {Background}
 
 
-[./federation-background.gif] HDFS Layers
+[./images/federation-background.gif] HDFS Layers
 
 
   HDFS has two main layers:
   HDFS has two main layers:
 
 
@@ -72,7 +72,7 @@ HDFS Federation
   send periodic heartbeats and block reports and handles commands from the 
   send periodic heartbeats and block reports and handles commands from the 
   Namenodes.
   Namenodes.
 
 
-[./federation.gif] HDFS Federation Architecture
+[./images/federation.gif] HDFS Federation Architecture
 
 
 
 
   <<Block Pool>>
   <<Block Pool>>

二进制
hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/federation-background.gif


二进制
hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/federation.gif


+ 116 - 27
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestGlobPaths.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
 
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.StringUtils;
@@ -27,10 +28,16 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.*;
 import org.junit.*;
 
 
 public class TestGlobPaths {
 public class TestGlobPaths {
 
 
+  private static final UserGroupInformation unprivilegedUser =
+      UserGroupInformation.createRemoteUser("myuser");
+
   static class RegexPathFilter implements PathFilter {
   static class RegexPathFilter implements PathFilter {
 
 
     private final String regex;
     private final String regex;
@@ -47,17 +54,33 @@ public class TestGlobPaths {
 
 
   static private MiniDFSCluster dfsCluster;
   static private MiniDFSCluster dfsCluster;
   static private FileSystem fs;
   static private FileSystem fs;
+  static private FileSystem unprivilegedFs;
   static private FileContext fc;
   static private FileContext fc;
+  static private FileContext unprivilegedFc;
   static final private int NUM_OF_PATHS = 4;
   static final private int NUM_OF_PATHS = 4;
   static private String USER_DIR;
   static private String USER_DIR;
   private Path[] path = new Path[NUM_OF_PATHS];
   private Path[] path = new Path[NUM_OF_PATHS];
 
 
   @BeforeClass
   @BeforeClass
   public static void setUp() throws Exception {
   public static void setUp() throws Exception {
-    Configuration conf = new HdfsConfiguration();
+    final Configuration conf = new HdfsConfiguration();
     dfsCluster = new MiniDFSCluster.Builder(conf).build();
     dfsCluster = new MiniDFSCluster.Builder(conf).build();
     fs = FileSystem.get(conf);
     fs = FileSystem.get(conf);
+    unprivilegedFs =
+      unprivilegedUser.doAs(new PrivilegedExceptionAction<FileSystem>() {
+        @Override
+        public FileSystem run() throws IOException {
+          return FileSystem.get(conf);
+        }
+      });
     fc = FileContext.getFileContext(conf);
     fc = FileContext.getFileContext(conf);
+    unprivilegedFc =
+      unprivilegedUser.doAs(new PrivilegedExceptionAction<FileContext>() {
+        @Override
+        public FileContext run() throws IOException {
+          return FileContext.getFileContext(conf);
+        }
+      });
     USER_DIR = fs.getHomeDirectory().toUri().getPath().toString();
     USER_DIR = fs.getHomeDirectory().toUri().getPath().toString();
   }
   }
   
   
@@ -339,17 +362,6 @@ public class TestGlobPaths {
     status = fs.globStatus(new Path(USER_DIR+"{/dir*}"));
     status = fs.globStatus(new Path(USER_DIR+"{/dir*}"));
     checkStatus(status, d1, d2, d3, d4);
     checkStatus(status, d1, d2, d3, d4);
 
 
-    /* 
-     * true filter
-     */
-
-    PathFilter trueFilter = new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return true;
-      }
-    };
-
     status = fs.globStatus(new Path(Path.SEPARATOR), trueFilter);
     status = fs.globStatus(new Path(Path.SEPARATOR), trueFilter);
     checkStatus(status, new Path(Path.SEPARATOR));
     checkStatus(status, new Path(Path.SEPARATOR));
     
     
@@ -781,8 +793,8 @@ public class TestGlobPaths {
    * A glob test that can be run on either FileContext or FileSystem.
    * A glob test that can be run on either FileContext or FileSystem.
    */
    */
   private static interface FSTestWrapperGlobTest {
   private static interface FSTestWrapperGlobTest {
-    void run(FSTestWrapper wrap, FileSystem fs, FileContext fc)
-        throws Exception;
+    void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrapper,
+        FileSystem fs, FileContext fc) throws Exception;
   }
   }
 
 
   /**
   /**
@@ -791,7 +803,8 @@ public class TestGlobPaths {
   private void testOnFileSystem(FSTestWrapperGlobTest test) throws Exception {
   private void testOnFileSystem(FSTestWrapperGlobTest test) throws Exception {
     try {
     try {
       fc.mkdir(new Path(USER_DIR), FsPermission.getDefault(), true);
       fc.mkdir(new Path(USER_DIR), FsPermission.getDefault(), true);
-      test.run(new FileSystemTestWrapper(fs), fs, null);
+      test.run(new FileSystemTestWrapper(fs),
+          new FileSystemTestWrapper(unprivilegedFs), fs, null);
     } finally {
     } finally {
       fc.delete(new Path(USER_DIR), true);
       fc.delete(new Path(USER_DIR), true);
     }
     }
@@ -803,7 +816,8 @@ public class TestGlobPaths {
   private void testOnFileContext(FSTestWrapperGlobTest test) throws Exception {
   private void testOnFileContext(FSTestWrapperGlobTest test) throws Exception {
     try {
     try {
       fs.mkdirs(new Path(USER_DIR));
       fs.mkdirs(new Path(USER_DIR));
-      test.run(new FileContextTestWrapper(fc), null, fc);
+      test.run(new FileContextTestWrapper(fc),
+          new FileContextTestWrapper(unprivilegedFc), null, fc);
     } finally {
     } finally {
       cleanupDFS();
       cleanupDFS();
     }
     }
@@ -819,6 +833,8 @@ public class TestGlobPaths {
     }
     }
   }
   }
 
 
+  private static final PathFilter trueFilter = new AcceptAllPathFilter();
+
   /**
   /**
    * Accept only paths ending in Z.
    * Accept only paths ending in Z.
    */
    */
@@ -834,8 +850,8 @@ public class TestGlobPaths {
    * Test globbing through symlinks.
    * Test globbing through symlinks.
    */
    */
   private static class TestGlobWithSymlinks implements FSTestWrapperGlobTest {
   private static class TestGlobWithSymlinks implements FSTestWrapperGlobTest {
-    public void run(FSTestWrapper wrap, FileSystem fs, FileContext fc)
-        throws Exception {
+    public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
+        FileSystem fs, FileContext fc) throws Exception {
       // Test that globbing through a symlink to a directory yields a path
       // Test that globbing through a symlink to a directory yields a path
       // containing that symlink.
       // containing that symlink.
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
@@ -869,11 +885,13 @@ public class TestGlobPaths {
     }
     }
   }
   }
 
 
+  @Ignore
   @Test
   @Test
   public void testGlobWithSymlinksOnFS() throws Exception {
   public void testGlobWithSymlinksOnFS() throws Exception {
     testOnFileSystem(new TestGlobWithSymlinks());
     testOnFileSystem(new TestGlobWithSymlinks());
   }
   }
 
 
+  @Ignore
   @Test
   @Test
   public void testGlobWithSymlinksOnFC() throws Exception {
   public void testGlobWithSymlinksOnFC() throws Exception {
     testOnFileContext(new TestGlobWithSymlinks());
     testOnFileContext(new TestGlobWithSymlinks());
@@ -886,8 +904,8 @@ public class TestGlobPaths {
    */
    */
   private static class TestGlobWithSymlinksToSymlinks implements
   private static class TestGlobWithSymlinksToSymlinks implements
       FSTestWrapperGlobTest {
       FSTestWrapperGlobTest {
-    public void run(FSTestWrapper wrap, FileSystem fs, FileContext fc)
-        throws Exception {
+    public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
+        FileSystem fs, FileContext fc) throws Exception {
       // Test that globbing through a symlink to a symlink to a directory
       // Test that globbing through a symlink to a symlink to a directory
       // fully resolves
       // fully resolves
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
@@ -946,11 +964,13 @@ public class TestGlobPaths {
     }
     }
   }
   }
 
 
+  @Ignore
   @Test
   @Test
   public void testGlobWithSymlinksToSymlinksOnFS() throws Exception {
   public void testGlobWithSymlinksToSymlinksOnFS() throws Exception {
     testOnFileSystem(new TestGlobWithSymlinksToSymlinks());
     testOnFileSystem(new TestGlobWithSymlinksToSymlinks());
   }
   }
 
 
+  @Ignore
   @Test
   @Test
   public void testGlobWithSymlinksToSymlinksOnFC() throws Exception {
   public void testGlobWithSymlinksToSymlinksOnFC() throws Exception {
     testOnFileContext(new TestGlobWithSymlinksToSymlinks());
     testOnFileContext(new TestGlobWithSymlinksToSymlinks());
@@ -961,8 +981,8 @@ public class TestGlobPaths {
    */
    */
   private static class TestGlobSymlinksWithCustomPathFilter implements
   private static class TestGlobSymlinksWithCustomPathFilter implements
       FSTestWrapperGlobTest {
       FSTestWrapperGlobTest {
-    public void run(FSTestWrapper wrap, FileSystem fs, FileContext fc)
-        throws Exception {
+    public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
+        FileSystem fs, FileContext fc) throws Exception {
       // Test that globbing through a symlink to a symlink to a directory
       // Test that globbing through a symlink to a symlink to a directory
       // fully resolves
       // fully resolves
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
@@ -995,11 +1015,13 @@ public class TestGlobPaths {
     }
     }
   }
   }
 
 
+  @Ignore
   @Test
   @Test
   public void testGlobSymlinksWithCustomPathFilterOnFS() throws Exception {
   public void testGlobSymlinksWithCustomPathFilterOnFS() throws Exception {
     testOnFileSystem(new TestGlobSymlinksWithCustomPathFilter());
     testOnFileSystem(new TestGlobSymlinksWithCustomPathFilter());
   }
   }
 
 
+  @Ignore
   @Test
   @Test
   public void testGlobSymlinksWithCustomPathFilterOnFC() throws Exception {
   public void testGlobSymlinksWithCustomPathFilterOnFC() throws Exception {
     testOnFileContext(new TestGlobSymlinksWithCustomPathFilter());
     testOnFileContext(new TestGlobSymlinksWithCustomPathFilter());
@@ -1009,8 +1031,8 @@ public class TestGlobPaths {
    * Test that globStatus fills in the scheme even when it is not provided.
    * Test that globStatus fills in the scheme even when it is not provided.
    */
    */
   private static class TestGlobFillsInScheme implements FSTestWrapperGlobTest {
   private static class TestGlobFillsInScheme implements FSTestWrapperGlobTest {
-    public void run(FSTestWrapper wrap, FileSystem fs, FileContext fc)
-        throws Exception {
+    public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
+        FileSystem fs, FileContext fc) throws Exception {
       // Verify that the default scheme is hdfs, when we don't supply one.
       // Verify that the default scheme is hdfs, when we don't supply one.
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
       wrap.mkdir(new Path(USER_DIR + "/alpha"), FsPermission.getDirDefault(),
           false);
           false);
@@ -1020,7 +1042,7 @@ public class TestGlobPaths {
           new Path(USER_DIR + "/alphaLink"), new AcceptAllPathFilter());
           new Path(USER_DIR + "/alphaLink"), new AcceptAllPathFilter());
       Assert.assertEquals(1, statuses.length);
       Assert.assertEquals(1, statuses.length);
       Path path = statuses[0].getPath();
       Path path = statuses[0].getPath();
-      Assert.assertEquals(USER_DIR + "/alphaLink", path.toUri().getPath());
+      Assert.assertEquals(USER_DIR + "/alpha", path.toUri().getPath());
       Assert.assertEquals("hdfs", path.toUri().getScheme());
       Assert.assertEquals("hdfs", path.toUri().getScheme());
       if (fc != null) {
       if (fc != null) {
         // If we're using FileContext, then we can list a file:/// URI.
         // If we're using FileContext, then we can list a file:/// URI.
@@ -1052,8 +1074,8 @@ public class TestGlobPaths {
    * Test that globStatus works with relative paths.
    * Test that globStatus works with relative paths.
    **/
    **/
   private static class TestRelativePath implements FSTestWrapperGlobTest {
   private static class TestRelativePath implements FSTestWrapperGlobTest {
-    public void run(FSTestWrapper wrap, FileSystem fs, FileContext fc)
-      throws Exception {
+    public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
+        FileSystem fs, FileContext fc) throws Exception {
       String[] files = new String[] { "a", "abc", "abc.p", "bacd" };
       String[] files = new String[] { "a", "abc", "abc.p", "bacd" };
 
 
       Path[] path = new Path[files.length];
       Path[] path = new Path[files.length];
@@ -1086,4 +1108,71 @@ public class TestGlobPaths {
   public void testRelativePathOnFC() throws Exception {
   public void testRelativePathOnFC() throws Exception {
     testOnFileContext(new TestRelativePath());
     testOnFileContext(new TestRelativePath());
   }
   }
+  
+  /**
+   * Test that trying to glob through a directory we don't have permission
+   * to list fails with AccessControlException rather than succeeding or
+   * throwing any other exception.
+   **/
+  private static class TestGlobAccessDenied implements FSTestWrapperGlobTest {
+    public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
+        FileSystem fs, FileContext fc) throws Exception {
+      wrap.mkdir(new Path("/nopermission/val"),
+          new FsPermission((short)0777), true);
+      wrap.mkdir(new Path("/norestrictions/val"),
+          new FsPermission((short)0777), true);
+      wrap.setPermission(new Path("/nopermission"),
+          new FsPermission((short)0));
+      try {
+        unprivilegedWrap.globStatus(new Path("/no*/*"),
+            new AcceptAllPathFilter());
+        Assert.fail("expected to get an AccessControlException when " +
+            "globbing through a directory we don't have permissions " +
+            "to list.");
+      } catch (AccessControlException ioe) {
+      }
+
+      Assert.assertEquals("/norestrictions/val",
+        TestPath.mergeStatuses(unprivilegedWrap.globStatus(
+            new Path("/norestrictions/*"),
+                new AcceptAllPathFilter())));
+    }
+  }
+
+  @Test
+  public void testGlobAccessDeniedOnFS() throws Exception {
+    testOnFileSystem(new TestGlobAccessDenied());
+  }
+
+  @Test
+  public void testGlobAccessDeniedOnFC() throws Exception {
+    testOnFileContext(new TestGlobAccessDenied());
+  }
+
+  /**
+   * Test that trying to list a reserved path on HDFS via the globber works.
+   **/
+  private static class TestReservedHdfsPaths implements FSTestWrapperGlobTest {
+    public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
+        FileSystem fs, FileContext fc) throws Exception {
+      String reservedRoot = "/.reserved/.inodes/" + INodeId.ROOT_INODE_ID;
+      Assert.assertEquals(reservedRoot,
+        TestPath.mergeStatuses(unprivilegedWrap.
+            globStatus(new Path(reservedRoot), new AcceptAllPathFilter())));
+      // These inodes don't show up via listStatus.
+      Assert.assertEquals("",
+        TestPath.mergeStatuses(unprivilegedWrap.
+            globStatus(new Path("/.reserved/*"), new AcceptAllPathFilter())));
+    }
+  }
+
+  @Test
+  public void testReservedHdfsPathsOnFS() throws Exception {
+    testOnFileSystem(new TestReservedHdfsPaths());
+  }
+
+  @Test
+  public void testReservedHdfsPathsOnFC() throws Exception {
+    testOnFileContext(new TestReservedHdfsPaths());
+  }
 }
 }

+ 34 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java

@@ -19,8 +19,10 @@
 package org.apache.hadoop.fs;
 package org.apache.hadoop.fs;
 
 
 import static org.apache.hadoop.fs.FileContextTestHelper.exists;
 import static org.apache.hadoop.fs.FileContextTestHelper.exists;
+import static org.junit.Assert.fail;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 
 
 import javax.security.auth.login.LoginException;
 import javax.security.auth.login.LoginException;
@@ -55,7 +57,8 @@ public class TestHDFSFileContextMainOperations extends
       LoginException, URISyntaxException {
       LoginException, URISyntaxException {
     cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(2).build();
     cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(2).build();
     cluster.waitClusterUp();
     cluster.waitClusterUp();
-    fc = FileContext.getFileContext(cluster.getURI(0), CONF);
+    URI uri0 = cluster.getURI(0);
+    fc = FileContext.getFileContext(uri0, CONF);
     defaultWorkingDirectory = fc.makeQualified( new Path("/user/" + 
     defaultWorkingDirectory = fc.makeQualified( new Path("/user/" + 
         UserGroupInformation.getCurrentUser().getShortUserName()));
         UserGroupInformation.getCurrentUser().getShortUserName()));
     fc.mkdir(defaultWorkingDirectory, FileContext.DEFAULT_PERM, true);
     fc.mkdir(defaultWorkingDirectory, FileContext.DEFAULT_PERM, true);
@@ -77,7 +80,10 @@ public class TestHDFSFileContextMainOperations extends
       
       
   @AfterClass
   @AfterClass
   public static void ClusterShutdownAtEnd() throws Exception {
   public static void ClusterShutdownAtEnd() throws Exception {
-    cluster.shutdown();   
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }    
   }
   }
   
   
   @Override
   @Override
@@ -111,7 +117,7 @@ public class TestHDFSFileContextMainOperations extends
   
   
   @Test
   @Test
   public void testOldRenameWithQuota() throws Exception {
   public void testOldRenameWithQuota() throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     Path src1 = getTestRootPath(fc, "test/testOldRenameWithQuota/srcdir/src1");
     Path src1 = getTestRootPath(fc, "test/testOldRenameWithQuota/srcdir/src1");
     Path src2 = getTestRootPath(fc, "test/testOldRenameWithQuota/srcdir/src2");
     Path src2 = getTestRootPath(fc, "test/testOldRenameWithQuota/srcdir/src2");
     Path dst1 = getTestRootPath(fc, "test/testOldRenameWithQuota/dstdir/dst1");
     Path dst1 = getTestRootPath(fc, "test/testOldRenameWithQuota/dstdir/dst1");
@@ -146,7 +152,7 @@ public class TestHDFSFileContextMainOperations extends
   
   
   @Test
   @Test
   public void testRenameWithQuota() throws Exception {
   public void testRenameWithQuota() throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     Path src1 = getTestRootPath(fc, "test/testRenameWithQuota/srcdir/src1");
     Path src1 = getTestRootPath(fc, "test/testRenameWithQuota/srcdir/src1");
     Path src2 = getTestRootPath(fc, "test/testRenameWithQuota/srcdir/src2");
     Path src2 = getTestRootPath(fc, "test/testRenameWithQuota/srcdir/src2");
     Path dst1 = getTestRootPath(fc, "test/testRenameWithQuota/dstdir/dst1");
     Path dst1 = getTestRootPath(fc, "test/testRenameWithQuota/dstdir/dst1");
@@ -210,7 +216,7 @@ public class TestHDFSFileContextMainOperations extends
    */
    */
   @Test
   @Test
   public void testEditsLogOldRename() throws Exception {
   public void testEditsLogOldRename() throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     Path src1 = getTestRootPath(fc, "testEditsLogOldRename/srcdir/src1");
     Path src1 = getTestRootPath(fc, "testEditsLogOldRename/srcdir/src1");
     Path dst1 = getTestRootPath(fc, "testEditsLogOldRename/dstdir/dst1");
     Path dst1 = getTestRootPath(fc, "testEditsLogOldRename/dstdir/dst1");
     createFile(src1);
     createFile(src1);
@@ -226,7 +232,7 @@ public class TestHDFSFileContextMainOperations extends
     // Restart the cluster and ensure the above operations can be
     // Restart the cluster and ensure the above operations can be
     // loaded from the edits log
     // loaded from the edits log
     restartCluster();
     restartCluster();
-    fs = (DistributedFileSystem)cluster.getFileSystem();
+    fs = cluster.getFileSystem();
     src1 = getTestRootPath(fc, "testEditsLogOldRename/srcdir/src1");
     src1 = getTestRootPath(fc, "testEditsLogOldRename/srcdir/src1");
     dst1 = getTestRootPath(fc, "testEditsLogOldRename/dstdir/dst1");
     dst1 = getTestRootPath(fc, "testEditsLogOldRename/dstdir/dst1");
     Assert.assertFalse(fs.exists(src1));   // ensure src1 is already renamed
     Assert.assertFalse(fs.exists(src1));   // ensure src1 is already renamed
@@ -239,7 +245,7 @@ public class TestHDFSFileContextMainOperations extends
    */
    */
   @Test
   @Test
   public void testEditsLogRename() throws Exception {
   public void testEditsLogRename() throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     Path src1 = getTestRootPath(fc, "testEditsLogRename/srcdir/src1");
     Path src1 = getTestRootPath(fc, "testEditsLogRename/srcdir/src1");
     Path dst1 = getTestRootPath(fc, "testEditsLogRename/dstdir/dst1");
     Path dst1 = getTestRootPath(fc, "testEditsLogRename/dstdir/dst1");
     createFile(src1);
     createFile(src1);
@@ -255,7 +261,7 @@ public class TestHDFSFileContextMainOperations extends
     // Restart the cluster and ensure the above operations can be
     // Restart the cluster and ensure the above operations can be
     // loaded from the edits log
     // loaded from the edits log
     restartCluster();
     restartCluster();
-    fs = (DistributedFileSystem)cluster.getFileSystem();
+    fs = cluster.getFileSystem();
     src1 = getTestRootPath(fc, "testEditsLogRename/srcdir/src1");
     src1 = getTestRootPath(fc, "testEditsLogRename/srcdir/src1");
     dst1 = getTestRootPath(fc, "testEditsLogRename/dstdir/dst1");
     dst1 = getTestRootPath(fc, "testEditsLogRename/dstdir/dst1");
     Assert.assertFalse(fs.exists(src1));   // ensure src1 is already renamed
     Assert.assertFalse(fs.exists(src1));   // ensure src1 is already renamed
@@ -279,7 +285,7 @@ public class TestHDFSFileContextMainOperations extends
 
 
   private void oldRename(Path src, Path dst, boolean renameSucceeds,
   private void oldRename(Path src, Path dst, boolean renameSucceeds,
       boolean exception) throws Exception {
       boolean exception) throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     try {
     try {
       Assert.assertEquals(renameSucceeds, fs.rename(src, dst));
       Assert.assertEquals(renameSucceeds, fs.rename(src, dst));
     } catch (Exception ex) {
     } catch (Exception ex) {
@@ -301,4 +307,23 @@ public class TestHDFSFileContextMainOperations extends
     Assert.assertEquals(renameSucceeds, !exists(fc, src));
     Assert.assertEquals(renameSucceeds, !exists(fc, src));
     Assert.assertEquals((dstExists||renameSucceeds), exists(fc, dst));
     Assert.assertEquals((dstExists||renameSucceeds), exists(fc, dst));
   }
   }
+  
+  @Override
+  protected boolean listCorruptedBlocksSupported() {
+    return true;
+  }
+  
+  @Test
+  public void testCrossFileSystemRename() throws IOException {
+    try {
+      fc.rename(
+        new Path("hdfs://127.0.0.1/aaa/bbb/Foo"), 
+        new Path("file://aaa/bbb/Moo"), 
+        Options.Rename.OVERWRITE);
+      fail("IOexception expected.");
+    } catch (IOException ioe) {
+      // okay
+    }
+  }
+  
 }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/shell/TestTextCommand.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/shell/TestHdfsTextCommand.java

@@ -42,7 +42,7 @@ import org.junit.Test;
  * This class tests the logic for displaying the binary formats supported
  * This class tests the logic for displaying the binary formats supported
  * by the Text command.
  * by the Text command.
  */
  */
-public class TestTextCommand {
+public class TestHdfsTextCommand {
   private static final String TEST_ROOT_DIR =
   private static final String TEST_ROOT_DIR =
     System.getProperty("test.build.data", "build/test/data/") + "/testText";
     System.getProperty("test.build.data", "build/test/data/") + "/testText";
   private static final Path AVRO_FILENAME = new Path(TEST_ROOT_DIR, "weather.avro");
   private static final Path AVRO_FILENAME = new Path(TEST_ROOT_DIR, "weather.avro");

+ 24 - 49
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -18,60 +18,20 @@
 
 
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
-import static org.junit.Assert.assertEquals;
-
-import java.io.BufferedOutputStream;
-import java.io.BufferedReader;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.HttpURLConnection;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.net.URL;
-import java.net.URLConnection;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.TimeoutException;
-
+import com.google.common.base.Charsets;
+import com.google.common.base.Joiner;
 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.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
 import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -93,8 +53,15 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
 
 
-import com.google.common.base.Charsets;
-import com.google.common.base.Joiner;
+import java.io.*;
+import java.net.*;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
+import static org.junit.Assert.assertEquals;
 
 
 /** Utilities for HDFS tests */
 /** Utilities for HDFS tests */
 public class DFSTestUtil {
 public class DFSTestUtil {
@@ -114,10 +81,10 @@ public class DFSTestUtil {
   
   
   /** Creates a new instance of DFSTestUtil
   /** Creates a new instance of DFSTestUtil
    *
    *
-   * @param testName Name of the test from where this utility is used
    * @param nFiles Number of files to be created
    * @param nFiles Number of files to be created
    * @param maxLevels Maximum number of directory levels
    * @param maxLevels Maximum number of directory levels
    * @param maxSize Maximum size for file
    * @param maxSize Maximum size for file
+   * @param minSize Minimum size for file
    */
    */
   private DFSTestUtil(int nFiles, int maxLevels, int maxSize, int minSize) {
   private DFSTestUtil(int nFiles, int maxLevels, int maxSize, int minSize) {
     this.nFiles = nFiles;
     this.nFiles = nFiles;
@@ -143,7 +110,7 @@ public class DFSTestUtil {
   }
   }
   
   
   /**
   /**
-   * when formating a namenode - we must provide clusterid.
+   * when formatting a namenode - we must provide clusterid.
    * @param conf
    * @param conf
    * @throws IOException
    * @throws IOException
    */
    */
@@ -806,6 +773,7 @@ public class DFSTestUtil {
     return new DatanodeID(ipAddr, "localhost", "",
     return new DatanodeID(ipAddr, "localhost", "",
         DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
+        DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT);
         DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT);
   }
   }
 
 
@@ -815,7 +783,7 @@ public class DFSTestUtil {
 
 
   public static DatanodeID getLocalDatanodeID(int port) {
   public static DatanodeID getLocalDatanodeID(int port) {
     return new DatanodeID("127.0.0.1", "localhost", "",
     return new DatanodeID("127.0.0.1", "localhost", "",
-        port, port, port);
+        port, port, port, port);
   }
   }
 
 
   public static DatanodeDescriptor getLocalDatanodeDescriptor() {
   public static DatanodeDescriptor getLocalDatanodeDescriptor() {
@@ -838,6 +806,7 @@ public class DFSTestUtil {
       String host, int port) {
       String host, int port) {
     return new DatanodeInfo(new DatanodeID(ipAddr, host, "",
     return new DatanodeInfo(new DatanodeID(ipAddr, host, "",
         port, DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
         port, DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
+        DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT));
         DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT));
   }
   }
 
 
@@ -846,6 +815,7 @@ public class DFSTestUtil {
     return new DatanodeInfo(ipAddr, hostname, "",
     return new DatanodeInfo(ipAddr, hostname, "",
         DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
+        DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT,
         1l, 2l, 3l, 4l, 0l, 0l, 5, 6, "local", adminState);
         1l, 2l, 3l, 4l, 0l, 0l, 5, 6, "local", adminState);
   }
   }
@@ -860,6 +830,7 @@ public class DFSTestUtil {
       int port, String rackLocation) {
       int port, String rackLocation) {
     DatanodeID dnId = new DatanodeID(ipAddr, "host", "", port,
     DatanodeID dnId = new DatanodeID(ipAddr, "host", "", port,
         DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
+        DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT);
         DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT);
     return new DatanodeDescriptor(dnId, rackLocation);
     return new DatanodeDescriptor(dnId, rackLocation);
   }
   }
@@ -1023,4 +994,8 @@ public class DFSTestUtil {
           cluster.getNameNodeRpc(nnIndex), filePath, 0L, bytes.length);
           cluster.getNameNodeRpc(nnIndex), filePath, 0L, bytes.length);
     } while (locatedBlocks.isUnderConstruction());
     } while (locatedBlocks.isUnderConstruction());
   }
   }
+
+  public static void abortStream(DFSOutputStream out) throws IOException {
+    out.abort();
+  }
 }
 }

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

@@ -871,7 +871,11 @@ public class TestDFSClientRetries {
 
 
     final Path dir = new Path("/testNamenodeRestart");
     final Path dir = new Path("/testNamenodeRestart");
 
 
-    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_RETRY_POLICY_ENABLED_KEY, true);
+    if (isWebHDFS) {
+      conf.setBoolean(DFSConfigKeys.DFS_HTTP_CLIENT_RETRY_POLICY_ENABLED_KEY, true);
+    } else {
+      conf.setBoolean(DFSConfigKeys.DFS_CLIENT_RETRY_POLICY_ENABLED_KEY, true);
+    }
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, 1);
     conf.setInt(MiniDFSCluster.DFS_NAMENODE_SAFEMODE_EXTENSION_TESTING_KEY, 5000);
     conf.setInt(MiniDFSCluster.DFS_NAMENODE_SAFEMODE_EXTENSION_TESTING_KEY, 5000);
 
 

+ 74 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java

@@ -0,0 +1,74 @@
+/**
+ * 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.util.concurrent.atomic.AtomicReference;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+public class TestDFSOutputStream {
+  static MiniDFSCluster cluster;
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    Configuration conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf).build();
+  }
+
+  /**
+   * The close() method of DFSOutputStream should never throw the same exception
+   * twice. See HDFS-5335 for details.
+   */
+  @Test
+  public void testCloseTwice() throws IOException {
+    DistributedFileSystem fs = cluster.getFileSystem();
+    FSDataOutputStream os = fs.create(new Path("/test"));
+    DFSOutputStream dos = (DFSOutputStream) Whitebox.getInternalState(os,
+        "wrappedStream");
+    @SuppressWarnings("unchecked")
+    AtomicReference<IOException> ex = (AtomicReference<IOException>) Whitebox
+        .getInternalState(dos, "lastException");
+    Assert.assertEquals(null, ex.get());
+
+    dos.close();
+
+    IOException dummy = new IOException("dummy");
+    ex.set(dummy);
+    try {
+      dos.close();
+    } catch (IOException e) {
+      Assert.assertEquals(e, dummy);
+    }
+    Assert.assertEquals(null, ex.get());
+    dos.close();
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    cluster.shutdown();
+  }
+}

+ 50 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java

@@ -61,6 +61,8 @@ import static org.junit.Assert.*;
 public class TestDFSShell {
 public class TestDFSShell {
   private static final Log LOG = LogFactory.getLog(TestDFSShell.class);
   private static final Log LOG = LogFactory.getLog(TestDFSShell.class);
   private static AtomicInteger counter = new AtomicInteger();
   private static AtomicInteger counter = new AtomicInteger();
+  private final int SUCCESS = 0;
+  private final int ERROR = 1;
 
 
   static final String TEST_ROOT_DIR = PathUtils.getTestDirName(TestDFSShell.class);
   static final String TEST_ROOT_DIR = PathUtils.getTestDirName(TestDFSShell.class);
 
 
@@ -1619,9 +1621,6 @@ public class TestDFSShell {
   // force Copy Option is -f
   // force Copy Option is -f
   @Test (timeout = 30000)
   @Test (timeout = 30000)
   public void testCopyCommandsWithForceOption() throws Exception {
   public void testCopyCommandsWithForceOption() throws Exception {
-    final int SUCCESS = 0;
-    final int ERROR = 1;
-
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
         .format(true).build();
         .format(true).build();
@@ -1682,7 +1681,55 @@ public class TestDFSShell {
       }
       }
       cluster.shutdown();
       cluster.shutdown();
     }
     }
+  }
+
+  // setrep for file and directory.
+  @Test (timeout = 30000)
+  public void testSetrep() throws Exception {
+
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
+                                                             .format(true).build();
+    FsShell shell = null;
+    FileSystem fs = null;
+
+    final String testdir1 = "/tmp/TestDFSShell-testSetrep-" + counter.getAndIncrement();
+    final String testdir2 = testdir1 + "/nestedDir";
+    final Path hdfsFile1 = new Path(testdir1, "testFileForSetrep");
+    final Path hdfsFile2 = new Path(testdir2, "testFileForSetrep");
+    final Short oldRepFactor = new Short((short) 1);
+    final Short newRepFactor = new Short((short) 3);
+    try {
+      String[] argv;
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+      assertThat(fs.mkdirs(new Path(testdir2)), is(true));
+      shell = new FsShell(conf);
+
+      fs.create(hdfsFile1, true).close();
+      fs.create(hdfsFile2, true).close();
+
+      // Tests for setrep on a file.
+      argv = new String[] { "-setrep", newRepFactor.toString(), hdfsFile1.toString() };
+      assertThat(shell.run(argv), is(SUCCESS));
+      assertThat(fs.getFileStatus(hdfsFile1).getReplication(), is(newRepFactor));
+      assertThat(fs.getFileStatus(hdfsFile2).getReplication(), is(oldRepFactor));
 
 
+      // Tests for setrep
+
+      // Tests for setrep on a directory and make sure it is applied recursively.
+      argv = new String[] { "-setrep", newRepFactor.toString(), testdir1 };
+      assertThat(shell.run(argv), is(SUCCESS));
+      assertThat(fs.getFileStatus(hdfsFile1).getReplication(), is(newRepFactor));
+      assertThat(fs.getFileStatus(hdfsFile2).getReplication(), is(newRepFactor));
+
+    } finally {
+      if (shell != null) {
+        shell.close();
+      }
+
+      cluster.shutdown();
+    }
   }
   }
 
 
   /**
   /**

+ 62 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java

@@ -20,20 +20,25 @@ package org.apache.hadoop.hdfs;
 
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICES;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICES;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
-import org.apache.hadoop.util.Shell;
-
-import static org.junit.Assert.*;
-import org.junit.Assume;
-import static org.hamcrest.CoreMatchers.*;
+import static org.hamcrest.CoreMatchers.not;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
@@ -54,8 +59,11 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Shell;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
@@ -539,6 +547,55 @@ public class TestDFSUtil {
     assertEquals("ns1", DFSUtil.getSecondaryNameServiceId(conf));
     assertEquals("ns1", DFSUtil.getSecondaryNameServiceId(conf));
   }
   }
 
 
+  @Test
+  public void testGetHaNnHttpAddresses() throws IOException {
+    final String LOGICAL_HOST_NAME = "ns1";
+    final String NS1_NN1_ADDR      = "ns1-nn1.example.com:8020";
+    final String NS1_NN2_ADDR      = "ns1-nn2.example.com:8020";
+
+    Configuration conf = createWebHDFSHAConfiguration(LOGICAL_HOST_NAME, NS1_NN1_ADDR, NS1_NN2_ADDR);
+
+    Map<String, Map<String, InetSocketAddress>> map =
+        DFSUtil.getHaNnHttpAddresses(conf);
+
+    assertEquals(NS1_NN1_ADDR, map.get("ns1").get("nn1").toString());
+    assertEquals(NS1_NN2_ADDR, map.get("ns1").get("nn2").toString());
+  }
+
+  @Test
+  public void testResolve() throws IOException, URISyntaxException {
+    final String LOGICAL_HOST_NAME = "ns1";
+    final String NS1_NN1_HOST      = "ns1-nn1.example.com";
+    final String NS1_NN2_HOST      = "ns1-nn2.example.com";
+    final String NS1_NN1_ADDR      = "ns1-nn1.example.com:8020";
+    final String NS1_NN2_ADDR      = "ns1-nn2.example.com:8020";
+    final int DEFAULT_PORT         = NameNode.DEFAULT_PORT;
+
+    Configuration conf = createWebHDFSHAConfiguration(LOGICAL_HOST_NAME, NS1_NN1_ADDR, NS1_NN2_ADDR);
+    URI uri = new URI("webhdfs://ns1");
+    assertTrue(HAUtil.isLogicalUri(conf, uri));
+    InetSocketAddress[] addrs = DFSUtil.resolve(uri, DEFAULT_PORT, conf);
+    assertArrayEquals(new InetSocketAddress[] {
+      new InetSocketAddress(NS1_NN1_HOST, DEFAULT_PORT),
+      new InetSocketAddress(NS1_NN2_HOST, DEFAULT_PORT),
+    }, addrs);
+  }
+
+  private static Configuration createWebHDFSHAConfiguration(String logicalHostName, String nnaddr1, String nnaddr2) {
+    HdfsConfiguration conf = new HdfsConfiguration();
+
+    conf.set(DFS_NAMESERVICES, "ns1");
+    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"),"nn1,nn2");
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_HTTP_ADDRESS_KEY, "ns1", "nn1"), nnaddr1);
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_HTTP_ADDRESS_KEY, "ns1", "nn2"), nnaddr2);
+
+    conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalHostName,
+        ConfiguredFailoverProxyProvider.class.getName());
+    return conf;
+  }
+
   @Test
   @Test
   public void testSubstituteForWildcardAddress() throws IOException {
   public void testSubstituteForWildcardAddress() throws IOException {
     assertEquals("foo:12345",
     assertEquals("foo:12345",

+ 39 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java

@@ -459,4 +459,43 @@ public class TestDatanodeBlockScanner {
     assertArrayEquals(expectedSubDirs, ReplicaInfo.parseSubDirs(testFile).subDirs);
     assertArrayEquals(expectedSubDirs, ReplicaInfo.parseSubDirs(testFile).subDirs);
     assertEquals(BASE_PATH, ReplicaInfo.parseSubDirs(testFile).baseDirPath);
     assertEquals(BASE_PATH, ReplicaInfo.parseSubDirs(testFile).baseDirPath);
   }
   }
+
+  @Test
+  public void testDuplicateScans() throws Exception {
+    long startTime = Time.now();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(new Configuration())
+        .numDataNodes(1).build();
+    FileSystem fs = null;
+    try {
+      fs = cluster.getFileSystem();
+      DataNode dataNode = cluster.getDataNodes().get(0);
+      int infoPort = dataNode.getInfoPort();
+      long scanTimeBefore = 0, scanTimeAfter = 0;
+      for (int i = 1; i < 10; i++) {
+        Path fileName = new Path("/test" + i);
+        DFSTestUtil.createFile(fs, fileName, 1024, (short) 1, 1000L);
+        waitForVerification(infoPort, fs, fileName, i, startTime, TIMEOUT);
+        if (i > 1) {
+          scanTimeAfter = DataNodeTestUtils.getLatestScanTime(dataNode,
+              DFSTestUtil.getFirstBlock(fs, new Path("/test" + (i - 1))));
+          assertFalse("scan time shoud not be 0", scanTimeAfter == 0);
+          assertEquals("There should not be duplicate scan", scanTimeBefore,
+              scanTimeAfter);
+        }
+
+        scanTimeBefore = DataNodeTestUtils.getLatestScanTime(dataNode,
+            DFSTestUtil.getFirstBlock(fs, new Path("/test" + i)));
+      }
+      cluster.restartDataNode(0);
+      Thread.sleep(10000);
+      dataNode = cluster.getDataNodes().get(0);
+      scanTimeAfter = DataNodeTestUtils.getLatestScanTime(dataNode,
+          DFSTestUtil.getFirstBlock(fs, new Path("/test" + (9))));
+      assertEquals("There should not be duplicate scan", scanTimeBefore,
+          scanTimeAfter);
+    } finally {
+      IOUtils.closeStream(fs);
+      cluster.shutdown();
+    }
+  }
 }
 }

+ 13 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java

@@ -17,13 +17,6 @@
  */
  */
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-
-import java.net.InetSocketAddress;
-import java.security.Permission;
-
 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.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -40,6 +33,13 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import java.net.InetSocketAddress;
+import java.security.Permission;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+
 /**
 /**
  * This class tests data node registration.
  * This class tests data node registration.
  */
  */
@@ -157,7 +157,8 @@ public class TestDatanodeRegistration {
     final String DN_HOSTNAME = "localhost";
     final String DN_HOSTNAME = "localhost";
     final int DN_XFER_PORT = 12345;
     final int DN_XFER_PORT = 12345;
     final int DN_INFO_PORT = 12346;
     final int DN_INFO_PORT = 12346;
-    final int DN_IPC_PORT = 12347;
+    final int DN_INFO_SECURE_PORT = 12347;
+    final int DN_IPC_PORT = 12348;
     Configuration conf = new HdfsConfiguration();
     Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
     try {
     try {
@@ -172,7 +173,8 @@ public class TestDatanodeRegistration {
 
 
       // register a datanode
       // register a datanode
       DatanodeID dnId = new DatanodeID(DN_IP_ADDR, DN_HOSTNAME,
       DatanodeID dnId = new DatanodeID(DN_IP_ADDR, DN_HOSTNAME,
-          "fake-storage-id", DN_XFER_PORT, DN_INFO_PORT, DN_IPC_PORT);
+          "fake-storage-id", DN_XFER_PORT, DN_INFO_PORT, DN_INFO_SECURE_PORT,
+          DN_IPC_PORT);
       long nnCTime = cluster.getNamesystem().getFSImage().getStorage()
       long nnCTime = cluster.getNamesystem().getFSImage().getStorage()
           .getCTime();
           .getCTime();
       StorageInfo mockStorageInfo = mock(StorageInfo.class);
       StorageInfo mockStorageInfo = mock(StorageInfo.class);
@@ -188,7 +190,8 @@ public class TestDatanodeRegistration {
 
 
       // register the same datanode again with a different storage ID
       // register the same datanode again with a different storage ID
       dnId = new DatanodeID(DN_IP_ADDR, DN_HOSTNAME,
       dnId = new DatanodeID(DN_IP_ADDR, DN_HOSTNAME,
-          "changed-fake-storage-id", DN_XFER_PORT, DN_INFO_PORT, DN_IPC_PORT);
+          "changed-fake-storage-id", DN_XFER_PORT, DN_INFO_PORT,
+          DN_INFO_SECURE_PORT, DN_IPC_PORT);
       dnReg = new DatanodeRegistration(dnId,
       dnReg = new DatanodeRegistration(dnId,
           mockStorageInfo, null, VersionInfo.getVersion());
           mockStorageInfo, null, VersionInfo.getVersion());
       rpcServer.registerDatanode(dnReg);
       rpcServer.registerDatanode(dnReg);

+ 9 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java

@@ -370,13 +370,20 @@ public class TestDecommission {
       for (int i = 0; i < numNamenodes; i++) {
       for (int i = 0; i < numNamenodes; i++) {
         ArrayList<DatanodeInfo> decommissionedNodes = namenodeDecomList.get(i);
         ArrayList<DatanodeInfo> decommissionedNodes = namenodeDecomList.get(i);
         FileSystem fileSys = cluster.getFileSystem(i);
         FileSystem fileSys = cluster.getFileSystem(i);
+        FSNamesystem ns = cluster.getNamesystem(i);
+
         writeFile(fileSys, file1, replicas);
         writeFile(fileSys, file1, replicas);
-        
+
+        int deadDecomissioned = ns.getNumDecomDeadDataNodes();
+        int liveDecomissioned = ns.getNumDecomLiveDataNodes();
+
         // Decommission one node. Verify that node is decommissioned.
         // Decommission one node. Verify that node is decommissioned.
         DatanodeInfo decomNode = decommissionNode(i, decommissionedNodes,
         DatanodeInfo decomNode = decommissionNode(i, decommissionedNodes,
             AdminStates.DECOMMISSIONED);
             AdminStates.DECOMMISSIONED);
         decommissionedNodes.add(decomNode);
         decommissionedNodes.add(decomNode);
-        
+        assertEquals(deadDecomissioned, ns.getNumDecomDeadDataNodes());
+        assertEquals(liveDecomissioned + 1, ns.getNumDecomLiveDataNodes());
+
         // Ensure decommissioned datanode is not automatically shutdown
         // Ensure decommissioned datanode is not automatically shutdown
         DFSClient client = getDfsClient(cluster.getNameNode(i), conf);
         DFSClient client = getDfsClient(cluster.getNameNode(i), conf);
         assertEquals("All datanodes must be alive", numDatanodes, 
         assertEquals("All datanodes must be alive", numDatanodes, 

+ 40 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -31,8 +31,10 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Random;
 import java.util.Random;
 
 
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.ArrayUtils;
@@ -47,9 +49,11 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
@@ -90,6 +94,19 @@ public class TestDistributedFileSystem {
     return conf;
     return conf;
   }
   }
 
 
+  @Test
+  public void testEmptyDelegationToken() throws IOException {
+    Configuration conf = getTestConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      FileSystem fileSys = cluster.getFileSystem();
+      fileSys.getDelegationToken("");
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   @Test
   @Test
   public void testFileSystemCloseAll() throws Exception {
   public void testFileSystemCloseAll() throws Exception {
     Configuration conf = getTestConfiguration();
     Configuration conf = getTestConfiguration();
@@ -226,7 +243,7 @@ public class TestDistributedFileSystem {
       final long millis = Time.now();
       final long millis = Time.now();
 
 
       {
       {
-        DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
+        final DistributedFileSystem dfs = cluster.getFileSystem();
         dfs.dfs.getLeaseRenewer().setGraceSleepPeriod(grace);
         dfs.dfs.getLeaseRenewer().setGraceSleepPeriod(grace);
         assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
         assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
   
   
@@ -326,7 +343,7 @@ public class TestDistributedFileSystem {
       }
       }
 
 
       {
       {
-        DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
+        final DistributedFileSystem dfs = cluster.getFileSystem();
         assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
         assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
 
 
         //open and check the file
         //open and check the file
@@ -835,4 +852,25 @@ public class TestDistributedFileSystem {
     }
     }
   }
   }
   
   
+  @Test(timeout=60000)
+  public void testListFiles() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    
+    try {
+      DistributedFileSystem fs = cluster.getFileSystem();
+  
+      final Path relative = new Path("relative");
+      fs.create(new Path(relative, "foo")).close();
+  
+      final List<LocatedFileStatus> retVal = new ArrayList<LocatedFileStatus>();
+      final RemoteIterator<LocatedFileStatus> iter = fs.listFiles(relative, true);
+      while (iter.hasNext()) {
+        retVal.add(iter.next());
+      }
+      System.out.println("retVal = " + retVal);
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }
 }

+ 8 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileInputStreamCache.java

@@ -17,12 +17,7 @@
  */
  */
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-
 import junit.framework.Assert;
 import junit.framework.Assert;
-
 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.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -31,6 +26,10 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
 public class TestFileInputStreamCache {
 public class TestFileInputStreamCache {
   static final Log LOG = LogFactory.getLog(TestFileInputStreamCache.class);
   static final Log LOG = LogFactory.getLog(TestFileInputStreamCache.class);
 
 
@@ -80,7 +79,7 @@ public class TestFileInputStreamCache {
   public void testAddAndRetrieve() throws Exception {
   public void testAddAndRetrieve() throws Exception {
     FileInputStreamCache cache = new FileInputStreamCache(1, 1000000);
     FileInputStreamCache cache = new FileInputStreamCache(1, 1000000);
     DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost", 
     DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost", 
-        "xyzzy", 8080, 9090, 7070);
+        "xyzzy", 8080, 9090, 7070, 6060);
     ExtendedBlock block = new ExtendedBlock("poolid", 123);
     ExtendedBlock block = new ExtendedBlock("poolid", 123);
     TestFileDescriptorPair pair = new TestFileDescriptorPair();
     TestFileDescriptorPair pair = new TestFileDescriptorPair();
     cache.put(dnId, block, pair.getFileInputStreams());
     cache.put(dnId, block, pair.getFileInputStreams());
@@ -94,7 +93,7 @@ public class TestFileInputStreamCache {
   public void testExpiry() throws Exception {
   public void testExpiry() throws Exception {
     FileInputStreamCache cache = new FileInputStreamCache(1, 10);
     FileInputStreamCache cache = new FileInputStreamCache(1, 10);
     DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost", 
     DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost", 
-        "xyzzy", 8080, 9090, 7070);
+        "xyzzy", 8080, 9090, 7070, 6060);
     ExtendedBlock block = new ExtendedBlock("poolid", 123);
     ExtendedBlock block = new ExtendedBlock("poolid", 123);
     TestFileDescriptorPair pair = new TestFileDescriptorPair();
     TestFileDescriptorPair pair = new TestFileDescriptorPair();
     cache.put(dnId, block, pair.getFileInputStreams());
     cache.put(dnId, block, pair.getFileInputStreams());
@@ -109,12 +108,12 @@ public class TestFileInputStreamCache {
   public void testEviction() throws Exception {
   public void testEviction() throws Exception {
     FileInputStreamCache cache = new FileInputStreamCache(1, 10000000);
     FileInputStreamCache cache = new FileInputStreamCache(1, 10000000);
     DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost", 
     DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost", 
-        "xyzzy", 8080, 9090, 7070);
+        "xyzzy", 8080, 9090, 7070, 6060);
     ExtendedBlock block = new ExtendedBlock("poolid", 123);
     ExtendedBlock block = new ExtendedBlock("poolid", 123);
     TestFileDescriptorPair pair = new TestFileDescriptorPair();
     TestFileDescriptorPair pair = new TestFileDescriptorPair();
     cache.put(dnId, block, pair.getFileInputStreams());
     cache.put(dnId, block, pair.getFileInputStreams());
     DatanodeID dnId2 = new DatanodeID("127.0.0.1", "localhost", 
     DatanodeID dnId2 = new DatanodeID("127.0.0.1", "localhost", 
-        "xyzzy", 8081, 9091, 7071);
+        "xyzzy", 8081, 9091, 7071, 6061);
     TestFileDescriptorPair pair2 = new TestFileDescriptorPair();
     TestFileDescriptorPair pair2 = new TestFileDescriptorPair();
     cache.put(dnId2, block, pair2.getFileInputStreams());
     cache.put(dnId2, block, pair2.getFileInputStreams());
     FileInputStream fis[] = cache.get(dnId, block);
     FileInputStream fis[] = cache.get(dnId, block);

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

@@ -88,19 +88,21 @@ public class TestHftpDelegationToken {
     URI fsUri = URI.create("hftp://localhost");
     URI fsUri = URI.create("hftp://localhost");
     MyHftpFileSystem fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
     MyHftpFileSystem fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpPort, fs.getCanonicalUri().getPort());
     assertEquals(httpPort, fs.getCanonicalUri().getPort());
-    checkTokenSelection(fs, httpsPort, conf); // should still use secure port
+    checkTokenSelection(fs, httpPort, conf);
 
 
     // test with explicit default port
     // test with explicit default port
+    // Make sure it uses the port from the hftp URI.
     fsUri = URI.create("hftp://localhost:"+httpPort);
     fsUri = URI.create("hftp://localhost:"+httpPort);
     fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
     fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpPort, fs.getCanonicalUri().getPort());
     assertEquals(httpPort, fs.getCanonicalUri().getPort());
-    checkTokenSelection(fs, httpsPort, conf); // should still use secure port
+    checkTokenSelection(fs, httpPort, conf);
     
     
     // test with non-default port
     // test with non-default port
+    // Make sure it uses the port from the hftp URI.
     fsUri = URI.create("hftp://localhost:"+(httpPort+1));
     fsUri = URI.create("hftp://localhost:"+(httpPort+1));
     fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
     fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpPort+1, fs.getCanonicalUri().getPort());
     assertEquals(httpPort+1, fs.getCanonicalUri().getPort());
-    checkTokenSelection(fs, httpsPort, conf); // should still use secure port
+    checkTokenSelection(fs, httpPort + 1, conf);
     
     
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY, 5);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY, 5);
   }
   }
@@ -178,7 +180,7 @@ public class TestHftpDelegationToken {
       }
       }
       assertNotNull(ex);
       assertNotNull(ex);
       assertNotNull(ex.getCause());
       assertNotNull(ex.getCause());
-      assertEquals("Unexpected end of file from server",
+      assertEquals("Remote host closed connection during handshake",
                    ex.getCause().getMessage());
                    ex.getCause().getMessage());
     } finally {
     } finally {
       t.interrupt();
       t.interrupt();

+ 10 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpFileSystem.java

@@ -294,11 +294,13 @@ public class TestHftpFileSystem {
     HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf);
     HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf);
 
 
     assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, fs.getDefaultPort());
     assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, fs.getDefaultPort());
-    assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultSecurePort());
 
 
     assertEquals(uri, fs.getUri());
     assertEquals(uri, fs.getUri());
+
+    // HFTP uses http to get the token so canonical service name should
+    // return the http port.
     assertEquals(
     assertEquals(
-        "127.0.0.1:"+DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT,
+        "127.0.0.1:" + DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT,
         fs.getCanonicalServiceName()
         fs.getCanonicalServiceName()
     );
     );
   }
   }
@@ -307,17 +309,18 @@ public class TestHftpFileSystem {
   public void testHftpCustomDefaultPorts() throws IOException {
   public void testHftpCustomDefaultPorts() throws IOException {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY, 123);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY, 123);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY, 456);
 
 
     URI uri = URI.create("hftp://localhost");
     URI uri = URI.create("hftp://localhost");
     HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf);
     HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf);
 
 
     assertEquals(123, fs.getDefaultPort());
     assertEquals(123, fs.getDefaultPort());
-    assertEquals(456, fs.getDefaultSecurePort());
     
     
     assertEquals(uri, fs.getUri());
     assertEquals(uri, fs.getUri());
+
+    // HFTP uses http to get the token so canonical service name should
+    // return the http port.
     assertEquals(
     assertEquals(
-        "127.0.0.1:456",
+        "127.0.0.1:123",
         fs.getCanonicalServiceName()
         fs.getCanonicalServiceName()
     );
     );
   }
   }
@@ -329,11 +332,10 @@ public class TestHftpFileSystem {
     HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf);
     HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf);
 
 
     assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, fs.getDefaultPort());
     assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, fs.getDefaultPort());
-    assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultSecurePort());
 
 
     assertEquals(uri, fs.getUri());
     assertEquals(uri, fs.getUri());
     assertEquals(
     assertEquals(
-        "127.0.0.1:"+DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT,
+        "127.0.0.1:123",
         fs.getCanonicalServiceName()
         fs.getCanonicalServiceName()
     );
     );
   }
   }
@@ -342,17 +344,15 @@ public class TestHftpFileSystem {
   public void testHftpCustomUriPortWithCustomDefaultPorts() throws IOException {
   public void testHftpCustomUriPortWithCustomDefaultPorts() throws IOException {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY, 123);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY, 123);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY, 456);
 
 
     URI uri = URI.create("hftp://localhost:789");
     URI uri = URI.create("hftp://localhost:789");
     HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf);
     HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf);
 
 
     assertEquals(123, fs.getDefaultPort());
     assertEquals(123, fs.getDefaultPort());
-    assertEquals(456, fs.getDefaultSecurePort());
    
    
     assertEquals(uri, fs.getUri()); 
     assertEquals(uri, fs.getUri()); 
     assertEquals(
     assertEquals(
-        "127.0.0.1:456",
+        "127.0.0.1:789",
         fs.getCanonicalServiceName()
         fs.getCanonicalServiceName()
     );
     );
   }
   }
@@ -366,7 +366,6 @@ public class TestHftpFileSystem {
     HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf);
     HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf);
 
 
     assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultPort());
     assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultPort());
-    assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultSecurePort());
 
 
     assertEquals(uri, fs.getUri());
     assertEquals(uri, fs.getUri());
     assertEquals(
     assertEquals(
@@ -385,7 +384,6 @@ public class TestHftpFileSystem {
     HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf);
     HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf);
 
 
     assertEquals(456, fs.getDefaultPort());
     assertEquals(456, fs.getDefaultPort());
-    assertEquals(456, fs.getDefaultSecurePort());
     
     
     assertEquals(uri, fs.getUri());
     assertEquals(uri, fs.getUri());
     assertEquals(
     assertEquals(
@@ -401,7 +399,6 @@ public class TestHftpFileSystem {
     HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf);
     HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf);
 
 
     assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultPort());
     assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultPort());
-    assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultSecurePort());
 
 
     assertEquals(uri, fs.getUri());
     assertEquals(uri, fs.getUri());
     assertEquals(
     assertEquals(
@@ -420,7 +417,6 @@ public class TestHftpFileSystem {
     HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf);
     HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf);
 
 
     assertEquals(456, fs.getDefaultPort());
     assertEquals(456, fs.getDefaultPort());
-    assertEquals(456, fs.getDefaultSecurePort());
 
 
     assertEquals(uri, fs.getUri());
     assertEquals(uri, fs.getUri());
     assertEquals(
     assertEquals(

+ 39 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestNameNodeHttpServer.java

@@ -0,0 +1,39 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+public class TestNameNodeHttpServer {
+
+  @Test
+  public void testSslConfiguration() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY, true);
+    System.setProperty("jetty.ssl.password", "foo");
+    System.setProperty("jetty.ssl.keypassword", "bar");
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+        .build();
+
+    cluster.shutdown();
+  }
+}

+ 13 - 16
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPeerCache.java

@@ -17,26 +17,23 @@
  */
  */
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.channels.ReadableByteChannel;
-
+import com.google.common.collect.HashMultiset;
 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.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.mockito.stubbing.Answer;
 
 
-import com.google.common.collect.HashMultiset;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.channels.ReadableByteChannel;
+
+import static org.junit.Assert.*;
 
 
 public class TestPeerCache {
 public class TestPeerCache {
   static final Log LOG = LogFactory.getLog(TestPeerCache.class);
   static final Log LOG = LogFactory.getLog(TestPeerCache.class);
@@ -150,7 +147,7 @@ public class TestPeerCache {
     PeerCache cache = new PeerCache(3, 100000);
     PeerCache cache = new PeerCache(3, 100000);
     DatanodeID dnId = new DatanodeID("192.168.0.1",
     DatanodeID dnId = new DatanodeID("192.168.0.1",
           "fakehostname", "fake_storage_id",
           "fakehostname", "fake_storage_id",
-          100, 101, 102);
+          100, 101, 102, 103);
     FakePeer peer = new FakePeer(dnId, false);
     FakePeer peer = new FakePeer(dnId, false);
     cache.put(dnId, peer);
     cache.put(dnId, peer);
     assertTrue(!peer.isClosed());
     assertTrue(!peer.isClosed());
@@ -170,7 +167,7 @@ public class TestPeerCache {
     for (int i = 0; i < CAPACITY; ++i) {
     for (int i = 0; i < CAPACITY; ++i) {
       dnIds[i] = new DatanodeID("192.168.0.1",
       dnIds[i] = new DatanodeID("192.168.0.1",
           "fakehostname_" + i, "fake_storage_id",
           "fakehostname_" + i, "fake_storage_id",
-          100, 101, 102);
+          100, 101, 102, 103);
       peers[i] = new FakePeer(dnIds[i], false);
       peers[i] = new FakePeer(dnIds[i], false);
     }
     }
     for (int i = 0; i < CAPACITY; ++i) {
     for (int i = 0; i < CAPACITY; ++i) {
@@ -201,7 +198,7 @@ public class TestPeerCache {
     for (int i = 0; i < dnIds.length; ++i) {
     for (int i = 0; i < dnIds.length; ++i) {
       dnIds[i] = new DatanodeID("192.168.0.1",
       dnIds[i] = new DatanodeID("192.168.0.1",
           "fakehostname_" + i, "fake_storage_id_" + i,
           "fakehostname_" + i, "fake_storage_id_" + i,
-          100, 101, 102);
+          100, 101, 102, 103);
       peers[i] = new FakePeer(dnIds[i], false);
       peers[i] = new FakePeer(dnIds[i], false);
     }
     }
     for (int i = 0; i < CAPACITY; ++i) {
     for (int i = 0; i < CAPACITY; ++i) {
@@ -232,7 +229,7 @@ public class TestPeerCache {
     PeerCache cache = new PeerCache(CAPACITY, 100000);
     PeerCache cache = new PeerCache(CAPACITY, 100000);
     DatanodeID dnId = new DatanodeID("192.168.0.1",
     DatanodeID dnId = new DatanodeID("192.168.0.1",
           "fakehostname", "fake_storage_id",
           "fakehostname", "fake_storage_id",
-          100, 101, 102);
+          100, 101, 102, 103);
     HashMultiset<FakePeer> peers = HashMultiset.create(CAPACITY);
     HashMultiset<FakePeer> peers = HashMultiset.create(CAPACITY);
     for (int i = 0; i < CAPACITY; ++i) {
     for (int i = 0; i < CAPACITY; ++i) {
       FakePeer peer = new FakePeer(dnId, false);
       FakePeer peer = new FakePeer(dnId, false);
@@ -257,7 +254,7 @@ public class TestPeerCache {
     PeerCache cache = new PeerCache(CAPACITY, 100000);
     PeerCache cache = new PeerCache(CAPACITY, 100000);
     DatanodeID dnId = new DatanodeID("192.168.0.1",
     DatanodeID dnId = new DatanodeID("192.168.0.1",
           "fakehostname", "fake_storage_id",
           "fakehostname", "fake_storage_id",
-          100, 101, 102);
+          100, 101, 102, 103);
     HashMultiset<FakePeer> peers = HashMultiset.create(CAPACITY);
     HashMultiset<FakePeer> peers = HashMultiset.create(CAPACITY);
     for (int i = 0; i < CAPACITY; ++i) {
     for (int i = 0; i < CAPACITY; ++i) {
       FakePeer peer = new FakePeer(dnId, i == CAPACITY - 1);
       FakePeer peer = new FakePeer(dnId, i == CAPACITY - 1);

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java

@@ -577,4 +577,64 @@ public class TestShortCircuitLocalRead {
     System.out.println("Iteration " + iteration + " took " + (end - start));
     System.out.println("Iteration " + iteration + " took " + (end - start));
     fs.delete(file1, false);
     fs.delete(file1, false);
   }
   }
+
+  public void testReadWithRemoteBlockReader() throws IOException, InterruptedException {
+    doTestShortCircuitReadWithRemoteBlockReader(true, 3*blockSize+100, getCurrentUser(), 0, false);
+  }
+
+  /**
+   * Test that file data can be read by reading the block
+   * through RemoteBlockReader
+   * @throws IOException
+  */
+  public void doTestShortCircuitReadWithRemoteBlockReader(boolean ignoreChecksum, int size, String shortCircuitUser,
+                                                          int readOffset, boolean shortCircuitFails) throws IOException, InterruptedException {
+    Configuration conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER, true);
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
+             .format(true).build();
+    FileSystem fs = cluster.getFileSystem();
+    // check that / exists
+    Path path = new Path("/");
+    URI uri = cluster.getURI();
+    assertTrue("/ should be a directory", fs.getFileStatus(path)
+                .isDirectory() == true);
+
+    byte[] fileData = AppendTestUtil.randomBytes(seed, size);
+    Path file1 = new Path("filelocal.dat");
+    FSDataOutputStream stm = createFile(fs, file1, 1);
+
+    stm.write(fileData);
+    stm.close();
+    try {
+      checkFileContent(uri, file1, fileData, readOffset, shortCircuitUser, conf, shortCircuitFails);
+      //RemoteBlockReader have unsupported method read(ByteBuffer bf)
+      assertTrue("RemoteBlockReader unsupported method read(ByteBuffer bf) error",
+                    checkUnsupportedMethod(fs, file1, fileData, readOffset));
+    } catch(IOException e) {
+      throw new IOException("doTestShortCircuitReadWithRemoteBlockReader ex error ", e);
+    } catch(InterruptedException inEx) {
+      throw inEx;
+    } finally {
+      fs.close();
+      cluster.shutdown();
+    }
+  }
+
+  private boolean checkUnsupportedMethod(FileSystem fs, Path file,
+                                           byte[] expected, int readOffset) throws IOException {
+    HdfsDataInputStream stm = (HdfsDataInputStream)fs.open(file);
+    ByteBuffer actual = ByteBuffer.allocateDirect(expected.length - readOffset);
+    IOUtils.skipFully(stm, readOffset);
+    try {
+      stm.read(actual);
+    } catch(UnsupportedOperationException unex) {
+      return true;
+    }
+    return false;
+  }
+
+
 }
 }

+ 213 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java

@@ -0,0 +1,213 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.tools.DFSAdmin;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Tool;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * This class includes end-to-end tests for snapshot related FsShell and
+ * DFSAdmin commands.
+ */
+public class TestSnapshotCommands {
+
+  private static Configuration conf;
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem fs;
+  
+  @BeforeClass
+  public static void clusterSetUp() throws IOException {
+    conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+  }
+
+  @AfterClass
+  public static void clusterShutdown() throws IOException{
+    if(fs != null){
+      fs.close();
+    }
+    if(cluster != null){
+      cluster.shutdown();
+    }
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    fs.mkdirs(new Path("/sub1"));
+    fs.allowSnapshot(new Path("/sub1"));
+    fs.mkdirs(new Path("/sub1/sub1sub1"));
+    fs.mkdirs(new Path("/sub1/sub1sub2"));
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    if (fs.exists(new Path("/sub1"))) {
+      if (fs.exists(new Path("/sub1/.snapshot"))) {
+        for (FileStatus st : fs.listStatus(new Path("/sub1/.snapshot"))) {
+          fs.deleteSnapshot(new Path("/sub1"), st.getPath().getName());
+        }
+        fs.disallowSnapshot(new Path("/sub1"));
+      }
+      fs.delete(new Path("/sub1"), true);
+    }
+  }
+
+  private void toolRun(Tool tool, String cmd, int retcode, String contain)
+      throws Exception {
+    String [] cmds = StringUtils.split(cmd, ' ');
+    System.out.flush();
+    System.err.flush();
+    PrintStream origOut = System.out;
+    PrintStream origErr = System.err;
+    String output = null;
+    int ret = 0;
+    try {
+      ByteArrayOutputStream bs = new ByteArrayOutputStream(1024);
+      PrintStream out = new PrintStream(bs);
+      System.setOut(out);
+      System.setErr(out);
+      ret = tool.run(cmds);
+      System.out.flush();
+      System.err.flush();
+      out.close();
+      output = bs.toString();
+    } finally {
+      System.setOut(origOut);
+      System.setErr(origErr);
+    }
+    System.out.println("Output for command: " + cmd + " retcode: " + ret);
+    if (output != null) {
+      System.out.println(output);
+    }
+    assertEquals(retcode, ret);
+    if (contain != null) {
+      assertTrue(output.contains(contain));
+    }
+  }
+
+  private void FsShellRun(String cmd, int retcode, String contain)
+      throws Exception {
+    FsShell shell = new FsShell(new Configuration(conf));
+    toolRun(shell, cmd, retcode, contain);
+  }  
+
+  private void DFSAdminRun(String cmd, int retcode, String contain)
+      throws Exception {
+    DFSAdmin admin = new DFSAdmin(new Configuration(conf));
+    toolRun(admin, cmd, retcode, contain);
+  }
+
+  private void FsShellRun(String cmd) throws Exception {
+    FsShellRun(cmd, 0, null);
+  }
+
+  @Test
+  public void testAllowSnapshot() throws Exception {
+    // Idempotent test
+    DFSAdminRun("-allowSnapshot /sub1", 0, "Allowing snaphot on /sub1 succeeded");
+    // allow normal dir success 
+    FsShellRun("-mkdir /sub2");
+    DFSAdminRun("-allowSnapshot /sub2", 0, "Allowing snaphot on /sub2 succeeded");
+    // allow non-exists dir failed
+    DFSAdminRun("-allowSnapshot /sub3", -1, null);
+  }
+
+  @Test
+  public void testCreateSnapshot() throws Exception {
+    // test createSnapshot
+    FsShellRun("-createSnapshot /sub1 sn0", 0, "Created snapshot /sub1/.snapshot/sn0");
+    FsShellRun("-createSnapshot /sub1 sn0", 1, "there is already a snapshot with the same name \"sn0\"");
+    FsShellRun("-rmr /sub1/sub1sub2");
+    FsShellRun("-mkdir /sub1/sub1sub3");
+    FsShellRun("-createSnapshot /sub1 sn1", 0, "Created snapshot /sub1/.snapshot/sn1");
+    // check snapshot contents
+    FsShellRun("-ls /sub1", 0, "/sub1/sub1sub1");
+    FsShellRun("-ls /sub1", 0, "/sub1/sub1sub3");
+    FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn0");
+    FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn1");
+    FsShellRun("-ls /sub1/.snapshot/sn0", 0, "/sub1/.snapshot/sn0/sub1sub1");
+    FsShellRun("-ls /sub1/.snapshot/sn0", 0, "/sub1/.snapshot/sn0/sub1sub2");
+    FsShellRun("-ls /sub1/.snapshot/sn1", 0, "/sub1/.snapshot/sn1/sub1sub1");
+    FsShellRun("-ls /sub1/.snapshot/sn1", 0, "/sub1/.snapshot/sn1/sub1sub3");
+  }
+
+  @Test
+  public void testMkdirUsingReservedName() throws Exception {
+    // test can not create dir with reserved name: .snapshot
+    FsShellRun("-ls /");
+    FsShellRun("-mkdir /.snapshot", 1, "File exists");
+    FsShellRun("-mkdir /sub1/.snapshot", 1, "File exists");
+    // mkdir -p ignore reserved name check if dir already exists
+    FsShellRun("-mkdir -p /sub1/.snapshot");
+    FsShellRun("-mkdir -p /sub1/sub1sub1/.snapshot", 1, "mkdir: \".snapshot\" is a reserved name.");
+  }
+
+  @Test
+  public void testRenameSnapshot() throws Exception {
+    FsShellRun("-createSnapshot /sub1 sn.orig");
+    FsShellRun("-renameSnapshot /sub1 sn.orig sn.rename");
+    FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn.rename");
+    FsShellRun("-ls /sub1/.snapshot/sn.rename", 0, "/sub1/.snapshot/sn.rename/sub1sub1");
+    FsShellRun("-ls /sub1/.snapshot/sn.rename", 0, "/sub1/.snapshot/sn.rename/sub1sub2");
+  }
+
+  @Test
+  public void testDeleteSnapshot() throws Exception {
+    FsShellRun("-createSnapshot /sub1 sn1");
+    FsShellRun("-deleteSnapshot /sub1 sn1");
+    FsShellRun("-deleteSnapshot /sub1 sn1", 1,
+        "deleteSnapshot: Cannot delete snapshot sn1 from path /sub1: the snapshot does not exist.");
+  }
+
+  @Test
+  public void testDisallowSnapshot() throws Exception {
+    FsShellRun("-createSnapshot /sub1 sn1");
+    // cannot delete snapshotable dir
+    FsShellRun("-rmr /sub1", 1, "The directory /sub1 cannot be deleted since /sub1 is snapshottable and already has snapshots");
+    DFSAdminRun("-disallowSnapshot /sub1", -1,
+        "disallowSnapshot: The directory /sub1 has snapshot(s). Please redo the operation after removing all the snapshots.");
+    FsShellRun("-deleteSnapshot /sub1 sn1");
+    DFSAdminRun("-disallowSnapshot /sub1", 0, "Disallowing snaphot on /sub1 succeeded");
+    // Idempotent test
+    DFSAdminRun("-disallowSnapshot /sub1", 0, "Disallowing snaphot on /sub1 succeeded");
+    // now it can be deleted
+    FsShellRun("-rmr /sub1");
+  }
+}

+ 7 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithNodeGroup.java

@@ -31,6 +31,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -39,9 +40,11 @@ import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithNodeGroup;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.net.NetworkTopologyWithNodeGroup;
+import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
-import junit.framework.Assert;
 
 
 /**
 /**
  * This class tests if a balancer schedules tasks correctly.
  * This class tests if a balancer schedules tasks correctly.
@@ -75,10 +78,9 @@ public class TestBalancerWithNodeGroup {
     Configuration conf = new HdfsConfiguration();
     Configuration conf = new HdfsConfiguration();
     TestBalancer.initConf(conf);
     TestBalancer.initConf(conf);
     conf.set(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY, 
     conf.set(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY, 
-        "org.apache.hadoop.net.NetworkTopologyWithNodeGroup");
-    conf.set("dfs.block.replicator.classname", 
-        "org.apache.hadoop.hdfs.server.blockmanagement." +
-        "BlockPlacementPolicyWithNodeGroup");
+        NetworkTopologyWithNodeGroup.class.getName());
+    conf.set(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, 
+        BlockPlacementPolicyWithNodeGroup.class.getName());
     return conf;
     return conf;
   }
   }
 
 

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCachedBlocksList.java

@@ -36,7 +36,7 @@ public class TestCachedBlocksList {
   @Test(timeout=60000)
   @Test(timeout=60000)
   public void testSingleList() {
   public void testSingleList() {
     DatanodeDescriptor dn = new DatanodeDescriptor(
     DatanodeDescriptor dn = new DatanodeDescriptor(
-      new DatanodeID("127.0.0.1", "localhost", "abcd", 5000, 5001, 5002));
+      new DatanodeID("127.0.0.1", "localhost", "abcd", 5000, 5001, 5002, 5003));
     CachedBlock[] blocks = new CachedBlock[] {
     CachedBlock[] blocks = new CachedBlock[] {
           new CachedBlock(0L, (short)1, true),
           new CachedBlock(0L, (short)1, true),
           new CachedBlock(1L, (short)1, true),
           new CachedBlock(1L, (short)1, true),
@@ -124,9 +124,9 @@ public class TestCachedBlocksList {
   public void testMultipleLists() {
   public void testMultipleLists() {
     DatanodeDescriptor[] datanodes = new DatanodeDescriptor[] {
     DatanodeDescriptor[] datanodes = new DatanodeDescriptor[] {
       new DatanodeDescriptor(
       new DatanodeDescriptor(
-        new DatanodeID("127.0.0.1", "localhost", "abcd", 5000, 5001, 5002)),
+        new DatanodeID("127.0.0.1", "localhost", "abcd", 5000, 5001, 5002, 5003)),
       new DatanodeDescriptor(
       new DatanodeDescriptor(
-        new DatanodeID("127.0.1.1", "localhost", "efgh", 6000, 6001, 6002)),
+        new DatanodeID("127.0.1.1", "localhost", "efgh", 6000, 6001, 6002, 6003)),
     };
     };
     CachedBlocksList[] lists = new CachedBlocksList[] {
     CachedBlocksList[] lists = new CachedBlocksList[] {
         datanodes[0].getPendingCached(),
         datanodes[0].getPendingCached(),

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java

@@ -157,8 +157,8 @@ public class TestRBWBlockInvalidation {
     // in the context of the test, whereas a random one is more accurate
     // in the context of the test, whereas a random one is more accurate
     // to what is seen in real clusters (nodes have random amounts of free
     // to what is seen in real clusters (nodes have random amounts of free
     // space)
     // space)
-    conf.setClass("dfs.block.replicator.classname", RandomDeleterPolicy.class,
-        BlockPlacementPolicy.class); 
+    conf.setClass(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
+        RandomDeleterPolicy.class, BlockPlacementPolicy.class); 
 
 
     // Speed up the test a bit with faster heartbeats.
     // Speed up the test a bit with faster heartbeats.
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);

+ 88 - 110
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java

@@ -29,9 +29,11 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Random;
 import java.util.Random;
+import java.util.Set;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
@@ -137,30 +139,25 @@ public class TestReplicationPolicy {
         0L, 0L, 4, 0); // overloaded
         0L, 0L, 4, 0); // overloaded
 
 
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename, 0, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(0);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
     
     
-    targets = replicator.chooseTarget(filename, 1, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(1);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     
     
-    targets = replicator.chooseTarget(filename,
-                                      2, dataNodes[0], new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(2);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
     
-    targets = replicator.chooseTarget(filename, 3, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(3);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
 
 
-    targets = replicator.chooseTarget(filename, 4, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(4);
     assertEquals(targets.length, 4);
     assertEquals(targets.length, 4);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]) ||
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]) ||
@@ -172,15 +169,38 @@ public class TestReplicationPolicy {
         HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
         HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
   }
   }
 
 
+  private static DatanodeDescriptor[] chooseTarget(int numOfReplicas) {
+    return chooseTarget(numOfReplicas, dataNodes[0]);
+  }
+
+  private static DatanodeDescriptor[] chooseTarget(int numOfReplicas,
+      DatanodeDescriptor writer) {
+    return chooseTarget(numOfReplicas, writer,
+        new ArrayList<DatanodeDescriptor>());
+  }
+
+  private static DatanodeDescriptor[] chooseTarget(int numOfReplicas,
+      List<DatanodeDescriptor> chosenNodes) {
+    return chooseTarget(numOfReplicas, dataNodes[0], chosenNodes);
+  }
+
+  private static DatanodeDescriptor[] chooseTarget(int numOfReplicas,
+      DatanodeDescriptor writer, List<DatanodeDescriptor> chosenNodes) {
+    return chooseTarget(numOfReplicas, writer, chosenNodes, null);
+  }
+
+  private static DatanodeDescriptor[] chooseTarget(int numOfReplicas,
+      List<DatanodeDescriptor> chosenNodes, Set<Node> excludedNodes) {
+    return chooseTarget(numOfReplicas, dataNodes[0], chosenNodes, excludedNodes);
+  }
+
   private static DatanodeDescriptor[] chooseTarget(
   private static DatanodeDescriptor[] chooseTarget(
-      BlockPlacementPolicyDefault policy,
       int numOfReplicas,
       int numOfReplicas,
       DatanodeDescriptor writer,
       DatanodeDescriptor writer,
       List<DatanodeDescriptor> chosenNodes,
       List<DatanodeDescriptor> chosenNodes,
-      HashMap<Node, Node> excludedNodes,
-      long blocksize) {
-    return policy.chooseTarget(numOfReplicas, writer, chosenNodes, false,
-        excludedNodes, blocksize);
+      Set<Node> excludedNodes) {
+    return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes,
+        false, excludedNodes, BLOCK_SIZE);
   }
   }
 
 
   /**
   /**
@@ -193,39 +213,34 @@ public class TestReplicationPolicy {
    */
    */
   @Test
   @Test
   public void testChooseTarget2() throws Exception { 
   public void testChooseTarget2() throws Exception { 
-    HashMap<Node, Node> excludedNodes;
+    Set<Node> excludedNodes;
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    BlockPlacementPolicyDefault repl = (BlockPlacementPolicyDefault)replicator;
     List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
     List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
     
     
-    excludedNodes = new HashMap<Node, Node>();
-    excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = chooseTarget(repl, 0, dataNodes[0], chosenNodes, excludedNodes,
-        BLOCK_SIZE);
+    excludedNodes = new HashSet<Node>();
+    excludedNodes.add(dataNodes[1]); 
+    targets = chooseTarget(0, chosenNodes, excludedNodes);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
     
     
     excludedNodes.clear();
     excludedNodes.clear();
     chosenNodes.clear();
     chosenNodes.clear();
-    excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = chooseTarget(repl, 1, dataNodes[0], chosenNodes, excludedNodes,
-        BLOCK_SIZE);
+    excludedNodes.add(dataNodes[1]); 
+    targets = chooseTarget(1, chosenNodes, excludedNodes);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     
     
     excludedNodes.clear();
     excludedNodes.clear();
     chosenNodes.clear();
     chosenNodes.clear();
-    excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = chooseTarget(repl, 2, dataNodes[0], chosenNodes, excludedNodes,
-        BLOCK_SIZE);
+    excludedNodes.add(dataNodes[1]); 
+    targets = chooseTarget(2, chosenNodes, excludedNodes);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
     
     excludedNodes.clear();
     excludedNodes.clear();
     chosenNodes.clear();
     chosenNodes.clear();
-    excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = chooseTarget(repl, 3, dataNodes[0], chosenNodes, excludedNodes,
-        BLOCK_SIZE);
+    excludedNodes.add(dataNodes[1]); 
+    targets = chooseTarget(3, chosenNodes, excludedNodes);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
@@ -233,9 +248,8 @@ public class TestReplicationPolicy {
     
     
     excludedNodes.clear();
     excludedNodes.clear();
     chosenNodes.clear();
     chosenNodes.clear();
-    excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = chooseTarget(repl, 4, dataNodes[0], chosenNodes, excludedNodes,
-        BLOCK_SIZE);
+    excludedNodes.add(dataNodes[1]); 
+    targets = chooseTarget(4, chosenNodes, excludedNodes);
     assertEquals(targets.length, 4);
     assertEquals(targets.length, 4);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     for(int i=1; i<4; i++) {
     for(int i=1; i<4; i++) {
@@ -247,9 +261,9 @@ public class TestReplicationPolicy {
 
 
     excludedNodes.clear();
     excludedNodes.clear();
     chosenNodes.clear();
     chosenNodes.clear();
-    excludedNodes.put(dataNodes[1], dataNodes[1]); 
+    excludedNodes.add(dataNodes[1]); 
     chosenNodes.add(dataNodes[2]);
     chosenNodes.add(dataNodes[2]);
-    targets = repl.chooseTarget(1, dataNodes[0], chosenNodes, true,
+    targets = replicator.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true,
         excludedNodes, BLOCK_SIZE);
         excludedNodes, BLOCK_SIZE);
     System.out.println("targets=" + Arrays.asList(targets));
     System.out.println("targets=" + Arrays.asList(targets));
     assertEquals(2, targets.length);
     assertEquals(2, targets.length);
@@ -276,30 +290,25 @@ public class TestReplicationPolicy {
         0L, 0L, 0, 0); // no space
         0L, 0L, 0, 0); // no space
         
         
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename, 0, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(0);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
     
     
-    targets = replicator.chooseTarget(filename, 1, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(1);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertEquals(targets[0], dataNodes[1]);
     assertEquals(targets[0], dataNodes[1]);
     
     
-    targets = replicator.chooseTarget(filename, 2, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(2);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertEquals(targets[0], dataNodes[1]);
     assertEquals(targets[0], dataNodes[1]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
     
-    targets = replicator.chooseTarget(filename, 3, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(3);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertEquals(targets[0], dataNodes[1]);
     assertEquals(targets[0], dataNodes[1]);
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
     
-    targets = replicator.chooseTarget(filename, 4, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(4);
     assertEquals(targets.length, 4);
     assertEquals(targets.length, 4);
     assertEquals(targets[0], dataNodes[1]);
     assertEquals(targets[0], dataNodes[1]);
     for(int i=1; i<4; i++) {
     for(int i=1; i<4; i++) {
@@ -332,23 +341,19 @@ public class TestReplicationPolicy {
     }
     }
       
       
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename, 0, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(0);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
     
     
-    targets = replicator.chooseTarget(filename, 1, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(1);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
     assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
     
     
-    targets = replicator.chooseTarget(filename, 2, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(2);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
     assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
     
-    targets = replicator.chooseTarget(filename, 3, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(3);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     for(int i=0; i<3; i++) {
     for(int i=0; i<3; i++) {
       assertFalse(cluster.isOnSameRack(targets[i], dataNodes[0]));
       assertFalse(cluster.isOnSameRack(targets[i], dataNodes[0]));
@@ -377,21 +382,17 @@ public class TestReplicationPolicy {
       DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r4");
       DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r4");
 
 
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename, 0, writerDesc,
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(0, writerDesc);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
 
 
-    targets = replicator.chooseTarget(filename, 1, writerDesc,
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(1, writerDesc);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
 
 
-    targets = replicator.chooseTarget(filename, 2, writerDesc,
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(2, writerDesc);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
 
 
-    targets = replicator.chooseTarget(filename, 3, writerDesc,
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(3, writerDesc);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
@@ -435,9 +436,7 @@ public class TestReplicationPolicy {
     
     
     // try to choose NUM_OF_DATANODES which is more than actually available
     // try to choose NUM_OF_DATANODES which is more than actually available
     // nodes.
     // nodes.
-    DatanodeDescriptor[] targets = replicator.chooseTarget(filename, 
-        NUM_OF_DATANODES, dataNodes[0], new ArrayList<DatanodeDescriptor>(),
-        BLOCK_SIZE);
+    DatanodeDescriptor[] targets = chooseTarget(NUM_OF_DATANODES);
     assertEquals(targets.length, NUM_OF_DATANODES - 2);
     assertEquals(targets.length, NUM_OF_DATANODES - 2);
 
 
     final List<LoggingEvent> log = appender.getLog();
     final List<LoggingEvent> log = appender.getLog();
@@ -480,17 +479,14 @@ public class TestReplicationPolicy {
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
     // We set the datanode[0] as stale, thus should choose datanode[1] since
     // We set the datanode[0] as stale, thus should choose datanode[1] since
     // datanode[1] is on the same rack with datanode[0] (writer)
     // datanode[1] is on the same rack with datanode[0] (writer)
-    targets = replicator.chooseTarget(filename, 1, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(1);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertEquals(targets[0], dataNodes[1]);
     assertEquals(targets[0], dataNodes[1]);
 
 
-    HashMap<Node, Node> excludedNodes = new HashMap<Node, Node>();
-    excludedNodes.put(dataNodes[1], dataNodes[1]);
+    Set<Node> excludedNodes = new HashSet<Node>();
+    excludedNodes.add(dataNodes[1]);
     List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
     List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
-    BlockPlacementPolicyDefault repl = (BlockPlacementPolicyDefault)replicator;
-    targets = chooseTarget(repl, 1, dataNodes[0], chosenNodes, excludedNodes,
-        BLOCK_SIZE);
+    targets = chooseTarget(1, chosenNodes, excludedNodes);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
     assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
     
     
@@ -517,33 +513,27 @@ public class TestReplicationPolicy {
     namenode.getNamesystem().getBlockManager()
     namenode.getNamesystem().getBlockManager()
       .getDatanodeManager().getHeartbeatManager().heartbeatCheck();
       .getDatanodeManager().getHeartbeatManager().heartbeatCheck();
 
 
-    DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename, 0, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    DatanodeDescriptor[] targets = chooseTarget(0);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
 
 
     // Since we have 6 datanodes total, stale nodes should
     // Since we have 6 datanodes total, stale nodes should
     // not be returned until we ask for more than 3 targets
     // not be returned until we ask for more than 3 targets
-    targets = replicator.chooseTarget(filename, 1, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(1);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertFalse(containsWithinRange(targets[0], dataNodes, 0, 2));
     assertFalse(containsWithinRange(targets[0], dataNodes, 0, 2));
 
 
-    targets = replicator.chooseTarget(filename, 2, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(2);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertFalse(containsWithinRange(targets[0], dataNodes, 0, 2));
     assertFalse(containsWithinRange(targets[0], dataNodes, 0, 2));
     assertFalse(containsWithinRange(targets[1], dataNodes, 0, 2));
     assertFalse(containsWithinRange(targets[1], dataNodes, 0, 2));
 
 
-    targets = replicator.chooseTarget(filename, 3, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(3);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertTrue(containsWithinRange(targets[0], dataNodes, 3, 5));
     assertTrue(containsWithinRange(targets[0], dataNodes, 3, 5));
     assertTrue(containsWithinRange(targets[1], dataNodes, 3, 5));
     assertTrue(containsWithinRange(targets[1], dataNodes, 3, 5));
     assertTrue(containsWithinRange(targets[2], dataNodes, 3, 5));
     assertTrue(containsWithinRange(targets[2], dataNodes, 3, 5));
 
 
-    targets = replicator.chooseTarget(filename, 4, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(4);
     assertEquals(targets.length, 4);
     assertEquals(targets.length, 4);
     assertTrue(containsWithinRange(dataNodes[3], targets, 0, 3));
     assertTrue(containsWithinRange(dataNodes[3], targets, 0, 3));
     assertTrue(containsWithinRange(dataNodes[4], targets, 0, 3));
     assertTrue(containsWithinRange(dataNodes[4], targets, 0, 3));
@@ -596,7 +586,8 @@ public class TestReplicationPolicy {
       BlockPlacementPolicy replicator = miniCluster.getNameNode()
       BlockPlacementPolicy replicator = miniCluster.getNameNode()
           .getNamesystem().getBlockManager().getBlockPlacementPolicy();
           .getNamesystem().getBlockManager().getBlockPlacementPolicy();
       DatanodeDescriptor[] targets = replicator.chooseTarget(filename, 3,
       DatanodeDescriptor[] targets = replicator.chooseTarget(filename, 3,
-          staleNodeInfo, new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+          staleNodeInfo, new ArrayList<DatanodeDescriptor>(), false, null, BLOCK_SIZE);
+
       assertEquals(targets.length, 3);
       assertEquals(targets.length, 3);
       assertFalse(cluster.isOnSameRack(targets[0], staleNodeInfo));
       assertFalse(cluster.isOnSameRack(targets[0], staleNodeInfo));
       
       
@@ -620,7 +611,7 @@ public class TestReplicationPolicy {
           .getDatanodeManager().shouldAvoidStaleDataNodesForWrite());
           .getDatanodeManager().shouldAvoidStaleDataNodesForWrite());
       // Call chooseTarget
       // Call chooseTarget
       targets = replicator.chooseTarget(filename, 3,
       targets = replicator.chooseTarget(filename, 3,
-          staleNodeInfo, new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+          staleNodeInfo, new ArrayList<DatanodeDescriptor>(), false, null, BLOCK_SIZE);
       assertEquals(targets.length, 3);
       assertEquals(targets.length, 3);
       assertTrue(cluster.isOnSameRack(targets[0], staleNodeInfo));
       assertTrue(cluster.isOnSameRack(targets[0], staleNodeInfo));
       
       
@@ -642,8 +633,7 @@ public class TestReplicationPolicy {
       assertTrue(miniCluster.getNameNode().getNamesystem().getBlockManager()
       assertTrue(miniCluster.getNameNode().getNamesystem().getBlockManager()
           .getDatanodeManager().shouldAvoidStaleDataNodesForWrite());
           .getDatanodeManager().shouldAvoidStaleDataNodesForWrite());
       // Call chooseTarget
       // Call chooseTarget
-      targets = replicator.chooseTarget(filename, 3,
-          staleNodeInfo, new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+      targets = chooseTarget(3, staleNodeInfo);
       assertEquals(targets.length, 3);
       assertEquals(targets.length, 3);
       assertFalse(cluster.isOnSameRack(targets[0], staleNodeInfo));
       assertFalse(cluster.isOnSameRack(targets[0], staleNodeInfo));
     } finally {
     } finally {
@@ -664,23 +654,19 @@ public class TestReplicationPolicy {
     chosenNodes.add(dataNodes[0]);    
     chosenNodes.add(dataNodes[0]);    
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
     
     
-    targets = replicator.chooseTarget(filename,
-                                      0, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(0, chosenNodes);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
     
     
-    targets = replicator.chooseTarget(filename,
-                                      1, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(1, chosenNodes);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     
     
-    targets = replicator.chooseTarget(filename,
-                                      2, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(2, chosenNodes);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
     
-    targets = replicator.chooseTarget(filename,
-                                      3, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(3, chosenNodes);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[2]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[2]));
@@ -700,17 +686,14 @@ public class TestReplicationPolicy {
     chosenNodes.add(dataNodes[1]);
     chosenNodes.add(dataNodes[1]);
 
 
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename,
-                                      0, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(0, chosenNodes);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
     
     
-    targets = replicator.chooseTarget(filename,
-                                      1, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(1, chosenNodes);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     
     
-    targets = replicator.chooseTarget(filename,
-                                      2, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(2, chosenNodes);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[1]));
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[1]));
@@ -730,29 +713,24 @@ public class TestReplicationPolicy {
     chosenNodes.add(dataNodes[2]);
     chosenNodes.add(dataNodes[2]);
     
     
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename,
-                                      0, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(0, chosenNodes);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
     
     
-    targets = replicator.chooseTarget(filename,
-                                      1, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(1, chosenNodes);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[2], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[2], targets[0]));
     
     
-    targets = replicator.chooseTarget(filename,
-                               1, dataNodes[2], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(1, dataNodes[2], chosenNodes);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertTrue(cluster.isOnSameRack(dataNodes[2], targets[0]));
     assertTrue(cluster.isOnSameRack(dataNodes[2], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
 
 
-    targets = replicator.chooseTarget(filename,
-                                      2, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(2, chosenNodes);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     
     
-    targets = replicator.chooseTarget(filename,
-                               2, dataNodes[2], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(2, dataNodes[2], chosenNodes);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertTrue(cluster.isOnSameRack(dataNodes[2], targets[0]));
     assertTrue(cluster.isOnSameRack(dataNodes[2], targets[0]));
   }
   }

+ 77 - 84
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.net.NetworkTopologyWithNodeGroup;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.junit.After;
 import org.junit.After;
@@ -101,10 +102,10 @@ public class TestReplicationPolicyWithNodeGroup {
     FileSystem.setDefaultUri(CONF, "hdfs://localhost:0");
     FileSystem.setDefaultUri(CONF, "hdfs://localhost:0");
     CONF.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     CONF.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     // Set properties to make HDFS aware of NodeGroup.
     // Set properties to make HDFS aware of NodeGroup.
-    CONF.set("dfs.block.replicator.classname", 
-        "org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithNodeGroup");
+    CONF.set(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, 
+        BlockPlacementPolicyWithNodeGroup.class.getName());
     CONF.set(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY, 
     CONF.set(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY, 
-        "org.apache.hadoop.net.NetworkTopologyWithNodeGroup");
+        NetworkTopologyWithNodeGroup.class.getName());
     
     
     File baseDir = PathUtils.getTestDir(TestReplicationPolicyWithNodeGroup.class);
     File baseDir = PathUtils.getTestDir(TestReplicationPolicyWithNodeGroup.class);
     
     
@@ -156,6 +157,35 @@ public class TestReplicationPolicyWithNodeGroup {
     return true;
     return true;
   }
   }
   
   
+  private DatanodeDescriptor[] chooseTarget(int numOfReplicas) {
+    return chooseTarget(numOfReplicas, dataNodes[0]);
+  }
+
+  private DatanodeDescriptor[] chooseTarget(int numOfReplicas,
+      DatanodeDescriptor writer) {
+    return chooseTarget(numOfReplicas, writer,
+        new ArrayList<DatanodeDescriptor>());
+  }
+
+  private DatanodeDescriptor[] chooseTarget(int numOfReplicas,
+      List<DatanodeDescriptor> chosenNodes) {
+    return chooseTarget(numOfReplicas, dataNodes[0], chosenNodes);
+  }
+
+  private DatanodeDescriptor[] chooseTarget(int numOfReplicas,
+      DatanodeDescriptor writer, List<DatanodeDescriptor> chosenNodes) {
+    return chooseTarget(numOfReplicas, writer, chosenNodes, null);
+  }
+
+  private DatanodeDescriptor[] chooseTarget(
+      int numOfReplicas,
+      DatanodeDescriptor writer,
+      List<DatanodeDescriptor> chosenNodes,
+      Set<Node> excludedNodes) {
+    return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes,
+        false, excludedNodes, BLOCK_SIZE);
+  }
+
   /**
   /**
    * In this testcase, client is dataNodes[0]. So the 1st replica should be
    * In this testcase, client is dataNodes[0]. So the 1st replica should be
    * placed on dataNodes[0], the 2nd replica should be placed on 
    * placed on dataNodes[0], the 2nd replica should be placed on 
@@ -173,31 +203,26 @@ public class TestReplicationPolicyWithNodeGroup {
         0L, 0L, 4, 0); // overloaded
         0L, 0L, 4, 0); // overloaded
 
 
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename, 0, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(0);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
 
 
-    targets = replicator.chooseTarget(filename, 1, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(1);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
 
 
-    targets = replicator.chooseTarget(filename, 2, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(2);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
 
 
-    targets = replicator.chooseTarget(filename, 3, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(3);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertFalse(cluster.isOnSameNodeGroup(targets[1], targets[2]));
     assertFalse(cluster.isOnSameNodeGroup(targets[1], targets[2]));
 
 
-    targets = replicator.chooseTarget(filename, 4, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(4);
     assertEquals(targets.length, 4);
     assertEquals(targets.length, 4);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]) ||
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]) ||
@@ -228,15 +253,14 @@ public class TestReplicationPolicyWithNodeGroup {
    * @throws Exception
    * @throws Exception
    */
    */
   @Test
   @Test
-  public void testChooseTarget2() throws Exception { 
-    HashMap<Node, Node> excludedNodes;
+  public void testChooseTarget2() throws Exception {
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
     BlockPlacementPolicyDefault repl = (BlockPlacementPolicyDefault)replicator;
     BlockPlacementPolicyDefault repl = (BlockPlacementPolicyDefault)replicator;
     List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
     List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
 
 
-    excludedNodes = new HashMap<Node, Node>();
-    excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = repl.chooseTarget(4, dataNodes[0], chosenNodes, false, 
+    Set<Node> excludedNodes = new HashSet<Node>();
+    excludedNodes.add(dataNodes[1]); 
+    targets = repl.chooseTarget(filename, 4, dataNodes[0], chosenNodes, false, 
         excludedNodes, BLOCK_SIZE);
         excludedNodes, BLOCK_SIZE);
     assertEquals(targets.length, 4);
     assertEquals(targets.length, 4);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
@@ -251,9 +275,9 @@ public class TestReplicationPolicyWithNodeGroup {
 
 
     excludedNodes.clear();
     excludedNodes.clear();
     chosenNodes.clear();
     chosenNodes.clear();
-    excludedNodes.put(dataNodes[1], dataNodes[1]); 
+    excludedNodes.add(dataNodes[1]); 
     chosenNodes.add(dataNodes[2]);
     chosenNodes.add(dataNodes[2]);
-    targets = repl.chooseTarget(1, dataNodes[0], chosenNodes, true,
+    targets = repl.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true,
         excludedNodes, BLOCK_SIZE);
         excludedNodes, BLOCK_SIZE);
     System.out.println("targets=" + Arrays.asList(targets));
     System.out.println("targets=" + Arrays.asList(targets));
     assertEquals(2, targets.length);
     assertEquals(2, targets.length);
@@ -280,30 +304,25 @@ public class TestReplicationPolicyWithNodeGroup {
         0L, 0L, 0, 0); // no space
         0L, 0L, 0, 0); // no space
 
 
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename, 0, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(0);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
 
 
-    targets = replicator.chooseTarget(filename, 1, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(1);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertEquals(targets[0], dataNodes[1]);
     assertEquals(targets[0], dataNodes[1]);
 
 
-    targets = replicator.chooseTarget(filename, 2, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(2);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertEquals(targets[0], dataNodes[1]);
     assertEquals(targets[0], dataNodes[1]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
 
 
-    targets = replicator.chooseTarget(filename, 3, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(3);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertEquals(targets[0], dataNodes[1]);
     assertEquals(targets[0], dataNodes[1]);
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
 
 
-    targets = replicator.chooseTarget(filename, 4, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(4);
     assertEquals(targets.length, 4);
     assertEquals(targets.length, 4);
     assertEquals(targets[0], dataNodes[1]);
     assertEquals(targets[0], dataNodes[1]);
     assertTrue(cluster.isNodeGroupAware());
     assertTrue(cluster.isNodeGroupAware());
@@ -335,23 +354,19 @@ public class TestReplicationPolicyWithNodeGroup {
     }
     }
 
 
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename, 0, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(0);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
 
 
-    targets = replicator.chooseTarget(filename, 1, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(1);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
     assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
 
 
-    targets = replicator.chooseTarget(filename, 2, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(2);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
     assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
 
 
-    targets = replicator.chooseTarget(filename, 3, dataNodes[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(3);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     for(int i=0; i<3; i++) {
     for(int i=0; i<3; i++) {
       assertFalse(cluster.isOnSameRack(targets[i], dataNodes[0]));
       assertFalse(cluster.isOnSameRack(targets[i], dataNodes[0]));
@@ -373,21 +388,17 @@ public class TestReplicationPolicyWithNodeGroup {
   public void testChooseTarget5() throws Exception {
   public void testChooseTarget5() throws Exception {
     setupDataNodeCapacity();
     setupDataNodeCapacity();
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename, 0, NODE,
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(0, NODE);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
 
 
-    targets = replicator.chooseTarget(filename, 1, NODE,
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(1, NODE);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
 
 
-    targets = replicator.chooseTarget(filename, 2, NODE,
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(2, NODE);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
 
 
-    targets = replicator.chooseTarget(filename, 3, NODE,
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(3, NODE);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
@@ -408,23 +419,19 @@ public class TestReplicationPolicyWithNodeGroup {
     chosenNodes.add(dataNodes[0]);
     chosenNodes.add(dataNodes[0]);
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
     
     
-    targets = replicator.chooseTarget(filename,
-                                      0, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(0, chosenNodes);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
     
     
-    targets = replicator.chooseTarget(filename,
-                                      1, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(1, chosenNodes);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     
     
-    targets = replicator.chooseTarget(filename,
-                                      2, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(2, chosenNodes);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
     
-    targets = replicator.chooseTarget(filename,
-                                      3, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(3, chosenNodes);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameNodeGroup(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameNodeGroup(dataNodes[0], targets[0]));
@@ -446,17 +453,14 @@ public class TestReplicationPolicyWithNodeGroup {
     chosenNodes.add(dataNodes[1]);
     chosenNodes.add(dataNodes[1]);
 
 
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename,
-                                      0, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(0, chosenNodes);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
 
 
-    targets = replicator.chooseTarget(filename,
-                                      1, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(1, chosenNodes);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
 
 
-    targets = replicator.chooseTarget(filename,
-                                      2, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(2, chosenNodes);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]) && 
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]) && 
         cluster.isOnSameRack(dataNodes[0], targets[1]));
         cluster.isOnSameRack(dataNodes[0], targets[1]));
@@ -477,30 +481,26 @@ public class TestReplicationPolicyWithNodeGroup {
     chosenNodes.add(dataNodes[3]);
     chosenNodes.add(dataNodes[3]);
 
 
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename,
-                                      0, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(0, chosenNodes);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
 
 
-    targets = replicator.chooseTarget(filename,
-                                      1, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(1, chosenNodes);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[3], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[3], targets[0]));
 
 
-    targets = replicator.chooseTarget(filename,
-                               1, dataNodes[3], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(1, dataNodes[3], chosenNodes);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertTrue(cluster.isOnSameRack(dataNodes[3], targets[0]));
     assertTrue(cluster.isOnSameRack(dataNodes[3], targets[0]));
     assertFalse(cluster.isOnSameNodeGroup(dataNodes[3], targets[0]));
     assertFalse(cluster.isOnSameNodeGroup(dataNodes[3], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
 
 
-    targets = replicator.chooseTarget(filename,
-                                      2, dataNodes[0], chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(2, chosenNodes);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameNodeGroup(dataNodes[0], targets[0]));
     assertFalse(cluster.isOnSameNodeGroup(dataNodes[0], targets[0]));
-    targets = replicator.chooseTarget(filename,
-                               2, dataNodes[3], chosenNodes, BLOCK_SIZE);
+
+    targets = chooseTarget(2, dataNodes[3], chosenNodes);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertTrue(cluster.isOnSameRack(dataNodes[3], targets[0]));
     assertTrue(cluster.isOnSameRack(dataNodes[3], targets[0]));
   }
   }
@@ -586,21 +586,17 @@ public class TestReplicationPolicyWithNodeGroup {
     }
     }
 
 
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename, 0, dataNodesInBoundaryCase[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(0, dataNodesInBoundaryCase[0]);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
     
     
-    targets = replicator.chooseTarget(filename, 1, dataNodesInBoundaryCase[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(1, dataNodesInBoundaryCase[0]);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
 
 
-    targets = replicator.chooseTarget(filename, 2, dataNodesInBoundaryCase[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(2, dataNodesInBoundaryCase[0]);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
     
-    targets = replicator.chooseTarget(filename, 3, dataNodesInBoundaryCase[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(3, dataNodesInBoundaryCase[0]);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertTrue(checkTargetsOnDifferentNodeGroup(targets));
     assertTrue(checkTargetsOnDifferentNodeGroup(targets));
   }
   }
@@ -623,8 +619,7 @@ public class TestReplicationPolicyWithNodeGroup {
     chosenNodes.add(dataNodesInBoundaryCase[0]);
     chosenNodes.add(dataNodesInBoundaryCase[0]);
     chosenNodes.add(dataNodesInBoundaryCase[5]);
     chosenNodes.add(dataNodesInBoundaryCase[5]);
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
-    targets = replicator.chooseTarget(filename, 1, dataNodesInBoundaryCase[0],
-        chosenNodes, BLOCK_SIZE);
+    targets = chooseTarget(1, dataNodesInBoundaryCase[0], chosenNodes);
     assertFalse(cluster.isOnSameNodeGroup(targets[0], 
     assertFalse(cluster.isOnSameNodeGroup(targets[0], 
         dataNodesInBoundaryCase[0]));
         dataNodesInBoundaryCase[0]));
     assertFalse(cluster.isOnSameNodeGroup(targets[0],
     assertFalse(cluster.isOnSameNodeGroup(targets[0],
@@ -663,14 +658,12 @@ public class TestReplicationPolicyWithNodeGroup {
 
 
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
     // Test normal case -- 3 replicas
     // Test normal case -- 3 replicas
-    targets = replicator.chooseTarget(filename, 3, dataNodesInMoreTargetsCase[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(3, dataNodesInMoreTargetsCase[0]);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertTrue(checkTargetsOnDifferentNodeGroup(targets));
     assertTrue(checkTargetsOnDifferentNodeGroup(targets));
 
 
     // Test special case -- replica number over node groups.
     // Test special case -- replica number over node groups.
-    targets = replicator.chooseTarget(filename, 10, dataNodesInMoreTargetsCase[0],
-        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    targets = chooseTarget(10, dataNodesInMoreTargetsCase[0]);
     assertTrue(checkTargetsOnDifferentNodeGroup(targets));
     assertTrue(checkTargetsOnDifferentNodeGroup(targets));
     // Verify it only can find 6 targets for placing replicas.
     // Verify it only can find 6 targets for placing replicas.
     assertEquals(targets.length, 6);
     assertEquals(targets.length, 6);

+ 152 - 19
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestJspHelper.java

@@ -17,23 +17,7 @@
  */
  */
 package org.apache.hadoop.hdfs.server.common;
 package org.apache.hadoop.hdfs.server.common;
 
 
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.doAnswer;
-
-import java.io.IOException;
-import java.io.StringReader;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.jsp.JspWriter;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-
-
+import com.google.common.base.Strings;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -43,6 +27,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer;
 import org.apache.hadoop.hdfs.web.resources.DoAsParam;
 import org.apache.hadoop.hdfs.web.resources.DoAsParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
@@ -54,11 +40,27 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecret
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.mockito.stubbing.Answer;
 import org.xml.sax.InputSource;
 import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
 import org.xml.sax.SAXException;
 
 
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.jsp.JspWriter;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.IOException;
+import java.io.StringReader;
+import java.net.InetSocketAddress;
+import java.text.MessageFormat;
+import java.util.ArrayList;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
 
 
 public class TestJspHelper {
 public class TestJspHelper {
 
 
@@ -446,9 +448,9 @@ public class TestJspHelper {
   @Test
   @Test
   public void testSortNodeByFields() throws Exception {
   public void testSortNodeByFields() throws Exception {
     DatanodeID dnId1 = new DatanodeID("127.0.0.1", "localhost1", "storage1",
     DatanodeID dnId1 = new DatanodeID("127.0.0.1", "localhost1", "storage1",
-        1234, 2345, 3456);
+        1234, 2345, 3456, 4567);
     DatanodeID dnId2 = new DatanodeID("127.0.0.2", "localhost2", "storage2",
     DatanodeID dnId2 = new DatanodeID("127.0.0.2", "localhost2", "storage2",
-        1235, 2346, 3457);
+        1235, 2346, 3457, 4568);
     DatanodeDescriptor dnDesc1 = new DatanodeDescriptor(dnId1, "rack1", 1024,
     DatanodeDescriptor dnDesc1 = new DatanodeDescriptor(dnId1, "rack1", 1024,
         100, 924, 100, 5l, 3l, 10, 2);
         100, 924, 100, 5l, 3l, 10, 2);
     DatanodeDescriptor dnDesc2 = new DatanodeDescriptor(dnId2, "rack2", 2500,
     DatanodeDescriptor dnDesc2 = new DatanodeDescriptor(dnId2, "rack2", 2500,
@@ -480,5 +482,136 @@ public class TestJspHelper {
     JspHelper.sortNodeList(live, "pcbpused", "DSC");
     JspHelper.sortNodeList(live, "pcbpused", "DSC");
     Assert.assertEquals(dnDesc1, live.get(0));
     Assert.assertEquals(dnDesc1, live.get(0));
     Assert.assertEquals(dnDesc2, live.get(1));
     Assert.assertEquals(dnDesc2, live.get(1));
+    
+    //unexisted field comparition is d1.getHostName().compareTo(d2.getHostName());    
+    JspHelper.sortNodeList(live, "unexists", "ASC");
+    Assert.assertEquals(dnDesc1, live.get(0));
+    Assert.assertEquals(dnDesc2, live.get(1));
+    
+    JspHelper.sortNodeList(live, "unexists", "DSC");
+    Assert.assertEquals(dnDesc2, live.get(0));
+    Assert.assertEquals(dnDesc1, live.get(1));  
+    
+    // test sorting by capacity
+    JspHelper.sortNodeList(live, "capacity", "ASC");
+    Assert.assertEquals(dnDesc1, live.get(0));
+    Assert.assertEquals(dnDesc2, live.get(1));
+    
+    JspHelper.sortNodeList(live, "capacity", "DSC");
+    Assert.assertEquals(dnDesc2, live.get(0));
+    Assert.assertEquals(dnDesc1, live.get(1));
+
+    // test sorting by used
+    JspHelper.sortNodeList(live, "used", "ASC");
+    Assert.assertEquals(dnDesc1, live.get(0));
+    Assert.assertEquals(dnDesc2, live.get(1));
+    
+    JspHelper.sortNodeList(live, "used", "DSC");
+    Assert.assertEquals(dnDesc2, live.get(0));
+    Assert.assertEquals(dnDesc1, live.get(1)); 
+    
+    // test sorting by nondfsused
+    JspHelper.sortNodeList(live, "nondfsused", "ASC");
+    Assert.assertEquals(dnDesc1, live.get(0));
+    Assert.assertEquals(dnDesc2, live.get(1));
+    
+    JspHelper.sortNodeList(live, "nondfsused", "DSC");
+    Assert.assertEquals(dnDesc2, live.get(0));
+    Assert.assertEquals(dnDesc1, live.get(1));
+   
+    // test sorting by remaining
+    JspHelper.sortNodeList(live, "remaining", "ASC");
+    Assert.assertEquals(dnDesc1, live.get(0));
+    Assert.assertEquals(dnDesc2, live.get(1));
+    
+    JspHelper.sortNodeList(live, "remaining", "DSC");
+    Assert.assertEquals(dnDesc2, live.get(0));
+    Assert.assertEquals(dnDesc1, live.get(1));
+  }
+  
+  @Test
+  public void testPrintMethods() throws IOException {
+    JspWriter out = mock(JspWriter.class);      
+    HttpServletRequest req = mock(HttpServletRequest.class);
+    
+    final StringBuffer buffer = new StringBuffer();
+    
+    ArgumentCaptor<String> arg = ArgumentCaptor.forClass(String.class);
+    doAnswer(new Answer<Object>() {      
+      @Override
+      public Object answer(InvocationOnMock invok) {
+        Object[] args = invok.getArguments();
+        buffer.append((String)args[0]);
+        return null;
+      }
+    }).when(out).print(arg.capture());
+    
+    
+    JspHelper.createTitle(out, req, "testfile.txt");
+    Mockito.verify(out, Mockito.times(1)).print(Mockito.anyString());
+    
+    JspHelper.addTableHeader(out);
+    Mockito.verify(out, Mockito.times(1 + 2)).print(Mockito.anyString());                  
+     
+    JspHelper.addTableRow(out, new String[] {" row11", "row12 "});
+    Mockito.verify(out, Mockito.times(1 + 2 + 4)).print(Mockito.anyString());      
+    
+    JspHelper.addTableRow(out, new String[] {" row11", "row12 "}, 3);
+    Mockito.verify(out, Mockito.times(1 + 2 + 4 + 4)).print(Mockito.anyString());
+      
+    JspHelper.addTableRow(out, new String[] {" row21", "row22"});
+    Mockito.verify(out, Mockito.times(1 + 2 + 4 + 4 + 4)).print(Mockito.anyString());      
+      
+    JspHelper.addTableFooter(out);
+    Mockito.verify(out, Mockito.times(1 + 2 + 4 + 4 + 4 + 1)).print(Mockito.anyString());
+    
+    assertFalse(Strings.isNullOrEmpty(buffer.toString()));               
+  }
+  
+  @Test
+  public void testReadWriteReplicaState() {
+    try {
+      DataOutputBuffer out = new DataOutputBuffer();
+      DataInputBuffer in = new DataInputBuffer();
+      for (HdfsServerConstants.ReplicaState repState : HdfsServerConstants.ReplicaState
+          .values()) {
+        repState.write(out);
+        in.reset(out.getData(), out.getLength());
+        HdfsServerConstants.ReplicaState result = HdfsServerConstants.ReplicaState
+            .read(in);
+        assertTrue("testReadWrite error !!!", repState == result);
+        out.reset();
+        in.reset();
+      }
+    } catch (Exception ex) {
+      fail("testReadWrite ex error ReplicaState");
+    }
   }
   }
+
+  @Test
+  public void testUpgradeStatusReport() {
+    short status = 6;
+    int version = 15;
+    String EXPECTED__NOTF_PATTERN = "Upgrade for version {0} has been completed.\nUpgrade is not finalized.";
+    String EXPECTED_PATTERN = "Upgrade for version {0} is in progress. Status = {1}%";
+
+    UpgradeStatusReport upgradeStatusReport = new UpgradeStatusReport(version,
+        status, true);
+    assertTrue(upgradeStatusReport.getVersion() == version);
+    assertTrue(upgradeStatusReport.getUpgradeStatus() == status);
+    assertTrue(upgradeStatusReport.isFinalized());
+
+    assertEquals(MessageFormat.format(EXPECTED_PATTERN, version, status),
+        upgradeStatusReport.getStatusText(true));
+
+    status += 100;
+    upgradeStatusReport = new UpgradeStatusReport(version, status, false);
+    assertFalse(upgradeStatusReport.isFinalized());
+    assertTrue(upgradeStatusReport.toString().equals(
+        MessageFormat.format(EXPECTED__NOTF_PATTERN, version)));
+    assertTrue(upgradeStatusReport.getStatusText(false).equals(
+        MessageFormat.format(EXPECTED__NOTF_PATTERN, version)));
+    assertTrue(upgradeStatusReport.getStatusText(true).equals(
+        MessageFormat.format(EXPECTED__NOTF_PATTERN, version)));
+  }  
 }
 }

+ 13 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java

@@ -115,11 +115,22 @@ public class DataNodeTestUtils {
   }
   }
   
   
   public static void runBlockScannerForBlock(DataNode dn, ExtendedBlock b) {
   public static void runBlockScannerForBlock(DataNode dn, ExtendedBlock b) {
+    BlockPoolSliceScanner bpScanner = getBlockPoolScanner(dn, b);
+    bpScanner.verifyBlock(b);
+  }
+
+  private static BlockPoolSliceScanner getBlockPoolScanner(DataNode dn,
+      ExtendedBlock b) {
     DataBlockScanner scanner = dn.getBlockScanner();
     DataBlockScanner scanner = dn.getBlockScanner();
     BlockPoolSliceScanner bpScanner = scanner.getBPScanner(b.getBlockPoolId());
     BlockPoolSliceScanner bpScanner = scanner.getBPScanner(b.getBlockPoolId());
-    bpScanner.verifyBlock(b);
+    return bpScanner;
   }
   }
-  
+
+  public static long getLatestScanTime(DataNode dn, ExtendedBlock b) {
+    BlockPoolSliceScanner scanner = getBlockPoolScanner(dn, b);
+    return scanner.getLastScanTime(b.getLocalBlock());
+  }
+
   public static void shutdownBlockScanner(DataNode dn) {
   public static void shutdownBlockScanner(DataNode dn) {
     if (dn.blockScanner != null) {
     if (dn.blockScanner != null) {
       dn.blockScanner.shutdown();
       dn.blockScanner.shutdown();

+ 11 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java

@@ -61,9 +61,10 @@ public class TestDatanodeJsp {
     
     
     InetSocketAddress nnIpcAddress = cluster.getNameNode().getNameNodeAddress();
     InetSocketAddress nnIpcAddress = cluster.getNameNode().getNameNodeAddress();
     InetSocketAddress nnHttpAddress = cluster.getNameNode().getHttpAddress();
     InetSocketAddress nnHttpAddress = cluster.getNameNode().getHttpAddress();
-    int dnInfoPort = cluster.getDataNodes().get(0).getInfoPort();
-    
-    URL url = new URL("http://localhost:" + dnInfoPort + "/"
+    String base = JspHelper.Url.url("http", cluster.getDataNodes().get(0)
+        .getDatanodeId());
+
+    URL url = new URL(base + "/"
         + "browseDirectory.jsp" + JspHelper.getUrlParam("dir", 
         + "browseDirectory.jsp" + JspHelper.getUrlParam("dir", 
             URLEncoder.encode(testPath.toString(), "UTF-8"), true)
             URLEncoder.encode(testPath.toString(), "UTF-8"), true)
         + JspHelper.getUrlParam("namenodeInfoPort", Integer
         + JspHelper.getUrlParam("namenodeInfoPort", Integer
@@ -86,18 +87,21 @@ public class TestDatanodeJsp {
     // check whether able to 'Go Back to File View' after tailing the file
     // check whether able to 'Go Back to File View' after tailing the file
     regex = "<a.+href=\"(.+?)\">Go\\s*Back\\s*to\\s*File\\s*View\\<\\/a\\>";
     regex = "<a.+href=\"(.+?)\">Go\\s*Back\\s*to\\s*File\\s*View\\<\\/a\\>";
     assertFileContents(regex, "Go Back to File View");
     assertFileContents(regex, "Go Back to File View");
+
+    regex = "<a href=\"///localhost:" + nnHttpAddress.getPort() + "/dfshealth.jsp\">Go back to DFS home</a>";
+    assertTrue("page should generate DFS home scheme without explicit scheme", viewFilePage.contains(regex));
   }
   }
   
   
   private static void assertFileContents(String regex, String text)
   private static void assertFileContents(String regex, String text)
       throws IOException {
       throws IOException {
     Pattern compile = Pattern.compile(regex);
     Pattern compile = Pattern.compile(regex);
     Matcher matcher = compile.matcher(viewFilePage);
     Matcher matcher = compile.matcher(viewFilePage);
-    URL hyperlink = null;
     if (matcher.find()) {
     if (matcher.find()) {
       // got hyperlink for Tail this file
       // got hyperlink for Tail this file
-      hyperlink = new URL(matcher.group(1));
+      String u = matcher.group(1);
+      String urlString = u.startsWith("///") ? ("http://" + u.substring(3)) : u;
       viewFilePage = StringEscapeUtils.unescapeHtml(DFSTestUtil
       viewFilePage = StringEscapeUtils.unescapeHtml(DFSTestUtil
-          .urlGet(hyperlink));
+          .urlGet(new URL(urlString)));
       assertTrue("page should show preview of file contents", viewFilePage
       assertTrue("page should show preview of file contents", viewFilePage
           .contains(FILE_DATA));
           .contains(FILE_DATA));
     } else {
     } else {
@@ -166,6 +170,7 @@ public class TestDatanodeJsp {
     Mockito.doReturn(NetUtils.getHostPortString(NameNode.getAddress(CONF)))
     Mockito.doReturn(NetUtils.getHostPortString(NameNode.getAddress(CONF)))
         .when(reqMock).getParameter("nnaddr");
         .when(reqMock).getParameter("nnaddr");
     Mockito.doReturn(testFile.toString()).when(reqMock).getPathInfo();
     Mockito.doReturn(testFile.toString()).when(reqMock).getPathInfo();
+    Mockito.doReturn("http").when(reqMock).getScheme();
   }
   }
 
 
   static Path writeFile(FileSystem fs, Path f) throws IOException {
   static Path writeFile(FileSystem fs, Path f) throws IOException {

+ 13 - 33
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java

@@ -17,14 +17,6 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.EnumSet;
-import java.util.List;
-
 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.commons.logging.impl.Log4JLogger;
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -33,41 +25,28 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
-import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
-import org.apache.hadoop.hdfs.server.protocol.CacheReport;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
-import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
-import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
-import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.hdfs.server.protocol.*;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.Groups;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.Time;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.util.VersionInfo;
+import org.apache.hadoop.util.*;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.LogManager;
 
 
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.List;
+
 /**
 /**
  * Main class for a series of name-node benchmarks.
  * Main class for a series of name-node benchmarks.
  * 
  * 
@@ -840,6 +819,7 @@ public class NNThroughputBenchmark implements Tool {
               DNS.getDefaultHost("default", "default"),
               DNS.getDefaultHost("default", "default"),
               "", getNodePort(dnIdx),
               "", getNodePort(dnIdx),
               DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
               DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
+              DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
               DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT),
               DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT),
           new DataStorage(nsInfo, ""),
           new DataStorage(nsInfo, ""),
           new ExportedBlockKeys(), VersionInfo.getVersion());
           new ExportedBlockKeys(), VersionInfo.getVersion());
@@ -1308,7 +1288,7 @@ public class NNThroughputBenchmark implements Tool {
 
 
   /**
   /**
    * Main method of the benchmark.
    * Main method of the benchmark.
-   * @param args command line parameters
+   * @param aArgs command line parameters
    */
    */
   @Override // Tool
   @Override // Tool
   public int run(String[] aArgs) throws Exception {
   public int run(String[] aArgs) throws Exception {

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

@@ -25,7 +25,7 @@ import static org.mockito.Mockito.spy;
 
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Field;
 import java.util.EnumSet;
 import java.util.EnumSet;
-import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.List;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -119,7 +119,7 @@ public class TestAddBlockRetry {
         return ret;
         return ret;
       }
       }
     }).when(spyBM).chooseTarget(Mockito.anyString(), Mockito.anyInt(),
     }).when(spyBM).chooseTarget(Mockito.anyString(), Mockito.anyInt(),
-        Mockito.<DatanodeDescriptor>any(), Mockito.<HashMap<Node, Node>>any(),
+        Mockito.<DatanodeDescriptor>any(), Mockito.<HashSet<Node>>any(),
         Mockito.anyLong(), Mockito.<List<String>>any());
         Mockito.anyLong(), Mockito.<List<String>>any());
 
 
     // create file
     // create file

+ 59 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterJspHelper.java

@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.junit.Assert.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.ClusterJspHelper.ClusterStatus;
+import org.apache.hadoop.hdfs.server.namenode.ClusterJspHelper.DecommissionStatus;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestClusterJspHelper {
+
+  private MiniDFSCluster cluster;
+  private Configuration conf;
+    
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();  
+    cluster = new MiniDFSCluster.Builder(conf).build();
+    cluster.waitClusterUp();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null)
+      cluster.shutdown();    
+  }
+  
+  @Test(timeout = 15000)
+  public void testClusterJspHelperReports() {
+    ClusterJspHelper clusterJspHelper = new ClusterJspHelper();
+    ClusterStatus clusterStatus = clusterJspHelper
+     .generateClusterHealthReport();
+    assertNotNull("testClusterJspHelperReports ClusterStatus is null",
+        clusterStatus);       
+    DecommissionStatus decommissionStatus = clusterJspHelper
+        .generateDecommissioningReport();
+    assertNotNull("testClusterJspHelperReports DecommissionStatus is null",
+        decommissionStatus);    
+  }
+}

+ 8 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java

@@ -18,15 +18,6 @@
 
 
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-
-import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -37,6 +28,13 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import java.io.IOException;
+
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Mockito.*;
+
 /**
 /**
  * Verify that TestCommitBlockSynchronization is idempotent.
  * Verify that TestCommitBlockSynchronization is idempotent.
  */
  */
@@ -177,7 +175,7 @@ public class TestCommitBlockSynchronization {
     Block block = new Block(blockId, length, genStamp);
     Block block = new Block(blockId, length, genStamp);
     FSNamesystem namesystemSpy = makeNameSystemSpy(block, file);
     FSNamesystem namesystemSpy = makeNameSystemSpy(block, file);
     DatanodeID[] newTargets = new DatanodeID[]{
     DatanodeID[] newTargets = new DatanodeID[]{
-        new DatanodeID("0.0.0.0", "nonexistantHost", "1", 0, 0, 0)};
+        new DatanodeID("0.0.0.0", "nonexistantHost", "1", 0, 0, 0, 0)};
 
 
     ExtendedBlock lastBlock = new ExtendedBlock();
     ExtendedBlock lastBlock = new ExtendedBlock();
     namesystemSpy.commitBlockSynchronization(
     namesystemSpy.commitBlockSynchronization(

+ 18 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java

@@ -20,8 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_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.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
@@ -142,4 +141,21 @@ public class TestFSNamesystem {
     assertTrue("Replication queues weren't being populated after entering "
     assertTrue("Replication queues weren't being populated after entering "
       + "safemode 2nd time", fsn.isPopulatingReplQueues());
       + "safemode 2nd time", fsn.isPopulatingReplQueues());
   }
   }
+  
+  @Test
+  public void testFsLockFairness() throws IOException, InterruptedException{
+    Configuration conf = new Configuration();
+
+    FSEditLog fsEditLog = Mockito.mock(FSEditLog.class);
+    FSImage fsImage = Mockito.mock(FSImage.class);
+    Mockito.when(fsImage.getEditLog()).thenReturn(fsEditLog);
+
+    conf.setBoolean("dfs.namenode.fslock.fair", true);
+    FSNamesystem fsNamesystem = new FSNamesystem(conf, fsImage);
+    assertTrue(fsNamesystem.getFsLockForTests().isFair());
+    
+    conf.setBoolean("dfs.namenode.fslock.fair", false);
+    fsNamesystem = new FSNamesystem(conf, fsImage);
+    assertFalse(fsNamesystem.getFsLockForTests().isFair());
+  }  
 }
 }

+ 72 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemMBean.java

@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.junit.Assert.assertTrue;
+
+import java.lang.management.ManagementFactory;
+import java.util.Map;
+
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.Test;
+import org.mortbay.util.ajax.JSON;
+
+/**
+ * Class for testing {@link NameNodeMXBean} implementation
+ */
+public class TestFSNamesystemMBean {
+
+  @Test
+  public void test() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+
+      FSNamesystem fsn = cluster.getNameNode().namesystem;
+
+      MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+      ObjectName mxbeanName = new ObjectName(
+          "Hadoop:service=NameNode,name=FSNamesystemState");
+
+      String snapshotStats = (String) (mbs.getAttribute(mxbeanName,
+          "SnapshotStats"));
+
+      @SuppressWarnings("unchecked")
+      Map<String, Object> stat = (Map<String, Object>) JSON
+          .parse(snapshotStats);
+
+      assertTrue(stat.containsKey("SnapshottableDirectories")
+          && (Long) stat.get("SnapshottableDirectories") == fsn
+              .getNumSnapshottableDirs());
+      assertTrue(stat.containsKey("Snapshots")
+          && (Long) stat.get("Snapshots") == fsn.getNumSnapshots());
+
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+}

+ 16 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java

@@ -86,24 +86,27 @@ public class TestFileJournalManager {
     EditLogInputStream elis = null;
     EditLogInputStream elis = null;
     try {
     try {
       while ((elis = allStreams.poll()) != null) {
       while ((elis = allStreams.poll()) != null) {
-        elis.skipUntil(txId);
-        while (true) {
-          FSEditLogOp op = elis.readOp();
-          if (op == null) {
-            break;
-          }
-          if (abortOnGap && (op.getTransactionId() != txId)) {
-            LOG.info("getNumberOfTransactions: detected gap at txId " +
-                fromTxId);
-            return numTransactions;
+        try {
+          elis.skipUntil(txId);
+          while (true) {
+            FSEditLogOp op = elis.readOp();
+            if (op == null) {
+              break;
+            }
+            if (abortOnGap && (op.getTransactionId() != txId)) {
+              LOG.info("getNumberOfTransactions: detected gap at txId "
+                  + fromTxId);
+              return numTransactions;
+            }
+            txId = op.getTransactionId() + 1;
+            numTransactions++;
           }
           }
-          txId = op.getTransactionId() + 1;
-          numTransactions++;
+        } finally {
+          IOUtils.cleanup(LOG, elis);
         }
         }
       }
       }
     } finally {
     } finally {
       IOUtils.cleanup(LOG, allStreams.toArray(new EditLogInputStream[0]));
       IOUtils.cleanup(LOG, allStreams.toArray(new EditLogInputStream[0]));
-      IOUtils.cleanup(LOG, elis);
     }
     }
     return numTransactions;
     return numTransactions;
   }
   }

+ 54 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -32,6 +32,7 @@ import java.util.List;
 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.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
@@ -51,6 +52,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 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.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -930,7 +932,9 @@ public class TestINodeFile {
       dirNodeFromNode = fsdir.getInode(dirNode.getId());
       dirNodeFromNode = fsdir.getInode(dirNode.getId());
       assertSame(dirNode, dirNodeFromNode);
       assertSame(dirNode, dirNodeFromNode);
     } finally {
     } finally {
-      cluster.shutdown();
+      if (cluster != null) {
+        cluster.shutdown();
+      }
     }
     }
   }
   }
   
   
@@ -960,7 +964,55 @@ public class TestINodeFile {
       assertTrue(parentId == status.getFileId());
       assertTrue(parentId == status.getFileId());
       
       
     } finally {
     } finally {
-      cluster.shutdown();
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+  
+  @Test
+  public void testFilesInGetListingOps() throws Exception {
+    final Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      cluster.waitActive();
+      final DistributedFileSystem hdfs = cluster.getFileSystem();
+      final FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+
+      hdfs.mkdirs(new Path("/tmp"));
+      DFSTestUtil.createFile(hdfs, new Path("/tmp/f1"), 0, (short) 1, 0);
+      DFSTestUtil.createFile(hdfs, new Path("/tmp/f2"), 0, (short) 1, 0);
+      DFSTestUtil.createFile(hdfs, new Path("/tmp/f3"), 0, (short) 1, 0);
+
+      DirectoryListing dl = cluster.getNameNodeRpc().getListing("/tmp",
+          HdfsFileStatus.EMPTY_NAME, false);
+      assertTrue(dl.getPartialListing().length == 3);
+
+      String f2 = new String("f2");
+      dl = cluster.getNameNodeRpc().getListing("/tmp", f2.getBytes(), false);
+      assertTrue(dl.getPartialListing().length == 1);
+
+      INode f2INode = fsdir.getINode("/tmp/f2");
+      String f2InodePath = "/.reserved/.inodes/" + f2INode.getId();
+      dl = cluster.getNameNodeRpc().getListing("/tmp", f2InodePath.getBytes(),
+          false);
+      assertTrue(dl.getPartialListing().length == 1);
+
+      // Test the deleted startAfter file
+      hdfs.delete(new Path("/tmp/f2"), false);
+      try {
+        dl = cluster.getNameNodeRpc().getListing("/tmp",
+            f2InodePath.getBytes(), false);
+        fail("Didn't get exception for the deleted startAfter token.");
+      } catch (IOException e) {
+        assertTrue(e instanceof DirectoryListingStartAfterNotFoundException);
+      }
+
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
     }
     }
   }
   }
 }
 }

+ 268 - 22
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java

@@ -17,44 +17,71 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
-
-import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase.*;
-import static org.mockito.Mockito.*;
+import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase.LOADING_EDITS;
+import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase.LOADING_FSIMAGE;
+import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase.SAFEMODE;
+import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase.SAVING_CHECKPOINT;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
 
 
+import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
 import javax.servlet.jsp.JspWriter;
 import javax.servlet.jsp.JspWriter;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.After;
+import org.apache.hadoop.util.VersionInfo;
+import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Assert;
-import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.ArgumentCaptor;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.znerd.xmlenc.XMLOutputter;
 
 
-public class TestNameNodeJspHelper {
+import com.google.common.collect.ImmutableSet;
 
 
-  private MiniDFSCluster cluster = null;
-  Configuration conf = null;
+public class TestNameNodeJspHelper {
 
 
-  @Before
-  public void setUp() throws Exception {
+  private static final int DATA_NODES_AMOUNT = 2;
+  
+  private static MiniDFSCluster cluster;
+  private static Configuration conf;
+  private static final String NAMENODE_ATTRIBUTE_KEY = "name.node";  
+  
+  @BeforeClass
+  public static void setUp() throws Exception {
     conf = new HdfsConfiguration();
     conf = new HdfsConfiguration();
-    cluster  = new MiniDFSCluster.Builder(conf).build();
-    cluster.waitActive();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(DATA_NODES_AMOUNT).build();
+    cluster.waitClusterUp();
   }
   }
 
 
-  @After
-  public void tearDown() throws Exception {
+  @AfterClass
+  public static void tearDown() throws Exception {
     if (cluster != null)
     if (cluster != null)
       cluster.shutdown();
       cluster.shutdown();
   }
   }
@@ -66,23 +93,23 @@ public class TestNameNodeJspHelper {
     UserGroupInformation ugi = UserGroupInformation.createRemoteUser("auser");
     UserGroupInformation ugi = UserGroupInformation.createRemoteUser("auser");
     String tokenString = NamenodeJspHelper.getDelegationToken(nn, request,
     String tokenString = NamenodeJspHelper.getDelegationToken(nn, request,
         conf, ugi);
         conf, ugi);
-    //tokenString returned must be null because security is disabled
+    // tokenString returned must be null because security is disabled
     Assert.assertEquals(null, tokenString);
     Assert.assertEquals(null, tokenString);
   }
   }
-  
+
   @Test
   @Test
-  public void  tesSecurityModeText() {
+  public void testSecurityModeText() {
     conf.set(DFSConfigKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
     conf.set(DFSConfigKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
     UserGroupInformation.setConfiguration(conf);
     UserGroupInformation.setConfiguration(conf);
     String securityOnOff = NamenodeJspHelper.getSecurityModeText();
     String securityOnOff = NamenodeJspHelper.getSecurityModeText();
-    Assert.assertTrue("security mode doesn't match. Should be ON", 
+    Assert.assertTrue("security mode doesn't match. Should be ON",
         securityOnOff.contains("ON"));
         securityOnOff.contains("ON"));
-    //Security is enabled
+    // Security is enabled
     conf.set(DFSConfigKeys.HADOOP_SECURITY_AUTHENTICATION, "simple");
     conf.set(DFSConfigKeys.HADOOP_SECURITY_AUTHENTICATION, "simple");
     UserGroupInformation.setConfiguration(conf);
     UserGroupInformation.setConfiguration(conf);
-    
+
     securityOnOff = NamenodeJspHelper.getSecurityModeText();
     securityOnOff = NamenodeJspHelper.getSecurityModeText();
-    Assert.assertTrue("security mode doesn't match. Should be OFF", 
+    Assert.assertTrue("security mode doesn't match. Should be OFF",
         securityOnOff.contains("OFF"));
         securityOnOff.contains("OFF"));
   }
   }
 
 
@@ -107,9 +134,83 @@ public class TestNameNodeJspHelper {
     Assert.assertTrue(containsMatch(contents, SAFEMODE.getDescription()));
     Assert.assertTrue(containsMatch(contents, SAFEMODE.getDescription()));
   }
   }
 
 
+  @Test
+  public void testGetRollingUpgradeText() {
+    Assert.assertEquals("", NamenodeJspHelper.getRollingUpgradeText(null));
+  }
+
+  /**
+   * Tests for non-null, non-empty NameNode label.
+   */
+  @Test
+  public void testGetNameNodeLabel() {
+    String nameNodeLabel = NamenodeJspHelper.getNameNodeLabel(
+      cluster.getNameNode());
+    Assert.assertNotNull(nameNodeLabel);
+    Assert.assertFalse(nameNodeLabel.isEmpty());
+  }
+
+  /**
+   * Tests for non-null, non-empty NameNode label when called before
+   * initialization of the NameNode RPC server.
+   */
+  @Test
+  public void testGetNameNodeLabelNullRpcServer() {
+    NameNode nn = mock(NameNode.class);
+    when(nn.getRpcServer()).thenReturn(null);
+    String nameNodeLabel = NamenodeJspHelper.getNameNodeLabel(
+      cluster.getNameNode());
+    Assert.assertNotNull(nameNodeLabel);
+    Assert.assertFalse(nameNodeLabel.isEmpty());
+  }
+
+  /**
+   * Tests that passing a null FSNamesystem to generateSnapshotReport does not
+   * throw NullPointerException.
+   */
+  @Test
+  public void testGenerateSnapshotReportNullNamesystem() throws Exception {
+    NamenodeJspHelper.generateSnapshotReport(mock(JspWriter.class), null);
+  }
+
+  /**
+   * Tests that redirectToRandomDataNode does not throw NullPointerException if
+   * it finds a null FSNamesystem.
+   */
+  @Test(expected=IOException.class)
+  public void testRedirectToRandomDataNodeNullNamesystem() throws Exception {
+    NameNode nn = mock(NameNode.class);
+    when(nn.getNamesystem()).thenReturn(null);
+    ServletContext context = mock(ServletContext.class);
+    when(context.getAttribute("name.node")).thenReturn(nn);
+    NamenodeJspHelper.redirectToRandomDataNode(context,
+      mock(HttpServletRequest.class), mock(HttpServletResponse.class));
+  }
+
+  /**
+   * Tests that XMLBlockInfo does not throw NullPointerException if it finds a
+   * null FSNamesystem.
+   */
+  @Test
+  public void testXMLBlockInfoNullNamesystem() throws IOException {
+    XMLOutputter doc = new XMLOutputter(mock(JspWriter.class), "UTF-8");
+    new NamenodeJspHelper.XMLBlockInfo(null, 1L).toXML(doc);
+  }
+
+  /**
+   * Tests that XMLCorruptBlockInfo does not throw NullPointerException if it
+   * finds a null FSNamesystem.
+   */
+  @Test
+  public void testXMLCorruptBlockInfoNullNamesystem() throws IOException {
+    XMLOutputter doc = new XMLOutputter(mock(JspWriter.class), "UTF-8");
+    new NamenodeJspHelper.XMLCorruptBlockInfo(null, mock(Configuration.class),
+      10, 1L).toXML(doc);
+  }
+
   /**
   /**
    * Checks if the list contains any string that partially matches the regex.
    * Checks if the list contains any string that partially matches the regex.
-   * 
+   *
    * @param list List<String> containing strings to check
    * @param list List<String> containing strings to check
    * @param regex String regex to check
    * @param regex String regex to check
    * @return boolean true if some string in list partially matches regex
    * @return boolean true if some string in list partially matches regex
@@ -123,4 +224,149 @@ public class TestNameNodeJspHelper {
     }
     }
     return false;
     return false;
   }
   }
+
+  @Test(timeout = 15000)
+  public void testGetRandomDatanode() {
+    ImmutableSet<String> set = ImmutableSet.of();
+    NameNode nameNode = cluster.getNameNode();
+    ImmutableSet.Builder<String> builder = ImmutableSet.builder();
+    for (DataNode dataNode : cluster.getDataNodes()) {
+      builder.add(dataNode.getDisplayName());
+    }
+    set = builder.build();
+
+    for (int i = 0; i < 10; i++) {
+      DatanodeDescriptor dnDescriptor = NamenodeJspHelper
+          .getRandomDatanode(nameNode);
+      assertTrue("testGetRandomDatanode error",
+          set.contains(dnDescriptor.toString()));
+    }
+  }
+      
+  @Test(timeout = 15000)
+  public void testNamenodeJspHelperRedirectToRandomDataNode() throws IOException, InterruptedException {
+    final String urlPart = "browseDirectory.jsp?namenodeInfoPort=";                     
+    
+    ServletContext context = mock(ServletContext.class);
+    HttpServletRequest request = mock(HttpServletRequest.class);
+    HttpServletResponse resp = mock(HttpServletResponse.class);          
+    
+    when(request.getScheme()).thenReturn("http");
+    when(request.getParameter(UserParam.NAME)).thenReturn("localuser");
+    when(context.getAttribute(NAMENODE_ATTRIBUTE_KEY)).thenReturn(
+        cluster.getNameNode());
+    when(context.getAttribute(JspHelper.CURRENT_CONF)).thenReturn(conf);    
+    ArgumentCaptor<String> captor = ArgumentCaptor.forClass(String.class);
+    doAnswer(new Answer<String>() {
+      @Override
+     public String answer(InvocationOnMock invocation) throws Throwable {
+        return null;
+        }
+    }).when(resp).sendRedirect(captor.capture());
+
+    NamenodeJspHelper.redirectToRandomDataNode(context, request, resp);    
+    assertTrue(captor.getValue().contains(urlPart));    
+  }
+  
+  private enum DataNodeStatus {
+    LIVE("[Live Datanodes(| +):(| +)]\\d"), 
+    DEAD("[Dead Datanodes(| +):(| +)]\\d");
+
+    private Pattern pattern;
+
+    public Pattern getPattern() {
+      return pattern;
+    }
+
+    DataNodeStatus(String line) {
+      this.pattern = Pattern.compile(line);
+    }
+  }
+
+  private void checkDeadLiveNodes(NameNode nameNode, int deadCount,
+      int lifeCount) {
+    FSNamesystem ns = nameNode.getNamesystem();
+    DatanodeManager dm = ns.getBlockManager().getDatanodeManager();
+    List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
+    List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
+    dm.fetchDatanodes(live, dead, true);
+    assertTrue("checkDeadLiveNodes error !!!", (live.size() == lifeCount)
+        && dead.size() == deadCount);
+  }
+
+  @Test(timeout = 15000)
+  public void testNodeListJspGenerateNodesList() throws IOException {
+    String output;
+    NameNode nameNode = cluster.getNameNode();
+    ServletContext context = mock(ServletContext.class);
+    when(context.getAttribute("name.node")).thenReturn(nameNode);
+    when(context.getAttribute(NameNodeHttpServer.NAMENODE_ADDRESS_ATTRIBUTE_KEY))
+        .thenReturn(cluster.getNameNode().getHttpAddress());    
+    checkDeadLiveNodes(nameNode, 0, DATA_NODES_AMOUNT);
+    output = getOutputFromGeneratedNodesList(context, DataNodeStatus.LIVE);
+    assertCounts(DataNodeStatus.LIVE, output, DATA_NODES_AMOUNT);
+    output = getOutputFromGeneratedNodesList(context, DataNodeStatus.DEAD);
+    assertCounts(DataNodeStatus.DEAD, output, 0);    
+  }
+
+  private void assertCounts(DataNodeStatus dataNodeStatus, String output,
+      int expectedCount) {
+    Matcher matcher = DataNodeStatus.LIVE.getPattern().matcher(output);
+    if (matcher.find()) {
+      String digitLine = output.substring(matcher.start(), matcher.end())
+          .trim();
+      assertTrue("assertCounts error. actual != expected",
+          Integer.valueOf(digitLine) == expectedCount);
+    } else {
+      fail("assertCount matcher error");
+    }
+  }
+
+  private String getOutputFromGeneratedNodesList(ServletContext context,
+      DataNodeStatus dnStatus) throws IOException {
+    JspWriter out = mock(JspWriter.class);
+    ArgumentCaptor<String> captor = ArgumentCaptor.forClass(String.class);
+    NamenodeJspHelper.NodeListJsp nodelistjsp = new NamenodeJspHelper.NodeListJsp();
+    final StringBuffer buffer = new StringBuffer();
+    doAnswer(new Answer<String>() {
+      @Override
+      public String answer(InvocationOnMock invok) {
+        Object[] args = invok.getArguments();
+        buffer.append((String) args[0]);
+        return null;
+      }
+    }).when(out).print(captor.capture());
+    HttpServletRequest request = mock(HttpServletRequest.class);
+    when(request.getScheme()).thenReturn("http");
+    when(request.getParameter("whatNodes")).thenReturn(dnStatus.name());
+    nodelistjsp.generateNodesList(context, out, request);
+    return buffer.toString();
+  }
+
+  @Test(timeout = 15000)
+  public void testGetInodeLimitText() {
+    NameNode nameNode = cluster.getNameNode();
+    FSNamesystem fsn = nameNode.getNamesystem();
+    ImmutableSet<String> patterns = 
+        ImmutableSet.of("files and directories", "Heap Memory used", "Non Heap Memory used");        
+    String line = NamenodeJspHelper.getInodeLimitText(fsn);    
+    for(String pattern: patterns) {
+      assertTrue("testInodeLimitText error " + pattern,
+          line.contains(pattern));
+    }    
+  }
+  
+  @Test(timeout = 15000)
+  public void testGetVersionTable() {
+    NameNode nameNode = cluster.getNameNode();
+    FSNamesystem fsn = nameNode.getNamesystem();
+    ImmutableSet<String> patterns = ImmutableSet.of(VersionInfo.getVersion(), 
+        VersionInfo.getRevision(), VersionInfo.getUser(), VersionInfo.getBranch(),
+        fsn.getClusterId(), fsn.getBlockPoolId());
+    String line = NamenodeJspHelper.getVersionTable(fsn);
+    for(String pattern: patterns) {
+       assertTrue("testGetVersionTable error " + pattern,
+           line.contains(pattern));
+    }
+  }  
 }
 }

+ 43 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.EnumSet;
 import java.util.EnumSet;
@@ -35,11 +36,15 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 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.ipc.ClientId;
 import org.apache.hadoop.ipc.ClientId;
@@ -47,7 +52,9 @@ import org.apache.hadoop.ipc.RPC.RpcKind;
 import org.apache.hadoop.ipc.RetryCache.CacheEntry;
 import org.apache.hadoop.ipc.RetryCache.CacheEntry;
 import org.apache.hadoop.ipc.RpcConstants;
 import org.apache.hadoop.ipc.RpcConstants;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.LightWeightCache;
 import org.apache.hadoop.util.LightWeightCache;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
@@ -75,12 +82,13 @@ public class TestNamenodeRetryCache {
       "TestNamenodeRetryCache", null, FsPermission.getDefault());
       "TestNamenodeRetryCache", null, FsPermission.getDefault());
   private static DistributedFileSystem filesystem;
   private static DistributedFileSystem filesystem;
   private static int callId = 100;
   private static int callId = 100;
-  private static Configuration conf = new HdfsConfiguration();
+  private static Configuration conf;
   private static final int BlockSize = 512;
   private static final int BlockSize = 512;
   
   
   /** Start a cluster */
   /** Start a cluster */
   @Before
   @Before
   public void setup() throws Exception {
   public void setup() throws Exception {
+    conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BlockSize);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BlockSize);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_KEY, true);
     cluster = new MiniDFSCluster.Builder(conf).build();
     cluster = new MiniDFSCluster.Builder(conf).build();
@@ -293,6 +301,40 @@ public class TestNamenodeRetryCache {
     }
     }
   }
   }
   
   
+  /**
+   * Make sure a retry call does not hang because of the exception thrown in the
+   * first call.
+   */
+  @Test(timeout = 60000)
+  public void testUpdatePipelineWithFailOver() throws Exception {
+    cluster.shutdown();
+    namesystem = null;
+    filesystem = null;
+    cluster = new MiniDFSCluster.Builder(conf).nnTopology(
+        MiniDFSNNTopology.simpleHATopology()).numDataNodes(1).build();
+    FSNamesystem ns0 = cluster.getNamesystem(0);
+    ExtendedBlock oldBlock = new ExtendedBlock();
+    ExtendedBlock newBlock = new ExtendedBlock();
+    DatanodeID[] newNodes = new DatanodeID[2];
+    
+    newCall();
+    try {
+      ns0.updatePipeline("testClient", oldBlock, newBlock, newNodes);
+      fail("Expect StandbyException from the updatePipeline call");
+    } catch (StandbyException e) {
+      // expected, since in the beginning both nn are in standby state
+      GenericTestUtils.assertExceptionContains(
+          HAServiceState.STANDBY.toString(), e);
+    }
+    
+    cluster.transitionToActive(0);
+    try {
+      ns0.updatePipeline("testClient", oldBlock, newBlock, newNodes);
+    } catch (IOException e) {
+      // ignore call should not hang.
+    }
+  }
+  
   /**
   /**
    * Test for crateSnapshot
    * Test for crateSnapshot
    */
    */

+ 16 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupProgressServlet.java

@@ -73,24 +73,28 @@ public class TestStartupProgressServlet {
       .put("phases", Arrays.<Object>asList(
       .put("phases", Arrays.<Object>asList(
         ImmutableMap.<String, Object>builder()
         ImmutableMap.<String, Object>builder()
           .put("name", "LoadingFsImage")
           .put("name", "LoadingFsImage")
+          .put("desc", "Loading fsimage")
           .put("status", "PENDING")
           .put("status", "PENDING")
           .put("percentComplete", 0.0f)
           .put("percentComplete", 0.0f)
           .put("steps", Collections.emptyList())
           .put("steps", Collections.emptyList())
           .build(),
           .build(),
         ImmutableMap.<String, Object>builder()
         ImmutableMap.<String, Object>builder()
           .put("name", "LoadingEdits")
           .put("name", "LoadingEdits")
+          .put("desc", "Loading edits")
           .put("status", "PENDING")
           .put("status", "PENDING")
           .put("percentComplete", 0.0f)
           .put("percentComplete", 0.0f)
           .put("steps", Collections.emptyList())
           .put("steps", Collections.emptyList())
           .build(),
           .build(),
         ImmutableMap.<String, Object>builder()
         ImmutableMap.<String, Object>builder()
           .put("name", "SavingCheckpoint")
           .put("name", "SavingCheckpoint")
+          .put("desc", "Saving checkpoint")
           .put("status", "PENDING")
           .put("status", "PENDING")
           .put("percentComplete", 0.0f)
           .put("percentComplete", 0.0f)
           .put("steps", Collections.emptyList())
           .put("steps", Collections.emptyList())
           .build(),
           .build(),
         ImmutableMap.<String, Object>builder()
         ImmutableMap.<String, Object>builder()
           .put("name", "SafeMode")
           .put("name", "SafeMode")
+          .put("desc", "Safe mode")
           .put("status", "PENDING")
           .put("status", "PENDING")
           .put("percentComplete", 0.0f)
           .put("percentComplete", 0.0f)
           .put("steps", Collections.emptyList())
           .put("steps", Collections.emptyList())
@@ -111,11 +115,13 @@ public class TestStartupProgressServlet {
       .put("phases", Arrays.<Object>asList(
       .put("phases", Arrays.<Object>asList(
         ImmutableMap.<String, Object>builder()
         ImmutableMap.<String, Object>builder()
           .put("name", "LoadingFsImage")
           .put("name", "LoadingFsImage")
+          .put("desc", "Loading fsimage")
           .put("status", "COMPLETE")
           .put("status", "COMPLETE")
           .put("percentComplete", 1.0f)
           .put("percentComplete", 1.0f)
           .put("steps", Collections.<Object>singletonList(
           .put("steps", Collections.<Object>singletonList(
             ImmutableMap.<String, Object>builder()
             ImmutableMap.<String, Object>builder()
               .put("name", "Inodes")
               .put("name", "Inodes")
+              .put("desc", "inodes")
               .put("count", 100L)
               .put("count", 100L)
               .put("total", 100L)
               .put("total", 100L)
               .put("percentComplete", 1.0f)
               .put("percentComplete", 1.0f)
@@ -124,6 +130,7 @@ public class TestStartupProgressServlet {
           .build(),
           .build(),
         ImmutableMap.<String, Object>builder()
         ImmutableMap.<String, Object>builder()
           .put("name", "LoadingEdits")
           .put("name", "LoadingEdits")
+          .put("desc", "Loading edits")
           .put("status", "RUNNING")
           .put("status", "RUNNING")
           .put("percentComplete", 0.5f)
           .put("percentComplete", 0.5f)
           .put("steps", Collections.<Object>singletonList(
           .put("steps", Collections.<Object>singletonList(
@@ -138,12 +145,14 @@ public class TestStartupProgressServlet {
           .build(),
           .build(),
         ImmutableMap.<String, Object>builder()
         ImmutableMap.<String, Object>builder()
           .put("name", "SavingCheckpoint")
           .put("name", "SavingCheckpoint")
+          .put("desc", "Saving checkpoint")
           .put("status", "PENDING")
           .put("status", "PENDING")
           .put("percentComplete", 0.0f)
           .put("percentComplete", 0.0f)
           .put("steps", Collections.emptyList())
           .put("steps", Collections.emptyList())
           .build(),
           .build(),
         ImmutableMap.<String, Object>builder()
         ImmutableMap.<String, Object>builder()
           .put("name", "SafeMode")
           .put("name", "SafeMode")
+          .put("desc", "Safe mode")
           .put("status", "PENDING")
           .put("status", "PENDING")
           .put("percentComplete", 0.0f)
           .put("percentComplete", 0.0f)
           .put("steps", Collections.emptyList())
           .put("steps", Collections.emptyList())
@@ -164,11 +173,13 @@ public class TestStartupProgressServlet {
       .put("phases", Arrays.<Object>asList(
       .put("phases", Arrays.<Object>asList(
         ImmutableMap.<String, Object>builder()
         ImmutableMap.<String, Object>builder()
           .put("name", "LoadingFsImage")
           .put("name", "LoadingFsImage")
+          .put("desc", "Loading fsimage")
           .put("status", "COMPLETE")
           .put("status", "COMPLETE")
           .put("percentComplete", 1.0f)
           .put("percentComplete", 1.0f)
           .put("steps", Collections.<Object>singletonList(
           .put("steps", Collections.<Object>singletonList(
             ImmutableMap.<String, Object>builder()
             ImmutableMap.<String, Object>builder()
               .put("name", "Inodes")
               .put("name", "Inodes")
+              .put("desc", "inodes")
               .put("count", 100L)
               .put("count", 100L)
               .put("total", 100L)
               .put("total", 100L)
               .put("percentComplete", 1.0f)
               .put("percentComplete", 1.0f)
@@ -177,6 +188,7 @@ public class TestStartupProgressServlet {
           .build(),
           .build(),
         ImmutableMap.<String, Object>builder()
         ImmutableMap.<String, Object>builder()
           .put("name", "LoadingEdits")
           .put("name", "LoadingEdits")
+          .put("desc", "Loading edits")
           .put("status", "COMPLETE")
           .put("status", "COMPLETE")
           .put("percentComplete", 1.0f)
           .put("percentComplete", 1.0f)
           .put("steps", Collections.<Object>singletonList(
           .put("steps", Collections.<Object>singletonList(
@@ -191,11 +203,13 @@ public class TestStartupProgressServlet {
           .build(),
           .build(),
         ImmutableMap.<String, Object>builder()
         ImmutableMap.<String, Object>builder()
           .put("name", "SavingCheckpoint")
           .put("name", "SavingCheckpoint")
+          .put("desc", "Saving checkpoint")
           .put("status", "COMPLETE")
           .put("status", "COMPLETE")
           .put("percentComplete", 1.0f)
           .put("percentComplete", 1.0f)
           .put("steps", Collections.<Object>singletonList(
           .put("steps", Collections.<Object>singletonList(
             ImmutableMap.<String, Object>builder()
             ImmutableMap.<String, Object>builder()
               .put("name", "Inodes")
               .put("name", "Inodes")
+              .put("desc", "inodes")
               .put("count", 300L)
               .put("count", 300L)
               .put("total", 300L)
               .put("total", 300L)
               .put("percentComplete", 1.0f)
               .put("percentComplete", 1.0f)
@@ -204,11 +218,13 @@ public class TestStartupProgressServlet {
           .build(),
           .build(),
         ImmutableMap.<String, Object>builder()
         ImmutableMap.<String, Object>builder()
           .put("name", "SafeMode")
           .put("name", "SafeMode")
+          .put("desc", "Safe mode")
           .put("status", "COMPLETE")
           .put("status", "COMPLETE")
           .put("percentComplete", 1.0f)
           .put("percentComplete", 1.0f)
           .put("steps", Collections.<Object>singletonList(
           .put("steps", Collections.<Object>singletonList(
             ImmutableMap.<String, Object>builder()
             ImmutableMap.<String, Object>builder()
               .put("name", "AwaitingReportedBlocks")
               .put("name", "AwaitingReportedBlocks")
+              .put("desc", "awaiting reported blocks")
               .put("count", 400L)
               .put("count", 400L)
               .put("total", 400L)
               .put("total", 400L)
               .put("percentComplete", 1.0f)
               .put("percentComplete", 1.0f)

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java

@@ -96,8 +96,8 @@ public class TestDNFencing {
     // Increase max streams so that we re-replicate quickly.
     // Increase max streams so that we re-replicate quickly.
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 1000);
     // See RandomDeleterPolicy javadoc.
     // See RandomDeleterPolicy javadoc.
-    conf.setClass("dfs.block.replicator.classname", RandomDeleterPolicy.class,
-        BlockPlacementPolicy.class); 
+    conf.setClass(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
+        RandomDeleterPolicy.class, BlockPlacementPolicy.class); 
     conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
     cluster = new MiniDFSCluster.Builder(conf)
     cluster = new MiniDFSCluster.Builder(conf)
       .nnTopology(MiniDFSNNTopology.simpleHATopology())
       .nnTopology(MiniDFSNNTopology.simpleHATopology())

+ 104 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -47,19 +48,22 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.RetriableException;
+import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtilTestHelper;
 import org.apache.hadoop.security.SecurityUtilTestHelper;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.AfterClass;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
 
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Joiner;
 
 
@@ -78,8 +82,12 @@ public class TestDelegationTokensWithHA {
   private static DelegationTokenSecretManager dtSecretManager;
   private static DelegationTokenSecretManager dtSecretManager;
   private static DistributedFileSystem dfs;
   private static DistributedFileSystem dfs;
 
 
-  @BeforeClass
-  public static void setupCluster() throws Exception {
+  private volatile boolean catchup = false;
+  
+  @Before
+  public void setupCluster() throws Exception {
+    SecurityUtilTestHelper.setTokenServiceUseIp(true);
+    
     conf.setBoolean(
     conf.setBoolean(
         DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
         DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
     conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL,
     conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL,
@@ -101,18 +109,12 @@ public class TestDelegationTokensWithHA {
         nn0.getNamesystem());
         nn0.getNamesystem());
   }
   }
 
 
-  @AfterClass
-  public static void shutdownCluster() throws IOException {
+  @After
+  public void shutdownCluster() throws IOException {
     if (cluster != null) {
     if (cluster != null) {
       cluster.shutdown();
       cluster.shutdown();
     }
     }
   }
   }
-
-
-  @Before
-  public void prepTest() {
-    SecurityUtilTestHelper.setTokenServiceUseIp(true);
-  }
   
   
   @Test
   @Test
   public void testDelegationTokenDFSApi() throws Exception {
   public void testDelegationTokenDFSApi() throws Exception {
@@ -155,6 +157,96 @@ public class TestDelegationTokensWithHA {
     doRenewOrCancel(token, clientConf, TokenTestAction.CANCEL);
     doRenewOrCancel(token, clientConf, TokenTestAction.CANCEL);
   }
   }
   
   
+  private class EditLogTailerForTest extends EditLogTailer {
+    public EditLogTailerForTest(FSNamesystem namesystem, Configuration conf) {
+      super(namesystem, conf);
+    }
+    
+    public void catchupDuringFailover() throws IOException {
+      synchronized (TestDelegationTokensWithHA.this) {
+        while (!catchup) {
+          try {
+            LOG.info("The editlog tailer is waiting to catchup...");
+            TestDelegationTokensWithHA.this.wait();
+          } catch (InterruptedException e) {}
+        }
+      }
+      super.catchupDuringFailover();
+    }
+  }
+  
+  /**
+   * Test if correct exception (StandbyException or RetriableException) can be
+   * thrown during the NN failover. 
+   */
+  @Test
+  public void testDelegationTokenDuringNNFailover() throws Exception {
+    EditLogTailer editLogTailer = nn1.getNamesystem().getEditLogTailer();
+    // stop the editLogTailer of nn1
+    editLogTailer.stop();
+    Configuration conf = (Configuration) Whitebox.getInternalState(
+        editLogTailer, "conf");
+    nn1.getNamesystem().setEditLogTailerForTests(
+        new EditLogTailerForTest(nn1.getNamesystem(), conf));
+    
+    // create token
+    final Token<DelegationTokenIdentifier> token =
+        getDelegationToken(fs, "JobTracker");
+    DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
+    byte[] tokenId = token.getIdentifier();
+    identifier.readFields(new DataInputStream(
+             new ByteArrayInputStream(tokenId)));
+
+    // Ensure that it's present in the nn0 secret manager and can
+    // be renewed directly from there.
+    LOG.info("A valid token should have non-null password, " +
+        "and should be renewed successfully");
+    assertTrue(null != dtSecretManager.retrievePassword(identifier));
+    dtSecretManager.renewToken(token, "JobTracker");
+    
+    // transition nn0 to standby
+    cluster.transitionToStandby(0);
+    
+    try {
+      cluster.getNameNodeRpc(0).renewDelegationToken(token);
+      fail("StandbyException is expected since nn0 is in standby state");
+    } catch (StandbyException e) {
+      GenericTestUtils.assertExceptionContains(
+          HAServiceState.STANDBY.toString(), e);
+    }
+    
+    new Thread() {
+      @Override
+      public void run() {
+        try {
+          cluster.transitionToActive(1);
+        } catch (Exception e) {
+          LOG.error("Transition nn1 to active failed", e);
+        }    
+      }
+    }.start();
+    
+    Thread.sleep(1000);
+    try {
+      nn1.getNamesystem().verifyToken(token.decodeIdentifier(),
+          token.getPassword());
+      fail("RetriableException/StandbyException is expected since nn1 is in transition");
+    } catch (IOException e) {
+      assertTrue(e instanceof StandbyException
+          || e instanceof RetriableException);
+      LOG.info("Got expected exception", e);
+    }
+    
+    catchup = true;
+    synchronized (this) {
+      this.notifyAll();
+    }
+    
+    Configuration clientConf = dfs.getConf();
+    doRenewOrCancel(token, clientConf, TokenTestAction.RENEW);
+    doRenewOrCancel(token, clientConf, TokenTestAction.CANCEL);
+  }
+  
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
   @Test
   @Test
   public void testDelegationTokenWithDoAs() throws Exception {
   public void testDelegationTokenWithDoAs() throws Exception {

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

@@ -17,12 +17,18 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.List;
+import java.util.Map;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -101,6 +107,50 @@ public class TestHASafeMode {
     }
     }
   }
   }
   
   
+  /**
+   * Make sure the client retries when the active NN is in safemode
+   */
+  @Test (timeout=300000)
+  public void testClientRetrySafeMode() throws Exception {
+    final Map<Path, Boolean> results = Collections
+        .synchronizedMap(new HashMap<Path, Boolean>());
+    final Path test = new Path("/test");
+    // let nn0 enter safemode
+    NameNodeAdapter.enterSafeMode(nn0, false);
+    LOG.info("enter safemode");
+    new Thread() {
+      @Override
+      public void run() {
+        try {
+          boolean mkdir = fs.mkdirs(test);
+          LOG.info("mkdir finished, result is " + mkdir);
+          synchronized (TestHASafeMode.this) {
+            results.put(test, mkdir);
+            TestHASafeMode.this.notifyAll();
+          }
+        } catch (Exception e) {
+          LOG.info("Got Exception while calling mkdir", e);
+        }
+      }
+    }.start();
+    
+    // make sure the client's call has actually been handled by the active NN
+    assertFalse("The directory should not be created while NN in safemode",
+        fs.exists(test));
+    
+    Thread.sleep(1000);
+    // let nn0 leave safemode
+    NameNodeAdapter.leaveSafeMode(nn0);
+    LOG.info("leave safemode");
+    
+    synchronized (this) {
+      while (!results.containsKey(test)) {
+        this.wait();
+      }
+      assertTrue(results.get(test));
+    }
+  }
+  
   private void restartStandby() throws IOException {
   private void restartStandby() throws IOException {
     cluster.shutdownNameNode(1);
     cluster.shutdownNameNode(1);
     // Set the safemode extension to be lengthy, so that the tests
     // Set the safemode extension to be lengthy, so that the tests

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

@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.ha;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.junit.Test;
+
+/**
+ * This test makes sure that when
+ * {@link DFSConfigKeys#DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY} is set,
+ * DFSClient instances can still be created within NN/DN (e.g., the fs instance
+ * used by the trash emptier thread in NN)
+ */
+public class TestLossyRetryInvocationHandler {
+  
+  @Test
+  public void testStartNNWithTrashEmptier() throws Exception {
+    MiniDFSCluster cluster = null;
+    Configuration conf = new HdfsConfiguration();
+    
+    // enable both trash emptier and dropping response
+    conf.setLong("fs.trash.interval", 360);
+    conf.setInt(DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY, 2);
+    
+    try {
+      cluster = new MiniDFSCluster.Builder(conf)
+          .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(0)
+          .build();
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+  
+}

+ 9 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
 import java.lang.reflect.Method;
@@ -646,10 +647,14 @@ public class TestRetryCacheWithHA {
     @Override
     @Override
     boolean checkNamenodeBeforeReturn() throws Exception {
     boolean checkNamenodeBeforeReturn() throws Exception {
       Path linkPath = new Path(link);
       Path linkPath = new Path(link);
-      FileStatus linkStatus = dfs.getFileLinkStatus(linkPath);
+      FileStatus linkStatus = null;
       for (int i = 0; i < CHECKTIMES && linkStatus == null; i++) {
       for (int i = 0; i < CHECKTIMES && linkStatus == null; i++) {
-        Thread.sleep(1000);
-        linkStatus = dfs.getFileLinkStatus(linkPath);
+        try {
+          linkStatus = dfs.getFileLinkStatus(linkPath);
+        } catch (FileNotFoundException fnf) {
+          // Ignoring, this can be legitimate.
+          Thread.sleep(1000);
+        }
       }
       }
       return linkStatus != null;
       return linkStatus != null;
     }
     }
@@ -857,4 +862,4 @@ public class TestRetryCacheWithHA {
           + results.get(op.name));
           + results.get(op.name));
     }
     }
   }
   }
-}
+}

+ 0 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java

@@ -192,17 +192,10 @@ public class TestNameNodeMetrics {
     assertCounter("CreateFileOps", 1L, rb);
     assertCounter("CreateFileOps", 1L, rb);
     assertCounter("FilesCreated", (long)file.depth(), rb);
     assertCounter("FilesCreated", (long)file.depth(), rb);
 
 
-    // Blocks are stored in a hashmap. Compute its capacity, which
-    // doubles every time the number of entries reach the threshold.
-    int threshold = (int)(blockCapacity * BlockManager.DEFAULT_MAP_LOAD_FACTOR);
-    while (threshold < blockCount) {
-      blockCapacity <<= 1;
-    }
     long filesTotal = file.depth() + 1; // Add 1 for root
     long filesTotal = file.depth() + 1; // Add 1 for root
     rb = getMetrics(NS_METRICS);
     rb = getMetrics(NS_METRICS);
     assertGauge("FilesTotal", filesTotal, rb);
     assertGauge("FilesTotal", filesTotal, rb);
     assertGauge("BlocksTotal", blockCount, rb);
     assertGauge("BlocksTotal", blockCount, rb);
-    assertGauge("BlockCapacity", blockCapacity, rb);
     fs.delete(file, true);
     fs.delete(file, true);
     filesTotal--; // reduce the filecount for deleted file
     filesTotal--; // reduce the filecount for deleted file
 
 

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

@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.snapshot;
+
+import java.io.IOException;
+import java.util.EnumSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSOutputStream;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestOpenFilesWithSnapshot {
+  private Configuration conf = new Configuration();
+  MiniDFSCluster cluster = null;
+  DistributedFileSystem fs = null;
+
+  @Before
+  public void setup() throws IOException {
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+    conf.set("dfs.blocksize", "1048576");
+    fs = cluster.getFileSystem();
+  }
+
+  @After
+  public void teardown() throws IOException {
+    if (fs != null) {
+      fs.close();
+    }
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+
+  }
+
+  @Test
+  public void testUCFileDeleteWithSnapShot() throws Exception {
+    Path path = new Path("/test");
+    doWriteAndAbort(fs, path);
+
+    // delete files separately
+    fs.delete(new Path("/test/test/test2"), true);
+    fs.delete(new Path("/test/test/test3"), true);
+    cluster.restartNameNode();
+  }
+
+  @Test
+  public void testParentDirWithUCFileDeleteWithSnapShot() throws Exception {
+    Path path = new Path("/test");
+    doWriteAndAbort(fs, path);
+
+    // delete parent directory
+    fs.delete(new Path("/test/test"), true);
+    cluster.restartNameNode();
+  }
+
+  private void doWriteAndAbort(DistributedFileSystem fs, Path path)
+      throws IOException {
+    fs.mkdirs(path);
+    fs.allowSnapshot(path);
+    DFSTestUtil
+        .createFile(fs, new Path("/test/test1"), 100, (short) 2, 100024L);
+    DFSTestUtil
+        .createFile(fs, new Path("/test/test2"), 100, (short) 2, 100024L);
+    Path file = new Path("/test/test/test2");
+    FSDataOutputStream out = fs.create(file);
+    for (int i = 0; i < 2; i++) {
+      long count = 0;
+      while (count < 1048576) {
+        out.writeBytes("hell");
+        count += 4;
+      }
+    }
+    ((DFSOutputStream) out.getWrappedStream()).hsync(EnumSet
+        .of(SyncFlag.UPDATE_LENGTH));
+    DFSTestUtil.abortStream((DFSOutputStream) out.getWrappedStream());
+    Path file2 = new Path("/test/test/test3");
+    FSDataOutputStream out2 = fs.create(file2);
+    for (int i = 0; i < 2; i++) {
+      long count = 0;
+      while (count < 1048576) {
+        out2.writeBytes("hell");
+        count += 4;
+      }
+    }
+    ((DFSOutputStream) out2.getWrappedStream()).hsync(EnumSet
+        .of(SyncFlag.UPDATE_LENGTH));
+    DFSTestUtil.abortStream((DFSOutputStream) out2.getWrappedStream());
+    fs.createSnapshot(path, "s1");
+  }
+}

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